From 1e3ca11a53bdf14eed7b4ad0d87318ecd511e8a3 Mon Sep 17 00:00:00 2001 From: vatsal mevada Date: Sat, 10 Aug 2019 23:08:56 +0530 Subject: [PATCH 01/26] [SNAP-3110] Changes for SNAP-3110 with fix for backward compatibility (#1423) Re-merging changes for SNAP-3110 with the fix for backward compatibility. Earlier we introduced a new parameter for the procedure `CREATE_OR_DROP_RESERVOIR_REGION`. This can break the upgrade from older versions while DDL replay. Instead of exposing extra parameter for removal of sampler, we are not removing it always along with the reservoir region. Fixing failing test `AQPCatalogConsistencyDUnitTest` --- .../columnar/impl/StoreCallbacksImpl.scala | 18 ++++++++++++++++++ store | 2 +- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala index 02d2d371e4..897d48b064 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala @@ -16,12 +16,14 @@ */ package org.apache.spark.sql.execution.columnar.impl +import java.lang.reflect.Method import java.net.URLClassLoader import java.sql.SQLException import java.util.Collections import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal import com.gemstone.gemfire.cache.{EntryDestroyedException, RegionDestroyedException} import com.gemstone.gemfire.internal.cache.lru.LRUEntry @@ -44,6 +46,7 @@ import com.pivotal.gemfirexd.internal.impl.sql.execute.PrivilegeInfo import com.pivotal.gemfirexd.internal.shared.common.reference.SQLState import io.snappydata.SnappyTableStatsProviderService import io.snappydata.sql.catalog.{CatalogObjectType, SnappyExternalCatalog} +import org.apache.spark import org.apache.spark.Logging import org.apache.spark.memory.{MemoryManagerCallback, MemoryMode} @@ -581,6 +584,21 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable } else userId } else userId } + + private lazy val removeSamplerMethod: Method = { + val samplerClassName = "org.apache.spark.sql.execution.StratifiedSampler" + val samplerClass = spark.util.Utils.classForName(samplerClassName) + samplerClass.getMethod("removeSampler", classOf[String], classOf[Boolean]) + } + + override def removeSampler(sampleTableName: String): Unit = { + try { + removeSamplerMethod.invoke(null, sampleTableName, Boolean.box(false)) + } + catch { + case NonFatal(e) => logWarning("Failure while removing sampler:", e) + } + } } trait StoreCallback extends Serializable { diff --git a/store b/store index ba7e579fa5..5f1de4f528 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit ba7e579fa5fcb34dd69c858d169a4b8b4c614457 +Subproject commit 5f1de4f52884bf11fad9b4af5a2f434025e7c797 From b385a6640391fee727062e68fa99593c76b86ec0 Mon Sep 17 00:00:00 2001 From: Vatsal Mevada Date: Sat, 10 Aug 2019 23:10:50 +0530 Subject: [PATCH 02/26] Linking store --- store | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store b/store index 5f1de4f528..9b5b814702 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 5f1de4f52884bf11fad9b4af5a2f434025e7c797 +Subproject commit 9b5b8147027d5684cb3adae0cd800a8699885ce5 From 111a4b5da3cb93602fd1b6d6afc63ec5f651da8c Mon Sep 17 00:00:00 2001 From: ahshahid Date: Sat, 10 Aug 2019 17:14:21 -0700 Subject: [PATCH 03/26] Snap 3053 1 (#1425) * Fix for SNAP-3132 VM crash. The issue was that setting of the map as null at the end of the loop was causing the map to be GCed causing string locations in variable to become invalid. Adding the single map term to the overflow list to prevent it from being GCed prematurely * Using a static array of MathContext objects for all query instances * changes for test to enable codegenfallback conditionally --- .../cluster/ClusterManagerTestBase.scala | 1 - .../cluster/ClusterMgrDUnitTest.scala | 3 +- .../ExecutionEngineArbiterDUnitTest.scala | 2 +- .../SnappyTableStatsProviderDUnitTest.scala | 3 +- .../cluster/SplitSnappyClusterDUnitTest.scala | 33 +- .../cluster/ValidateMVCCDUnitTest.scala | 3 +- .../JDBCPreparedStatementDUnitTest.scala | 2 + .../DynamicJarInstallationDUnitTest.scala | 1 + .../spark/sql/SmartConnectorFunctions.scala | 1 - .../org/apache/spark/sql/TPCHDUnitTest.scala | 1 - .../sql/execution/benchmark/TAQTest.scala | 3 +- .../spark/sql/store/SHAByteBufferTest.scala | 1347 ++++++++++++++++ .../cluster/SplitClusterDUnitTestBase.scala | 1 + .../main/scala/io/snappydata/Literals.scala | 33 + .../apache/spark/sql/SnappyStrategies.scala | 6 +- .../spark/sql/execution/SHAMapAccessor.scala | 1353 +++++++++++++++++ .../aggregate/SnappyHashAggregateExec.scala | 694 ++++++++- .../impl/JDBCSourceAsColumnarStore.scala | 4 +- .../spark/sql/hive/SnappySessionState.scala | 6 +- .../apache/spark/sql/internal/session.scala | 5 +- .../spark/sql/types/TypeUtilities.scala | 24 + .../io/snappydata/core/LocalTestData.scala | 3 + .../BufferSizeLimitExceededException.scala | 24 + .../collection/ByteBufferHashMap.scala | 107 +- .../snappydata/collection/DictionaryMap.scala | 6 +- .../io/snappydata/collection/SHAMap.scala | 41 + 26 files changed, 3647 insertions(+), 60 deletions(-) create mode 100644 cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala create mode 100644 core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala create mode 100644 encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala create mode 100644 encoders/src/main/scala/io/snappydata/collection/SHAMap.scala diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala index fefef812a1..58deca8651 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala @@ -202,7 +202,6 @@ abstract class ClusterManagerTestBase(s: String) Array(vm3, vm2, vm1, vm0).foreach(_.invoke(getClass, "stopNetworkServers")) stopNetworkServers() } - bootProps.clear() } diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala index 8ba74300a5..4638eaf432 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala @@ -26,7 +26,8 @@ import com.gemstone.gemfire.cache.LowMemoryException import org.apache.spark.sql.{Row, SnappyContext} import org.apache.spark.{Logging, SparkConf, SparkContext} -class ClusterMgrDUnitTest(s: String) extends ClusterManagerTestBase(s) with Logging { +class ClusterMgrDUnitTest(s: String) extends ClusterManagerTestBase(s) with Logging +{ import ClusterMgrDUnitTest._ diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala index ae1f70aeda..fb9a9d3436 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala @@ -714,7 +714,7 @@ trait ExecutionEngineArbiterTestBase { createTables_SNAP1507(snc, "COLUMN") val conn = DriverManager.getConnection( "jdbc:snappydata://" + startNetServer) - + val s = conn.createStatement() val query = "select T1_COL1, T1_COL2, T1_COL3, T1_COL4, T1_COL5, T1_COL6, T1_COL7," + " T1_COL8 from TABLE1 as tab1 where exists (select * from " + "TABLE2 as tab2 where exists (select * from " + diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala index b6a00b7548..cb3b79df4a 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.ColumnFormatRelation import org.apache.spark.sql.{SaveMode, SnappyContext} -class SnappyTableStatsProviderDUnitTest(s: String) extends ClusterManagerTestBase(s) { +class SnappyTableStatsProviderDUnitTest(s: String) extends ClusterManagerTestBase(s) +{ val table = "test.test_table" diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 72ee536e1a..7e49eb9452 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -49,7 +49,8 @@ import org.apache.spark.{Logging, SparkConf, SparkContext} * Basic tests for non-embedded mode connections to an embedded cluster. */ class SplitSnappyClusterDUnitTest(s: String) - extends ClusterManagerTestBase(s) with SplitClusterDUnitTestBase with Serializable { + extends ClusterManagerTestBase(s) with SplitClusterDUnitTestBase + with Serializable { override val locatorNetPort: Int = testObject.locatorNetPort @@ -72,7 +73,7 @@ class SplitSnappyClusterDUnitTest(s: String) } override def afterClass(): Unit = { - Array(vm2, vm1, vm0).foreach(_.invoke(getClass, "stopNetworkServers")) + Array(vm2, vm1, vm0, vm3).foreach(_.invoke(getClass, "stopNetworkServers")) ClusterManagerTestBase.stopNetworkServers() vm3.invoke(classOf[ClusterManagerTestBase], "stopSparkCluster", sparkProductDir) super.afterClass() @@ -388,8 +389,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testStaleCatalog(): Unit = { - - val snc = SnappyContext(sc) + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") snc.sql(s"CREATE TABLE T5(COL1 STRING, COL2 STRING) USING column OPTIONS" + s" (key_columns 'col1', PARTITION_BY 'COL1', COLUMN_MAX_DELTA_ROWS '1')") @@ -419,7 +420,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testStaleCatalogRetryForStreamingSink(): Unit = { - val snc = SnappyContext(sc) + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") import scala.concurrent.ExecutionContext.Implicits.global val testTempDirectory = "/tmp/SplitSnappyClusterDUnitTest" @@ -456,7 +458,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testSNAP3024(): Unit = { - val snc = SnappyContext(sc) + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") snc.sql(s"CREATE TABLE T5(COL1 STRING, COL2 STRING) USING column OPTIONS" + s" (key_columns 'col1', PARTITION_BY 'COL1', COLUMN_MAX_DELTA_ROWS '1')") snc.sql("insert into t5 values('1', '1')") @@ -489,7 +492,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testSmartConnectorAfterBucketRebalance(): Unit = { - val snc = SnappyContext(sc) + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") snc.sql(s"CREATE TABLE T5(COL1 STRING, COL2 STRING) USING column OPTIONS" + s" (key_columns 'col1', PARTITION_BY 'COL1', COLUMN_MAX_DELTA_ROWS '1')") snc.sql("insert into t5 values('1', '1')") @@ -530,8 +534,8 @@ class SplitSnappyClusterDUnitTest(s: String) } private def insertDataAfterStaleCatalog(tableType: String) = { - val snc = SnappyContext(sc) - + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") logInfo(s"insertDataAfterStaleCatalog: invoked for $tableType table") if (tableType == "COLUMN") { snc.sql(s"CREATE TABLE T5(COL1 STRING, COL2 STRING) USING column OPTIONS" + @@ -576,8 +580,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testDeleteAfterStaleCatalog(): Unit = { - val snc = SnappyContext(sc) - + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") snc.sql(s"CREATE TABLE T6(COL1 STRING, COL2 STRING) USING column OPTIONS" + s" (key_columns 'COL1', PARTITION_BY 'COL1', COLUMN_MAX_DELTA_ROWS '1')") snc.sql("insert into t6 values('1', '1')") @@ -612,8 +616,8 @@ class SplitSnappyClusterDUnitTest(s: String) } def testUpdateAfterStaleCatalog(): Unit = { - val snc = SnappyContext(sc) - + val snc = SnappyContext(sc).newSession() + snc.setConf(Property.TestDisableCodeGenFlag.name, "false") snc.sql(s"CREATE TABLE T7(COL1 STRING, COL2 STRING) USING column OPTIONS" + s" (key_columns 'COL1', PARTITION_BY 'COL1', COLUMN_MAX_DELTA_ROWS '1')") snc.sql("insert into t7 values('1', '1')") @@ -922,7 +926,7 @@ object SplitSnappyClusterDUnitTest .set("spark.sql.autoBroadcastJoinThreshold", "-1") .set("snappydata.connection", connectionURL) .set("snapptdata.sql.planCaching", random.nextBoolean().toString) - + .set(Property.TestDisableCodeGenFlag.name, "false") logInfo("Spark conf:" + conf.getAll.toString) val sc = SparkContext.getOrCreate(conf) @@ -1467,3 +1471,4 @@ object SplitSnappyClusterDUnitTest } } } + diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala index 9bc078d9d0..45a9b7fa08 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.{SaveMode, SnappyContext} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.ColumnFormatRelation -class ValidateMVCCDUnitTest(val s: String) extends ClusterManagerTestBase(s) with Logging { +class ValidateMVCCDUnitTest(val s: String) extends ClusterManagerTestBase(s) + with Logging { // set default batch size for this test bootProps.setProperty(io.snappydata.Property.ColumnBatchSize.name, "100") diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala index 1c8e356a71..9d59decc96 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala @@ -17,9 +17,11 @@ package io.snappydata.externalstore import java.sql.PreparedStatement + import io.snappydata.cluster.ClusterManagerTestBase import io.snappydata.test.dunit.AvailablePortHelper import org.junit.Assert.assertEquals + import org.apache.spark.Logging diff --git a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala index 37ff2c2b94..73307d28b4 100644 --- a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala @@ -21,6 +21,7 @@ package org.apache.spark import java.io.File import java.net.URL import java.sql.{Connection, DriverManager} + import _root_.io.snappydata.Constant import _root_.io.snappydata.cluster.ClusterManagerTestBase import org.joda.time.DateTime diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala b/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala index 05adafa6f5..a7e7e91b5f 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala @@ -56,7 +56,6 @@ object SmartConnectorFunctions { .set("spark.executor.extraClassPath", getEnvironmentVariable("SNAPPY_DIST_CLASSPATH")) .set("snappydata.connection", s"localhost:$locatorNetPort") - val sc = SparkContext.getOrCreate(conf) val snc = SnappyContext(sc) diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala index 3b392fe42e..09181c76a1 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala @@ -59,7 +59,6 @@ class TPCHDUnitTest(s: String) extends ClusterManagerTestBase(s) def testSnappy(): Unit = { val snc = SnappyContext(sc) - // create table randomly either using smart connector or // from embedded mode if ((System.currentTimeMillis() % 2) == 0) { diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala index 6281c7520e..2039f11800 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala @@ -22,7 +22,7 @@ import java.time.{ZoneId, ZonedDateTime} import scala.util.Random import com.typesafe.config.Config -import io.snappydata.SnappyFunSuite +import io.snappydata.{Property, SnappyFunSuite} import org.scalatest.Assertions import org.apache.spark.memory.SnappyUnifiedMemoryManager @@ -285,6 +285,7 @@ object TAQTest extends Logging with Assertions { .set("spark.serializer", "org.apache.spark.serializer.PooledKryoSerializer") .set("spark.closure.serializer", "org.apache.spark.serializer.PooledKryoSerializer") .set("snappydata.sql.planCaching", random.nextBoolean().toString) + if (addOn != null) { addOn(conf) } diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala new file mode 100644 index 0000000000..b2f36e642b --- /dev/null +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala @@ -0,0 +1,1347 @@ +/* + * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.sql.store + +import java.math.BigDecimal +import java.sql.{Connection, Date, DriverManager, SQLException, SQLType, Timestamp, Types} +import java.util.Random + +import scala.collection.mutable + +import com.pivotal.gemfirexd.{Attribute, TestUtil} +import com.pivotal.gemfirexd.security.SecurityTestUtils +import io.snappydata.{Constant, Property, SnappyFunSuite} +import org.scalatest.BeforeAndAfterAll +import org.junit.Assert._ + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Row, SaveMode, SnappyContext, SnappySession} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.benchmark.ColumnCacheBenchmark +import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.types.{ArrayType, CalendarIntervalType, IntegerType, LongType, StringType, StructField, StructType} +import org.apache.spark.unsafe.types.CalendarInterval + +class SHAByteBufferTest extends SnappyFunSuite with BeforeAndAfterAll { + + var serverHostPort2: String = _ + + override def beforeAll(): Unit = { + super.beforeAll() + serverHostPort2 = TestUtil.startNetServer() + } + + protected override def newSparkConf(addOn: (SparkConf) => SparkConf): SparkConf = { + + System.setProperty("spark.testing", "true") + super.newSparkConf((conf: SparkConf) => { + conf.set("spark.sql.codegen.maxFields", "110") + .set("spark.sql.codegen.fallback", "false") + .set("snappydata.sql.useOptimizedHashAggregateForSingleKey", "true") + .set(Property.TestDisableCodeGenFlag.name , "true") + conf + }) + } + + override def afterAll(): Unit = { + TestUtil.stopNetServer() + super.afterAll() + System.clearProperty("spark.testing") + } + + test("simple aggregate query") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 int) " + + "using column ") + val range = 50 + val groupingDivisor = 10 + val insertDF = snc.range(50).selectExpr("id", "id*2", "id * 3", + s"id % $groupingDivisor ") + insertDF.write.insertInto("test1") + val rs = snc.sql("select col4, sum(col1) as summ1, sum(col2) as summ2 " + + " from test1 group by col4") + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor, results.length) + results.foreach(row => { + val groupKey = row.getInt(0) + val n = range / groupingDivisor + val sum1 = ((n / 2.0f) * ((2 * groupKey) + (n - 1) * groupingDivisor)).toLong + val sum2 = ((n / 2.0f) * ((2 * 2 * groupKey) + (n - 1) * 2 * groupingDivisor)).toLong + val sumAgg1 = row.getLong(1) + val sumAgg2 = row.getLong(2) + assertEquals(sum1, sumAgg1) + assertEquals(sum2, sumAgg2) + }) + snc.dropTable("test1") + } + + test("not nullable multiple strings as grouping key") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int not null, col2 String not null , " + + "col3 int not null, col4 String not null) using column ") + val range = 50 + val groupingDivisor = 10 + val groupingDivisor1 = 5 + val ps = getSqlConnection.prepareStatement("insert into test1 values (?, ?, ?, ?) ") + for (i <- 0 until range) { + for (j <- 0 until range) { + ps.setInt(1, i * j) + ps.setString(2, s"col2_${j % groupingDivisor1}") + ps.setInt(3, j) + ps.setString(4, s"col4_${i % groupingDivisor}") + ps.addBatch() + } + } + ps.executeBatch() + val rs = snc.sql("select col4, col2 , sum(col1) as summ1 from test1 group by col4, col2") + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor * groupingDivisor1, results.length) + + snc.dropTable("test1") + } + + test("multiple aggregates query") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 String) " + + "using column ") + val range = 50 + val groupingDivisor = 10 + val insertDF = snc.range(50).selectExpr("id", "id*2", "id * 3", + s"Concat( 'test', Cast(id % $groupingDivisor as string) ) ") + insertDF.write.insertInto("test1") + val rs = snc.sql("select col4, sum(col1) as summ1, sum(col2) as summ2 " + + " from test1 group by col4") + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor, results.length) + results.foreach(row => { + val groupKey = row.getString(0).substring("test".length).toInt + val n = range / groupingDivisor + val sum1 = ((n / 2.0f) * ((2 * groupKey) + (n - 1) * groupingDivisor)).toLong + val sum2 = ((n / 2.0f) * ((2 * 2 * groupKey) + (n - 1) * 2 * groupingDivisor)).toLong + val sumAgg1 = row.getLong(1) + val sumAgg2 = row.getLong(2) + assertEquals(sum1, sumAgg1) + assertEquals(sum2, sumAgg2) + }) + snc.dropTable("test1") + } + + test("multiple aggregates query with null grouping keys") { + snc + val conn = getSqlConnection + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 String) " + + "using column ") + val range = 200 + val groupingDivisor = 10 + val insertPs = conn.prepareStatement("insert into test1 values(?,?,?,?)") + for (i <- 0 until range) { + insertPs.setInt(1, i) + insertPs.setInt(2, i * 2) + insertPs.setInt(3, i * 3) + if (i % groupingDivisor == 0) { + insertPs.setNull(4, Types.VARCHAR) + } else { + insertPs.setString(4, s"test${i % groupingDivisor}") + } + insertPs.addBatch() + } + insertPs.executeBatch() + + val rs = snc.sql("select col4, sum(col1) as summ1, sum(col2) as summ2 " + + " from test1 group by col4") + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor, results.length) + results.foreach(row => { + val groupKey = if (row.isNullAt(0)) { + 0 + } else { + row.getString(0).substring("test".length).toInt + } + val n = range / groupingDivisor + val sum1 = ((n / 2.0f) * ((2 * groupKey) + (n - 1) * groupingDivisor)).toLong + val sum2 = ((n / 2.0f) * ((2 * 2 * groupKey) + (n - 1) * 2 * groupingDivisor)).toLong + val sumAgg1 = row.getLong(1) + val sumAgg2 = row.getLong(2) + assertEquals(sum1, sumAgg1) + assertEquals(sum2, sumAgg2) + }) + snc.dropTable("test1") + } + + test("multiple aggregates query with null and two grouping keys") { + snc + val conn = getSqlConnection + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 String, col5 String) " + + "using column ") + val range = 50 + val groupingDivisor1 = 10 + val groupingDivisor2 = 10 + val insertPs = conn.prepareStatement("insert into test1 values(?,?,?,?,?)") + for (i <- 0 until range; j <- 0 until groupingDivisor2) { + insertPs.setInt(1, i) + insertPs.setInt(2, i * 2) + insertPs.setInt(3, i * 3) + if (i % groupingDivisor1 == 0) { + insertPs.setNull(4, Types.VARCHAR) + } else { + insertPs.setString(4, s"test${i % groupingDivisor1}") + } + + if (j % groupingDivisor2 == 0) { + insertPs.setNull(5, Types.VARCHAR) + } else { + insertPs.setString(5, s"test${j % groupingDivisor2}") + } + insertPs.addBatch() + } + insertPs.executeBatch() + + val rs = snc.sql("select col4, col5, sum(col1) as summ1, sum(col2) as summ2 " + + " from test1 group by col4, col5") + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor1 * groupingDivisor2, results.length) + results.foreach(row => { + val groupKey = if (row.isNullAt(0)) { + 0 + } else { + row.getString(0).substring("test".length).toInt + } + val n = range / groupingDivisor1 + val sum1 = ((n / 2.0f) * ((2 * groupKey) + (n - 1) * groupingDivisor1)).toLong + val sum2 = ((n / 2.0f) * ((2 * 2 * groupKey) + (n - 1) * 2 * groupingDivisor1)).toLong + val sumAgg1 = row.getLong(2) + val sumAgg2 = row.getLong(3) + assertEquals(sum1, sumAgg1) + assertEquals(sum2, sumAgg2) + }) + snc.dropTable("test1") + } + + + test("Test incremental addition of grouping columns with the last grouping column as null") { + val numKeyCols = 100 + snc + val conn = getSqlConnection + + snc.sql("drop table if exists test1") + + val createTableStr = "create table test1 ( num1 int, num2 int," + + (for (i <- 3 until numKeyCols + 3) yield s"col$i string").mkString(",") + ")" + + snc.sql(s" $createTableStr using column ") + + + val insertStr = "insert into test1 values(?, ?," + + (for (i <- 0 until numKeyCols) yield "?").mkString(",") + ")" + + val insertPs = conn.prepareStatement(insertStr) + + for (i <- 3 until numKeyCols + 3) { + val groupingCols = (for (j <- 3 to i) yield s"col$j").mkString(", ") + val sqlStr = s"select sum(num1) as summ1, sum(num2) as summ2, $groupingCols " + + s" from test1 group by $groupingCols" + // println(s"executing query making $i grouping column as null") + insertPs.setInt(1, i) + insertPs.setInt(2, i * 2) + for (j <- 3 until i) { + insertPs.setString(j, s"col$j") + } + insertPs.setNull(i, Types.VARCHAR) + insertPs.addBatch() + insertPs.executeBatch() + // now query + + val rs = snc.sql(sqlStr) + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(1, results.length) + for (j <- 2 until i - 1) { + assertEquals(s"col${j + 1}", results(0).getString(j)) + } + assertTrue(results(0).isNullAt(i - 1)) + snc.sql("delete from test1") + } + snc.dropTable("test1") + } + + test("null grouping key bit masking for 1 to 100 columns in group by") { + val numKeyCols = 100 + snc + val conn = getSqlConnection + + snc.sql("drop table if exists test1") + + val createTableStr = "create table test1 ( num1 int, num2 int," + + (for (i <- 3 until numKeyCols + 3) yield s"col$i string").mkString(",") + ")" + + snc.sql(s" $createTableStr using column ") + val range = 100 + val groupingDivisor = 10 + val insertStr = "insert into test1 values(?, ?," + + (for (_ <- 0 until numKeyCols) yield "?").mkString(",") + ")" + val insertPs = conn.prepareStatement(insertStr) + for (i <- 0 until range) { + insertPs.setInt(1, i) + insertPs.setInt(2, i * 2) + for (j <- 3 until numKeyCols + 3) { + if (j == i % numKeyCols) { + insertPs.setNull(j, Types.VARCHAR) + } else { + insertPs.setString(j, s"col$j-${i % groupingDivisor}") + } + } + insertPs.addBatch() + } + + insertPs.executeBatch() + + for (i <- 3 until numKeyCols + 3) { + // println(s"executing query number ${i -2} ") + val groupingCols = (for (j <- 3 to i) yield s"col$j").mkString(", ") + val sqlStr = s"select sum(num1) as summ1, sum(num2) as summ2, $groupingCols " + + s" from test1 group by $groupingCols" + val rs = snc.sql(sqlStr) + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertTrue(results.length > 0) + } + + + for (i <- 3 until numKeyCols + 3) { + // println(s"executing query number ${i -2} ") + snc.sql("delete from test1") + // insert 100 rows with num1 as 1 and num2 as 2 + // string cols as col-n & rest as null + val numRowsInBatch = 100 + val const1 = 1 + val const2 = 2 + val const4 = 4 + val const8 = 8 + for (_ <- 0 until numRowsInBatch) { + insertPs.setInt(1, const1) + insertPs.setInt(2, const2) + for (k <- 3 to i) { + insertPs.setString(k, s"col-$k") + } + for (p <- i + 1 until numKeyCols + 3) { + insertPs.setNull(p, Types.VARCHAR) + } + insertPs.addBatch() + } + + // insert 100 rows with num1 as 4 and num2 as 8 + // string cols as col-n for n - 1 cols, & the nth col as null + + for (_ <- 0 until numRowsInBatch) { + insertPs.setInt(1, const4) + insertPs.setInt(2, const8) + for (k <- 3 until i) { + insertPs.setString(k, s"col-$k") + } + for (p <- i until numKeyCols + 3) { + insertPs.setNull(p, Types.VARCHAR) + } + insertPs.addBatch() + } + insertPs.executeBatch() + + val groupingCols = (for (j <- 3 to i) yield s"col$j").mkString(", ") + val sqlStr = s"select sum(num1) as summ1, sum(num2) as summ2, $groupingCols " + + s" from test1 group by $groupingCols" + val rs = snc.sql(sqlStr) + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(2, results.length) + val row1 = results(0) + val row2 = results(1) + var foundNullKey = false + if (row1.isNullAt(i - 1)) { + foundNullKey = true + assertEquals(numRowsInBatch * const4, row1.getLong(0)) + for (j <- 2 until i - 1) { + assertEquals(s"col-${j + 1}", row1.getString(j)) + } + } else { + assertEquals(numRowsInBatch * const1, row1.getLong(0)) + for (j <- 2 until i) { + assertEquals(s"col-${j + 1}", row1.getString(j)) + } + } + + if (row2.isNullAt(i - 1)) { + foundNullKey = true + assertEquals(numRowsInBatch * const4, row2.getLong(0)) + for (j <- 2 until i - 1) { + assertEquals(s"col-${j + 1}", row2.getString(j)) + } + } else { + assertEquals(numRowsInBatch * const1, row2.getLong(0)) + for (j <- 2 until i) { + assertEquals(s"col-${j + 1}", row2.getString(j)) + } + } + assert(foundNullKey) + + } + + snc.dropTable("test1") + } + + + // missing types = struct type, maptype,userdefinedtype, hivestringtype, + // array type, calendarinterval + test("aggregate functions & grouping on each of spark data type") { + + snc + val conn = getSqlConnection + + type DataMap = Map[Int, Any] + + snc.sql("drop table if exists test1") + + val numCols = 14 + + val createTableStr = s"create table test1 ( col000 int, " + + s"col${Types.TINYINT.toString.replaceAll("-", "_")} byte," + + s" col${Types.SMALLINT.toString.replaceAll("-", "_")} short, " + + s"col${Types.INTEGER.toString.replaceAll("-", "_")} int," + + s"col${Types.BIGINT.toString.replaceAll("-", "_")} long, " + + s"col${Types.FLOAT.toString.replaceAll("-", "_")} float, " + + s"col${Types.DOUBLE.toString.replaceAll("-", "_")} double, " + + s"col${Types.DECIMAL.toString.replaceAll("-", "_")}_1 decimal(12, 5)," + + s" col${Types.DECIMAL.toString.replaceAll("-", "_")}_2 decimal(28, 25)," + + s"col${Types.TIMESTAMP.toString.replaceAll("-", "_")} timestamp," + + s" col${Types.VARCHAR.toString.replaceAll("-", "_")} string, " + + s"col${Types.BOOLEAN.toString.replaceAll("-", "_")} boolean, " + + s"col${Types.DATE.toString.replaceAll("-", "_")} date, " + + s"col${Types.BINARY.toString.replaceAll("-", "_")} binary) using column" + + + snc.sql(createTableStr) + + + val insertStr = s"insert into test1 " + + s"values (${(for (_ <- Range(0, numCols, 1)) yield "?").mkString(",")} )" + + val insertPs = conn.prepareStatement(insertStr) + + val posToTypeMapping = Map[Int, Int]( + 1 -> Types.INTEGER, 2 -> Types.TINYINT, 3 -> Types.SMALLINT, + 4 -> Types.INTEGER, 5 -> Types.BIGINT, 6 -> Types.FLOAT, + 7 -> Types.DOUBLE, 8 -> Types.DECIMAL, 9 -> Types.DECIMAL, 10 -> Types.TIMESTAMP, + 11 -> Types.VARCHAR, 12 -> Types.BOOLEAN, 13 -> Types.DATE, 14 -> Types.BINARY + ) + + val typeMapping: Map[Int, (Int, Any) => Unit] = + Map( + Types.TINYINT -> ((i: Int, o: Any) => insertPs.setByte(i, o.asInstanceOf[Byte])), + Types.SMALLINT -> ((i: Int, o: Any) => insertPs.setShort(i, o.asInstanceOf[Short])), + Types.INTEGER -> ((i: Int, o: Any) => insertPs.setInt(i, o.asInstanceOf[Int])), + Types.BIGINT -> ((i: Int, o: Any) => insertPs.setLong(i, o.asInstanceOf[Long])), + Types.FLOAT -> ((i: Int, o: Any) => insertPs.setFloat(i, o.asInstanceOf[Float])), + Types.DOUBLE -> ((i: Int, o: Any) => insertPs.setDouble(i, o.asInstanceOf[Double])), + Types.DECIMAL -> ((i: Int, o: Any) => + insertPs.setBigDecimal(i, o.asInstanceOf[java.math.BigDecimal])), + Types.TIMESTAMP -> ((i: Int, o: Any) => + insertPs.setTimestamp(i, o.asInstanceOf[Timestamp])), + Types.VARCHAR -> ((i: Int, o: Any) => insertPs.setString(i, o.asInstanceOf[String])), + Types.BOOLEAN -> ((i: Int, o: Any) => insertPs.setBoolean(i, o.asInstanceOf[Boolean])), + Types.DATE -> ((i: Int, o: Any) => insertPs.setDate(i, o.asInstanceOf[Date])), + Types.BINARY -> ((i: Int, o: Any) => insertPs.setBytes(i, o.asInstanceOf[Array[Byte]])) + ) + + def setInInsertStatement(dataMap: Map[Int, Any]): Unit = { + for (entry <- dataMap) { + val pos = entry._1 + val value = entry._2 + typeMapping(posToTypeMapping(pos))(pos, value) + } + + for (i <- 1 until numCols + 1) { + if (!dataMap.contains(i)) { + insertPs.setNull(i, posToTypeMapping(i)) + } + } + } + + def colName(pos: Int): String = if (pos == 1) "col000" + else s"col${posToTypeMapping(pos).toString.replaceAll("-", "_")}" + + var expectedResult: mutable.Map[Any, Any] = null + + // check behaviour of byte as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 2 -> i.toByte, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + var q = s"select sum(${colName(2)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map[Any, Any]("col0" -> 10L, "col1" -> 35L) + var rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + var rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(row.getString(1)), row.getLong(0)) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of short as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 3 -> i.toShort, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(3)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> 10L, "col1" -> 35L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(row.getString(1)), row.getLong(0)) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of long as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 5 -> i.toLong, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(5)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> 10L, "col1" -> 35L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(row.getString(1)), row.getLong(0)) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + + // check behaviour of float as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 6 -> i.toFloat, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(6)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> 10.toDouble, "col1" -> 35.toDouble) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(row.getString(1)), row.getDouble(0)) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + + // check behaviour of double as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 7 -> i.toDouble, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(7)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> 10.toDouble, "col1" -> 35.toDouble) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(row.getString(1)), row.getDouble(0)) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of Big Decimal with precision < 18 as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 8 -> new java.math.BigDecimal(s"${.3 * i}"), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(8)}_1), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> new java.math.BigDecimal(s"${.3 * 10}"), + "col1" -> new java.math.BigDecimal(s"${.3 * 35}")) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertTrue( + Math.abs(expectedResult(row.getString(1)).asInstanceOf[java.math.BigDecimal]. + subtract(row.getDecimal(0)).doubleValue()) < .1) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of Big Decimal with precision > 18 as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 9 -> new java.math.BigDecimal(s"${.3 * i}"), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(9)}_2), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> new java.math.BigDecimal(s"${.3 * 10}"), + "col1" -> new java.math.BigDecimal(s"${.3 * 35}")) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertTrue( + Math.abs(expectedResult(row.getString(1)).asInstanceOf[java.math.BigDecimal]. + subtract(row.getDecimal(0)).doubleValue()) < .1) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + + // check behaviour of Timestamp as aggregate column + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 10 -> new Timestamp(1234567 * i), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + val expected1 = (for (i <- 0 until 5) yield { + DateTimeUtils.fromJavaTimestamp(new Timestamp(1234567 * i)) + }).foldLeft(0L)(_ + _) / 1000000d + + val expected2 = (for (i <- 5 until 10) yield { + DateTimeUtils.fromJavaTimestamp(new Timestamp(1234567 * i)) + }).foldLeft(0L)(_ + _) / 1000000d + + q = s"select sum(${colName(10)}), ${colName(11)} from test1 group by ${colName(11)} " + expectedResult = mutable.Map("col0" -> expected1, + "col1" -> expected2) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(2, rows.length) + rows.foreach(row => { + assertTrue(Math.abs(expectedResult( + row.getString(1)).asInstanceOf[Double] - row.getDouble(0)) < 1) + expectedResult.remove(row.getString(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + + // check behaviour of byte as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 2 -> (i / 5).toByte, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(2)} from test1 group by ${colName(2)} " + expectedResult = mutable.Map(0 -> 10L, 1 -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getByte(1)), row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getByte(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of short as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 3 -> (i / 5).toShort, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(3)} from test1 group by ${colName(3)} " + expectedResult = mutable.Map(0 -> 10L, 1 -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getShort(1)), row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getShort(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of int as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 4 -> (i / 5), 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(4)} from test1 group by ${colName(4)} " + expectedResult = mutable.Map(0 -> 10L, 1 -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getInt(1)), row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getInt(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of long as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 5 -> (i / 5).toLong, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(5)} from test1 group by ${colName(5)} " + expectedResult = mutable.Map(0 -> 10L, 1 -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getLong(1)), row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getLong(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of float as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 6 -> (i / 5) * .7F, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(6)} from test1 group by ${colName(6)} " + expectedResult = mutable.Map(0 -> 10L, .7f -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getFloat(1)), row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getFloat(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of double as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 7 -> (i / 5) * .7D, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(7)} from test1 group by ${colName(7)} " + expectedResult = mutable.Map(0 -> 10L, .7D -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getDouble(1)), + row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getDouble(1)) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of BigDecimal as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 8 -> new java.math.BigDecimal(s"${12.3E+3 + i / 5 + 1}"), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(8)}_1 from test1 group by ${colName(8)}_1 " + expectedResult = mutable.Map(new java.math.BigDecimal(s"${12.3E+3 + 1}").doubleValue() -> 10L, + new java.math.BigDecimal(s"${12.3E+3 + 2}").doubleValue() -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult(if (row.isNullAt(1)) "null" else row.getDecimal(1).doubleValue()), + row.getLong(0)) + expectedResult.remove(if (row.isNullAt(1)) "null" else row.getDecimal(1).doubleValue()) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of binary byte array as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 14 -> Array.fill[Byte](100)((i / 5).toByte), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(14)} from test1 group by ${colName(14)} " + expectedResult = mutable.Map(Array.fill[Byte](100)(0.toByte).sum -> 10L, + Array.fill[Byte](100)(1.toByte).sum -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult( + if (row.isNullAt(1)) { + "null" + } else { + row.getAs[Array[Byte]](1).sum + }), row.getLong(0)) + expectedResult.remove( + if (row.isNullAt(1)) { + "null" + } else { + row.getAs[Array[Byte]](1).sum + }) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + // check behaviour of boolean as grouping column with null & two not nulls + for (i <- 0 until 10) { + val dataMap: DataMap = Map(1 -> i, 12 -> (i / 5 == 1), + 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + for (i <- 10 until 15) { + val dataMap: DataMap = Map(1 -> i, 11 -> s"col${i / 5}") + setInInsertStatement(dataMap) + insertPs.executeUpdate() + } + q = s"select sum(${colName(1)}), ${colName(12)} from test1 group by ${colName(12)} " + expectedResult = mutable.Map(false -> 10L, + true -> 35L, "null" -> 60L) + rs = snc.sql(q) + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rows = rs.collect + assertEquals(3, rows.length) + rows.foreach(row => { + assertEquals(expectedResult( + if (row.isNullAt(1)) { + "null" + } else { + row.getBoolean(1) + }), row.getLong(0)) + expectedResult.remove( + if (row.isNullAt(1)) { + "null" + } else { + row.getBoolean(1) + }) + }) + assertTrue(expectedResult.isEmpty) + snc.sql("delete from test1") + + snc.dropTable("test1") + } + + + test("simple aggregate query with struct type as grouping key") { + snc.sql("drop table if exists test1") + val data = for (i <- 0 until 15) yield { + Row(i, Row(s"col${i / 5}", i / 5)) + } + val strucType1 = StructType(Seq(StructField("col2_1", StringType), + StructField("col2_2", IntegerType))) + val structType = StructType(Seq(StructField("col1", IntegerType), + StructField("col2", strucType1))) + val df = snc.createDataFrame(snc.sparkContext.parallelize(data), structType) + df.write.format("column").mode(SaveMode.Overwrite).saveAsTable("test1") + + def runTest(session: SnappyContext): Unit = { + val rs = session.sql("select col2, sum(col1) as summ1 from test1 group by col2") + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + val expectedResult = mutable.Map[(String, Int), Int](("col0", 0) -> 10, + ("col1", 1) -> 35, ("col2", 2) -> 60) + assertEquals(expectedResult.size, results.length) + results.foreach(row => { + val str = row.getStruct(0) + val key = (str.getString(0), str.getInt(1)) + assertEquals(expectedResult.getOrElse(key, fail("key does not exist")), row.getLong(1)) + expectedResult.remove(key) + }) + assertTrue(expectedResult.isEmpty) + } + + val newSession1 = snc.newSession() + newSession1.setConf(Property.TestExplodeComplexDataTypeInSHA.name, true.toString) + runTest(newSession1) + + val newSession2 = snc.newSession() + newSession2.setConf(Property.TestExplodeComplexDataTypeInSHA.name, false.toString) + runTest(newSession2) + snc.dropTable("test1") + + } + + test("test array type field as grouping key") { + snc.sql("drop table if exists test1") + + val structType = StructType(Seq(StructField("col1", IntegerType), + StructField("stringarray", ArrayType(StringType)), + StructField("longarray", ArrayType(LongType)))) + val data = for (i <- 0 until 15) yield { + Row(i, Array.fill[String](10)(s"col${i / 5}"), Array.fill[Long](10)(i / 5.toLong)) + } + + val data1 = for (i <- 15 until 20) yield { + Row(i, Array.tabulate[String](7)(indx => if (indx % 2 == 0) null else s"col${i / 5}"), + Array.fill[Long](10)(i / 5.toLong)) + } + + val data2 = for (i <- 20 until 25) yield { + Row(i, Array.tabulate[String](100)(indx => if (indx % 2 == 0) null else s"col${i / 5}"), + Array.fill[Long](10)(i / 5.toLong)) + } + + val df = snc.createDataFrame(snc.sparkContext.parallelize(data ++ data1 ++ data2), structType) + df.write.format("column").mode(SaveMode.Overwrite).saveAsTable("test1") + + def runTest1(session: SnappyContext): Unit = { + val rs = session.sql("select longarray, sum(col1) as summ1 from test1 group by longarray") + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + + val expectedResult1 = mutable.Map[Long, Int](0L -> 10, 1L * 10 -> 35, + 2L * 10 -> 60, 3L * 10 -> 85, 4L * 10 -> 110) + assertEquals(expectedResult1.size, results.length) + + results.foreach(row => { + val key = row.getSeq[Long](0).foldLeft(0L)(_ + _) + assertEquals(expectedResult1.getOrElse(key, fail("key does not exist")), row.getLong(1)) + expectedResult1.remove(key) + }) + assertTrue(expectedResult1.isEmpty) + } + + val newSession1 = snc.newSession() + newSession1.setConf(Property.TestExplodeComplexDataTypeInSHA.name, true.toString) + runTest1(newSession1) + + val newSession2 = snc.newSession() + newSession2.setConf(Property.TestExplodeComplexDataTypeInSHA.name, false.toString) + runTest1(newSession2) + + + def runTest2(session: SnappyContext): Unit = { + val rs = session.sql("select stringarray, longarray," + + " sum(col1) as summ1 from test1 group by stringarray, longarray") + + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + val expectedResult2 = mutable.Map[(String, Long), Long]( + (Array.fill[String](10)("col0").mkString(","), 10 * 0L) -> 10L, + (Array.fill[String](10)("col1").mkString(","), 10 * 1L) -> 35L, + (Array.fill[String](10)("col2").mkString(","), 10 * 2L) -> 60L, + (Array.tabulate[String](7)(indx => if (indx % 2 == 0) "null" else "col3"). + mkString(","), 10 * 3L) -> 85L, + (Array.tabulate[String](100)(indx => if (indx % 2 == 0) "null" else "col4"). + mkString(","), 10 * 4L) -> 110L + ) + + assertEquals(expectedResult2.size, results.length) + results.foreach(row => { + val key = (row.getSeq[String](0).map(x => if (x == null) "null" else x).mkString(","), + row.getSeq[Long](1).foldLeft(0L)(_ + _)) + assertEquals(expectedResult2.getOrElse(key, fail("key does not exist")), row.getLong(2)) + expectedResult2.remove(key) + }) + + assertTrue(expectedResult2.isEmpty) + } + + runTest2(newSession1) + runTest2(newSession2) + + + snc.dropTable("test1") + } + + test("aggregate query with nested struct type as grouping key") { + snc.sql("drop table if exists test1") + val structType2 = StructType(Seq(StructField("wife", StringType), + StructField("numOffsprings", IntegerType))) + val strucType1 = StructType(Seq(StructField("name", StringType), + StructField("zip", IntegerType), StructField("family", structType2))) + val structType = StructType(Seq(StructField("id", IntegerType), + StructField("details", strucType1))) + + val data = for (i <- 0 until 15) yield { + val num = i / 5 + Row(i, Row(s"name$num", num, Row(s"spouse$num", num))) + } + + val df = snc.createDataFrame(snc.sparkContext.parallelize(data), structType) + df.write.format("column").mode(SaveMode.Overwrite).saveAsTable("test1") + + def runTest(session: SnappyContext): Unit = { + val rs = session.sql("select details, sum(id) as summ1 from test1 group by details") + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + val expectedResult = mutable.Map[(String, Int, (String, Int)), Int]( + ("name0", 0, ("spouse0", 0)) -> 10, + ("name1", 1, ("spouse1", 1)) -> 35, ("name2", 2, ("spouse2", 2)) -> 60) + assertEquals(expectedResult.size, results.length) + results.foreach(row => { + val str1 = row.getStruct(0) + val str2 = str1.getStruct(2) + val key = (str1.getString(0), str1.getInt(1), (str2.getString(0), str2.getInt(1))) + assertEquals(expectedResult.getOrElse(key, fail("key does not exist")), row.getLong(1)) + expectedResult.remove(key) + }) + assertTrue(expectedResult.isEmpty) + } + + val newSession1 = snc.newSession() + newSession1.setConf(Property.TestExplodeComplexDataTypeInSHA.name, true.toString) + runTest(newSession1) + + val newSession2 = snc.newSession() + newSession2.setConf(Property.TestExplodeComplexDataTypeInSHA.name, false.toString) + runTest(newSession2) + snc.dropTable("test1") + } + + test("grouping columns and aggregate with expressions") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 int, col5 string) " + + "using column ") + val range = 50 + val groupingDivisor = 10 + val insertDF = snc.range(50).selectExpr("id", "id * 2", "id * 3", + s"id % $groupingDivisor", s"concat('col_',id % $groupingDivisor)") + + insertDF.write.insertInto("test1") + val rs = snc.sql(s"select " + + s"concat(Cast((cast(col2 / 2 as int)) % $groupingDivisor as string), col5) as x," + + s" col5, sum(col1) as summ1 from test1 group by x, col5") + val results = rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor, results.length) + rs.foreach(row => { + val x = row.getString(1) + val groupKey = x.substring("col_".length).toInt + val n = range / groupingDivisor + val sum1 = ((n / 2.0f) * ((2 * groupKey) + (n - 1) * groupingDivisor)).toLong + val sumAgg1 = row.getLong(2) + val firstCol = s"${groupKey}col_$groupKey" + assertEquals(firstCol, row.getString(0)) + assertEquals(sum1, sumAgg1) + }) + snc.dropTable("test1") + } + + + test("test large data with rehash of ByteBufferHashMap") { + val hfileTaxi: String = getClass.getResource("/trip_fare_ParquetEmptyData").getPath + // TODO: Use following with full data + // val hfile: String = "../../data/NYC_trip_ParquetData" + // val hfile1: String = "../../data/trip_fare_ParquetData" + + val taxiFare = "taxifare" + val tripFareDF = snc.read.parquet(hfileTaxi) + + tripFareDF.registerTempTable(taxiFare) + tripFareDF.cache() + + var rs = snc.sql(s"SELECT hack_license, sum(fare_amount) as daily_fare_amount" + + s" FROM $taxiFare group by hack_license") + rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + + rs = snc.sql(s"SELECT hack_license, to_date(pickup_datetime) as pickup_date," + + s" sum(fare_amount) as daily_fare_amount FROM $taxiFare " + + s"group by hack_license, pickup_date") + rs.collect() + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + snc.dropTable(taxiFare, true) + tripFareDF.unpersist() + } + + test("bug negative size of string grouping key") { + val hfile: String = getClass.getResource("/2015.parquet").getPath + val snContext = snc + snContext.sql("set spark.sql.shuffle.partitions=6") + val airlineTable = "airline" + val airlineDF = snContext.read.load(hfile) + airlineDF.registerTempTable(airlineTable) + val rs = snc.sql(s"select dest from $airlineTable " + + s" group by dest having count(*) > 1000000") + + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + rs.collect() + snc.dropTable(airlineTable, true) + } + + test("group by query without having aggregate functions") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 int, col3 int, col4 String) " + + "using column ") + val range = 50 + val groupingDivisor = 10 + val insertDF = snc.range(50).selectExpr("id", "id*2", "id * 3", + s"Concat( 'test', Cast(id % $groupingDivisor as string) ) ") + insertDF.write.insertInto("test1") + val rs = snc.sql("select distinct col4 from test1 ") + // import org.apache.spark.sql.execution.debug._ + // rs.debugCodegen() + val results = rs.collect() + val expectedResults = Array.tabulate[String](groupingDivisor)(i => s"test$i").toBuffer + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + assertEquals(groupingDivisor, results.length) + results.foreach(row => { + assertTrue(expectedResults.exists(_ == row.getString(0))) + expectedResults.remove(expectedResults.indexOf(row.getString(0))) + }) + assertTrue(expectedResults.isEmpty) + snc.dropTable("test1") + } + + test("GITHUB-534") { + val session = SnappyContext(sc).snappySession + session.sql("CREATE TABLE yes_with(device_id VARCHAR(200), " + + "sdk_version VARCHAR(200)) USING COLUMN OPTIONS(PARTITION_BY 'device_id')") + session.insert("yes_with", Row("id1", "v1"), Row("id1", "v2"), + Row("id2", "v1"), Row("id2", "v1"), Row("id2", "v3")) + val rs = session.sql("select sdk_version, count(distinct device_id) from (" + + "select sdk_version,device_id from YES_WITH group by sdk_version, " + + "device_id) a group by sdk_version") + // assertEquals(2, getNumCodeGenTrees(rs.queryExecution.executedPlan)) + + ColumnCacheBenchmark.collect(rs, + Seq(Row("v1", 2), Row("v2", 1), Row("v3", 1))) + snc.dropTable("yes_with") + } + + test("BigDecimal bug SNAP-3047") { + val rand = new Random(System.currentTimeMillis()) + + def getPrice(): BigDecimal = new BigDecimal(java.lang.Double.toString( + (rand.nextInt(10000) + 1) * .01)) + + snc + snc.sql("drop table if exists securities") + snc.sql("create table securities(sec_id int not null, symbol varchar(10) not null, " + + "price decimal (30, 20))") + val conn = getSqlConnection + val insertPs = conn.prepareStatement("insert into securities values (?, ?, ?)") + for (i <- 0 until 5000) { + insertPs.setInt(1, i) + insertPs.setString(2, s"symbol_${i % 10}") + insertPs.setBigDecimal(3, getPrice()) + insertPs.addBatch() + } + insertPs.executeBatch() + + val snc2 = snc.newSession() + snc2.setConf("snappydata.sql.optimizedHashAggregate", "false") + + val query1 = "select cast(avg( distinct price) as decimal (30, 20)) as " + + "avg_distinct_price from securities " + + var rs1 = snc.sql(query1).collect()(0) + var rs2 = snc2.sql(query1).collect()(0) + assertTrue(rs1.getDecimal(0).equals(rs2.getDecimal(0))) + + val query2 = "select count( distinct price) from securities " + rs1 = snc.sql(query2).collect()(0) + rs2 = snc2.sql(query2).collect()(0) + assertEquals(rs1.getLong(0), rs2.getLong(0)) + snc.dropTable("securities") + } + + test("SNAP-3077 test list of overflow BBMaps if capacity is reached") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 binary) using column ") + val range: Byte = 100 + val byteArraySize = 512 + val numRowDuplication = 10 + val conn = getSqlConnection + val insertPs = conn.prepareStatement("insert into test1 values (?,?)") + for( i <- 1 to range) { + val bytes = Array.ofDim[Byte](byteArraySize) + bytes(i) = i.toByte + for (j <- 0 until numRowDuplication) { + insertPs.setInt(1, i) + insertPs.setBytes(2, bytes) + insertPs.addBatch() + } + } + insertPs.executeBatch() + val snc1 = snc.newSession() + snc1.setConf("snappydata.sql.approxMaxCapacityOfBBMap", "4096") + snc1.setConf("snappydata.sql.initialCapacityOfSHABBMap", "4") + val rs = snc1.sql("select col2, sum(col1) from test1 group by col2").collect + assertEquals(100, rs.length) + rs.foreach(row => { + val byteArr = row.getAs[Array[Byte]](0) + val sum = row.getLong(1) + val key = byteArr.find(_ > 0).get + assertEquals(10 * key, sum) + }) + snc.dropTable("test1") + } + + test("SNAP-3132") { + snc + snc.sql("drop table if exists test1") + snc.sql("drop table if exists test2") + snc.sql("drop table if exists test3") + snc.sql("create table test1 (col1 int, col2 int, col3 Decimal(7,2)) using column") + + val conn = getSqlConnection + val ps1 = conn.prepareStatement("insert into test1 values (?,?, ?)") + for(i <- 0 until 500) { + ps1.setInt(1, i % 5) + ps1.setInt(2, i % 10) + val bd = new BigDecimal(174.576d * i) + ps1.setBigDecimal(3, bd) + ps1.addBatch + } + + ps1.executeBatch + + + snc.sql(s" select 'asif' as name, col1, col2, Cast(sum(col3) as Decimal(15,4)) from test1" + + s" group by name, col1, col2 ").collect + snc.dropTable("test1") + + } + + ignore("SNAP-3077 test if default max capacity nearing Integer.MAX_VALUE is reached." + + " Disabled due to heap requirements being more than 16G") { + snc + snc.sql("drop table if exists test1") + snc.sql("create table test1 (col1 int, col2 binary) using column ") + + val conn = getSqlConnection + val insertPs = conn.prepareStatement("insert into test1 values (?,?)") + + + for (j <- 0 until 8) { + insertPs.setInt(1, j) + val bytes = Array.ofDim[Byte](Integer.MAX_VALUE/4 - 8) + bytes(j) = 1 + insertPs.setBytes(2, bytes) + insertPs.addBatch() + } + insertPs.executeBatch() + val snc1 = snc.newSession() + snc1.setConf("snappydata.sql.initialCapacityOfSHABBMap", "4") + val rs = snc1.sql("select col2, sum(col1) from test1 group by col2").collect + assertEquals(8, rs.length) + snc.dropTable("test1") + } + + + def getSqlConnection: Connection = + DriverManager.getConnection(s"jdbc:snappydata://$serverHostPort2") + + def getNumCodeGenTrees(plan: SparkPlan): Int = { + var numcodegenSubtrees = 0 + plan transform { + case s: WholeStageCodegenExec => + numcodegenSubtrees += 1 + s + case s => s + } + numcodegenSubtrees + } + +} diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index cb681d032f..00fe5dfe4d 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -402,6 +402,7 @@ trait SplitClusterDUnitTestObject extends Logging { getEnvironmentVariable("SNAPPY_DIST_CLASSPATH")) .set("snappydata.connection", connectionURL) .set("snapptdata.sql.planCaching", random.nextBoolean().toString) + .set(io.snappydata.Property.TestDisableCodeGenFlag.name, "false") if (props != null) { val user = props.getProperty(Attribute.USERNAME_ATTR, "") diff --git a/core/src/main/scala/io/snappydata/Literals.scala b/core/src/main/scala/io/snappydata/Literals.scala index 0c5431cb78..8ee24c3dee 100644 --- a/core/src/main/scala/io/snappydata/Literals.scala +++ b/core/src/main/scala/io/snappydata/Literals.scala @@ -18,6 +18,8 @@ package io.snappydata import scala.reflect.ClassTag +import com.gemstone.gemfire.internal.shared.unsafe.DirectBufferAllocator + import org.apache.spark.sql.execution.columnar.ExternalStoreUtils import org.apache.spark.sql.internal.{AltName, SQLAltName, SQLConfigEntry} @@ -262,6 +264,37 @@ object Property extends Enumeration { "The putInto inner join would be cached if the result of " + "join with incoming Dataset is of size less " + "than PutIntoInnerJoinCacheSize. Default value is 100 MB.", Some("100m")) + + val TestExplodeComplexDataTypeInSHA: SQLValue[Boolean] = SQLVal[Boolean]( + s"${Constant.PROPERTY_PREFIX}sql.explodeStructInSHA", + "Explodes the Struct or Array Field in Group By Keys even if the struct object is " + + "UnsafeRow or UnsafeArrayData", Some(false)) + + val UseOptimzedHashAggregate: SQLValue[Boolean] = SQLVal[Boolean]( + s"${Constant.PROPERTY_PREFIX}sql.optimizedHashAggregate", + "Enables the use of ByteBufferMap based SnappyHashAggregateExec", + Some(true)) + + val UseOptimizedHashAggregateForSingleKey: SQLValue[Boolean] = SQLVal[Boolean]( + s"${Constant.PROPERTY_PREFIX}sql.useOptimizedHashAggregateForSingleKey", + "use ByteBufferMap based SnappyHashAggregateExec even for single string group by", + Some(false)) + + val ApproxMaxCapacityOfBBMap: SQLValue[Int] = SQLVal[Int]( + s"${Constant.PROPERTY_PREFIX}sql.approxMaxCapacityOfBBMap", + s"The max capacity of value byte array in ByteBufferHashMap. " + + s"Default value is ${Integer.MAX_VALUE}", + Some(((Integer.MAX_VALUE -DirectBufferAllocator.DIRECT_OBJECT_OVERHEAD - 7) >>> 3) << 3)) + + val initialCapacityOfSHABBMap: SQLValue[Int] = SQLVal[Int]( + s"${Constant.PROPERTY_PREFIX}sql.initialCapacityOfSHABBMap", + s"The initial capacity of SHAMap. " + + s"Default value is 8192", Some(8192)) + + val TestDisableCodeGenFlag: SQLValue[Boolean] = SQLVal[Boolean]( + s"${Constant.PROPERTY_PREFIX}sql.disableCodegenFallback", + s"The test flag if set to true will throw Exception instead of creating CodegenSparkFallback " + + s"Default value is false", Some(false)) } // extractors for properties diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala index 886ccacade..51d4b75a74 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala @@ -798,7 +798,11 @@ case class InsertCachedPlanFallback(session: SnappySession, topLevel: Boolean) else plan match { // TODO: disabled for StreamPlans due to issues but can it require fallback? case _: StreamPlan => plan - case _ => CodegenSparkFallback(plan, session) + case _: CollectAggregateExec => CodegenSparkFallback(plan, session) + case _ if !Property.TestDisableCodeGenFlag.get(session.sessionState.conf) || + session.sessionState.conf.contains("snappydata.connection") => + CodegenSparkFallback(plan, session) + case _ => plan } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala new file mode 100644 index 0000000000..1d27c4ce57 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala @@ -0,0 +1,1353 @@ +/* + * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.sql.execution + +import java.nio.ByteBuffer + +import scala.reflect.runtime.universe._ + +import com.gemstone.gemfire.internal.shared.{BufferSizeLimitExceededException, ClientResolverUtils} +import io.snappydata.Property +import io.snappydata.collection.{ByteBufferData, SHAMap} + + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SnappySession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, UnsafeArrayData, UnsafeRow} +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.types.UTF8String + +case class SHAMapAccessor(@transient session: SnappySession, + @transient ctx: CodegenContext, @transient keyExprs: Seq[Expression], + @transient valueExprs: Seq[Expression], classPrefix: String, + hashMapTerm: String, overflowHashMapsTerm: String, keyValSize: Int, + valueOffsetTerm: String, numKeyBytesTerm: String, numValueBytes: Int, + currentOffSetForMapLookupUpdt: String, valueDataTerm: String, + vdBaseObjectTerm: String, vdBaseOffsetTerm: String, + nullKeysBitsetTerm: String, numBytesForNullKeyBits: Int, + allocatorTerm: String, numBytesForNullAggBits: Int, + nullAggsBitsetTerm: String, sizeAndNumNotNullFuncForStringArr: String, + keyBytesHolderVarTerm: String, baseKeyObject: String, + baseKeyHolderOffset: String, keyExistedTerm: String, + skipLenForAttribIndex: Int, codeForLenOfSkippedTerm: String, + valueDataCapacityTerm: String, storedAggNullBitsTerm: Option[String], + storedKeyNullBitsTerm: Option[String], + aggregateBufferVars: Seq[String], keyHolderCapacityTerm: String) + extends CodegenSupport { + + private val alwaysExplode = Property.TestExplodeComplexDataTypeInSHA. + get(session.sessionState.conf) + private[this] val hashingClass = classOf[ClientResolverUtils].getName + + override def children: Seq[SparkPlan] = Nil + override def output: Seq[Attribute] = Nil + + override protected def doExecute(): RDD[InternalRow] = + throw new UnsupportedOperationException("unexpected invocation") + + override def inputRDDs(): Seq[RDD[InternalRow]] = Nil + + override protected def doProduce(ctx: CodegenContext): String = + throw new UnsupportedOperationException("unexpected invocation") + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], + row: ExprCode): String = { + throw new UnsupportedOperationException("unexpected invocation") + } + + def getBufferVars(dataTypes: Seq[DataType], varNames: Seq[String], + currentValueOffsetTerm: String, isKey: Boolean, nullBitTerm: String, + numBytesForNullBits: Int, skipNullBitsCode: Boolean, nestingLevel: Int = 0): + Seq[ExprCode] = { + val plaformClass = classOf[Platform].getName + val decimalClass = classOf[Decimal].getName + val bigDecimalObjectClass = s"$decimalClass$$.MODULE$$" + val typeUtiltiesObjectClass = + s"${org.apache.spark.sql.types.TypeUtilities.getClass.getName}.MODULE$$" + val bigDecimalClass = classOf[java.math.BigDecimal].getName + val bigIntegerClass = classOf[java.math.BigInteger].getName + val byteBufferClass = classOf[ByteBuffer].getName + val unsafeClass = classOf[UnsafeRow].getName + val castTerm = SHAMapAccessor.getNullBitsCastTerm(numBytesForNullBits) + dataTypes.zip(varNames).zipWithIndex.map { case ((dt, varName), i) => + val nullVar = if (isKey) { + if (nestingLevel == 0 && skipNullBitsCode) { + "false" + } else { + ctx.freshName("isNull") + } + } else s"$varName${SHAMapAccessor.nullVarSuffix}" + // if it is aggregate value buffer do not declare null var as + // they are already declared at start + // also aggregate vars cannot be nested in any case. + val booleanStr = if (isKey) "boolean" else "" + val nullVarCode = if (skipNullBitsCode) { + "" + } else { + s"""$booleanStr $nullVar = ${SHAMapAccessor.getExpressionForNullEvalFromMask(i, + numBytesForNullBits, nullBitTerm)};""".stripMargin + } + + val evaluationCode = (dt match { + /* case StringType => + val holder = ctx.freshName("holder") + val holderBaseObject = ctx.freshName("holderBaseObject") + val holderBaseOffset = ctx.freshName("holderBaseOffset") + val len = ctx.freshName("len") + val readLenCode = if (nestingLevel == 0 && i == skipLenForAttribIndex) { + s"int $len = $codeForLenOfSkippedTerm" + } else { + s"""int $len = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4; + """ + } + s"""$readLenCode + |$byteBufferClass $holder = $allocatorTerm.allocate($len, "SHA"); + | Object $holderBaseObject = $allocatorTerm.baseObject($holder); + | long $holderBaseOffset = $allocatorTerm.baseOffset($holder); + | $plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + | $holderBaseObject, $holderBaseOffset , $len); + | $varName = ${classOf[UTF8String].getName}.fromAddress($holderBaseObject, + | $holderBaseOffset, $len); + |$currentValueOffsetTerm += $len; + """.stripMargin + */ + case StringType => + val len = ctx.freshName("len") + val readLenCode = if (nestingLevel == 0 && i == skipLenForAttribIndex) { + s"int $len = $codeForLenOfSkippedTerm" + } else { + s"""int $len = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4; + """ + } + s"""$readLenCode + | $varName = ${classOf[UTF8String].getName}.fromAddress($vdBaseObjectTerm, + | $currentValueOffsetTerm, $len); + |$currentValueOffsetTerm += $len; + """.stripMargin + case BinaryType => + s"""$varName = new byte[$plaformClass.getInt($vdBaseObjectTerm, + | $currentValueOffsetTerm)]; + |$currentValueOffsetTerm += 4; + |$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + | $varName, ${Platform.BYTE_ARRAY_OFFSET}, $varName.length); + | $currentValueOffsetTerm += $varName.length; + """.stripMargin + case x: AtomicType => { + (typeOf(x.tag) match { + case t if t =:= typeOf[Boolean] => + s"""$varName = $plaformClass.getBoolean($vdBaseObjectTerm, $currentValueOffsetTerm); + """ + case t if t =:= typeOf[Byte] => + s"""$varName = $plaformClass.getByte($vdBaseObjectTerm, $currentValueOffsetTerm); + """ + case t if t =:= typeOf[Short] => + s"""$varName = $plaformClass.getShort($vdBaseObjectTerm, $currentValueOffsetTerm); + """.stripMargin + case t if t =:= typeOf[Int] => + s"""$varName = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + """.stripMargin + case t if t =:= typeOf[Long] => + s"""$varName = $plaformClass.getLong($vdBaseObjectTerm,$currentValueOffsetTerm); + """.stripMargin + case t if t =:= typeOf[Float] => + s"""$varName = $plaformClass.getFloat($vdBaseObjectTerm, $currentValueOffsetTerm); + """.stripMargin + case t if t =:= typeOf[Double] => + s"""$varName = $plaformClass.getDouble($vdBaseObjectTerm, $currentValueOffsetTerm); + """.stripMargin + case t if t =:= typeOf[Decimal] => + if (dt.asInstanceOf[DecimalType].precision <= Decimal.MAX_LONG_DIGITS) { + s"""$varName = $bigDecimalObjectClass.apply( + |$plaformClass.getLong($vdBaseObjectTerm, $currentValueOffsetTerm), + ${dt.asInstanceOf[DecimalType].precision}, + ${dt.asInstanceOf[DecimalType].scale});""".stripMargin + } else { + val tempByteArrayTerm = ctx.freshName("tempByteArray") + + val len = ctx.freshName("len") + s""" + |byte[] $tempByteArrayTerm = new byte[${dt.asInstanceOf[DecimalType]. + defaultSize}]; + |$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + |$tempByteArrayTerm, ${Platform.BYTE_ARRAY_OFFSET} , $tempByteArrayTerm.length); + |$varName = $bigDecimalObjectClass.apply(new $bigDecimalClass( + |new $bigIntegerClass($tempByteArrayTerm), + |${dt.asInstanceOf[DecimalType].scale}, + | $typeUtiltiesObjectClass.mathContextCache()[${dt.asInstanceOf[DecimalType].precision - 1}])); + """.stripMargin + } + + case _ => throw new UnsupportedOperationException("unknown type " + dt) + }) + + s"""$currentValueOffsetTerm += ${dt.defaultSize};""" + } + case ArrayType(elementType, containsNull) => + val isExploded = ctx.freshName("isExplodedArray") + val arraySize = ctx.freshName("arraySize") + val holder = ctx.freshName("holder") + val byteBufferClass = classOf[ByteBuffer].getName + val unsafeArrayDataClass = classOf[UnsafeArrayData].getName + val genericArrayDataClass = classOf[GenericArrayData].getName + val objectArray = ctx.freshName("objArray") + val objectClass = classOf[Object].getName + val counter = ctx.freshName("counter") + val readingCodeExprs = getBufferVars(Seq(elementType), Seq(s"$objectArray[$counter]"), + currentValueOffsetTerm, true, "", -1, + true, nestingLevel) + val varWidthNumNullBytes = ctx.freshName("numNullBytes") + val varWidthNullBits = ctx.freshName("nullBits") + val remainder = ctx.freshName("remainder") + val indx = ctx.freshName("indx") + + s"""boolean $isExploded = $plaformClass.getBoolean($vdBaseObjectTerm, + |$currentValueOffsetTerm); + |++$currentValueOffsetTerm; + |if ($isExploded) { + |int $arraySize = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4; + |$objectClass[] $objectArray = new $objectClass[$arraySize]; + |if ($containsNull) { + |int $varWidthNumNullBytes = $arraySize/8 + ($arraySize % 8 > 0 ? 1 : 0); + |byte[] $varWidthNullBits = new byte[$varWidthNumNullBytes]; + |$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + | $varWidthNullBits, ${Platform.BYTE_ARRAY_OFFSET}, $varWidthNumNullBytes); + |$currentValueOffsetTerm += $varWidthNumNullBytes; + |for (int $counter = 0; $counter < $arraySize; ++$counter ) { + |int $remainder = $counter % 8; + |int $indx = $counter / 8; + |if ( ($varWidthNullBits[$indx] & (0x01 << $remainder)) == 0) { + |${readingCodeExprs.map(_.code).mkString("\n")} + |} + |} + |} else { + |for (int $counter = 0; $counter < $arraySize; ++$counter ) { + |${readingCodeExprs.map(_.code).mkString("\n")} + |} + |} + + $varName = new $genericArrayDataClass($objectArray); + |} else { + |int $arraySize = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4; + |$byteBufferClass $holder = $allocatorTerm.allocate($arraySize, "SHA"); + |$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + |$allocatorTerm.baseObject($holder), $allocatorTerm.baseOffset($holder),$arraySize); + |$currentValueOffsetTerm += $arraySize; + |$varName = new $unsafeArrayDataClass(); + |(($unsafeArrayDataClass)$varName).pointTo($allocatorTerm.baseObject($holder), + |$allocatorTerm.baseOffset($holder), $arraySize); + |}""".stripMargin + case st: StructType => + val objectArray = ctx.freshName("objectArray") + val byteBufferClass = classOf[ByteBuffer].getName + val currentOffset = ctx.freshName("currentOffset") + val nullBitSetTermForStruct = SHAMapAccessor.generateNullKeysBitTermForStruct( + varName) + val numNullKeyBytesForStruct = SHAMapAccessor.calculateNumberOfBytesForNullBits(st.length) + val genericInternalRowClass = classOf[GenericInternalRow].getName + val internalRowClass = classOf[InternalRow].getName + val objectClass = classOf[Object].getName + val keyVarNamesWithStructFlags = st.zipWithIndex.map { case (sf, indx) => + sf.dataType match { + case _: StructType => SHAMapAccessor.generateVarNameForStructField(varName, + nestingLevel, indx) -> true + case _ => s"$objectArray[$indx]" -> false + } + } + + val isExploded = ctx.freshName("isUnsafeRow") + val unsafeRowLength = ctx.freshName("unsafeRowLength") + val holder = ctx.freshName("holder") + // ${SHAMapAccessor.initNullBitsetCode(newNullBitSetTerm, newNumNullKeyBytes)} + s"""boolean $isExploded = $plaformClass.getBoolean($vdBaseObjectTerm, + |$currentValueOffsetTerm); + |++$currentValueOffsetTerm; + |if ($isExploded) { + |${ + readNullBitsCode(currentValueOffsetTerm, nullBitSetTermForStruct, + numNullKeyBytesForStruct) + } + |$objectClass[] $objectArray = new $objectClass[${st.length}]; + |$varName = new $genericInternalRowClass($objectArray); + // declare child struct variables + |${ + keyVarNamesWithStructFlags.filter(_._2).map { + case (name, _) => s"$internalRowClass $name = null;" + }.mkString("\n") + } + ${ + getBufferVars(st.map(_.dataType), keyVarNamesWithStructFlags.unzip._1, + currentValueOffsetTerm, true, nullBitSetTermForStruct, + numNullKeyBytesForStruct, false, nestingLevel + 1). + map(_.code).mkString("\n") + } + //add child Internal Rows to parent struct's object array + ${ + keyVarNamesWithStructFlags.zipWithIndex.map { case ((name, isStruct), indx) => + if (isStruct) { + s"$objectArray[$indx] = $name;" + } else { + "" + } + }.mkString("\n") + } + } + |else { + |int $unsafeRowLength = $plaformClass.getInt($vdBaseObjectTerm, + | $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4; + |$byteBufferClass $holder = $allocatorTerm.allocate($unsafeRowLength, "SHA"); + |$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, + |$allocatorTerm.baseObject($holder), $allocatorTerm.baseOffset($holder), + |$unsafeRowLength); + |$currentValueOffsetTerm += $unsafeRowLength; + |$varName = new $unsafeClass(${st.length}); + |(($unsafeClass)$varName).pointTo($allocatorTerm.baseObject($holder), + | $allocatorTerm.baseOffset($holder), $unsafeRowLength); + |} """.stripMargin + }).trim + + val exprCode = if (skipNullBitsCode) { + evaluationCode + } else { + s"""$nullVarCode + if (!$nullVar) { + $evaluationCode + }${ + if (!isKey) { + s""" + else { + ${SHAMapAccessor.getOffsetIncrementCodeForNullAgg(currentValueOffsetTerm, dt)} + } """.stripMargin + } else { + "" + } + }""".stripMargin + } + ExprCode(exprCode, nullVar, varName) + } + } + + def readNullBitsCode(currentValueOffsetTerm: String, nullBitsetTerm: String, + numBytesForNullBits: Int): String = { + val plaformClass = classOf[Platform].getName + if (numBytesForNullBits == 0) { + "" + } else if (numBytesForNullBits == 1) { + s"""$nullBitsetTerm = $plaformClass.getByte($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 1;""".stripMargin + } else if (numBytesForNullBits == 2) { + s"""$nullBitsetTerm = $plaformClass.getShort($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 2;""".stripMargin + } else if (numBytesForNullBits <= 4) { + s"""|$nullBitsetTerm = $plaformClass.getInt($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 4;""".stripMargin + } else if (numBytesForNullBits <= 8) { + s"""$nullBitsetTerm = $plaformClass.getLong($vdBaseObjectTerm, $currentValueOffsetTerm); + |$currentValueOffsetTerm += 8;""".stripMargin + } else { + s"""$plaformClass.copyMemory($vdBaseObjectTerm, $currentValueOffsetTerm, $nullBitsetTerm, + |${Platform.BYTE_ARRAY_OFFSET}, $numBytesForNullBits); + |$currentValueOffsetTerm += $numBytesForNullBits;""".stripMargin + } + } + + + def initKeyOrBufferVal(dataTypes: Seq[DataType], varNames: Seq[String]): + String = dataTypes.zip(varNames).map { case (dt, varName) => + s"${ctx.javaType(dt)} $varName = ${ctx.defaultValue(dt)};" + }.mkString("\n") + + def declareNullVarsForAggBuffer(varNames: Seq[String]): String = + varNames.map(varName => s"boolean ${varName}${SHAMapAccessor.nullVarSuffix};").mkString("\n") + /** + * Generate code to lookup the map or insert a new key, value if not found. + */ + def generateMapGetOrInsert(valueInitVars: Seq[ExprCode], + valueInitCode: String, input: Seq[ExprCode], keyVars: Seq[ExprCode], + keysDataType: Seq[DataType], aggregateDataTypes: Seq[DataType]): String = { + val hashVar = Array(ctx.freshName("hash")) + val tempValueData = ctx.freshName("tempValueData") + val linkedListClass = classOf[java.util.LinkedList[SHAMap]].getName + val exceptionName = classOf[BufferSizeLimitExceededException].getName + val bbDataClass = classOf[ByteBufferData].getName + val shaMapClassName = classOf[SHAMap].getName + // val valueInit = valueInitCode + '\n' + val insertDoneTerm = ctx.freshName("insertDone"); + /* generateUpdate(objVar, Nil, + valueInitVars, forKey = false, doCopy = false) */ + val inputEvals = evaluateVariables(input) + + s"""|$valueInitCode + |${SHAMapAccessor.resetNullBitsetCode(nullKeysBitsetTerm, numBytesForNullKeyBits)} + |${SHAMapAccessor.resetNullBitsetCode(nullAggsBitsetTerm, numBytesForNullAggBits)} + // evaluate input row vars + |$inputEvals + // evaluate key vars + |${evaluateVariables(keyVars)} + |${keyVars.zip(keysDataType).filter(_._2 match { + case x: StructType => true + case _ => false + }).map { + case (exprCode, dt) => explodeStruct(exprCode.value, exprCode.isNull, + dt.asInstanceOf[StructType]) + }.mkString("\n") + } + // evaluate hash code of the lookup key + |${generateHashCode(hashVar, keyVars, this.keyExprs, keysDataType)} + |// System.out.println("hash code for key = " +${hashVar(0)}); + |// get key size code + |$numKeyBytesTerm = ${generateKeySizeCode(keyVars, keysDataType, numBytesForNullKeyBits)}; + + |// prepare the key + |${generateKeyBytesHolderCode(numKeyBytesTerm, numValueBytes, + keyVars, keysDataType, aggregateDataTypes, valueInitVars) + } + |long $valueOffsetTerm = 0; + // insert or lookup + |boolean $keyExistedTerm = false; + |if($overflowHashMapsTerm == null) { + |try { + |$valueOffsetTerm = $hashMapTerm.putBufferIfAbsent($baseKeyObject, + |$baseKeyHolderOffset, $numKeyBytesTerm, $numValueBytes + $numKeyBytesTerm, + |${hashVar(0)}); + |$keyExistedTerm = $valueOffsetTerm >= 0; + |if (!$keyExistedTerm) { + |$valueOffsetTerm = -1 * $valueOffsetTerm; + |if (($valueOffsetTerm + $numValueBytes + $numKeyBytesTerm) >= + |$valueDataCapacityTerm) { + |//$valueDataTerm = $tempValueData; + |$valueDataTerm = $hashMapTerm.getValueData(); + |$vdBaseObjectTerm = $valueDataTerm.baseObject(); + |$vdBaseOffsetTerm = $valueDataTerm.baseOffset(); + |$valueDataCapacityTerm = $valueDataTerm.capacity(); + |} + |} + |} catch ($exceptionName bsle) { + |$overflowHashMapsTerm = new $linkedListClass<$shaMapClassName>(); + |$overflowHashMapsTerm.add($hashMapTerm); + |$hashMapTerm = new $shaMapClassName(${Property.initialCapacityOfSHABBMap.get(session.sessionState.conf)}, + |$keyValSize, + |${Property.ApproxMaxCapacityOfBBMap.get(session.sessionState.conf)}); + |$overflowHashMapsTerm.add($hashMapTerm); + |$valueOffsetTerm = $hashMapTerm.putBufferIfAbsent($baseKeyObject, + |$baseKeyHolderOffset, $numKeyBytesTerm, $numValueBytes + $numKeyBytesTerm, + |${hashVar(0)}); + |$valueOffsetTerm = -1 * $valueOffsetTerm; + |$valueDataTerm = $hashMapTerm.getValueData(); + |$vdBaseObjectTerm = $valueDataTerm.baseObject(); + |$vdBaseOffsetTerm = $valueDataTerm.baseOffset(); + |$keyExistedTerm = false; + |} + |} else { + |boolean $insertDoneTerm = false; + |for($shaMapClassName shaMap : $overflowHashMapsTerm ) { + |try { + |$valueOffsetTerm = shaMap.putBufferIfAbsent($baseKeyObject, + |$baseKeyHolderOffset, $numKeyBytesTerm, $numValueBytes + $numKeyBytesTerm, + |${hashVar(0)}); + |$keyExistedTerm = $valueOffsetTerm >= 0; + |if (!$keyExistedTerm) { + |$valueOffsetTerm = -1 * $valueOffsetTerm; + |} + |$hashMapTerm = shaMap; + |$valueDataTerm = $hashMapTerm.getValueData(); + |$vdBaseObjectTerm = $valueDataTerm.baseObject(); + |$vdBaseOffsetTerm = $valueDataTerm.baseOffset(); + |$insertDoneTerm = true; + |break; + |} catch ($exceptionName bsle) { + |//ignore + |} + |} + |if (!$insertDoneTerm) { + |$hashMapTerm = new $shaMapClassName(${Property.initialCapacityOfSHABBMap.get(session.sessionState.conf)}, + | $keyValSize, + | ${Property.ApproxMaxCapacityOfBBMap.get(session.sessionState.conf)}); + |$overflowHashMapsTerm.add($hashMapTerm); + |$valueOffsetTerm = $hashMapTerm.putBufferIfAbsent($baseKeyObject, + |$baseKeyHolderOffset, $numKeyBytesTerm, $numValueBytes + $numKeyBytesTerm, + |${hashVar(0)}); + |$valueOffsetTerm = -1 * $valueOffsetTerm; + |$keyExistedTerm = false; + |$valueDataTerm = $hashMapTerm.getValueData(); + |$vdBaseObjectTerm = $valueDataTerm.baseObject(); + |$vdBaseOffsetTerm = $valueDataTerm.baseOffset(); + |} + |} + | + | + |// position the offset to start of aggregate value + |$valueOffsetTerm += $numKeyBytesTerm + $vdBaseOffsetTerm; + |long $currentOffSetForMapLookupUpdt = $valueOffsetTerm;""".stripMargin + + } + + // handle arraydata , map , object + def explodeStruct(structVarName: String, structNullVarName: String, structType: StructType, + nestingLevel: Int = 0): String = { + val unsafeRowClass = classOf[UnsafeRow].getName + val explodedStructCode = structType.zipWithIndex.map { case (sf, index) => + (sf.dataType, index, SHAMapAccessor.generateExplodedStructFieldVars(structVarName, + nestingLevel, index)) + }.map { case (dt, index, (varName, nullVarName)) => + val valueExtractCode = dt match { + case x: AtomicType => typeOf(x.tag) match { + case t if t =:= typeOf[Boolean] => s"$structVarName.getBoolean($index); \n" + case t if t =:= typeOf[Byte] => s"$structVarName.getByte($index); \n" + case t if t =:= typeOf[Short] => s"$structVarName.getShort($index); \n" + case t if t =:= typeOf[Int] => s"$structVarName.getInt($index); \n" + case t if t =:= typeOf[Long] => s"$structVarName.getLong($index); \n" + case t if t =:= typeOf[Float] => s"$structVarName.getFloat$index); \n" + case t if t =:= typeOf[Double] => s"$structVarName.getDouble($index); \n" + case t if t =:= typeOf[Decimal] => s"$structVarName.getDecimal($index, " + + s"${dt.asInstanceOf[DecimalType].precision}," + + s"${dt.asInstanceOf[DecimalType].scale}); \n" + case t if t =:= typeOf[UTF8String] => s"$structVarName.getUTF8String($index); \n" + case _ => throw new UnsupportedOperationException("unknown type " + dt) + } + case BinaryType => s"$structVarName.getBinary($index); \n" + case CalendarIntervalType => s"$structVarName.getInterval($index); \n" + case st: StructType => s"$structVarName.getStruct($index, ${st.length}); \n" + + case _ => throw new UnsupportedOperationException("unknown type " + dt) + } + + val snippet = + s"""|boolean $nullVarName = $structNullVarName || + | (!$alwaysExplode && $structVarName instanceof $unsafeRowClass) || + | $structVarName.isNullAt($index); + | ${ctx.javaType(dt)} $varName = ${ctx.defaultValue(dt)}; + | if ($alwaysExplode|| !($structVarName instanceof $unsafeRowClass)) { + |if (!$nullVarName) { + |$varName = $valueExtractCode; + |} + |} + """.stripMargin + + snippet + (dt match { + case st: StructType => explodeStruct(varName, nullVarName, + st, nestingLevel + 1) + case _ => "" + }) + }.mkString("\n") + s""" + ${ + SHAMapAccessor.initNullBitsetCode( + SHAMapAccessor.generateNullKeysBitTermForStruct(structVarName), + SHAMapAccessor.calculateNumberOfBytesForNullBits(structType.length)) + } + $explodedStructCode + """.stripMargin + + } + + + + + + def generateUpdate(bufferVars: Seq[ExprCode], aggBufferDataType: Seq[DataType]): String = { + val plaformClass = classOf[Platform].getName + val setStoredAggNullBitsTerm = storedAggNullBitsTerm.map(storedNullBit => { + s"""// If key did not exist, make cachedAggBit -1 , so that the update will always write + // the right state of agg bit , else it will be that stored Agg Bit will match the + // after update aggBit, but will not reflect it in the HashMap bits + if ($keyExistedTerm) { + $storedNullBit = $nullAggsBitsetTerm; + } else { + $storedNullBit = -1; + } + """.stripMargin + }).getOrElse("") + + + s""" + |$setStoredAggNullBitsTerm + ${SHAMapAccessor.resetNullBitsetCode(nullAggsBitsetTerm, numBytesForNullAggBits)} + ${ + writeKeyOrValue(vdBaseObjectTerm, currentOffSetForMapLookupUpdt, + aggBufferDataType, bufferVars, nullAggsBitsetTerm, numBytesForNullAggBits, + false, false) + } + """.stripMargin + + } + + + + def writeKeyOrValue(baseObjectTerm: String, offsetTerm: String, + dataTypes: Seq[DataType], varsToWrite: Seq[ExprCode], nullBitsTerm: String, + numBytesForNullBits: Int, isKey: Boolean, skipNullEvalCode: Boolean, + nestingLevel: Int = 0): String = { + // Move the offset at the end of num Null Bytes space, we will fill that space later + // store the starting value of offset + val unsafeArrayClass = classOf[UnsafeArrayData].getName + val unsafeRowClass = classOf[UnsafeRow].getName + val startingOffsetTerm = ctx.freshName("startingOffset") + val tempBigDecArrayTerm = ctx.freshName("tempBigDecArray") + val plaformClass = classOf[Platform].getName + val storeNullBitStartOffsetAndRepositionOffset = if (skipNullEvalCode) { + "" + } else { + s"""long $startingOffsetTerm = $offsetTerm; + |// move current offset to end of null bits + |$offsetTerm += ${SHAMapAccessor.sizeForNullBits(numBytesForNullBits)};""".stripMargin + } + s"""$storeNullBitStartOffsetAndRepositionOffset + |${dataTypes.zip(varsToWrite).zipWithIndex.map { + case ((dt, expr), i) => + val variable = expr.value + val writingCode = (dt match { + case x: AtomicType => + val snippet = typeOf(x.tag) match { + case t if t =:= typeOf[Boolean] => + s"""$plaformClass.putBoolean($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Byte] => + s"""$plaformClass.putByte($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Array[Byte]] => + s"""$plaformClass.putInt($baseObjectTerm, $offsetTerm, $variable.length); + |$offsetTerm += 4; + |$plaformClass.copyMemory($variable, ${Platform.BYTE_ARRAY_OFFSET}, + |$baseObjectTerm, $offsetTerm, $variable.length); + |$offsetTerm += $variable.length; + """.stripMargin + case t if t =:= typeOf[Short] => + s"""$plaformClass.putShort($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Int] => + s""" + $plaformClass.putInt($baseObjectTerm, $offsetTerm, $variable); + $offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Long] => + s"""$plaformClass.putLong($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Float] => + s"""$plaformClass.putFloat($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Double] => + s"""$plaformClass.putDouble($baseObjectTerm, $offsetTerm, $variable); + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[Decimal] => + s""" + |if (${dt.asInstanceOf[DecimalType].precision} != $variable.precision() || + | ${dt.asInstanceOf[DecimalType].scale} != $variable.scale()) { + | if (!$variable.changePrecision(${dt.asInstanceOf[DecimalType].precision}, + | ${dt.asInstanceOf[DecimalType].scale})) { + | throw new java.lang.IllegalStateException("unable to change precision"); + | } + |} + """.stripMargin + + (if (dt.asInstanceOf[DecimalType].precision <= Decimal.MAX_LONG_DIGITS) { + s"""$plaformClass.putLong($baseObjectTerm, $offsetTerm, + | $variable.toUnscaledLong()); + """.stripMargin + } else { + s"""byte[] $tempBigDecArrayTerm = $variable.toJavaBigDecimal(). + |unscaledValue().toByteArray(); + |assert ($tempBigDecArrayTerm.length <= 16); + |$plaformClass.putLong($baseObjectTerm, $offsetTerm,0); + |$plaformClass.putLong($baseObjectTerm, $offsetTerm + 8,0); + |$plaformClass.copyMemory($tempBigDecArrayTerm, + |$plaformClass.BYTE_ARRAY_OFFSET, $baseObjectTerm, $offsetTerm + + |${dt.asInstanceOf[DecimalType].defaultSize} - $tempBigDecArrayTerm.length , + | $tempBigDecArrayTerm.length); + """.stripMargin + }) + + s""" + |$offsetTerm += ${dt.defaultSize}; + """.stripMargin + case t if t =:= typeOf[UTF8String] => + val tempLenTerm = ctx.freshName("tempLen") + + val lengthWritingPart = if (nestingLevel > 0 || i != skipLenForAttribIndex) { + s"""$plaformClass.putInt($baseObjectTerm, $offsetTerm, $tempLenTerm); + |$offsetTerm += 4;""".stripMargin + } else "" + + s"""int $tempLenTerm = $variable.numBytes(); + |$lengthWritingPart + |$variable.writeToMemory($baseObjectTerm, $offsetTerm); + |$offsetTerm += $tempLenTerm; + """.stripMargin + case _ => throw new UnsupportedOperationException("unknown type " + dt) + } + snippet + case st: StructType => val (childExprCodes, childDataTypes) = + getExplodedExprCodeAndDataTypeForStruct(variable, st, nestingLevel) + val newNullBitTerm = SHAMapAccessor.generateNullKeysBitTermForStruct(variable) + val newNumBytesForNullBits = SHAMapAccessor. + calculateNumberOfBytesForNullBits(st.length) + val explodeStructSnipet = + s"""$plaformClass.putBoolean($baseObjectTerm, $offsetTerm, true); + |$offsetTerm += 1; + |${ + writeKeyOrValue(baseObjectTerm, offsetTerm, childDataTypes, childExprCodes, + newNullBitTerm, newNumBytesForNullBits, true, false, + nestingLevel + 1) + } + """.stripMargin + val unexplodedStructSnippet = + s"""|$plaformClass.putBoolean($baseObjectTerm, $offsetTerm, false); + |$offsetTerm += 1; + |$plaformClass.putInt($baseObjectTerm, $offsetTerm, + |(($unsafeRowClass)$variable).getSizeInBytes()); + |$offsetTerm += 4; + |(($unsafeRowClass)$variable).writeToMemory($baseObjectTerm, $offsetTerm); + |$offsetTerm += (($unsafeRowClass)$variable).getSizeInBytes(); + """.stripMargin + if (alwaysExplode) { + explodeStructSnipet + } else { + s"""if (!($variable instanceof $unsafeRowClass)) { + $explodeStructSnipet + } else { + $unexplodedStructSnippet + } + """.stripMargin + } + + + case at@ArrayType(elementType, containsNull) => + val varWidthNullBitStartPos = ctx.freshName("nullBitBeginPos") + val varWidthNumNullBytes = ctx.freshName("numNullBytes") + val varWidthNullBits = ctx.freshName("nullBits") + val arrElement = ctx.freshName("arrElement") + val tempObj = ctx.freshName("temp") + val array = ctx.freshName("array") + val counter = ctx.freshName("counter") + val remainder = ctx.freshName("remainder") + val arrIndex = ctx.freshName("arrIndex") + val dataTypeAsJson = elementType.json + val strippedQuotesJson = dataTypeAsJson.substring(1, dataTypeAsJson.length - 1) + val dataType = ctx.freshName("dataType") + val dataTypeClass = classOf[DataType].getName + val elementWitingCode = writeKeyOrValue(baseObjectTerm, offsetTerm, Seq(elementType), + Seq(ExprCode("", "false", arrElement)), "", -1, + true, true, nestingLevel) + val explodeArraySnippet = + s"""|$plaformClass.putBoolean($baseObjectTerm, $offsetTerm, true); + |$offsetTerm += 1; + |$plaformClass.putInt($baseObjectTerm, $offsetTerm, $variable.numElements()); + |$offsetTerm += 4; + |long $varWidthNullBitStartPos = $offsetTerm; + |int $varWidthNumNullBytes = $variable.numElements() / 8 + + |($variable.numElements() % 8 > 0 ? 1 : 0); + |byte[] $varWidthNullBits = null; + |${ if (containsNull) { + s""" + |$varWidthNullBits = new byte[$varWidthNumNullBytes]; + |$offsetTerm += $varWidthNumNullBytes; + """.stripMargin + } else "" + } + + |$dataTypeClass $dataType = $dataTypeClass$$.MODULE$$. + |fromJson("\\"$strippedQuotesJson\\""); + |for( int $counter = 0; $counter < $variable.numElements(); ++$counter) { + |int $remainder = $counter % 8; + |int $arrIndex = $counter / 8; + |if ($variable.isNullAt($counter)) { + |if ($containsNull) { + |$varWidthNullBits[$arrIndex] |= (byte)((0x01 << $remainder)); + |} else { + | throw new IllegalStateException("Not null Array element contains null"); + |} + |} else { + |${ctx.javaType(elementType)} $arrElement = + |(${ctx.boxedType(elementType)}) $variable.get($counter, $dataType); + |$elementWitingCode + |} + |} + |${ if (containsNull ) { + s""" + |$plaformClass.copyMemory($varWidthNullBits, + |${Platform.BYTE_ARRAY_OFFSET}, + |$baseObjectTerm, $varWidthNullBitStartPos, $varWidthNumNullBytes); + """.stripMargin + } else "" + } + """.stripMargin + val unexplodedArraySnippet = + s"""$plaformClass.putBoolean($baseObjectTerm, $offsetTerm, false); + |$offsetTerm += 1; + |$plaformClass.putInt($baseObjectTerm, $offsetTerm, + |(($unsafeArrayClass)$variable).getSizeInBytes()); + |$offsetTerm += 4; + |(($unsafeArrayClass)$variable).writeToMemory($baseObjectTerm, $offsetTerm); + |$offsetTerm += (($unsafeArrayClass)$variable).getSizeInBytes(); + """.stripMargin + + if (alwaysExplode) { + explodeArraySnippet + } else { + s"""if (!($variable instanceof $unsafeArrayClass)) { + $explodeArraySnippet + |} else { + $unexplodedArraySnippet + |} + """.stripMargin + } + case _ => throw new UnsupportedOperationException("unknown type " + dt) + }).trim + + + // Now do the actual writing based on whether the variable is null or not + if (skipNullEvalCode) { + writingCode + } else { + SHAMapAccessor.evaluateNullBitsAndEmbedWrite(numBytesForNullBits, expr, + i, nullBitsTerm, offsetTerm, dt, isKey, writingCode) + } + + }.mkString("\n") + } + // now write the nullBitsTerm + ${if (!skipNullEvalCode) { + val nullBitsWritingCode = writeNullBitsAt(baseObjectTerm, startingOffsetTerm, + nullBitsTerm, numBytesForNullBits) + if(isKey) { + if (nestingLevel == 0) { + storedKeyNullBitsTerm.map(storedBit => + s""" + | if ($storedBit != $nullBitsTerm) { + | $nullBitsWritingCode + | $storedBit = $nullBitsTerm; + | } + """.stripMargin).getOrElse(nullBitsWritingCode) + } else { + nullBitsWritingCode + } + } else { + storedAggNullBitsTerm.map(storedAggBit => + s""" + | if ($storedAggBit != $nullAggsBitsetTerm) { + | $nullBitsWritingCode + | } + """.stripMargin + ).getOrElse(nullBitsWritingCode) + } + } else "" + }""" + + } + + def generateKeyBytesHolderCode(numKeyBytesVar: String, numValueBytes: Int, + keyVars: Seq[ExprCode], keysDataType: Seq[DataType], + aggregatesDataType: Seq[DataType], valueInitVars: Seq[ExprCode]): String = { + + val byteBufferClass = classOf[ByteBuffer].getName + val currentOffset = ctx.freshName("currentOffset") + val plaformClass = classOf[Platform].getName + s""" + if ($keyBytesHolderVarTerm == null || $keyHolderCapacityTerm < + $numKeyBytesVar + $numValueBytes) { + //$keyBytesHolderVarTerm = + //$allocatorTerm.allocate($numKeyBytesVar + $numValueBytes, "SHA"); + //$baseKeyObject = $allocatorTerm.baseObject($keyBytesHolderVarTerm); + //$baseKeyHolderOffset = $allocatorTerm.baseOffset($keyBytesHolderVarTerm); + $keyHolderCapacityTerm = $numKeyBytesVar + $numValueBytes; + $keyBytesHolderVarTerm = $byteBufferClass.allocate($keyHolderCapacityTerm); + $baseKeyObject = $keyBytesHolderVarTerm.array(); + $baseKeyHolderOffset = $plaformClass.BYTE_ARRAY_OFFSET; + ${storedKeyNullBitsTerm.map(x => s"$x = -1;").getOrElse("")} + } + + long $currentOffset = $baseKeyHolderOffset; + // first write key data + ${ writeKeyOrValue(baseKeyObject, currentOffset, keysDataType, keyVars, + nullKeysBitsetTerm, numBytesForNullKeyBits, true, numBytesForNullKeyBits == 0) + } + // write value data + ${"" /* writeKeyOrValue(baseKeyObject, currentOffset, aggregatesDataType, valueInitVars, + nullAggsBitsetTerm, numBytesForNullAggBits, false, false) */ + } + """.stripMargin + } + + def writeNullBitsAt(baseObjectTerm: String, offsetToWriteTerm: String, + nullBitsTerm: String, numBytesForNullBits: Int): String = { + val plaformClass = classOf[Platform].getName + if (numBytesForNullBits == 0) { + "" + } else if (numBytesForNullBits == 1) { + s"$plaformClass.putByte($baseObjectTerm, $offsetToWriteTerm, $nullBitsTerm);" + } else if (numBytesForNullBits == 2) { + s"$plaformClass.putShort($baseObjectTerm, $offsetToWriteTerm, $nullBitsTerm);" + } else if (numBytesForNullBits <= 4) { + s"$plaformClass.putInt($baseObjectTerm, $offsetToWriteTerm, $nullBitsTerm);" + } else if (numBytesForNullBits <= 8) { + s"$plaformClass.putLong($baseObjectTerm, $offsetToWriteTerm, $nullBitsTerm);" + } else { + s"$plaformClass.copyMemory($nullBitsTerm, ${Platform.BYTE_ARRAY_OFFSET}," + + s" $baseObjectTerm, $offsetToWriteTerm, $numBytesForNullBits);" + } + } + + + + def generateKeySizeCode(keyVars: Seq[ExprCode], keysDataType: Seq[DataType], + numBytesForNullBits: Int, nestingLevel: Int = 0): String = { + val unsafeRowClass = classOf[UnsafeRow].getName + val unsafeArrayDataClass = classOf[UnsafeArrayData].getName + + keysDataType.zip(keyVars).zipWithIndex.map { case ((dt, expr), i) => + val nullVar = expr.isNull + val notNullSizeExpr = if (TypeUtilities.isFixedWidth(dt)) { + dt.defaultSize.toString + } else { + dt match { + case StringType => + val strPart = s"${expr.value}.numBytes()" + if (nestingLevel == 0 && i == skipLenForAttribIndex) { + strPart + } else { + s"($strPart + 4)" + } + case BinaryType => s"(${expr.value}.length + 4) " + case st: StructType => val (childKeysVars, childDataTypes) = + getExplodedExprCodeAndDataTypeForStruct(expr.value, st, nestingLevel) + val explodedStructSizeCode = generateKeySizeCode(childKeysVars, childDataTypes, + SHAMapAccessor.calculateNumberOfBytesForNullBits(st.length), nestingLevel + 1) + val unexplodedStructSizeCode = s"(($unsafeRowClass) ${expr.value}).getSizeInBytes() + 4" + + "1 + " + (if (alwaysExplode) { + explodedStructSizeCode + } else { + s"""(${expr.value} instanceof $unsafeRowClass ? $unexplodedStructSizeCode + |: $explodedStructSizeCode) + """.stripMargin + } + ) + + case at@ArrayType(elementType, containsNull) => + // The array serialization format is following + /** + * Boolean (exploded or not) + * | + * -------------------------------------- + * False| | true + * 4 bytes for num bytes ---------- + * all bytes no null | | may be null + * allowed | 4 bytes for total elements + * | + num bytes for null bit mask + * 4 bytes for + inidividual not null elements + * num elements + * + each element + * serialzied + * + */ + val (isFixedWidth, unitSize) = if (TypeUtilities.isFixedWidth(elementType)) { + (true, dt.defaultSize) + } else { + (false, 0) + } + val snippetNullBitsSizeCode = + s"""${expr.value}.numElements()/8 + (${expr.value}.numElements() % 8 > 0 ? 1 : 0) + """.stripMargin + + val snippetNotNullFixedWidth = s"4 + ${expr.value}.numElements() * $unitSize" + val snippetNotNullVarWidth = + s"""4 + (int)($sizeAndNumNotNullFuncForStringArr(${expr.value}, true) >>> 32L) + """.stripMargin + val snippetNullVarWidth = s" $snippetNullBitsSizeCode + $snippetNotNullVarWidth" + val snippetNullFixedWidth = + s"""4 + $snippetNullBitsSizeCode + + |$unitSize * (int)($sizeAndNumNotNullFuncForStringArr( + |${expr.value}, false) & 0xffffffffL) + """.stripMargin + + "( 1 + " + (if (alwaysExplode) { + if (isFixedWidth) { + if (containsNull) { + snippetNullFixedWidth + } else { + snippetNotNullFixedWidth + } + } else { + if (containsNull) { + snippetNullVarWidth + } else { + snippetNotNullVarWidth + } + } + } else { + s"""(${expr.value} instanceof $unsafeArrayDataClass ? + |(($unsafeArrayDataClass) ${expr.value}).getSizeInBytes() + 4 + |: ${ if (isFixedWidth) { + s"""$containsNull ? ($snippetNullFixedWidth) + |: ($snippetNotNullFixedWidth)) + """.stripMargin + } else { + s"""$containsNull ? ($snippetNullVarWidth) + |: ($snippetNotNullVarWidth)) + """.stripMargin + } + } + """.stripMargin + }) + ")" + + } + } + if (nullVar.isEmpty || nullVar == "false") { + notNullSizeExpr + } else { + s"($nullVar? 0 : $notNullSizeExpr)" + } + }.mkString(" + ") + s" + ${SHAMapAccessor.sizeForNullBits(numBytesForNullBits)}" + } + + def getExplodedExprCodeAndDataTypeForStruct(parentStructVarName: String, st: StructType, + nestingLevel: Int): (Seq[ExprCode], Seq[DataType]) = st.zipWithIndex.map { + case (sf, index) => val (varName, nullVarName) = + SHAMapAccessor.generateExplodedStructFieldVars(parentStructVarName, nestingLevel, index) + ExprCode("", nullVarName, varName) -> sf.dataType + }.unzip + + + + + /** + * Generate code to calculate the hash code for given column variables that + * correspond to the key columns in this class. + */ + def generateHashCode(hashVar: Array[String], keyVars: Seq[ExprCode], + keyExpressions: Seq[Expression], keysDataType: Seq[DataType], + skipDeclaration: Boolean = false, register: Boolean = true): String = { + var hash = hashVar(0) + val hashDeclaration = if (skipDeclaration) "" else s"int $hash = 0;\n" + // check if hash has already been generated for keyExpressions + var doRegister = register + val vars = keyVars.map(_.value) + val (prefix, suffix) = session.getHashVar(ctx, vars) match { + case Some(h) => + hashVar(0) = h + hash = h + doRegister = false + (s"if ($hash == 0) {\n", "}\n") + case _ => (hashDeclaration, "") + } + + // register the hash variable for the key expressions + if (doRegister) { + session.addHashVar(ctx, vars, hash) + } + + // optimize for first column to use fast hashing + val expr = keyVars.head + val colVar = expr.value + val nullVar = expr.isNull + val firstColumnHash = keysDataType.head match { + case BooleanType => + hashSingleInt(s"($colVar) ? 1 : 0", nullVar, hash) + case ByteType | ShortType | IntegerType | DateType => + hashSingleInt(colVar, nullVar, hash) + case LongType | TimestampType => + hashSingleLong(colVar, nullVar, hash) + case FloatType => + hashSingleInt(s"Float.floatToIntBits($colVar)", nullVar, hash) + case BinaryType => + hashBinary(colVar, nullVar, hash) + case DoubleType => + hashSingleLong(s"Double.doubleToLongBits($colVar)", nullVar, hash) + case _: DecimalType => + hashSingleInt(s"$colVar.fastHashCode()", nullVar, hash) + // single column types that use murmur hash already, + // so no need to further apply mixing on top of it + case _: StringType | _: ArrayType | _: StructType => + s"$hash = ${hashCodeSingleInt(s"$colVar.hashCode()", nullVar)};\n" + case _ => + hashSingleInt(s"$colVar.hashCode()", nullVar, hash) + } + if (keyVars.length > 1) { + keysDataType.tail.zip(keyVars.tail).map { + case (BooleanType, ev) => + addHashInt(s"${ev.value} ? 1 : 0", ev.isNull, hash) + case (ByteType | ShortType | IntegerType | DateType, ev) => + addHashInt(ev.value, ev.isNull, hash) + case (BinaryType, ev) => + hashBinary(ev.value, ev.isNull, hash) + case (LongType | TimestampType, ev) => + addHashLong(ctx, ev.value, ev.isNull, hash) + case (FloatType, ev) => + addHashInt(s"Float.floatToIntBits(${ev.value})", ev.isNull, hash) + case (DoubleType, ev) => + addHashLong(ctx, s"Double.doubleToLongBits(${ev.value})", ev.isNull, + hash) + case (_: DecimalType, ev) => + addHashInt(s"${ev.value}.fastHashCode()", ev.isNull, hash) + case (_, ev) => + addHashInt(s"${ev.value}.hashCode()", ev.isNull, hash) + }.mkString(prefix + firstColumnHash, "", suffix) + } else prefix + firstColumnHash + suffix + } + + + private def hashSingleInt(colVar: String, nullVar: String, + hashVar: String): String = { + if (nullVar.isEmpty || nullVar == "false") { + s"$hashVar = $hashingClass.fastHashInt($colVar);\n" + } else { + s"$hashVar = ($nullVar) ? -1 : $hashingClass.fastHashInt($colVar);\n" + } + } + + private def hashCodeSingleInt(hashExpr: String, nullVar: String): String = { + if (nullVar.isEmpty || nullVar == "false") hashExpr + else s"($nullVar) ? -1 : $hashExpr" + } + + private def hashSingleLong(colVar: String, nullVar: String, + hashVar: String): String = { + if (nullVar.isEmpty || nullVar == "false") { + s"$hashVar = $hashingClass.fastHashLong($colVar);\n" + } else { + s"$hashVar = ($nullVar) ? -1 : $hashingClass.fastHashLong($colVar);\n" + } + } + + private def hashBinary(colVar: String, nullVar: String, + hashVar: String): String = { + if (nullVar.isEmpty || nullVar == "false") { + s"$hashVar = $hashingClass.addBytesToHash($colVar, $hashVar);\n" + } else { + s"$hashVar = ($nullVar) ? -1 : $hashingClass.addBytesToHash($colVar, $hashVar);\n" + } + } + + private def addHashInt(hashExpr: String, nullVar: String, + hashVar: String): String = { + if (nullVar.isEmpty || nullVar == "false") { + s""" + $hashVar = ($hashVar ^ 0x9e3779b9) + ($hashExpr) + + ($hashVar << 6) + ($hashVar >>> 2); + """ + } else { + s""" + $hashVar = ($hashVar ^ 0x9e3779b9) + (($nullVar) ? -1 : ($hashExpr)) + + ($hashVar << 6) + ($hashVar >>> 2); + """ + } + } + + private def addHashLong(ctx: CodegenContext, hashExpr: String, nullVar: String, + hashVar: String): String = { + val longVar = ctx.freshName("longVar") + if (nullVar.isEmpty || nullVar == "false") { + s""" + final long $longVar = $hashExpr; + $hashVar = ($hashVar ^ 0x9e3779b9) + (int)($longVar ^ ($longVar >>> 32)) + + ($hashVar << 6) + ($hashVar >>> 2); + """ + } else { + s""" + final long $longVar; + $hashVar = ($hashVar ^ 0x9e3779b9) + (($nullVar) ? -1 + : (int)(($longVar = ($hashExpr)) ^ ($longVar >>> 32))) + + ($hashVar << 6) + ($hashVar >>> 2); + """ + } + } + +} + +object SHAMapAccessor { + + val nullVarSuffix = "_isNull" + val supportedDataTypes: DataType => Boolean = dt => + dt match { + case _: MapType => false + case _: UserDefinedType[_] => false + case CalendarIntervalType => false + case NullType => false + case _: ObjectType => false + case ArrayType(elementType, _) => elementType match { + case _: StructType => false + case _ => true + } + case _ => true + + // includes atomic types, string type, array type + // ( depends on element type) , struct type ( depends on fields) + } + + def initNullBitsetCode(nullBitsetTerm: String, + numBytesForNullBits: Int): String = if (numBytesForNullBits == 0) { + "" + } else if (numBytesForNullBits == 1) { + s"byte $nullBitsetTerm = 0;" + } else if (numBytesForNullBits == 2) { + s"short $nullBitsetTerm = 0;" + } else if (numBytesForNullBits <= 4) { + s"int $nullBitsetTerm = 0;" + } else if (numBytesForNullBits <= 8) { + s"long $nullBitsetTerm = 0l;" + } else { + s""" + |for( int i = 0 ; i < $numBytesForNullBits; ++i) { + |$nullBitsetTerm[i] = 0; + |}""".stripMargin + } + + def resetNullBitsetCode(nullBitsetTerm: String, + numBytesForNullBits: Int): String = if (numBytesForNullBits == 0) { + "" + } else if (numBytesForNullBits <= 8) { + s"$nullBitsetTerm = 0; \n" + } else { + s""" + for( int i = 0 ; i < $numBytesForNullBits; ++i) { + $nullBitsetTerm[i] = 0; + } + + """.stripMargin + } + + def calculateNumberOfBytesForNullBits(numAttributes: Int): Int = (numAttributes + 7 )/ 8 + + def generateNullKeysBitTermForStruct(structName: String): String = s"${structName}_nullKeysBitset" + + def generateVarNameForStructField(parentVar: String, + nestingLevel: Int, index: Int): String = s"${parentVar}_${nestingLevel}_$index" + + def generateExplodedStructFieldVars(parentVar: String, + nestingLevel: Int, index: Int): (String, String) = { + val varName = generateVarNameForStructField(parentVar, nestingLevel, index) + val isNullVarName = s"${varName}_isNull" + (varName, isNullVarName) + } + + def isByteArrayNeededForNullBits(numBytes: Int): Boolean = numBytes > 8 + + def sizeForNullBits(numBytesForNullBits: Int): Int = + if (numBytesForNullBits == 0) { + 0 + } else if (numBytesForNullBits < 3 || numBytesForNullBits > 8) { + numBytesForNullBits + } else if (numBytesForNullBits <= 4) { + 4 + } else { + 8 + } + + + def getExpressionForNullEvalFromMask(i: Int, numBytesForNullBits: Int, + nullBitTerm: String ): String = { + val castTerm = getNullBitsCastTerm(numBytesForNullBits) + if (numBytesForNullBits <= 8) { + s"""($nullBitTerm & ((($castTerm)0x01) << $i)) != 0""" + } else { + val remainder = i % 8 + val index = i / 8 + s"""($nullBitTerm[$index] & (0x01 << $remainder)) != 0""" + } + } + + def getSizeOfValueBytes(aggDataTypes: Seq[DataType], numBytesForNullAggBits: Int): Int = { + aggDataTypes.foldLeft(0)((size, dt) => size + dt.defaultSize) + + SHAMapAccessor.sizeForNullBits(numBytesForNullAggBits) + } + + def getNullBitsCastTerm(numBytesForNullBits: Int): String = if (numBytesForNullBits == 1) { + "byte" + } else if (numBytesForNullBits == 2) { + "short" + } else if (numBytesForNullBits <= 4) { + "int" + } else { + "long" + } + + def getOffsetIncrementCodeForNullAgg(offsetTerm: String, dt: DataType): String = { + s"""$offsetTerm += ${dt.defaultSize};""" + } + + def evaluateNullBitsAndEmbedWrite(numBytesForNullBits: Int, expr: ExprCode, + i: Int, nullBitsTerm: String, offsetTerm: String, dt: DataType, + isKey: Boolean, writingCodeToEmbed: String): String = { + val castTerm = SHAMapAccessor.getNullBitsCastTerm(numBytesForNullBits) + val nullVar = expr.isNull + if (numBytesForNullBits > 8) { + val remainder = i % 8 + val index = i / 8 + if (nullVar.isEmpty || nullVar == "false") { + s"""$writingCodeToEmbed""" + } else if (nullVar == "true") { + s""" + |$nullBitsTerm[$index] |= (byte)((0x01 << $remainder)); + ${ + if (isKey) { + "" + } else { + SHAMapAccessor.getOffsetIncrementCodeForNullAgg(offsetTerm, dt) + } + } + """.stripMargin + } + else { + s""" if ($nullVar) { + |$nullBitsTerm[$index] |= (byte)((0x01 << $remainder)); + |${ + if (isKey) "" + else SHAMapAccessor.getOffsetIncrementCodeForNullAgg(offsetTerm, dt) + } + } else { + |$writingCodeToEmbed + } + """.stripMargin + } + } + else { + if (nullVar.isEmpty || nullVar == "false") { + s"""$writingCodeToEmbed""" + } else if (nullVar == "true") { + s""""$nullBitsTerm |= ($castTerm)(( (($castTerm)0x01) << $i)); + |${ if (isKey) "" + else SHAMapAccessor.getOffsetIncrementCodeForNullAgg(offsetTerm, dt) + } + """.stripMargin + + } else { + s""" + |if ($nullVar) { + |$nullBitsTerm |= ($castTerm)(( (($castTerm)0x01) << $i)); + |${ if (isKey) "" + else SHAMapAccessor.getOffsetIncrementCodeForNullAgg(offsetTerm, dt) + } + |} else { + | $writingCodeToEmbed + |} + """.stripMargin + } + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index ab0dba769d..c89c2320df 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -36,7 +36,13 @@ package org.apache.spark.sql.execution.aggregate -import io.snappydata.collection.ObjectHashSet + +import java.nio.ByteBuffer + +import com.gemstone.gemfire.internal.cache.GemFireCacheImpl +import com.gemstone.gemfire.internal.shared.BufferAllocator +import io.snappydata.Property +import io.snappydata.collection.{ByteBufferData, ObjectHashSet, SHAMap} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -44,10 +50,14 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.columnar.encoding.ColumnEncoding import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.{ArrayType, BinaryType, MapType, StringType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{SnappySession, collection} +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils /** @@ -69,6 +79,27 @@ case class SnappyHashAggregateExec( hasDistinct: Boolean) extends NonRecursivePlans with UnaryExecNode with BatchConsumer { + + val useByteBufferMapBasedAggregation: Boolean = { + val useDictionaryOptimizationForSingleKey = (groupingExpressions.length == 1 && + groupingExpressions.exists(_.dataType match { + case StringType => true + case _ => false + }) && !Property.UseOptimizedHashAggregateForSingleKey.get( + sqlContext.sparkSession.asInstanceOf[SnappySession].sessionState.conf)) + + aggregateBufferAttributes.forall(attr => + TypeUtilities.isFixedWidth(attr.dataType)) && + Property.UseOptimzedHashAggregate.get( + sqlContext.sparkSession.asInstanceOf[SnappySession].sessionState.conf) && + !groupingExpressions.isEmpty && + groupingExpressions.forall(_.dataType. + existsRecursively(SHAMapAccessor.supportedDataTypes)) && + !useDictionaryOptimizationForSingleKey + } + + + override def nodeName: String = "SnappyHashAggregate" @transient def resultExpressions: Seq[NamedExpression] = __resultExpressions @@ -202,7 +233,11 @@ case class SnappyHashAggregateExec( if (groupingExpressions.isEmpty) { doProduceWithoutKeys(ctx) } else { - doProduceWithKeys(ctx) + if (useByteBufferMapBasedAggregation) { + doProduceWithKeysForSHAMap(ctx) + } else { + doProduceWithKeys(ctx) + } } } @@ -211,7 +246,11 @@ case class SnappyHashAggregateExec( if (groupingExpressions.isEmpty) { doConsumeWithoutKeys(ctx, input) } else { - doConsumeWithKeys(ctx, input) + if (useByteBufferMapBasedAggregation) { + doConsumeWithKeysForSHAMap(ctx, input) + } else { + doConsumeWithKeys(ctx, input) + } } } @@ -220,13 +259,17 @@ case class SnappyHashAggregateExec( // check for possible optimized dictionary code path; // below is a loose search while actual decision will be taken as per // availability of ExprCodeEx with DictionaryCode in doConsume - DictionaryOptimizedMapAccessor.canHaveSingleKeyCase( - keyBufferAccessor.keyExpressions) + if (useByteBufferMapBasedAggregation) { + false + } else { + DictionaryOptimizedMapAccessor.canHaveSingleKeyCase( + keyBufferAccessor.keyExpressions) + } } override def batchConsume(ctx: CodegenContext, plan: SparkPlan, input: Seq[ExprCode]): String = { - if (groupingExpressions.isEmpty || !canConsume(plan)) "" + if (groupingExpressions.isEmpty || !canConsume(plan) ) "" else { // create an empty method to populate the dictionary array // which will be actually filled with code in consume if the dictionary @@ -431,6 +474,7 @@ case class SnappyHashAggregateExec( // utility to generate class for optimized map, and hash map access methods @transient private var keyBufferAccessor: ObjectHashMapAccessor = _ + @transient private var byteBufferAccessor: SHAMapAccessor = _ @transient private var mapDataTerm: String = _ @transient private var maskTerm: String = _ @transient private var dictionaryArrayTerm: String = _ @@ -490,6 +534,523 @@ case class SnappyHashAggregateExec( } } + private def generateResultCodeForSHAMap( + ctx: CodegenContext, keyBufferVars: Seq[ExprCode], + aggBufferVars: Seq[ExprCode], iterValueOffsetTerm: String): String = { + /* Asif: It appears that we have to put the code of materilization of each grouping column + & aggreagte before we can send it to parent. The reason is following: + 1) In the byte buffer hashmap data is written consecitively i.e key1, key2 agg1 etc. + Now the pointer cannot jump arbitrarily to just read key2 without reading key1 + So suppose we have a nested query such that inner query produces code for outputting key1 , key2, + while outer query is going to use only key2. If we do not write the code of materialzing key1, + the pointer will not move forward, as the outer query is going to try to materialzie only key2, + but the pointer will not move to key2 unleass key1 has been consumed. + We need to resolve this issue. I suppose we can declare local variable pointers pointing to start location + of each key/aggregate & use those declared pointers in the materialization code for each key + */ + if (modes.contains(Final) || modes.contains(Complete)) { + // generate output extracting from ExprCodes + ctx.INPUT_ROW = null + ctx.currentVars = keyBufferVars + val keyVars = groupingExpressions.zipWithIndex.map { + case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) + } + val evaluateKeyVars = evaluateVariables(keyVars) + ctx.currentVars = aggBufferVars + val bufferVars = aggregateBufferAttributesForGroup.zipWithIndex.map { + case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) + } + val evaluateBufferVars = evaluateVariables(bufferVars) + // evaluate the aggregation result + ctx.currentVars = bufferVars + val aggResults = declFunctions.map(_.evaluateExpression).map { e => + BindReferences.bindReference(e, aggregateBufferAttributesForGroup) + .genCode(ctx) + } + val evaluateAggResults = evaluateVariables(aggResults) + // generate the final result + ctx.currentVars = keyVars ++ aggResults + val inputAttrs = groupingAttributes ++ aggregateAttributes + val resultVars = resultExpressions.map { e => + BindReferences.bindReference(e, inputAttrs).genCode(ctx) + } + s""" + ${byteBufferAccessor.readNullBitsCode(iterValueOffsetTerm, + byteBufferAccessor.nullKeysBitsetTerm, byteBufferAccessor.numBytesForNullKeyBits)} + $evaluateKeyVars + ${byteBufferAccessor.readNullBitsCode(iterValueOffsetTerm, + byteBufferAccessor.nullAggsBitsetTerm, byteBufferAccessor.numBytesForNullAggBits)} + $evaluateBufferVars + $evaluateAggResults + ${consume(ctx, resultVars)} + """ + + } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { + // Combined grouping keys and aggregate values in buffer + ctx.INPUT_ROW = null + ctx.currentVars = keyBufferVars + val keyVars = groupingExpressions.zipWithIndex.map { + case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) + } + val evaluateKeyVars = evaluateVariables(keyVars) + ctx.currentVars = aggBufferVars + val bufferVars = aggregateBufferAttributesForGroup.zipWithIndex.map { + case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) + } + val evaluateBufferVars = evaluateVariables(bufferVars) + s""" + ${byteBufferAccessor.readNullBitsCode(iterValueOffsetTerm, + byteBufferAccessor.nullKeysBitsetTerm, byteBufferAccessor.numBytesForNullKeyBits)} + $evaluateKeyVars + ${byteBufferAccessor.readNullBitsCode(iterValueOffsetTerm, + byteBufferAccessor.nullAggsBitsetTerm, byteBufferAccessor.numBytesForNullAggBits)} + $evaluateBufferVars + ${consume(ctx, keyBufferVars ++ aggBufferVars)} + """ + + + } else { + // generate result based on grouping key + ctx.INPUT_ROW = null + ctx.currentVars = keyBufferVars + val keyVars = groupingExpressions.zipWithIndex.map { + case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) + } + val evaluateKeyVars = evaluateVariables(keyVars) + ctx.currentVars = keyVars + val resultVars = resultExpressions.map { e => + BindReferences.bindReference(e, groupingAttributes).genCode(ctx) + } + + s""" + ${byteBufferAccessor.readNullBitsCode(iterValueOffsetTerm, + byteBufferAccessor.nullKeysBitsetTerm, byteBufferAccessor.numBytesForNullKeyBits)} + $evaluateKeyVars + ${consume(ctx, resultVars)} + """ + } + } + + private def doProduceWithKeysForSHAMap(ctx: CodegenContext): String = { + val initAgg = ctx.freshName("initAgg") + ctx.addMutableState("boolean", initAgg, s"$initAgg = false;") + // Create a name for iterator from HashMap + + val endIterValueOffset = ctx.freshName("endIterValueOffset") + val localIterValueOffsetTerm = ctx.freshName("localIterValueOffsetTerm") + val localIterValueStartOffsetTerm = ctx.freshName("localIterValueStartOffsetTerm") + val iterValueOffsetTerm = ctx.freshName("iterValueOffsetTerm") + ctx.addMutableState("long", iterValueOffsetTerm, s"$iterValueOffsetTerm = 0;") + + val nullKeysBitsetTerm = ctx.freshName("nullKeysBitset") + val nullAggsBitsetTerm = ctx.freshName("nullAggsBitset") + + val numBytesForNullKeyBits = if (this.groupingAttributes.forall(!_.nullable)) { + 0 + } else { + SHAMapAccessor.calculateNumberOfBytesForNullBits(this.groupingAttributes.length) + } + + val numBytesForNullAggsBits = SHAMapAccessor.calculateNumberOfBytesForNullBits( + this.aggregateBufferAttributesForGroup.length) + + if (SHAMapAccessor.isByteArrayNeededForNullBits(numBytesForNullKeyBits)) { + ctx.addMutableState("byte[]", nullKeysBitsetTerm, + s"$nullKeysBitsetTerm = new byte[$numBytesForNullKeyBits];") + } + + if (SHAMapAccessor.isByteArrayNeededForNullBits(numBytesForNullAggsBits)) { + ctx.addMutableState("byte[]", nullAggsBitsetTerm, + s"$nullKeysBitsetTerm = new byte[$numBytesForNullAggsBits];") + } + val probableSkipLen = this.groupingAttributes. + lastIndexWhere(attr => !TypeUtilities.isFixedWidth(attr.dataType)) + + val skipLenForAttrib = if (probableSkipLen != - 1 && + this.groupingAttributes(probableSkipLen).dataType == StringType) { + probableSkipLen + } else -1 + + val keyLengthTerm = ctx.freshName("keyLength") + + val utf8Class = classOf[UTF8String].getName + val bbDataClass = classOf[ByteBufferData].getName + val arrayDataClass = classOf[ArrayData].getName + val platformClass = classOf[Platform].getName + + val sizeAndNumNotNullFuncForStringArr = ctx.freshName("calculateStringArrSizeAndNumNotNulls") + + if (groupingAttributes.exists(attrib => attrib.dataType.existsRecursively(_ match { + case ArrayType(StringType, _) | ArrayType(_, true) => true + case _ => false + }))) { + ctx.addNewFunction(sizeAndNumNotNullFuncForStringArr, + s""" + private long $sizeAndNumNotNullFuncForStringArr($arrayDataClass arrayData, + boolean isStringData) { + long size = 0L; + int numNulls = 0; + for(int i = 0; i < arrayData.numElements(); ++i) { + if (!arrayData.isNullAt(i)) { + if (isStringData) { + $utf8Class o = arrayData.getUTF8String(i); + size += o.numBytes() + 4; + } + } else { + ++numNulls; + } + } + return (size << 32L) | ((arrayData.numElements() - numNulls) & 0xffffffffL); + } + """) + } + + + + val valueOffsetTerm = ctx.freshName("valueOffset") + val currentValueOffSetTerm = ctx.freshName("currentValueOffSet") + val valueDataTerm = ctx.freshName("valueData") + val vdBaseObjectTerm = ctx.freshName("vdBaseObjectTerm") + val vdBaseOffsetTerm = ctx.freshName("vdBaseOffsetTerm") + val valueDataCapacityTerm = ctx.freshName("valueDataCapacity") + val doAgg = ctx.freshName("doAggregateWithKeys") + val setBBMap = ctx.freshName("setBBMap") + + // generate variable name for hash map for use here and in consume + hashMapTerm = ctx.freshName("hashMap") + val hashSetClassName = classOf[SHAMap].getName + + val overflowHashMapsTerm = ctx.freshName("overflowHashMaps") + val listClassName = classOf[java.util.List[SHAMap]].getName + val overflowMapIter = ctx.freshName("overflowMapIter") + val iterClassName = classOf[java.util.Iterator[SHAMap]].getName + // generate variable names for holding data from the Map buffer + val aggregateBufferVars = for (i <- this.aggregateBufferAttributesForGroup.indices) yield { + ctx.freshName(s"buffer_$i") + } + + val KeyBufferVars = for (i <- groupingExpressions.indices) yield { + ctx.freshName(s"key_$i") + } + + val keysDataType = this.groupingAttributes.map(_.dataType) + // declare nullbitset terms for nested structs if required + val nestedStructNullBitsTermCreator: ((String, StructType, Int) => Any) => (String, StructType, Int) => Any = + (f: (String, StructType, Int) => Any) => + (structVarName: String, structType: StructType, nestingLevel: Int) => { + val numBytesForNullBits = SHAMapAccessor. + calculateNumberOfBytesForNullBits(structType.length) + if (SHAMapAccessor.isByteArrayNeededForNullBits(numBytesForNullBits)) { + val nullBitTerm = SHAMapAccessor. + generateNullKeysBitTermForStruct(structVarName) + ctx.addMutableState("byte[]", nullBitTerm, + s"$nullBitTerm = new byte[$numBytesForNullBits];") + } + structType.zipWithIndex.foreach { case (sf, index) => sf.dataType match { + case stt: StructType => val structtVarName = SHAMapAccessor. + generateExplodedStructFieldVars(structVarName, nestingLevel + 1, index)._1 + f(structtVarName, stt, nestingLevel + 1) + null + case _ => null + } + + } + } + val nestedStructNullBitsTermInitializer: ((String, StructType, Int) => Any) => + (String, StructType, Int) => Any = + (f: (String, StructType, Int) => Any) => + (structVarName: String, structType: StructType, nestingLevel: Int) => { + val numBytesForNullBits = SHAMapAccessor. + calculateNumberOfBytesForNullBits(structType.length) + val nullBitTerm = SHAMapAccessor. + generateNullKeysBitTermForStruct(structVarName) + val snippet1 = SHAMapAccessor.initNullBitsetCode(nullBitTerm, numBytesForNullBits) + + val snippet2 = structType.zipWithIndex.map { case (sf, index) => sf.dataType match { + case stt: StructType => val structtVarName = SHAMapAccessor. + generateVarNameForStructField(structVarName, nestingLevel , index) + f(structtVarName, stt, nestingLevel + 1).toString + case _ => "" + } + }.mkString("\n") + s""" + ${snippet1} + $snippet2 + """.stripMargin + } + + def recursiveApply(f: + ((String, StructType, Int) => Any) => (String, StructType, Int) => Any): + (String, StructType, Int) => Any = f(recursiveApply(f))(_, _, _) + + // Now create nullBitTerms + KeyBufferVars.zip(keysDataType).foreach { + case (varName, dataType) => dataType match { + case st: StructType => recursiveApply( + nestedStructNullBitsTermCreator)(varName, st, 0) + case _ => + } + } + + val aggBuffDataTypes = this.aggregateBufferAttributesForGroup.map(_.dataType) + val allocatorTerm = ctx.freshName("bufferAllocator") + val allocatorClass = classOf[BufferAllocator].getName + val gfeCacheImplClass = classOf[GemFireCacheImpl].getName + val byteBufferClass = classOf[ByteBuffer].getName + + val keyBytesHolderVar = ctx.freshName("keyBytesHolder") + val baseKeyHolderOffset = ctx.freshName("baseKeyHolderOffset") + val baseKeyObject = ctx.freshName("baseKeyHolderObject") + val keyHolderCapacityTerm = ctx.freshName("keyholderCapacity") + val keyExistedTerm = ctx.freshName("keyExisted") + + val codeForLenOfSkippedTerm = if (skipLenForAttrib != -1) { + val numToDrop = skipLenForAttrib + 1 + val keysToProcessSize = this.groupingAttributes.drop(numToDrop) + val suffixSize = if (numBytesForNullKeyBits == 0) { + keysToProcessSize.foldLeft(0) { + case (size, attrib) => size + attrib.dataType.defaultSize + (attrib.dataType match { + case dec: DecimalType if (dec.precision > Decimal.MAX_LONG_DIGITS) => 1 + case _ => 0 + }) + }.toString + } else { + keysToProcessSize.zipWithIndex.map { + case(attrib, i) => { + val sizeTerm = attrib.dataType.defaultSize + (attrib.dataType match { + case dec: DecimalType if (dec.precision > Decimal.MAX_LONG_DIGITS) => 1 + case _ => 0 + }) + s"""(int)(${SHAMapAccessor.getExpressionForNullEvalFromMask(i + numToDrop, + numBytesForNullKeyBits, nullKeysBitsetTerm)} ? 0 : $sizeTerm) + """ + } + }.mkString("+") + } + + s"""$keyLengthTerm - + |(int)($localIterValueOffsetTerm - $localIterValueStartOffsetTerm) + |${ if (keysToProcessSize.length > 0) s" - ($suffixSize)" else ""};""".stripMargin + } else "" + + + ctx.addMutableState(hashSetClassName, hashMapTerm, s"$hashMapTerm = null;") + ctx.addMutableState(listClassName + s"<$hashSetClassName>", overflowHashMapsTerm, + s"$overflowHashMapsTerm = null;") + ctx.addMutableState(iterClassName + s"<$hashSetClassName>", overflowMapIter, + s"$overflowMapIter = null;") + + val storedAggNullBitsTerm = ctx.freshName("storedAggNullBit") + val cacheStoredAggNullBits = !SHAMapAccessor.isByteArrayNeededForNullBits( + numBytesForNullAggsBits) && numBytesForNullAggsBits > 0 + + val storedKeyNullBitsTerm = ctx.freshName("storedKeyNullBit") + val cacheStoredKeyNullBits = !SHAMapAccessor.isByteArrayNeededForNullBits( + numBytesForNullKeyBits) && numBytesForNullKeyBits > 0 + + // generate the map accessor to generate key/value class + // and get map access methods + val session = sqlContext.sparkSession.asInstanceOf[SnappySession] + val numKeyBytesTerm = ctx.freshName("numKeyBytes") + val numValueBytes = SHAMapAccessor.getSizeOfValueBytes(aggBuffDataTypes, + numBytesForNullAggsBits) + + val keyValSize = groupingAttributes.foldLeft(0)((len, attrib) => + len + attrib.dataType.defaultSize + + (if (TypeUtilities.isFixedWidth(attrib.dataType)) 0 else 4)) + + SHAMapAccessor.sizeForNullBits(numBytesForNullKeyBits) + numValueBytes - + (if (skipLenForAttrib != -1) 4 else 0) + + + byteBufferAccessor = SHAMapAccessor(session, ctx, groupingExpressions, + aggregateBufferAttributesForGroup, "ByteBuffer", hashMapTerm, overflowHashMapsTerm, + keyValSize, valueOffsetTerm, numKeyBytesTerm, numValueBytes, + currentValueOffSetTerm, valueDataTerm, vdBaseObjectTerm, vdBaseOffsetTerm, + nullKeysBitsetTerm, numBytesForNullKeyBits, allocatorTerm, + numBytesForNullAggsBits, nullAggsBitsetTerm, sizeAndNumNotNullFuncForStringArr, + keyBytesHolderVar, baseKeyObject, baseKeyHolderOffset, keyExistedTerm, + skipLenForAttrib, codeForLenOfSkippedTerm, valueDataCapacityTerm, + if (cacheStoredAggNullBits) Some(storedAggNullBitsTerm) else None, + if (cacheStoredKeyNullBits) Some(storedKeyNullBitsTerm) else None, + aggregateBufferVars, keyHolderCapacityTerm) + + val maxMemory = ctx.freshName("maxMemory") + val peakMemory = metricTerm(ctx, "peakMemory") + + val childProduce = + childProducer.asInstanceOf[CodegenSupport].produce(ctx, this) + ctx.addNewFunction(doAgg, + s"""private void $doAgg() throws java.io.IOException { + |$hashMapTerm = new $hashSetClassName(${Property.initialCapacityOfSHABBMap.get( + sqlContext.sparkSession.asInstanceOf[SnappySession].sessionState.conf)}, + $keyValSize, ${Property.ApproxMaxCapacityOfBBMap.get(sqlContext.sparkSession. + asInstanceOf[SnappySession].sessionState.conf)}); + |$allocatorClass $allocatorTerm = $gfeCacheImplClass. + |getCurrentBufferAllocator(); + |$byteBufferClass $keyBytesHolderVar = null; + |int $keyHolderCapacityTerm = 0; + |Object $baseKeyObject = null; + |long $baseKeyHolderOffset = -1L; + |$bbDataClass $valueDataTerm = $hashMapTerm.getValueData(); + |Object $vdBaseObjectTerm = $valueDataTerm.baseObject(); + |long $vdBaseOffsetTerm = $valueDataTerm.baseOffset(); + |int $valueDataCapacityTerm = $valueDataTerm.capacity(); + |${SHAMapAccessor.initNullBitsetCode(nullKeysBitsetTerm, numBytesForNullKeyBits)} + |${SHAMapAccessor.initNullBitsetCode(nullAggsBitsetTerm, numBytesForNullAggsBits)} + |${byteBufferAccessor.initKeyOrBufferVal(aggBuffDataTypes, aggregateBufferVars)} + |${byteBufferAccessor.declareNullVarsForAggBuffer(aggregateBufferVars)} + |${ if (cacheStoredAggNullBits) { + SHAMapAccessor.initNullBitsetCode(storedAggNullBitsTerm, numBytesForNullAggsBits) + } else "" + } + |${ if (cacheStoredKeyNullBits) { + SHAMapAccessor.initNullBitsetCode(storedKeyNullBitsTerm, numBytesForNullKeyBits) + } else "" + } + |int $numKeyBytesTerm = 0; + |$childProduce + |if ($overflowHashMapsTerm == null) { + | long $maxMemory = $hashMapTerm.maxMemory(); + | $peakMemory.add($maxMemory); + | if ($hashMapTerm.taskContext() != null) { + | $hashMapTerm.taskContext().taskMetrics().incPeakExecutionMemory($maxMemory); + | } + |} else { + $iterClassName tempIter = $overflowHashMapsTerm.iterator(); + while(tempIter.hasNext()) { + $hashSetClassName tempMap = ($hashSetClassName)tempIter.next(); + long $maxMemory = tempMap.maxMemory(); + $peakMemory.add($maxMemory); + if (tempMap.taskContext() != null) { + tempMap.taskContext().taskMetrics().incPeakExecutionMemory($maxMemory); + } + } + |} + |}""".stripMargin) + + ctx.addNewFunction(setBBMap, + s"""private boolean $setBBMap() { + |if ($hashMapTerm != null) { + |return true; + |} else { + |if ($overflowMapIter.hasNext()) { + |$hashMapTerm = ($hashSetClassName)$overflowMapIter.next(); + |$bbDataClass $valueDataTerm = $hashMapTerm.getValueData(); + |Object $vdBaseObjectTerm = $valueDataTerm.baseObject(); + |$iterValueOffsetTerm = $valueDataTerm.baseOffset(); + return true; + |} else { + return false; + |} + |} + |}""".stripMargin) + + // generate code for output + /* val keyBufferTerm = ctx.freshName("keyBuffer") + val (initCode, keyBufferVars, _) = keyBufferAccessor.getColumnVars( + keyBufferTerm, keyBufferTerm, onlyKeyVars = false, onlyValueVars = false) */ + + val keysExpr = byteBufferAccessor.getBufferVars(keysDataType, KeyBufferVars, + localIterValueOffsetTerm, true, byteBufferAccessor.nullKeysBitsetTerm, + byteBufferAccessor.numBytesForNullKeyBits, byteBufferAccessor.numBytesForNullKeyBits == 0) + val aggsExpr = byteBufferAccessor.getBufferVars(aggBuffDataTypes, + aggregateBufferVars, localIterValueOffsetTerm, false, byteBufferAccessor.nullAggsBitsetTerm, + byteBufferAccessor.numBytesForNullAggBits, false) + val outputCode = generateResultCodeForSHAMap(ctx, keysExpr, aggsExpr, localIterValueOffsetTerm) + val numOutput = metricTerm(ctx, "numOutputRows") + val localNumRowsIterated = ctx.freshName("localNumRowsIterated") + // The child could change `copyResult` to true, but we had already + // consumed all the rows, so `copyResult` should be reset to `false`. + ctx.copyResult = false + + val aggTime = metricTerm(ctx, "aggTime") + val beforeAgg = ctx.freshName("beforeAgg") + + val readKeyLengthCode = if (skipLenForAttrib != -1) { + if (ColumnEncoding.littleEndian) { + s"int $keyLengthTerm = $platformClass.getInt($vdBaseObjectTerm, $localIterValueOffsetTerm);" + } else { + s"""int $keyLengthTerm = java.lang.Integer.reverseBytes($platformClass.getInt( + $vdBaseObjectTerm, $localIterValueOffsetTerm)); + """ + } + } else "" + s""" + if (!$initAgg) { + $initAgg = true; + long $beforeAgg = System.nanoTime(); + $doAgg(); + $aggTime.${metricAdd(s"(System.nanoTime() - $beforeAgg) / 1000000")}; + if ($overflowHashMapsTerm != null) { + $overflowMapIter = $overflowHashMapsTerm.iterator(); + $hashMapTerm = ($hashSetClassName)$overflowMapIter.next(); + } else { + // add the single hashmap case to the list to fix the issue SNAP-3132 + // where the single hashmap gets GCed due to setting of null at the end + // of loop, causing the TPCHD query to crash the server as the string positions + // referenced in the map are no longer valid. Adding it in the list will + // prevent single hashmap from being gced + $overflowHashMapsTerm = ${classOf[java.util.Collections].getName}.<$hashSetClassName>singletonList( + $hashMapTerm); + $overflowMapIter = $overflowHashMapsTerm.iterator(); + $overflowMapIter.next(); + } + $bbDataClass $valueDataTerm = $hashMapTerm.getValueData(); + Object $vdBaseObjectTerm = $valueDataTerm.baseObject(); + $iterValueOffsetTerm += $valueDataTerm.baseOffset(); + } + if ($hashMapTerm == null) { + return; + } + $allocatorClass $allocatorTerm = $gfeCacheImplClass. + getCurrentBufferAllocator(); + ${byteBufferAccessor.initKeyOrBufferVal(aggBuffDataTypes, aggregateBufferVars)} + ${byteBufferAccessor.initKeyOrBufferVal(keysDataType, KeyBufferVars)} + ${SHAMapAccessor.initNullBitsetCode(nullKeysBitsetTerm, numBytesForNullKeyBits)} + ${SHAMapAccessor.initNullBitsetCode(nullAggsBitsetTerm, numBytesForNullAggsBits)} + ${KeyBufferVars.zip(keysDataType).map { + case (varName, dataType) => dataType match { + case st: StructType => + recursiveApply(nestedStructNullBitsTermInitializer)(varName, st, 0).toString + case _ => "" + } + }.mkString("\n")} + + // output the result + while($setBBMap()) { + $bbDataClass $valueDataTerm = $hashMapTerm.getValueData(); + Object $vdBaseObjectTerm = $valueDataTerm.baseObject(); + long $endIterValueOffset = $hashMapTerm.valueDataSize() + $valueDataTerm.baseOffset(); + long $localIterValueOffsetTerm = $iterValueOffsetTerm; + ${byteBufferAccessor.declareNullVarsForAggBuffer(aggregateBufferVars)} + int $localNumRowsIterated = 0; + while ($localIterValueOffsetTerm != $endIterValueOffset) { + ++$localNumRowsIterated; + $readKeyLengthCode + // skip the key length + $localIterValueOffsetTerm += 4; + long $localIterValueStartOffsetTerm = $localIterValueOffsetTerm; + $outputCode + if (shouldStop()) { + | $numOutput.${metricAdd(String.valueOf(localNumRowsIterated))}; + $localNumRowsIterated = 0; + $iterValueOffsetTerm = $localIterValueOffsetTerm; + return; + } + } + + + if ($localIterValueOffsetTerm == $endIterValueOffset) { + // Not releasing memory here as it will be freed by the task completion listener + $numOutput.${metricAdd(String.valueOf(localNumRowsIterated))}; + $localNumRowsIterated = 0; + //$hashMapTerm.release(); + $hashMapTerm = null; + } + } + """ + } + private def doProduceWithKeys(ctx: CodegenContext): String = { val initAgg = ctx.freshName("initAgg") ctx.addMutableState("boolean", initAgg, s"$initAgg = false;") @@ -506,6 +1067,7 @@ case class SnappyHashAggregateExec( // generate variable name for hash map for use here and in consume hashMapTerm = ctx.freshName("hashMap") val hashSetClassName = classOf[ObjectHashSet[_]].getName + ctx.addMutableState(hashSetClassName, hashMapTerm, "") // generate variables for HashMap data array and mask @@ -517,9 +1079,11 @@ case class SnappyHashAggregateExec( // generate the map accessor to generate key/value class // and get map access methods val session = sqlContext.sparkSession.asInstanceOf[SnappySession] + keyBufferAccessor = ObjectHashMapAccessor(session, ctx, groupingExpressions, - aggregateBufferAttributesForGroup, "KeyBuffer", hashMapTerm, - mapDataTerm, maskTerm, multiMap = false, this, this.parent, child) + aggregateBufferAttributesForGroup, "KeyBuffer", hashMapTerm, + mapDataTerm, maskTerm, multiMap = false, this, this.parent, child) + val entryClass = keyBufferAccessor.getClassName val numKeyColumns = groupingExpressions.length @@ -583,6 +1147,103 @@ case class SnappyHashAggregateExec( """ } + private def doConsumeWithKeysForSHAMap(ctx: CodegenContext, + input: Seq[ExprCode]): String = { + + + // only have DeclarativeAggregate + val updateExpr = aggregateExpressions.flatMap { e => + e.mode match { + case Partial | Complete => + e.aggregateFunction.asInstanceOf[DeclarativeAggregate] + .updateExpressions + case PartialMerge | Final => + e.aggregateFunction.asInstanceOf[DeclarativeAggregate] + .mergeExpressions + } + } + + ctx.currentVars = input + val keysExpr = ctx.generateExpressions( + groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) + // generate class for key, buffer and hash code evaluation of key columns + val inputAttr = aggregateBufferAttributesForGroup ++ child.output + val initVars = ctx.generateExpressions(declFunctions.flatMap( + bufferInitialValuesForGroup(_).map(BindReferences.bindReference(_, + inputAttr)))) + val initCode = evaluateVariables(initVars) + val keysDataType = this.groupingAttributes.map(_.dataType) + val aggBuffDataTypes = this.aggregateBufferAttributesForGroup.map(_.dataType) + + + // if aggregate expressions uses some of the key variables then signal those + // to be materialized explicitly for the dictionary optimization case (AQP-292) + val updateAttrs = AttributeSet(updateExpr) + // evaluate map lookup code before updateEvals possibly modifies the keyVars + val mapCode = byteBufferAccessor.generateMapGetOrInsert(initVars, initCode, input, + keysExpr, keysDataType, aggBuffDataTypes) + + val bufferVars = byteBufferAccessor.getBufferVars(aggBuffDataTypes, + byteBufferAccessor.aggregateBufferVars, + byteBufferAccessor.currentOffSetForMapLookupUpdt, + false, byteBufferAccessor.nullAggsBitsetTerm, byteBufferAccessor.numBytesForNullAggBits, + false) + val bufferEval = evaluateVariables(bufferVars) + val bufferVarsFromInitVars = byteBufferAccessor.aggregateBufferVars.zip(initVars).map { + case (bufferVarName, initExpr) => ExprCode( + s""" + |$bufferVarName${SHAMapAccessor.nullVarSuffix} = ${initExpr.isNull}; + |$bufferVarName = ${initExpr.value};""".stripMargin, + s"$bufferVarName${SHAMapAccessor.nullVarSuffix}", bufferVarName) + } + val bufferEvalFromInitVars = evaluateVariables(bufferVarsFromInitVars) + ctx.currentVars = bufferVars ++ input + // pre-evaluate input variables used by child expressions and updateExpr + val inputCodes = evaluateRequiredVariables(child.output, + ctx.currentVars.takeRight(child.output.length), + child.references ++ updateAttrs) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, + inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen( + boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val updateEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } + + // We first generate code to probe and update the hash map. If the probe is + // successful, the corresponding buffer will hold buffer class object. + // We try to do hash map based in-memory aggregation first. If there is not + // enough memory (the hash map will return null for new key), we spill the + // hash map to disk to free memory, then continue to do in-memory + // aggregation and spilling until all the rows had been processed. + // Finally, sort the spilled aggregate buffers by key, and merge + // them together for same key. + s""" + |$mapCode + |// initialization for buffer fields from the hashmap + |if (${byteBufferAccessor.keyExistedTerm}) { + | ${ + byteBufferAccessor.readNullBitsCode(byteBufferAccessor. + currentOffSetForMapLookupUpdt, byteBufferAccessor.nullAggsBitsetTerm, + byteBufferAccessor.numBytesForNullAggBits) + } + |$bufferEval + |} else { + | $bufferEvalFromInitVars + |} + | // reset the offset position to start of values for writing update + |${byteBufferAccessor.currentOffSetForMapLookupUpdt} = ${byteBufferAccessor.valueOffsetTerm}; + | // common sub-expressions + |$inputCodes + |$effectiveCodes + | + |// evaluate aggregate functions + |${evaluateVariables(updateEvals)} + |${byteBufferAccessor.generateUpdate(updateEvals, aggBuffDataTypes)} + """.stripMargin + } + private def doConsumeWithKeys(ctx: CodegenContext, input: Seq[ExprCode]): String = { val keyBufferTerm = ctx.freshName("keyBuffer") @@ -690,15 +1351,18 @@ case class SnappyHashAggregateExec( val outputString = Utils.truncatedString(output, "[", ", ", "]") val modesStr = modes.mkString(",") if (verbose) { - s"$name(keys=$keyString, modes=$modesStr, " + - s"functions=$functionString, output=$outputString)" + s"$name(ByteBufferHashMap used = $useByteBufferMapBasedAggregation;" + + s" keys=$keyString, modes=$modesStr, " + + s"functions=$functionString, output=$outputString)" } else { - s"$name(keys=$keyString, modes=$modesStr, functions=$functionString)" + s"$name(ByteBufferHashMap used = $useByteBufferMapBasedAggregation; " + + s"keys=$keyString, modes=$modesStr, functions=$functionString)" } case Some(fallbackStartsAt) => - s"${name}WithControlledFallback $groupingExpressions " + - s"$allAggregateExpressions $resultExpressions " + - s"fallbackStartsAt=$fallbackStartsAt" + s"${name}WithControlledFallback;" + + s"ByteBufferHashMap used = $useByteBufferMapBasedAggregation; $groupingExpressions " + + s"$allAggregateExpressions $resultExpressions " + + s"fallbackStartsAt=$fallbackStartsAt" } } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index 834c366df3..29f6cd963d 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -234,7 +234,8 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie connectionType match { case ConnectionType.Embedded => val rgn = Misc.getRegionForTable( - JdbcExtendedUtils.toUpperCase(tableName), true).asInstanceOf[LocalRegion] + JdbcExtendedUtils.toUpperCase(tableName), true). + asInstanceOf[LocalRegion] try { Misc.getGemFireCache.getCacheTransactionManager.rollback() } finally { @@ -243,7 +244,6 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie ds.releaseDiskStoreReadLock() } } - case _ => logDebug(s"Going to rollback transaction $txId on server using $conn") var ps: PreparedStatement = null diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala index 52cc270de0..d638ddffd2 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala @@ -425,8 +425,8 @@ class SnappySessionState(val snappySession: SnappySession) object RowLevelSecurity extends Rule[LogicalPlan] { // noinspection ScalaUnnecessaryParentheses // Y combinator - val conditionEvaluator: (Expression => Boolean) => (Expression => Boolean) = - (f: (Expression => Boolean)) => + val conditionEvaluator: (Expression => Boolean) => Expression => Boolean = + (f: Expression => Boolean) => (exp: Expression) => exp.eq(PolicyProperties.rlsAppliedCondition) || (exp match { case And(left, _) => f(left) @@ -437,7 +437,7 @@ class SnappySessionState(val snappySession: SnappySession) // noinspection ScalaUnnecessaryParentheses def rlsConditionChecker(f: (Expression => Boolean) => - (Expression => Boolean)): Expression => Boolean = f(rlsConditionChecker(f))(_: Expression) + Expression => Boolean): Expression => Boolean = f(rlsConditionChecker(f))(_: Expression) def apply(plan: LogicalPlan): LogicalPlan = { val memStore = GemFireStore.getBootingInstance diff --git a/core/src/main/scala/org/apache/spark/sql/internal/session.scala b/core/src/main/scala/org/apache/spark/sql/internal/session.scala index 6c3267014b..28825f2770 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/session.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/session.scala @@ -105,7 +105,10 @@ class SnappyConf(@transient val session: SnappySession) case SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key | Property.HashJoinSize.name | Property.HashAggregateSize.name | - Property.ForceLinkPartitionsToBuckets.name => + Property.ForceLinkPartitionsToBuckets.name | + Property.UseOptimzedHashAggregate.name | + Property.UseOptimizedHashAggregateForSingleKey.name | + Property.TestExplodeComplexDataTypeInSHA.name => session.clearPlanCache() key case SQLConf.SHUFFLE_PARTITIONS.key => diff --git a/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala b/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala index 3eec4c482e..1b8eb9d60f 100644 --- a/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala +++ b/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala @@ -16,13 +16,17 @@ */ package org.apache.spark.sql.types +import java.math.MathContext import java.util.Properties +import scala.reflect.runtime.universe._ + import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.CodegenSupport +import org.apache.spark.unsafe.types.UTF8String object TypeUtilities { @@ -108,4 +112,24 @@ object TypeUtilities { } props } + + def isFixedWidth(dataType: DataType): Boolean = { + dataType match { + case x: AtomicType => typeOf(x.tag) match { + case t if t =:= typeOf[Boolean] => true + case t if t =:= typeOf[Byte] => true + case t if t =:= typeOf[Short] => true + case t if t =:= typeOf[Int] => true + case t if t =:= typeOf[Long] => true + case t if t =:= typeOf[Float] => true + case t if t =:= typeOf[Double] => true + case t if t =:= typeOf[Decimal] => true + case _ => false + } + case _ => false + } + } + + val mathContextCache: Array[MathContext] = Array.tabulate[MathContext]( + DecimalType.MAX_PRECISION)(i => new MathContext(i + 1)) } diff --git a/core/src/test/scala/io/snappydata/core/LocalTestData.scala b/core/src/test/scala/io/snappydata/core/LocalTestData.scala index 49fa5eb641..99f4d66992 100644 --- a/core/src/test/scala/io/snappydata/core/LocalTestData.scala +++ b/core/src/test/scala/io/snappydata/core/LocalTestData.scala @@ -19,6 +19,8 @@ package io.snappydata.core import scala.reflect.io.Path import scala.util.Random +import io.snappydata.Property + import org.apache.spark.SparkConf /** @@ -85,6 +87,7 @@ object LocalSparkConf { .setIfMissing("spark.master", "local[4]") .setIfMissing("spark.memory.debugFill", "true") .set("snappydata.sql.planCaching", random.nextBoolean().toString) + .set(Property.TestDisableCodeGenFlag.name , "true") .setAppName(getClass.getName) if (addOn != null) { addOn(conf) diff --git a/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala b/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala new file mode 100644 index 0000000000..b72cad72ea --- /dev/null +++ b/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package com.gemstone.gemfire.internal.shared + +class BufferSizeLimitExceededException(message: String) extends RuntimeException(message) { + def this(currentUsed: Int, required: Int, maxCapacity: Int) { + this(s"The bytes size exceeded max capacity($maxCapacity) by" + + s" ${currentUsed + required - maxCapacity}") + } +} diff --git a/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala b/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala index 1231605b20..2374dc9cee 100644 --- a/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala +++ b/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala @@ -19,8 +19,12 @@ package io.snappydata.collection import java.nio.ByteBuffer -import com.gemstone.gemfire.internal.shared.BufferAllocator +import com.gemstone.gemfire.internal.cache.GemFireCacheImpl +import com.gemstone.gemfire.internal.shared.unsafe.DirectBufferAllocator +import com.gemstone.gemfire.internal.shared.{BufferAllocator, BufferSizeLimitExceededException} +import org.apache.spark.TaskContext +import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} import org.apache.spark.sql.collection.SharedUtils import org.apache.spark.sql.execution.columnar.encoding.ColumnEncoding import org.apache.spark.unsafe.Platform @@ -61,28 +65,46 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, protected val allocator: BufferAllocator, protected var keyData: ByteBufferData = null, protected var valueData: ByteBufferData = null, - protected var valueDataPosition: Long = 0L) { - + protected var valueDataPosition: Long = 0L, + val approxMaxCapacity: Int = ((Integer.MAX_VALUE - DirectBufferAllocator.DIRECT_OBJECT_OVERHEAD + - 7) >>> 3) << 3) { + val taskContext: TaskContext = TaskContext.get() + private var maxSizeReached: Boolean = false + private[this] val consumer = if ((taskContext ne null) && !GemFireCacheImpl.hasNewOffHeap) { + new ByteBufferHashMapMemoryConsumer(SharedUtils.taskMemoryManager(taskContext)) + } else null + + if ((taskContext ne null)) { + freeMemoryOnTaskCompletion() + } // round to word size adding 8 bytes for header (offset + hashcode) private val fixedKeySize = ((keySize + 15) >>> 3) << 3 private var _capacity = SharedUtils.nextPowerOf2(initialCapacity) + private var _size = 0 private var growThreshold = (loadFactor * _capacity).toInt private var mask = _capacity - 1 - + private[this] var _maxMemory: Long = _ if (keyData eq null) { val buffer = allocator.allocate(_capacity * fixedKeySize, "HASHMAP") // clear the key data allocator.clearPostAllocate(buffer, 0) keyData = new ByteBufferData(buffer, allocator) + acquireMemory(keyData.capacity) + _maxMemory += keyData.capacity } if (valueData eq null) { + require(_capacity * valueSize < approxMaxCapacity) valueData = new ByteBufferData(allocator.allocate(_capacity * valueSize, "HASHMAP"), allocator) valueDataPosition = valueData.baseOffset + acquireMemory(valueData.capacity) + _maxMemory += valueData.capacity } + + final def getKeyData: ByteBufferData = this.keyData final def getValueData: ByteBufferData = this.valueData @@ -92,6 +114,7 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, final def valueDataSize: Long = valueDataPosition - valueData.baseOffset final def capacity: Int = _capacity + def maxMemory: Long = _maxMemory /** * Insert raw bytes with given hash code into the map if not present. @@ -128,20 +151,31 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, if (mapKey != 0L) { // first compare the hash codes followed by "equalsSize" that will // include the check for 4 bytes of numKeyBytes itself - if (hash == mapKey.toInt && valueData.equalsSize((mapKey >>> 32L).toInt - 4, + val valueStartOffset = (mapKey >>> 32L).toInt - 4 + if (hash == mapKey.toInt && valueData.equalsSize(valueStartOffset, baseObject, baseOffset, numKeyBytes)) { - return handleExisting(mapKeyObject, mapKeyOffset) + return handleExisting(mapKeyObject, mapKeyOffset, valueStartOffset + 4) } else { // quadratic probing (increase delta) pos = (pos + delta) & mask delta += 1 } } else { + if (maxSizeReached) { + throw ByteBufferHashMap.bsle + } // insert into the map and rehash if required val relativeOffset = newInsert(baseObject, baseOffset, numKeyBytes, numBytes) Platform.putLong(mapKeyObject, mapKeyOffset, (relativeOffset << 32L) | (hash & 0xffffffffL)) - return handleNew(mapKeyObject, mapKeyOffset) + try { + return handleNew(mapKeyObject, mapKeyOffset, relativeOffset) + } catch { + case bsle: BufferSizeLimitExceededException => + maxSizeReached = true + Platform.putLong(mapKeyObject, mapKeyOffset, 0L) + throw bsle + } } } 0 // not expected to reach @@ -153,21 +187,30 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, valueData.reset(clearMemory = false) valueDataPosition = valueData.baseOffset _size = 0 + this.maxSizeReached = false } final def release(): Unit = { - keyData.release(allocator) - valueData.release(allocator) + // The fields could be null, if the ByteBufferData could not be + // created due to insufficient memory + if (keyData ne null ) { + keyData.release(allocator) + } + if (valueData ne null) { + valueData.release(allocator) + } keyData = null valueData = null + this.maxSizeReached = false } - protected def handleExisting(mapKeyObject: AnyRef, mapKeyOffset: Long): Int = { + protected def handleExisting(mapKeyObject: AnyRef, mapKeyOffset: Long, + valueStartOffset: Int): Int = { // 0 indicates existing 0 } - protected def handleNew(mapKeyObject: AnyRef, mapKeyOffset: Long): Int = { + protected def handleNew(mapKeyObject: AnyRef, mapKeyOffset: Long, valueStartOffset: Int): Int = { handleNewInsert() // 1 indicates new insert 1 @@ -179,8 +222,11 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, var position = valueDataPosition val dataSize = position - valueData.baseOffset if (position + numBytes + 4 > valueData.endPosition) { - valueData = valueData.resize(numBytes + 4, allocator) + val oldCapacity = valueData.capacity + valueData = valueData.resize(numBytes + 4, allocator, approxMaxCapacity) position = valueData.baseOffset + dataSize + acquireMemory(valueData.capacity - oldCapacity) + _maxMemory += valueData.capacity - oldCapacity } val valueBaseObject = valueData.baseObject // write the key size followed by the full key+value bytes @@ -203,13 +249,15 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, val fixedKeySize = this.fixedKeySize val newCapacity = SharedUtils.checkCapacity(_capacity << 1) val newKeyBuffer = allocator.allocate(newCapacity * fixedKeySize, "HASHMAP") + // clear the key data allocator.clearPostAllocate(newKeyBuffer, 0) val newKeyData = new ByteBufferData(newKeyBuffer, allocator) val newKeyObject = newKeyData.baseObject val newKeyBaseOffset = newKeyData.baseOffset val newMask = newCapacity - 1 - + acquireMemory(newKeyData.capacity - this.keyData.capacity) + _maxMemory += newKeyData.capacity - this.keyData.capacity val keyData = this.keyData val keyObject = keyData.baseObject var keyOffset = keyData.baseOffset @@ -246,6 +294,21 @@ class ByteBufferHashMap(initialCapacity: Int, val loadFactor: Double, keyData.release(allocator) this.keyData = newKeyData } + + private def acquireMemory(required: Long): Unit = { + if (consumer ne null) { + consumer.acquireMemory(required) + } + } + + private def freeMemoryOnTaskCompletion(): Unit = { + taskContext.addTaskCompletionListener { _ => + if (consumer ne null) { + consumer.freeMemory(_maxMemory) + } + this.release() + } + } } final class ByteBufferData private(val buffer: ByteBuffer, @@ -279,7 +342,13 @@ final class ByteBufferData private(val buffer: ByteBuffer, .arrayEquals(baseObject, offset + 4, oBase, oBaseOffset, size) } - def resize(required: Int, allocator: BufferAllocator): ByteBufferData = { + def resize(required: Int, allocator: BufferAllocator, maxCapacity: Int): ByteBufferData = { + val currentUsed = this.buffer.limit; + val currentCapacity = this.buffer.capacity; + if (currentCapacity > maxCapacity || maxCapacity - currentUsed < required ) { + throw new BufferSizeLimitExceededException(currentUsed, required, maxCapacity); + } + // the expand will not full respect max capacity , but that is ok , ... val buffer = allocator.expand(this.buffer, required, "HASHMAP") val baseOffset = allocator.baseOffset(buffer) new ByteBufferData(buffer, allocator.baseObject(buffer), baseOffset, @@ -304,3 +373,13 @@ final class ByteBufferData private(val buffer: ByteBuffer, allocator.release(buffer) } } + +object ByteBufferHashMap { + val bsle = new BufferSizeLimitExceededException("ByteBufferData capacity reached to max") +} + + +final class ByteBufferHashMapMemoryConsumer(taskMemoryManager: TaskMemoryManager) + extends MemoryConsumer(taskMemoryManager) { + override def spill(size: Long, trigger: MemoryConsumer): Long = 0L +} diff --git a/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala b/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala index e457fbd0c7..b89389646d 100644 --- a/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala +++ b/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala @@ -53,12 +53,14 @@ final class DictionaryMap(_initialCapacity: Int, _loadFactor: Double, key.hashCode()) } - override protected def handleExisting(mapKeyObject: AnyRef, mapKeyOffset: Long): Int = { + override protected def handleExisting(mapKeyObject: AnyRef, mapKeyOffset: Long, + valueStartOffset: Int): Int = { // get the dictionary index from the key bytes Platform.getInt(mapKeyObject, mapKeyOffset + 8) } - override protected def handleNew(mapKeyObject: AnyRef, mapKeyOffset: Long): Int = { + override protected def handleNew(mapKeyObject: AnyRef, mapKeyOffset: Long, + valueStartOffset: Int): Int = { val newIndex = size // write the dictionary index in the key bytes Platform.putInt(mapKeyObject, mapKeyOffset + 8, newIndex) diff --git a/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala b/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala new file mode 100644 index 0000000000..daee315ec9 --- /dev/null +++ b/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package io.snappydata.collection + +import com.gemstone.gemfire.internal.cache.GemFireCacheImpl + +final class SHAMap(initialCapacity: Int, valueSize: Int, + maxCapacity: Int) extends ByteBufferHashMap(initialCapacity,0.75, + 0, valueSize, GemFireCacheImpl.getCurrentBufferAllocator, + null,null,0L, maxCapacity) { + + override protected def handleExisting(mapKeyObject: AnyRef, mapKeyOffset: Long, + valueOffset: Int): Int = { + // Get the valueOffSet + // (Platform.getLong(mapKeyObject, mapKeyOffset) >>> 32L).toInt + valueOffset + } + + override protected def handleNew(mapKeyObject: AnyRef, mapKeyOffset: Long, + valueOffset: Int): Int = { + // Read the value start offset before invoking handleNewInsert which may cause rehash + // & make the mayKeyObject & mapKeyOffset invalid + // val valueOffset = (Platform.getLong(mapKeyObject, mapKeyOffset) >>> 32L).toInt + handleNewInsert() + -1 * valueOffset + } +} From 8c4adb1210c57dc8a6ac191d43bc7506816ef6f6 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 10 Aug 2019 04:17:20 +0530 Subject: [PATCH 04/26] Disallow table entry in external hive catalog if present in builtin catalog - use test-log4j.properties for logger of the "master" JVM in dunit tests - update store link --- .../sql/catalog/SnappyExternalCatalog.scala | 11 +++++++++++ .../sql/hive/SnappyHiveExternalCatalog.scala | 6 +++--- .../sql/internal/SnappySessionCatalog.scala | 13 ++++++++++++- .../test/dunit/DistributedTestBase.java | 18 +++++++++++++++--- store | 2 +- 5 files changed, 42 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala b/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala index 71f3d9b8bc..df50e9ec71 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala @@ -341,6 +341,12 @@ object SnappyExternalCatalog { Utils.analysisException(s"Schema or database '$schema' not found") } else Utils.analysisException(s"Schema '$schema' not found") } + + def objectExistsException(tableIdentifier: TableIdentifier, + objectType: CatalogObjectType.Type): AnalysisException = { + Utils.analysisException(s"Object with name '${tableIdentifier.table}' (requested type = " + + s"$objectType) already exists in schema/database '${tableIdentifier.database}'") + } } object CatalogObjectType extends Enumeration { @@ -398,4 +404,9 @@ object CatalogObjectType extends Enumeration { def isPolicy(table: CatalogTable): Boolean = { table.properties.contains(PolicyProperties.policyApplyTo) } + + def isTableOrView(tableType: CatalogObjectType.Type): Boolean = tableType match { + case Index | Policy => false + case _ => true + } } diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala index 24a039b419..eaae0c76ff 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala @@ -381,10 +381,10 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, try { withHiveExceptionHandling(super.createTable(catalogTable, ifExists)) } catch { - case _: TableAlreadyExistsException => + case e: TableAlreadyExistsException => val objectType = CatalogObjectType.getTableType(tableDefinition) - throw new AnalysisException(s"Object '${tableDefinition.identifier.table}' of type " + - s"$objectType already exists in schema '${tableDefinition.database}'") + if (CatalogObjectType.isTableOrView(objectType)) throw e + else throw objectExistsException(tableDefinition.identifier, objectType) } // refresh cache for required tables diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala index f7b6f25a02..ad720b3f19 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala @@ -36,7 +36,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql._ import org.apache.spark.sql.catalog.Column import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExpressionInfo} @@ -558,6 +558,17 @@ class SnappySessionCatalog(val externalCatalog: SnappyExternalCatalog, table.provider match { case Some(DDLUtils.HIVE_PROVIDER) => if (snappySession.enableHiveSupport) { + + // check for existing table else for hive table it could create in both catalogs + if (!ignoreIfExists && super.tableExists(table.identifier)) { + val objectType = CatalogObjectType.getTableType(table) + if (CatalogObjectType.isTableOrView(objectType)) { + throw new TableAlreadyExistsException(db = schemaName, table = tableName) + } else { + throw SnappyExternalCatalog.objectExistsException(table.identifier, objectType) + } + } + // resolve table fully as per current schema in this session hiveSessionCatalog.createTable(resolveCatalogTable(table, schemaName), ignoreIfExists) } else { diff --git a/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java b/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java index 7658550140..2e367ac631 100755 --- a/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java @@ -17,6 +17,8 @@ package io.snappydata.test.dunit; import java.io.File; +import java.io.IOException; +import java.io.InputStream; import java.io.PrintWriter; import java.io.Serializable; import java.io.StringWriter; @@ -28,7 +30,6 @@ import java.lang.reflect.Method; import java.net.Inet4Address; import java.net.InetAddress; -import java.net.UnknownHostException; import java.security.PrivilegedAction; import java.text.DecimalFormat; import java.util.HashMap; @@ -55,6 +56,7 @@ import org.apache.log4j.Level; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; import org.junit.internal.MethodSorter; /** @@ -646,8 +648,18 @@ public static Logger getGlobalLogger() { return (globalLogger = newGlobalLogger()); } - private Logger newLogWriter() { - Logger logger = LogManager.getLogger(getClass()); + private static synchronized Logger newLogWriter() { + Logger logger = LogManager.getLogger("DUnitTest"); + try { + Properties props = new Properties(); + // fallback to defaults + try (InputStream in = DistributedTestBase.class.getResourceAsStream( + "/test-log4j.properties")) { + props.load(in); + } + new PropertyConfigurator().doConfigure(props, logger.getLoggerRepository()); + } catch (IOException ignored) { + } logger.setLevel(getLevel(DUnitLauncher.LOG_LEVEL)); return logger; } diff --git a/store b/store index 9b5b814702..be153871c3 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 9b5b8147027d5684cb3adae0cd800a8699885ce5 +Subproject commit be153871c35ed6e6627323c39d0131a7c46b1e50 From a965cc43acbdfc5b42e80d79416339f577236293 Mon Sep 17 00:00:00 2001 From: Amogh Shetkar Date: Mon, 12 Aug 2019 18:14:45 +0530 Subject: [PATCH 05/26] SNAP-3134: Ensure package is deployed as a package and not as a jar, during cluster startup. (#1422) * Fix for SNAP-3134: During cluster restart, ensure package is deployed as a package and not as a jar. * Also, now allow cluster to come up even if an existing package or jar could not be deployed as a default behavior. * Also, clean up the jar files (deployed via DEPLOY JAR) from executors directory during cluster startup, when this happens. * Rename the work directory created in SnappySinkProviderDUnitTest after it completes so that it does not conflict with the next test which again creates a new work directory. * Updated the test cases. --- .../io/snappydata/ToolsCallbackImpl.scala | 16 +- .../cluster/CassandraSnappyDUnitTest.scala | 154 +++++++++++++----- .../SnappySinkProviderDUnitTest.scala | 4 +- .../org/apache/spark/sql/SnappyContext.scala | 78 ++++++--- .../org/apache/spark/sql/execution/ddl.scala | 14 +- 5 files changed, 193 insertions(+), 73 deletions(-) diff --git a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala index 672f034ed6..333e617317 100644 --- a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala +++ b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala @@ -164,10 +164,22 @@ object ToolsCallbackImpl extends ToolsCallback with Logging { } override def removeURIsFromExecutorClassLoader(jars: Array[String]): Unit = { - if (ExecutorInitiator.snappyExecBackend != null) { + if (!jars.isEmpty && "__REMOVE_FILES_ONLY__" != jars.head + && ExecutorInitiator.snappyExecBackend != null) { val snappyexecutor = ExecutorInitiator.snappyExecBackend.executor.asInstanceOf[SnappyExecutor] snappyexecutor.removeJarsFromExecutorLoader(jars) - } + } else removeJarFiles(jars) + } + + def removeJarFiles(jars: Array[String]): Unit = { + jars.foreach(name => { + val localName = name.split("/").last + val jarFile = new File(SparkFiles.getRootDirectory(), localName) + if (jarFile.exists()) { + jarFile.delete() + logDebug(s"Deleted jarFile $jarFile") + } + }) } override def getAllGlobalCmnds: Array[String] = { diff --git a/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala b/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala index 8e46ee0ccc..a88da0cdad 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala @@ -18,15 +18,17 @@ package io.snappydata.cluster import java.io._ import java.nio.file.{Files, Paths} -import java.sql.{Connection, DriverManager, ResultSet, SQLException} +import java.sql.{Connection, DriverManager, ResultSet, SQLException, Statement} import java.util import scala.language.postfixOps import scala.sys.process._ + import io.snappydata.Constant import io.snappydata.test.dunit.{AvailablePortHelper, DistributedTestBase} import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.{IOFileFilter, TrueFileFilter, WildcardFileFilter} + import org.apache.spark.Logging class CassandraSnappyDUnitTest(val s: String) @@ -39,6 +41,7 @@ class CassandraSnappyDUnitTest(val s: String) override val snappyProductDir = System.getenv("SNAPPY_HOME") val scriptPath = s"$snappyProductDir/../../../cluster/src/test/resources/scripts" + val downloadPath = s"$snappyProductDir/../../../dist" lazy val downloadLoc = { val path = if (System.getenv().containsKey("GRADLE_USER_HOME")) { @@ -56,6 +59,7 @@ class CassandraSnappyDUnitTest(val s: String) var cassandraClusterLoc = "" var cassandraConnectorJarLoc = "" + var sparkXmlJarPath = "" private val commandOutput = "command-output.txt" @@ -84,10 +88,15 @@ class CassandraSnappyDUnitTest(val s: String) Thread.sleep(10000) logInfo("Download Location : " + downloadLoc) + logInfo(s"Creating $downloadPath") + new File(downloadPath).mkdir() + new File(snappyProductDir, "books.xml").createNewFile() + sparkXmlJarPath = downloadURI("https://repo1.maven.org/maven2/com/databricks/" + + "spark-xml_2.11/0.5.0/spark-xml_2.11-0.5.0.jar") val cassandraJarLoc = getLoc(downloadLoc) cassandraConnectorJarLoc = getUserAppJarLocation("spark-cassandra-connector_2.11-2.0.7.jar", downloadLoc) - if(cassandraJarLoc.nonEmpty && cassandraConnectorJarLoc != null) { + if (cassandraJarLoc.nonEmpty && cassandraConnectorJarLoc != null) { cassandraClusterLoc = cassandraJarLoc.head } else { ("curl -OL http://www-us.apache.org/dist/cassandra/" + @@ -139,6 +148,22 @@ class CassandraSnappyDUnitTest(val s: String) res } + private def downloadURI(url: String): String = { + val jarName = url.split("/").last + val jar = new File(downloadPath, jarName) + if (!jar.exists()) { + logInfo(s"Downloading $url ...") + s"curl -OL $url".!! + val cmd = s"find $currDir -name $jarName" + logInfo(s"Executing $cmd") + val tempPath = cmd.lineStream_!.toList + val tempJar = new File(tempPath.head) + assert(tempJar.exists(), s"Did not find $jarName at $tempPath") + assert(tempJar.renameTo(jar), s"Could not move $jarName to $downloadPath") + } + jar.getAbsolutePath + } + protected def getUserAppJarLocation(jarName: String, jarPath: String) = { var userAppJarPath: String = null if (new File(jarName).exists) jarName @@ -196,26 +221,31 @@ class CassandraSnappyDUnitTest(val s: String) count } + private var user1Conn: Connection = null + private var stmt1: Statement = null + def testDeployPackageWithCassandra(): Unit = { (cassandraClusterLoc + s"/bin/cqlsh -f $scriptPath/cassandra_script1").!! - snap_2772BugTest_deployPkg_createExternalTable() - snap_2772BugTest_deployJar_createExternalTable() - snap_2772BugTest_deployJar_snappyJob() - snap_2772BugTest_deployPkg_snappyJob() - snappyJobTest() - externalTableCreateTest() + user1Conn = getConnection(netPort) + stmt1 = user1Conn.createStatement() + doTestDeployPackageWithExternalTable() + doTestDeployJarWithExternalTable() + doTestDeployJarWithSnappyJob() + doTestDeployPackageWithSnappyJob() + doTestPackageViaSnappyJobCommand() + doTestDeployPackageWithExternalTableInSnappyShell() } - def snappyJobTest(): Unit = { - logInfo("Running snappyJobTest") + def doTestPackageViaSnappyJobCommand(): Unit = { + logInfo("Running testPackageViaSnappyJobCommand") submitAndWaitForCompletion("io.snappydata.cluster.jobs.CassandraSnappyConnectionJob" , "--packages com.datastax.spark:spark-cassandra-connector_2.11:2.4.1" + " --conf spark.cassandra.connection.host=localhost") logInfo("Job completed") } - def externalTableCreateTest(): Unit = { - logInfo("Running externalTableCreateTest") + def doTestDeployPackageWithExternalTableInSnappyShell(): Unit = { + logInfo("Running testDeployPackageWithExternalTableInSnappyShell") SnappyShell("CreateExternalTable", Seq(s"connect client 'localhost:$netPort';", "deploy package cassandraJar 'com.datastax.spark:spark-cassandra-connector_2.11:2.0.7';", @@ -229,10 +259,8 @@ class CassandraSnappyDUnitTest(val s: String) "exit;")) } - def snap_2772BugTest_deployPkg_createExternalTable(): Unit = { - logInfo("Running snap_2772BugTest_deployPkg_createExternalTable") - val user1Conn = getConnection(netPort) - val stmt1 = user1Conn.createStatement() + def doTestDeployPackageWithExternalTable(): Unit = { + logInfo("Running testDeployPackageWithExternalTable") stmt1.execute("deploy package cassandraJar " + "'com.datastax.spark:spark-cassandra-connector_2.11:2.0.7'") stmt1.execute("drop table if exists customer2") @@ -266,32 +294,43 @@ class CassandraSnappyDUnitTest(val s: String) stmt1.execute("deploy package cassandraJar " + "'com.datastax.spark:spark-cassandra-connector_2.11:2.0.7'") stmt1.execute("deploy package GoogleGSONAndAvro " + - "'com.google.code.gson:gson:2.8.5,com.databricks:spark-avro_2.11:4.0.0'") + "'com.google.code.gson:gson:2.8.5,com.databricks:spark-avro_2.11:4.0.0' " + + s"path '$snappyProductDir/testdeploypackagepath'") stmt1.execute("deploy package MSSQL 'com.microsoft.sqlserver:sqljdbc4:4.0'" + " repos 'http://clojars.org/repo/'") stmt1.execute("list packages") assert(getCount(stmt1.getResultSet) == 3) + logInfo("Restarting the cluster for " + + "CassandraSnappyDUnitTest.doTestDeployPackageWithExternalTable()") + logInfo((snappyProductDir + "/sbin/snappy-stop-all.sh").!!) + logInfo((snappyProductDir + "/sbin/snappy-start-all.sh").!!) + + user1Conn = getConnection(netPort) + stmt1 = user1Conn.createStatement() + + stmt1.execute("drop table if exists customer2") + stmt1.execute("create external table customer2 using org.apache.spark.sql.cassandra options" + + " (table 'customer', keyspace 'test', spark.cassandra.input.fetch.size_in_rows '200000'," + + " spark.cassandra.read.timeout_ms '10000')") + stmt1.execute("select * from customer2") + assert(getCount(stmt1.getResultSet) == 3) + + stmt1.execute("list packages") + assert(getCount(stmt1.getResultSet) == 3, s"After restart, packages expected 3," + + s" found ${stmt1.getResultSet}") + stmt1.execute("undeploy mssql") stmt1.execute("undeploy cassandrajar") stmt1.execute("undeploy googlegsonandavro") stmt1.execute("list packages") assert(getCount(stmt1.getResultSet) == 0) - - val jarPath = s"$snappyProductDir/jars/hadoop-client-2.7.7.jar" - stmt1.execute(s"""deploy jar avro-v_1.0 '$jarPath'""") - stmt1.execute("list jars") - assert(getCount(stmt1.getResultSet) == 1) - stmt1.execute("undeploy avro-v_1.0 ") - stmt1.execute("list jars") - assert(getCount(stmt1.getResultSet) == 0) } - - def snap_2772BugTest_deployJar_createExternalTable(): Unit = { - logInfo("Running snap_2772BugTest_deployJar_createExternalTable") - val user1Conn = getConnection(netPort) - val stmt1 = user1Conn.createStatement() + + def doTestDeployJarWithExternalTable(): Unit = { + logInfo("Running testDeployJarWithExternalTable") stmt1.execute(s"deploy jar cassJar '$cassandraConnectorJarLoc'") + stmt1.execute(s"deploy jar xmlJar '$sparkXmlJarPath'") stmt1.execute("drop table if exists customer3") stmt1.execute("create external table customer3 using org.apache.spark.sql.cassandra options" + " (table 'customer', keyspace 'test', spark.cassandra.input.fetch.size_in_rows '200000'," + @@ -299,6 +338,33 @@ class CassandraSnappyDUnitTest(val s: String) stmt1.execute("select * from customer3") assert(getCount(stmt1.getResultSet) == 3) + stmt1.execute("list packages") + assert(getCount(stmt1.getResultSet) == 2) + + stmt1.execute("create external table books using com.databricks.spark.xml options" + + s" (path '$snappyProductDir/books.xml')") + + // Move xml jar and verify the deploy fails upon restart. + val xmlPath = new File(sparkXmlJarPath) + val tempXmlPath = new File(s"$xmlPath.bak") + assert(xmlPath.renameTo(tempXmlPath), + s"Could not move ${xmlPath.getName} to ${tempXmlPath.getName}") + + logInfo("Restarting the cluster for " + + "CassandraSnappyDUnitTest.doTestDeployJarWithExternalTable()") + logInfo((snappyProductDir + "/sbin/snappy-stop-all.sh").!!) + logInfo((snappyProductDir + "/sbin/snappy-start-all.sh").!!) + + user1Conn = getConnection(netPort) + stmt1 = user1Conn.createStatement() + stmt1.execute("drop table if exists customer3") + + stmt1.execute("create external table customer4 using org.apache.spark.sql.cassandra options" + + " (table 'customer', keyspace 'test', spark.cassandra.input.fetch.size_in_rows '200000'," + + " spark.cassandra.read.timeout_ms '10000')") + stmt1.execute("select * from customer4") + assert(getCount(stmt1.getResultSet) == 3) + stmt1.execute("list packages") assert(getCount(stmt1.getResultSet) == 1) @@ -306,9 +372,12 @@ class CassandraSnappyDUnitTest(val s: String) stmt1.execute("list packages") assert(getCount(stmt1.getResultSet) == 0) - stmt1.execute("drop table if exists customer3") + assert(tempXmlPath.renameTo(xmlPath), + s"Could not move ${tempXmlPath.getName} to ${xmlPath.getName}") + + stmt1.execute("drop table if exists customer4") try { - stmt1.execute("create external table customer3 using org.apache.spark.sql.cassandra options" + + stmt1.execute("create external table customer5 using org.apache.spark.sql.cassandra options" + " (table 'customer', keyspace 'test', " + "spark.cassandra.input.fetch.size_in_rows '200000'," + " spark.cassandra.read.timeout_ms '10000')") @@ -319,12 +388,19 @@ class CassandraSnappyDUnitTest(val s: String) "data source: org.apache.spark.sql.cassandra") => // expected case t: Throwable => assert(assertion = false, s"Unexpected exception $t") } + + try { + stmt1.execute("create external table books2 using com.databricks.spark.xml options" + + s" (path '$snappyProductDir/books.xml')") + assert(false, "External table on xml should have failed.") + } catch { + case sqle: SQLException if (sqle.getSQLState == "42000") => // expected + case t: Throwable => throw t + } } - def snap_2772BugTest_deployJar_snappyJob(): Unit = { - logInfo("Running snap_2772BugTest_deployJar_snappyJob") - val user1Conn = getConnection(netPort) - val stmt1 = user1Conn.createStatement() + def doTestDeployJarWithSnappyJob(): Unit = { + logInfo("Running testDeployJarWithSnappyJob") stmt1.execute(s"deploy jar cassJar '$cassandraConnectorJarLoc'") stmt1.execute("drop table if exists customer") submitAndWaitForCompletion("io.snappydata.cluster.jobs.CassandraSnappyConnectionJob" , @@ -351,10 +427,8 @@ class CassandraSnappyDUnitTest(val s: String) } } - def snap_2772BugTest_deployPkg_snappyJob(): Unit = { - logInfo("Running snap_2772BugTest_deployPkg_snappyJob") - val user1Conn = getConnection(netPort) - val stmt1 = user1Conn.createStatement() + def doTestDeployPackageWithSnappyJob(): Unit = { + logInfo("Running testDeployPackageWithSnappyJob") stmt1.execute("deploy package cassandraJar " + "'com.datastax.spark:spark-cassandra-connector_2.11:2.0.7'") stmt1.execute("drop table if exists customer") diff --git a/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala b/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala index 15c47a4d8d..532684a471 100644 --- a/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala +++ b/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala @@ -149,6 +149,8 @@ class SnappySinkProviderDUnitTest(s: String) Files.deleteIfExists(Paths.get(snappyProductDir, "conf", "locators")) Files.deleteIfExists(Paths.get(snappyProductDir, "conf", "leads")) Files.deleteIfExists(Paths.get(snappyProductDir, "conf", "servers")) + Files.move(Paths.get(snappyProductDir, "work"), Paths.get(snappyProductDir, + "work-SnappySinkProviderDUnitTest")) } private val snappyProductDir = getEnvironmentVariable("SNAPPY_HOME") @@ -491,4 +493,4 @@ object SnappySinkProviderDUnitTest extends Logging { } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala index 1140d5630c..8862e18300 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.lang +import java.util.Map.Entry import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.function.Consumer import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} @@ -32,6 +36,7 @@ import com.gemstone.gemfire.distributed.internal.MembershipListener import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember import com.pivotal.gemfirexd.Attribute import com.pivotal.gemfirexd.internal.engine.Misc +import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils import com.pivotal.gemfirexd.internal.shared.common.SharedUtils import io.snappydata.sql.catalog.{CatalogObjectType, ConnectorExternalCatalog} import io.snappydata.util.ServiceUtils @@ -48,9 +53,9 @@ import org.apache.spark.sql.catalyst.expressions.SortDirection import org.apache.spark.sql.collection.{ToolsCallbackInit, Utils} import org.apache.spark.sql.execution.columnar.ExternalStoreUtils.CaseInsensitiveMutableHashMap import org.apache.spark.sql.execution.joins.HashedObjectCache -import org.apache.spark.sql.execution.{ConnectionPool, DeployCommand, DeployJarCommand} +import org.apache.spark.sql.execution.{ConnectionPool, DeployCommand, DeployJarCommand, RefreshMetadata} import org.apache.spark.sql.hive.{HiveExternalCatalog, SnappyHiveExternalCatalog, SnappySessionState} -import org.apache.spark.sql.internal.{SharedState, SnappySharedState, StaticSQLConf} +import org.apache.spark.sql.internal.{ContextJarUtils, SharedState, SnappySharedState, StaticSQLConf} import org.apache.spark.sql.store.CodeGeneration import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{StructField, StructType} @@ -1097,24 +1102,39 @@ object SnappyContext extends Logging { SnappyContext.getClusterMode(sc) match { case _: SnappyEmbeddedMode => val deployCmds = ToolsCallbackInit.toolsCallback.getAllGlobalCmnds - val nonEmpty = deployCmds.length > 0 - if (nonEmpty) { - logInfo(s"deploycmnds size = ${deployCmds.length}") - deployCmds.foreach(s => logDebug(s"$s")) + if (deployCmds.length > 0) { + logInfo(s"Deployed commands size = ${deployCmds.length}") + val commandSet = ToolsCallbackInit.toolsCallback.getGlobalCmndsSet + commandSet.forEach(new Consumer[Entry[String, String]] { + override def accept(t: Entry[String, String]): Unit = { + if (!t.getKey.startsWith(ContextJarUtils.functionKeyPrefix)) { + val d = t.getValue + val cmdFields = d.split('|') // split() removes empty elements + if (d.contains('|')) { + val coordinate = cmdFields(0) + val repos = if (cmdFields.length > 1 && !cmdFields(1).isEmpty) { + Some(cmdFields(1)) + } else None + val cache = if (cmdFields.length > 2 && !cmdFields(2).isEmpty) { + Some(cmdFields(2)) + } else None + try { + DeployCommand(coordinate, null, repos, cache, restart = true) + .run(session) + } catch { + case e: Throwable => failOnJarUnavailability(t.getKey, Array.empty, e) + } + } else { // Jars we have + try { + DeployJarCommand(null, cmdFields(0), restart = true).run(session) + } catch { + case e: Throwable => failOnJarUnavailability(t.getKey, cmdFields, e) + } + } + } + } + }) } - if (nonEmpty) deployCmds.foreach(d => { - val cmdFields = d.split('|') - if (cmdFields.length > 1) { - val coordinate = cmdFields(0) - val repos = if (cmdFields(1).isEmpty) None else Some(cmdFields(1)) - val cache = if (cmdFields(2).isEmpty) None else Some(cmdFields(2)) - DeployCommand(coordinate, null, repos, cache, restart = true).run(session) - } - else { - // Jars we have - DeployJarCommand(null, cmdFields(0), restart = true).run(session) - } - }) case _ => // Nothing } } @@ -1124,6 +1144,26 @@ object SnappyContext extends Logging { } } + private def failOnJarUnavailability(k: String, jars: Array[String], e: Throwable): Unit = { + GemFireXDUtils.waitForNodeInitialization() + Misc.getMemStore.getGlobalCmdRgn.remove(k) + if (!jars.isEmpty) { + val mutable = new ArrayBuffer[String]() + mutable += "__REMOVE_FILES_ONLY__" + jars.foreach(e => mutable += e) + try { + RefreshMetadata.executeOnAll(globalSparkContext, + RefreshMetadata.REMOVE_URIS_FROM_CLASSLOADER, mutable.toArray) + } catch { + case e: Throwable => logWarning(s"Could not delete jar files of '$k'. You may need to" + + s" delete those manually.", e) + } + } + if (lang.Boolean.parseBoolean(System.getProperty("FAIL_ON_JAR_UNAVAILABILITY", "false"))) { + throw e + } + } + private[sql] def sharedState(sc: SparkContext): SnappySharedState = { var state = _sharedState if ((state ne null) && (state.sparkContext eq sc)) state diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala index 204e4bfb38..94a693e0d6 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala @@ -514,13 +514,8 @@ case class DeployCommand( if (restart) { logWarning(s"Following mvn coordinate" + s" could not be resolved during restart: $coordinates", ex) - if (lang.Boolean.parseBoolean(System.getProperty("FAIL_ON_JAR_UNAVAILABILITY", "true"))) { - throw ex - } - Nil - } else { - throw ex } + throw ex } } } @@ -537,14 +532,11 @@ case class DeployJarCommand( if (unavailableUris.nonEmpty) { logWarning(s"Following jars are unavailable" + s" for deployment during restart: ${unavailableUris.deep.mkString(",")}") - if (restart && lang.Boolean.parseBoolean( - System.getProperty("FAIL_ON_JAR_UNAVAILABILITY", "true"))) { + if (restart) { throw new IllegalStateException( s"Could not find deployed jars: ${unavailableUris.mkString(",")}") } - if (!restart) { - throw new IllegalArgumentException(s"jars not readable: ${unavailableUris.mkString(",")}") - } + throw new IllegalArgumentException(s"jars not readable: ${unavailableUris.mkString(",")}") } val sc = sparkSession.sparkContext val uris = availableUris.map(j => sc.env.rpcEnv.fileServer.addFile(new File(j))) From 3b322ffef220638add8a0fcea1ae8068aa2d4f61 Mon Sep 17 00:00:00 2001 From: ahshahid Date: Mon, 12 Aug 2019 19:57:41 -0700 Subject: [PATCH 06/26] Fix for SNAP-3132. The size calculation for DecimalType in a special case of optimization was incorrect (#1426) --- .../execution/aggregate/SnappyHashAggregateExec.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index c89c2320df..36d851747a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -809,18 +809,12 @@ case class SnappyHashAggregateExec( val keysToProcessSize = this.groupingAttributes.drop(numToDrop) val suffixSize = if (numBytesForNullKeyBits == 0) { keysToProcessSize.foldLeft(0) { - case (size, attrib) => size + attrib.dataType.defaultSize + (attrib.dataType match { - case dec: DecimalType if (dec.precision > Decimal.MAX_LONG_DIGITS) => 1 - case _ => 0 - }) + case (size, attrib) => size + attrib.dataType.defaultSize }.toString } else { keysToProcessSize.zipWithIndex.map { case(attrib, i) => { - val sizeTerm = attrib.dataType.defaultSize + (attrib.dataType match { - case dec: DecimalType if (dec.precision > Decimal.MAX_LONG_DIGITS) => 1 - case _ => 0 - }) + val sizeTerm = attrib.dataType.defaultSize s"""(int)(${SHAMapAccessor.getExpressionForNullEvalFromMask(i + numToDrop, numBytesForNullKeyBits, nullKeysBitsetTerm)} ? 0 : $sizeTerm) """ From a0114d1e77abc67265a875f89ca76c1c631a1c06 Mon Sep 17 00:00:00 2001 From: ahshahid Date: Tue, 13 Aug 2019 23:30:46 -0700 Subject: [PATCH 07/26] Tests enhancements (#1429) * enabled flag to use optimized byte buffer map for cases with single string group by column * Modified the tests to use flag Disable Codegenfallback as true --- .../PreparedQueryRoutingSingleNodeSuite.scala | 12 +++++++++++- .../apache/spark/sql/SQLFunctionsTestSuite.scala | 13 ++++++++++++- .../sql/kafka010/SnappyStructuredKafkaSuite.scala | 9 ++++++++- .../spark/sql/store/ColumnUpdateDeleteTest.scala | 10 ++++++++++ .../apache/spark/sql/store/SHAByteBufferTest.scala | 8 ++++++++ .../spark/sql/test/SharedSnappySessionContext.scala | 12 +++++++++++- .../scala/io/snappydata/core/LocalTestData.scala | 9 +++++++++ 7 files changed, 69 insertions(+), 4 deletions(-) diff --git a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala index 799d72bae1..14898876e4 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala @@ -38,8 +38,18 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA * Change of 'n' will influence results if they are dependent on weights - derived * from hidden column in sample table. */ + /** + * Pls do not change the flag values of Property.TestDisableCodeGenFlag.name + * and Property.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ new org.apache.spark.SparkConf().setAppName("PreparedQueryRoutingSingleNodeSuite") - .setMaster("local[6]") + .setMaster("local[6]"). + set(io.snappydata.Property.TestDisableCodeGenFlag.name, "true"). + set(io.snappydata.Property.UseOptimizedHashAggregateForSingleKey.name, "true") // .set("spark.logConf", "true") // .set("mcast-port", "4958") } diff --git a/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala b/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala index 0748b68d6d..abe80a4603 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala @@ -41,7 +41,18 @@ class SQLFunctionsTestSuite extends SnappyFunSuite // scalastyle:off println - val sparkSession = SparkSession.builder().master("local[*]").getOrCreate() + /** + * Pls do not change the flag values of Property.TestDisableCodeGenFlag.name + * and Property.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ + val sparkSession = SparkSession.builder(). + config(io.snappydata.Property.TestDisableCodeGenFlag.name, "true"). + config(io.snappydata.Property.UseOptimizedHashAggregateForSingleKey.name, "true"). + master("local[*]").getOrCreate() // snc.sql("set snappydata.sql.tokenize=true") // snc.sql("set snappydata.sql.planCaching=true") diff --git a/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala b/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala index 27754afb50..0615195772 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.kafka010 import java.util.concurrent.atomic.AtomicInteger -import io.snappydata.SnappyFunSuite +import io.snappydata.{Property, SnappyFunSuite} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.Eventually import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.apache.spark.SparkConf import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.functions.{count, window} @@ -38,6 +39,12 @@ class SnappyStructuredKafkaSuite extends SnappyFunSuite with Eventually private var kafkaTestUtils: KafkaTestUtils = _ + protected override def newSparkConf(addOn: (SparkConf) => SparkConf): SparkConf = { + super.newSparkConf((conf: SparkConf) => { + // conf.set(Property.TestDisableCodeGenFlag.name , "false") + conf + }) + } override def beforeAll() { super.beforeAll() kafkaTestUtils = new KafkaTestUtils diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala index 27ec9a649f..a563809310 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala @@ -41,6 +41,14 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase { } override protected def newSparkConf(addOn: SparkConf => SparkConf): SparkConf = { + /** + * Pls do not change the flag values of Property.TestDisableCodeGenFlag.name + * and Property.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ val conf = new SparkConf() conf.setIfMissing("spark.master", "local[*]") .setAppName(getClass.getName) @@ -51,6 +59,8 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase { conf.set("spark.memory.manager", classOf[SnappyUnifiedMemoryManager].getName) conf.set("spark.serializer", "org.apache.spark.serializer.PooledKryoSerializer") conf.set("spark.closure.serializer", "org.apache.spark.serializer.PooledKryoSerializer") + conf.set(io.snappydata.Property.TestDisableCodeGenFlag.name, "true") + conf.set(io.snappydata.Property.UseOptimizedHashAggregateForSingleKey.name, "true") conf } diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala index b2f36e642b..ae2ede78db 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala @@ -47,6 +47,14 @@ class SHAByteBufferTest extends SnappyFunSuite with BeforeAndAfterAll { protected override def newSparkConf(addOn: (SparkConf) => SparkConf): SparkConf = { + /** + * Pls do not change the flag values of Property.TestDisableCodeGenFlag.name + * and Property.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ System.setProperty("spark.testing", "true") super.newSparkConf((conf: SparkConf) => { conf.set("spark.sql.codegen.maxFields", "110") diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala index 960867a2b2..ee20eee69f 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala @@ -30,10 +30,20 @@ trait SharedSnappySessionContext extends SharedSQLContext { protected def codegenFallback: Boolean = false override protected def createSparkSession: SnappySession = { + /** + * Pls do not change the flag values of snappydaya.sql.TestDisableCodeGenFlag + * and snappydaya.sql.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ val session = new TestSnappySession(sparkConf .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) .set("spark.sql.codegen.fallback", codegenFallback.toString) - .set("snappydata.sql.planCaching.", random.nextBoolean().toString)) + .set("snappydata.sql.planCaching.", random.nextBoolean().toString) + .set("snappydata.sql.disableCodegenFallback", "true") + .set("snappydata.sql.useOptimizedHashAggregateForSingleKey", "true")) session.setCurrentSchema("default") session } diff --git a/core/src/test/scala/io/snappydata/core/LocalTestData.scala b/core/src/test/scala/io/snappydata/core/LocalTestData.scala index 99f4d66992..93863ee380 100644 --- a/core/src/test/scala/io/snappydata/core/LocalTestData.scala +++ b/core/src/test/scala/io/snappydata/core/LocalTestData.scala @@ -83,11 +83,20 @@ object LocalSparkConf { private val random = new Random() def newConf(addOn: (SparkConf) => SparkConf = null): SparkConf = { + /** + * Pls do not change the flag values of Property.TestDisableCodeGenFlag.name + * and Property.UseOptimizedHashAggregateForSingleKey.name + * They are meant to suppress CodegenFallback Plan so that optimized + * byte buffer code path is tested & prevented from false passing. + * If your test needs CodegenFallback, then override the newConf function + * & clear the flag from the conf of the test locally. + */ val conf = new SparkConf() .setIfMissing("spark.master", "local[4]") .setIfMissing("spark.memory.debugFill", "true") .set("snappydata.sql.planCaching", random.nextBoolean().toString) .set(Property.TestDisableCodeGenFlag.name , "true") + .set(Property.UseOptimizedHashAggregateForSingleKey.name, "true") .setAppName(getClass.getName) if (addOn != null) { addOn(conf) From 1c9df64f6c356869647e7024d61ada2b95d15260 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 14 Aug 2019 15:11:00 +0530 Subject: [PATCH 08/26] [SNAP-3137] use "SampleTableScan" in GUI for sample table scan (#1430) - change nodeName to "SampleTableScan" instead of ColumnTableScan for sample table scans - also fixed CTAS for sample table to show up in GUI by posting separate GUI plan for the same --- .../org/apache/spark/sql/SnappySession.scala | 37 +++++++++++++------ .../execution/columnar/ColumnTableScan.scala | 5 ++- .../org/apache/spark/sql/execution/ddl.scala | 3 +- .../apache/spark/sql/internal/session.scala | 7 +++- 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index 253322db48..b8db3a82d6 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -2106,6 +2106,17 @@ object SnappySession extends Logging { } } + private def handleCTAS(tableType: CatalogObjectType.Type): (Boolean, Boolean) = { + if (CatalogObjectType.isTableBackedByRegion(tableType)) { + if (tableType == CatalogObjectType.Sample) false -> true else true -> false + } + // most CTAS writers (FileFormatWriter, KafkaWriter) post their own plans for insert + // but it does not include SQL string or the top-level plan for ExecuteCommand + // so post the GUI plans but evaluate the toRdd outside else it will be nested + // withNewExecutionId() that will fail + else true -> true + } + private def evaluatePlan(qe: QueryExecution, session: SnappySession, sqlShortText: String, sqlText: String, paramLiterals: Array[ParamLiteral], paramsId: Int): CachedDataFrame = { val (executedPlan, withFallback) = getExecutedPlan(qe.executedPlan) @@ -2129,18 +2140,20 @@ object SnappySession extends Logging { val executionStr = replaceParamLiterals(origExecutionStr, paramLiterals, paramsId) val planInfo = PartitionedPhysicalScan.updatePlanInfo(origPlanInfo, paramLiterals, paramsId) - // different Command types will post their own plans in toRdd evaluation - val isCommand = executedPlan.isInstanceOf[ExecutedCommandExec] - var rdd = if (isCommand) qe.toRdd else null // don't post separate plan for CTAS since it already has posted one for the insert - val postGUIPlans = if (isCommand) executedPlan.asInstanceOf[ExecutedCommandExec].cmd match { - case c: CreateTableUsingCommand if c.query.isDefined && CatalogObjectType - .isTableBackedByRegion(SnappyContext.getProviderType(c.provider)) => false - case c: CreateDataSourceTableAsSelectCommand if CatalogObjectType.isTableBackedByRegion( - CatalogObjectType.getTableType(c.table)) => false - case _: SnappyCacheTableCommand => false - case _ => true - } else true + val (eagerToRDD, postGUIPlans) = executedPlan match { + case ExecutedCommandExec(c: CreateTableUsingCommand) if c.query.isDefined => + handleCTAS(SnappyContext.getProviderType(c.provider)) + case ExecutedCommandExec(c: CreateDataSourceTableAsSelectCommand) => + handleCTAS(CatalogObjectType.getTableType(c.table)) + case ExecutedCommandExec(c: SnappyCacheTableCommand) if !c.isLazy => true -> false + // other commands may have their own withNewExecutionId but still post GUI + // plans to see the command with proper SQL string in the GUI + case _: ExecutedCommandExec => true -> true + case _ => false -> true + } + var rdd = if (eagerToRDD) qe.toRdd else null + // post final execution immediately (collect for these plans will post nothing) CachedDataFrame.withNewExecutionId(session, sqlShortText, sqlText, executionStr, planInfo, postGUIPlans = postGUIPlans) { @@ -2149,7 +2162,7 @@ object SnappySession extends Logging { // around the LogicalPlan and not the executedPlan; it works for plans using // ExecutedCommandExec though because Spark layer has special check for it in // Dataset hasSideEffects) - if (!isCommand) rdd = qe.toRdd + if (rdd eq null) rdd = qe.toRdd val newPlan = LogicalRDD(qe.analyzed.output, rdd)(session) val execution = session.sessionState.executePlan(newPlan) (null, execution, origExecutionStr, origPlanInfo, executionStr, planInfo, diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala index c44dfe0ec4..b86ba85491 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala @@ -81,7 +81,10 @@ private[sql] final case class ColumnTableScan( partitionColumns, partitionColumnAliases, baseRelation.asInstanceOf[BaseRelation]) with CodegenSupport { - override val nodeName: String = "ColumnTableScan" + override val nodeName: String = { + if (baseRelation != null && baseRelation.getClass.getName.contains("Sampl")) "SampleTableScan" + else "ColumnTableScan" + } override def sameResult(plan: SparkPlan): Boolean = plan match { case r: ColumnTableScan => r.baseRelation.table == baseRelation.table && diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala index 94a693e0d6..2355f75724 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala @@ -345,7 +345,8 @@ case class SnappyCacheTableCommand(tableIdent: TableIdentifier, queryString: Str // Dummy op to materialize the cache. This does the minimal job of count on // the actual cached data (RDD[CachedBatch]) to force materialization of cache // while avoiding creation of any new SparkPlan. - (memoryPlan.cachedColumnBuffers.count(), System.nanoTime() - start) + val count = memoryPlan.cachedColumnBuffers.count() + (count, System.nanoTime() - start) }))._1) :: Nil } finally { if (previousJobDescription ne null) { diff --git a/core/src/main/scala/org/apache/spark/sql/internal/session.scala b/core/src/main/scala/org/apache/spark/sql/internal/session.scala index 28825f2770..1c526c4217 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/session.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/session.scala @@ -111,6 +111,7 @@ class SnappyConf(@transient val session: SnappySession) Property.TestExplodeComplexDataTypeInSHA.name => session.clearPlanCache() key + case SQLConf.SHUFFLE_PARTITIONS.key => // stop dynamic determination of shuffle partitions if (doSet) { @@ -121,6 +122,7 @@ class SnappyConf(@transient val session: SnappySession) } session.clearPlanCache() key + case Property.SchedulerPool.name => schedulerPool = value match { case None => Property.SchedulerPool.defaultValue.get @@ -140,10 +142,11 @@ class SnappyConf(@transient val session: SnappySession) case Property.PlanCaching.name => value match { case Some(boolVal) => - if (boolVal.toString.toBoolean) { + val b = boolVal.toString.toBoolean + if (b) { session.clearPlanCache() } - session.planCaching = boolVal.toString.toBoolean + session.planCaching = b case None => session.planCaching = Property.PlanCaching.defaultValue.get } key From e18647f4d1f10484712c41b9ec4ffdaeffac86d6 Mon Sep 17 00:00:00 2001 From: vatsal mevada Date: Wed, 14 Aug 2019 15:11:33 +0530 Subject: [PATCH 09/26] [SNAP-3071] Streaming examples for CSV and JSON (#1427) Added structured streaming examples for the following cases: - JSON file source - JSON data from Kafka source - CSV file source - CSV data from Kafka source --- .../examples/JavaAirlineDataJob.java | 17 ++ .../JavaCreateAndLoadAirlineDataJob.java | 17 ++ .../examples/JavaTwitterPopularTagsJob.java | 17 ++ .../src/main/resources/people.csv/people1.csv | 2 + .../src/main/resources/people.csv/people2.csv | 3 + .../main/resources/people.json/people1.json | 2 + .../main/resources/people.json/people2.json | 3 + .../CDCExample.scala} | 37 ++-- .../CSVFileSourceExampleWithSnappySink.scala | 104 ++++++++++++ .../CSVKafkaSourceExampleWithSnappySink.scala | 127 ++++++++++++++ .../JSONFileSourceExampleWithSnappySink.scala | 159 ++++++++++++++++++ ...JSONKafkaSourceExampleWithSnappySink.scala | 122 ++++++++++++++ .../SocketSourceExample.scala} | 46 +++-- .../SocketSourceExampleWithSnappySink.scala} | 47 +++--- 14 files changed, 633 insertions(+), 70 deletions(-) create mode 100644 examples/src/main/resources/people.csv/people1.csv create mode 100644 examples/src/main/resources/people.csv/people2.csv create mode 100644 examples/src/main/resources/people.json/people1.json create mode 100644 examples/src/main/resources/people.json/people2.json rename examples/src/main/scala/org/apache/spark/examples/snappydata/{StructuredStreamingCDCExample.scala => structuredstreaming/CDCExample.scala} (83%) create mode 100644 examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVFileSourceExampleWithSnappySink.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVKafkaSourceExampleWithSnappySink.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONFileSourceExampleWithSnappySink.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONKafkaSourceExampleWithSnappySink.scala rename examples/src/main/scala/org/apache/spark/examples/snappydata/{StructuredStreamingExample.scala => structuredstreaming/SocketSourceExample.scala} (77%) rename examples/src/main/scala/org/apache/spark/examples/snappydata/{StructuredStreamingWithSnappySink.scala => structuredstreaming/SocketSourceExampleWithSnappySink.scala} (78%) diff --git a/examples/src/main/java/io/snappydata/examples/JavaAirlineDataJob.java b/examples/src/main/java/io/snappydata/examples/JavaAirlineDataJob.java index 8851e5eb23..87bbefe6dc 100644 --- a/examples/src/main/java/io/snappydata/examples/JavaAirlineDataJob.java +++ b/examples/src/main/java/io/snappydata/examples/JavaAirlineDataJob.java @@ -1,3 +1,20 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ + package io.snappydata.examples; import java.util.Collections; diff --git a/examples/src/main/java/io/snappydata/examples/JavaCreateAndLoadAirlineDataJob.java b/examples/src/main/java/io/snappydata/examples/JavaCreateAndLoadAirlineDataJob.java index 9ca3b615a3..2b97400947 100644 --- a/examples/src/main/java/io/snappydata/examples/JavaCreateAndLoadAirlineDataJob.java +++ b/examples/src/main/java/io/snappydata/examples/JavaCreateAndLoadAirlineDataJob.java @@ -1,3 +1,20 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ + package io.snappydata.examples; import java.io.File; diff --git a/examples/src/main/java/io/snappydata/examples/JavaTwitterPopularTagsJob.java b/examples/src/main/java/io/snappydata/examples/JavaTwitterPopularTagsJob.java index 33d53fb924..56373042b3 100644 --- a/examples/src/main/java/io/snappydata/examples/JavaTwitterPopularTagsJob.java +++ b/examples/src/main/java/io/snappydata/examples/JavaTwitterPopularTagsJob.java @@ -1,3 +1,20 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ + package io.snappydata.examples; import java.io.IOException; diff --git a/examples/src/main/resources/people.csv/people1.csv b/examples/src/main/resources/people.csv/people1.csv new file mode 100644 index 0000000000..d0ea2a0213 --- /dev/null +++ b/examples/src/main/resources/people.csv/people1.csv @@ -0,0 +1,2 @@ +Yin,31,Columbus,Ohio +Michael,38,"San Jose",California \ No newline at end of file diff --git a/examples/src/main/resources/people.csv/people2.csv b/examples/src/main/resources/people.csv/people2.csv new file mode 100644 index 0000000000..71d1415cec --- /dev/null +++ b/examples/src/main/resources/people.csv/people2.csv @@ -0,0 +1,3 @@ +Adam,21,Columbus,Ohio +John,44,"San Diego",California +Carol,37,"San Diego",California \ No newline at end of file diff --git a/examples/src/main/resources/people.json/people1.json b/examples/src/main/resources/people.json/people1.json new file mode 100644 index 0000000000..6422e81ba3 --- /dev/null +++ b/examples/src/main/resources/people.json/people1.json @@ -0,0 +1,2 @@ +{"name":"Yin", "age":31, "address":{"city":"Columbus","state":"Ohio", "district" :"Cincinnati"}} +{"name":"Michael", "age":38, "address":{"city":null, "state":"California", "lane" :"15"}} \ No newline at end of file diff --git a/examples/src/main/resources/people.json/people2.json b/examples/src/main/resources/people.json/people2.json new file mode 100644 index 0000000000..970769e875 --- /dev/null +++ b/examples/src/main/resources/people.json/people2.json @@ -0,0 +1,3 @@ +{"name":"Adam", "age":21, "address":{"city":"Columbus","state":"Ohio"}} +{"name":"John", "age":44, "address":{"city":"San Diego", "state":"California"}} +{"name":"Carol", "age":37, "address":{"city":"San Diego", "state":"California"}} \ No newline at end of file diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingCDCExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala similarity index 83% rename from examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingCDCExample.scala rename to examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala index e7de9a5549..2431516e6e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingCDCExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -15,7 +15,7 @@ * LICENSE file. */ -package org.apache.spark.examples.snappydata +package org.apache.spark.examples.snappydata.structuredstreaming import org.apache.log4j.{Level, Logger} @@ -25,36 +25,35 @@ import scala.language.postfixOps import scala.reflect.io.Path /** - * An example showing CDC usage with SnappyData structured streaming + * An example showing CDC usage with SnappyData streaming sink. * - *

- * To run the example in local mode go to your SnappyData product distribution - * directory and type following command on the command prompt - *
- * bin/run-example snappydata.StructuredStreamingCDCExample
- * 
- *

* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999` - *

- * Sample input data: - * {{{ + * + * Example input data: + * * 1,user1,23,0 * 2,user2,45,0 * 1,user1,23,2 * 2,user2,46,1 - * }}} + * + * To run the example in local mode go to your SnappyData product distribution + * directory and execute the following command: + *

+ * `bin/run-example snappydata.structuredstreaming.CDCExample` + *

* */ -object StructuredStreamingCDCExample{ +// scalastyle:off println +object CDCExample{ def main(args: Array[String]) { // reducing the log level to minimize the messages on console Logger.getLogger("org").setLevel(Level.ERROR) Logger.getLogger("akka").setLevel(Level.ERROR) - println("Initializing a SnappySesion") - val checkpointDirectory = "/tmp/StructuredStreamingCDCExample" + println("Initializing a SnappySession") + val checkpointDirectory = this.getClass.getSimpleName val spark: SparkSession = SparkSession .builder() .appName(getClass.getSimpleName) @@ -65,7 +64,8 @@ object StructuredStreamingCDCExample{ val snappy = new SnappySession(spark.sparkContext) - snappy.sql("create table users (id long , name varchar(40), age int) using column options(key_columns 'id')") + snappy.sql("create table users (id long , name varchar(40), age int) using" + + " column options(key_columns 'id')") // Create DataFrame representing the stream of input lines from connection to host:port val socketDF = snappy @@ -107,4 +107,3 @@ object StructuredStreamingCDCExample{ } case class User(is: Long, name: String, age: Int, _eventType: Int) - diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVFileSourceExampleWithSnappySink.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVFileSourceExampleWithSnappySink.scala new file mode 100644 index 0000000000..a99bf722fd --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVFileSourceExampleWithSnappySink.scala @@ -0,0 +1,104 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.examples.snappydata.structuredstreaming + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.streaming.ProcessingTime +import org.apache.spark.sql.{SnappySession, SparkSession} + +/** + * An example of structured streaming depicting CSV file processing with Snappy sink. + * + * Example input data: + * + * Yin,31,Columbus,Ohio + * Michael,38,"San Jose",California + * + * Usage: CSVFileSourceExampleWithSnappySink [checkpoint-directory] [input-directory] + * + * [checkpoint-directory] Optional argument providing checkpoint directory where the + * state of the steaming query will be stored. Note that this + * directory needs to be deleted manually to reset the state + * of the streaming query. + * Default: `CSVFileSourceExampleWithSnappySink` directory + * in working directory. + * [input-directory] Optional argument pointing to input directory path where incoming + * CSV files should be dumped to get picked up for processing. + * Default: `people.csv` directory under resources + * + * Example: + * $ bin/run-example snappydata.structuredstreaming.CSVFileSourceExampleWithSnappySink \ + * "checkpoint_dir" "CSV_input_dir" + */ +// scalastyle:off println +object CSVFileSourceExampleWithSnappySink extends Logging { + + def main(args: Array[String]) { + // reducing the log level to minimize the messages on console + Logger.getLogger("org").setLevel(Level.ERROR) + Logger.getLogger("akka").setLevel(Level.ERROR) + + val checkpointDirectory = if (args.length >= 1) args(0) + else getClass.getSimpleName + val inputDirectory = if (args.length >= 2) args(1) + else "quickstart/src/main/resources/people.csv" + println("Initializing SnappySession ...") + val spark: SparkSession = SparkSession + .builder() + .appName(getClass.getSimpleName) + .master("local[*]") + .getOrCreate() + val snappy = new SnappySession(spark.sparkContext) + println("Initializing SnappySession ... Done.") + + try { + snappy.sql("create table people (name string , age int," + + " city string, state string)") + + val schema = snappy.read.csv(inputDirectory).schema + // Create DataFrame representing the stream of CSV records + val df = snappy.readStream + .option("maxFilesPerTrigger", 1) // controls number of files to be processed per batch + .schema(schema) + .csv(inputDirectory) + + val streamingQuery = df + .writeStream + .format("snappysink") + .queryName(getClass.getSimpleName) // must be unique across a snappydata cluster + .trigger(ProcessingTime("1 seconds")) // streaming query trigger interval + .option("tableName", "people") // name of the target table + .option("checkpointLocation", checkpointDirectory) + .start() + + println("Streaming started.") + // Following line will make streaming query terminate after 15 seconds. + // This can be replaced by streamingQuery.awaitTermination() to keep the streaming query + // running. + streamingQuery.awaitTermination(15000) + + println("Data loaded in table:") + snappy.sql("select * from people").show() + } finally { + snappy.sql("drop table if exists people") + } + println("Exiting") + System.exit(0) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVKafkaSourceExampleWithSnappySink.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVKafkaSourceExampleWithSnappySink.scala new file mode 100644 index 0000000000..ce25575bd2 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CSVKafkaSourceExampleWithSnappySink.scala @@ -0,0 +1,127 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.examples.snappydata.structuredstreaming + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.streaming.ProcessingTime +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.{SnappySession, SparkSession} + +/** + * An example of structured streaming depicting processing of JSON coming from kafka source + * using snappy sink. + * + * Example input data: + * + * Key: USA, Value: Yin,31,Columbus,Ohio + * Key: USA, Value: John,44,"San Jose",California + * + * Usage: CSVKafkaSourceExampleWithSnappySink [checkpoint-directory] + * + * Compulsory argument providing comma separate list of kafka brokers + * Compulsory argument providing comma separated list of kafka topics to subscribe + * [checkpoint-directory] Optional argument providing checkpoint directory where the state of + * the steaming query will be stored. Note that this directory needs to + * be deleted manually to reset the state of the streaming query. + * Default: `CSVKafkaSourceExampleWithSnappySink` directory under + * working directory. + * + * Example: + * $ bin/run-example snappydata.structuredstreaming.CSVKafkaSourceExampleWithSnappySink \ + * "broker-1:9092,broker-2:9092" "topic1,topic2" "checkpoint_dir" + */ +// scalastyle:off println +object CSVKafkaSourceExampleWithSnappySink extends Logging { + + def main(args: Array[String]) { + + if (args.length < 2) { + println("Usage: CSVKafkaSourceExampleWithSnappySink " + + " [checkpoint-directory]") + System.exit(1) + } + + // reducing the log level to minimize the messages on console + Logger.getLogger("org").setLevel(Level.ERROR) + Logger.getLogger("akka").setLevel(Level.ERROR) + + val checkpointDirectory = if (args.length >= 3) args(2) else this.getClass.getSimpleName + println("Initializing SnappySession ... ") + val spark: SparkSession = SparkSession + .builder() + .appName(this.getClass.getSimpleName) + .master("local[*]") + .getOrCreate() + val snappy = new SnappySession(spark.sparkContext) + println("Initializing SnappySession ... Done.") + try { + + snappy.sql("create table people(name string , age int," + + "city string, state string, country string)") + + val keySchema = StructType(Seq(StructField("country", StringType, nullable = false))) + val addressSchema = StructType(Seq(StructField("city", StringType, nullable = false), + StructField("state", StringType, nullable = false))) + val valueSchema = StructType(Seq(StructField("name", StringType, nullable = false), + StructField("age", IntegerType, nullable = false), + StructField("address", addressSchema, nullable = false) + )) + + val df = snappy.readStream + .format("kafka") + .option("kafka.bootstrap.servers", args(0)) + .option("startingOffsets", "earliest") + // more kafka consumer options can be provided here + .option("subscribe", args(1)) + .option("maxOffsetsPerTrigger", 100) // to restrict the streaming batch size + .load() + + import snappy.implicits._ + val structDF = df.select("key", "value").as[(String, String)].map(s => { + // Note: this split won't handle CSV containing comma character as part of quotes values. + val fields = s._2.split(",") + People(fields(0), fields(1).toInt, fields(2), fields(3), s._1) + }) + + val streamingQuery = structDF + .writeStream + .format("snappysink") + .queryName(this.getClass.getSimpleName) // must be unique across the Snappydata cluster + .trigger(ProcessingTime("1 seconds")) // streaming query trigger interval + .option("tableName", "people") // name of the target table + .option("checkpointLocation", checkpointDirectory) + .start() + + println("Streaming started.") + // Following line will make streaming query terminate after 15 seconds. + // This can be replaced by streamingQuery.awaitTermination() to keep the streaming query + // running. + streamingQuery.awaitTermination(15000) + + println("Data loaded in table:") + snappy.sql("select * from people").show() + } finally { + snappy.sql("drop table if exists people") + } + println("Exiting") + System.exit(0) + } + case class People(name: String, age : Int, city : String, state : String, country : String) +} + diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONFileSourceExampleWithSnappySink.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONFileSourceExampleWithSnappySink.scala new file mode 100644 index 0000000000..9b31a300f0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONFileSourceExampleWithSnappySink.scala @@ -0,0 +1,159 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.examples.snappydata.structuredstreaming + +import com.typesafe.config.Config +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery} +import org.apache.spark.sql.{SnappyJobValid, SnappyJobValidation, SnappySQLJob, SnappySession, SparkSession} + +/** + * An example of structured streaming depicting JSON file processing with Snappy sink. + * + * This example can be run either in local mode (in which case the example runs + * collocated with Spark+SnappyData Store in the same JVM) or can be submitted as a job + * to an already running SnappyData cluster. + * + * Example input data: + * + * {"name":"Yin", "age":31, "address":{"city":"Columbus","state":"Ohio", "district" :"Cincinnati"}} + * {"name":"Michael", "age":38, "address":{"city":"San Jose", "state":"California", "lane" :"15"}} + * + * Running locally: + * + * Usage: JSONFileSourceExampleWithSnappySink [checkpoint-directory] [input-directory] + * + * [checkpoint-directory] Optional argument providing checkpoint directory where the + * state of the steaming query will be stored. Note that this + * directory needs to be deleted manually to reset the state + * of the streaming query. + * Default: `JSONFileSourceExampleWithSnappySink` directory + * in working directory. + * [input-directory] Optional argument pointing to input directory path where incoming + * JSON files should be dumped to get picked up for processing. + * Default: `people.json` directory under resources + * + * Example: + * $ bin/run-example snappydata.structuredstreaming.JSONFileSourceExampleWithSnappySink \ + * "checkpoint_dir" "JSON_input_dir" + * + * Submitting as a snappy job to already running cluster: + * cd $SNAPPY_HOME + * bin/snappy-job.sh submit \ + * --app-name JSONFileSourceExampleWsithSnappySink \ + * --class org.apache.spark.examples.snappydata.structuredstreaming.JSONFileSourceExampleWithSnappySink \ + * --app-jar examples/jars/quickstart.jar \ + * --conf checkpoint-directory= \ + * --conf input-directory= + * + * Note that the checkpoint directory and input directory are mandatory options while submitting snappy job. + * Check the status of your job id + * bin/snappy-job.sh status --lead [leadHost:port] --job-id [job-id] + * + * To stop the job: + * bin/snappy-job.sh stop --lead [leadHost:port] --job-id [job-id] + * + * The content of the sink table can be checked from snappy-sql using a select query: + * select * from people; + * + * Resetting the streaming query: + * To reset streaming query progress delete the checkpoint directory. + * While running this example from as snappy job, you will also need to clear the state from state + * table using following query: + * + * delete from app.snappysys_internal____sink_state_table where stream_query_id = 'query1'; + * + */ +// scalastyle:off println +object JSONFileSourceExampleWithSnappySink extends SnappySQLJob with Logging { + + // Entry point for local mode + def main(args: Array[String]) { + // reducing the log level to minimize the messages on console + Logger.getLogger("org").setLevel(Level.ERROR) + Logger.getLogger("akka").setLevel(Level.ERROR) + + val checkpointDirectory = if (args.length >= 1) args(0) + else getClass.getSimpleName + val inputDirectory = if (args.length >= 2) args(1) + else "quickstart/src/main/resources/people.json" + println("Initializing SnappySession ...") + val spark: SparkSession = SparkSession + .builder() + .appName(getClass.getSimpleName) + .master("local[*]") + .getOrCreate() + val snappy = new SnappySession(spark.sparkContext) + println("Initializing SnappySession ... Done.") + try { + snappy.sql("create table people (name string , age int, lane string," + + " city string, district string, state string)") + + val streamingQuery: StreamingQuery = startStreaming(snappy, checkpointDirectory, + inputDirectory) + // Following line will make streaming query terminate after 15 seconds. + // This can be replaced by streamingQuery.awaitTermination() to keep the streaming query + // running. + streamingQuery.awaitTermination(15000) + + println("Data loaded in table:") + snappy.sql("select * from people").show() + } finally { + snappy.sql("drop table if exists people") + } + println("Exiting") + System.exit(0) + } + + // Entry point for snappy job submission + override def runSnappyJob(sc: SnappySession, jobConfig: Config): Any = { + sc.sql("create table if not exists people (name string , age int, lane string," + + " city string, district string, state string)") + val streamingQuery = startStreaming(sc, jobConfig.getString("checkpoint-directory"), + jobConfig.getString("input-directory")) + streamingQuery.awaitTermination() + } + + override def isValidJob(sc: SnappySession, config: Config): SnappyJobValidation = SnappyJobValid() + + private def startStreaming(snappy: SnappySession, checkpointDirectory: String, + inputDirectory: String) = { + val schema = snappy.read.json(inputDirectory).schema + + // Create DataFrame representing the stream of JSON + val jsonDF = snappy.readStream + .option("maxFilesPerTrigger", 1) // Controls number of files to be processed per batch + .schema(schema) + .json(inputDirectory) + + val streamingQuery = jsonDF + .select("name", "age", "address.lane", "address.city", "address.district", + "address.state") + .writeStream + .format("snappysink") + .queryName("query1") // must be unique across a snappydata cluster + .trigger(ProcessingTime("1 seconds")) // streaming query trigger interval + .option("tableName", "people") // name of the target table + .option("checkpointLocation", checkpointDirectory) + .start() + + println("Streaming started.") + streamingQuery + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONKafkaSourceExampleWithSnappySink.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONKafkaSourceExampleWithSnappySink.scala new file mode 100644 index 0000000000..4a2933ae02 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/JSONKafkaSourceExampleWithSnappySink.scala @@ -0,0 +1,122 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ +package org.apache.spark.examples.snappydata.structuredstreaming + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.streaming.ProcessingTime +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.{SnappySession, SparkSession} + +/** + * An example of structured streaming depicting processing of JSON coming from kafka source + * using snappy sink. + * + * Example input data: + * key: {"country" : "USA"}, value: {"name":"Adam", "age":21, "address":{"city":"Columbus","state":"Ohio"}} + * key: {"country" : "England"}, value: {"name":"John", "age":44, "address":{"city":"London"}} + * key: {"country" : "USA"}, value: {"name":"Carol", "age":37, "address":{"city":"San Diego", "state":"California"}} + * + * Usage: JSONKafkaSourceExampleWithSnappySink [checkpoint-directory] + * + * Compulsory argument providing comma separate list of kafka brokers + * Compulsory argument providing comma separated list of kafka topics to subscribe + * [checkpoint-directory] Optional argument providing checkpoint directory where the state of + * the steaming query will be stored. Note that this directory needs to + * be deleted manually to reset the state of the streaming query. + * Default: `JSONKafkaSourceExampleWithSnappySink` directory under + * working directory. + * + * Example: + * $ bin/run-example snappydata.structuredstreaming.JSONKafkaSourceExampleWithSnappySink \ + * "broker-1:9092,broker-2:9092" "topic1,topic2" "checkpoint_dir" + */ +// scalastyle:off println +object JSONKafkaSourceExampleWithSnappySink extends Logging { + + def main(args: Array[String]) { + + if (args.length < 2) { + println("Usage: JSONKafkaSourceExampleWithSnappySink " + + " [checkpoint-directory]") + System.exit(1) + } + + // reducing the log level to minimize the messages on console + Logger.getLogger("org").setLevel(Level.ERROR) + Logger.getLogger("akka").setLevel(Level.ERROR) + + val checkpointDirectory = if (args.length >= 3) args(2) else getClass.getSimpleName + println("Initializing SnappySession ... ") + val spark: SparkSession = SparkSession + .builder() + .appName(getClass.getSimpleName) + .master("local[*]") + .getOrCreate() + val snappy = new SnappySession(spark.sparkContext) + println("Initializing SnappySession ... Done.") + try { + snappy.sql("create table people(name string , age int," + + "city string, state string, country string)") + + val keySchema = StructType(Seq(StructField("country", StringType, nullable = false))) + val addressSchema = StructType(Seq(StructField("city", StringType, nullable = false), + StructField("state", StringType, nullable = false))) + val valueSchema = StructType(Seq(StructField("name", StringType, nullable = false), + StructField("age", IntegerType, nullable = false), + StructField("address", addressSchema, nullable = false) + )) + + + val df = snappy.readStream + .format("kafka") + .option("kafka.bootstrap.servers", args(0)) + .option("startingOffsets", "earliest") + // more kafka consumer options can be provided here + .option("subscribe", args(1)) + .option("maxOffsetsPerTrigger", 100) // to restrict the batch size + .load() + + import org.apache.spark.sql.functions.from_json + val streamingQuery = df + .select(from_json(df.col("key").cast("string"), keySchema).alias("key"), + from_json(df.col("value").cast("string"), valueSchema).alias("value")) + .select("value.name", "value.age", "value.address.*", "key.country") + .writeStream + .format("snappysink") + .queryName(getClass.getSimpleName) // must be unique across the Snappydata cluster + .trigger(ProcessingTime("1 seconds")) // streaming query trigger interval + .option("tableName", "people") // name of the target table + .option("checkpointLocation", checkpointDirectory) + .start() + + println("Streaming started.") + // Following line will make streaming query terminate after 15 seconds. + // This can be replaced by streamingQuery.awaitTermination() to keep the streaming query + // running. + streamingQuery.awaitTermination(15000) + + println("Data loaded in table:") + snappy.sql("select * from people").show() + } finally { + snappy.sql("drop table if exists people") + } + println("Exiting") + System.exit(0) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExample.scala similarity index 77% rename from examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingExample.scala rename to examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExample.scala index d6b6a6cb0c..8521719e3a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -14,53 +14,52 @@ * permissions and limitations under the License. See accompanying * LICENSE file. */ - -package org.apache.spark.examples.snappydata +package org.apache.spark.examples.snappydata.structuredstreaming import org.apache.log4j.{Level, Logger} + import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.sql.{SnappySession, SparkSession} -import scala.language.postfixOps - /** - * An example showing usage of structured streaming with SnappyData + * An example showing usage of structured streaming with console sink. * - *

- * To run the example in local mode go to your SnappyData product distribution - * directory and type following command on the command prompt - *
- * bin/run-example snappydata.StructuredStreamingExample
- * 
- *

- * To run this on your local machine, you need to first run a Netcat server
+ * To run this example on your local machine, you need to first start a Netcat server:
* `$ nc -lk 9999` *

* Sample input data: - * {{{ + *

  * device1,45
  * device2,67
  * device3,35
- * }}}
+ * 
+ * + * To run the example in local mode go to your SnappyData product distribution + * directory and run the following command: + *
+ * bin/run-example snappydata.structuredstreaming.SocketSourceExample
+ * 
+ * * For more details on streaming with SnappyData refer to: * http://snappydatainc.github.io/snappydata/programming_guide * /stream_processing_using_sql/#stream-processing-using-sql * */ -object StructuredStreamingExample { +// scalastyle:off println +object SocketSourceExample { def main(args: Array[String]) { // reducing the log level to minimize the messages on console Logger.getLogger("org").setLevel(Level.ERROR) Logger.getLogger("akka").setLevel(Level.ERROR) - println("Initializing a SnappySesion") + println("Initializing SnappySession ... ") val spark: SparkSession = SparkSession - .builder + .builder() .appName(getClass.getSimpleName) .master("local[*]") - .getOrCreate - + .getOrCreate() + println("Initializing SnappySession ... Done.") import spark.implicits._ val snappy = new SnappySession(spark.sparkContext) @@ -86,9 +85,9 @@ object StructuredStreamingExample { .format("console") .outputMode("append") .trigger(ProcessingTime("1 seconds")) - .start + .start() - streamingQuery.awaitTermination(timeoutMs = 15000) + streamingQuery.awaitTermination(15000) println("Exiting") System.exit(0) @@ -96,4 +95,3 @@ object StructuredStreamingExample { case class DeviceData(device: String, signal: Int) } - diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingWithSnappySink.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExampleWithSnappySink.scala similarity index 78% rename from examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingWithSnappySink.scala rename to examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExampleWithSnappySink.scala index fc46cd1d3f..a57f6b3e7d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingWithSnappySink.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExampleWithSnappySink.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -14,47 +14,44 @@ * permissions and limitations under the License. See accompanying * LICENSE file. */ +package org.apache.spark.examples.snappydata.structuredstreaming -package org.apache.spark.examples.snappydata +import scala.reflect.io.Path import org.apache.log4j.{Level, Logger} +import org.apache.spark.internal.Logging import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.sql.{SnappySession, SparkSession} -import scala.language.postfixOps -import scala.reflect.io.Path /** - * An example showing usage of structured streaming with SnappyData - * - *

- * To run the example in local mode go to your SnappyData product distribution - * directory and type following command on the command prompt - *
- * bin/run-example snappydata.StructuredStreamingWithSnappySink
- * 
- *

- * To run this on your local machine, you need to first run a Netcat server
+ * An example showing usage of structured streaming with SnappyData.
+ * To run this on your local machine, you need to first start a Netcat server:
* `$ nc -lk 9999` *

* Sample input data: - * {{{ + *

  * device1,45
  * device2,67
  * device3,35
- * }}}
+ * 
+ * To run the example in local mode go to your SnappyData product distribution + * directory and run the following command: + *
+ * bin/run-example snappydata.structuredstreaming.SocketSourceExampleWithSnappySink
+ * 
* */ -object StructuredStreamingWithSnappySink { +// scalastyle:off println +object SocketSourceExampleWithSnappySink extends Logging { def main(args: Array[String]) { // reducing the log level to minimize the messages on console Logger.getLogger("org").setLevel(Level.ERROR) Logger.getLogger("akka").setLevel(Level.ERROR) - println("Initializing a SnappySesion") - - val checkpointDirectory = "/tmp/StructuredStreamingWithSnappySink" + println("Initializing SnappySession ... ") + val checkpointDirectory = this.getClass.getSimpleName val spark: SparkSession = SparkSession .builder() .appName(getClass.getSimpleName) @@ -63,7 +60,7 @@ object StructuredStreamingWithSnappySink { import spark.implicits._ val snappy = new SnappySession(spark.sparkContext) - + println("Initializing SnappySession ... Done.") try { snappy.sql("create table devices (device varchar(30) , signal int)") @@ -87,14 +84,13 @@ object StructuredStreamingWithSnappySink { .writeStream .format("snappysink") .outputMode("append") - .queryName("Devices") + .queryName("Devices") // must be unique across a snappydata cluster .trigger(ProcessingTime("1 seconds")) - .option("streamQueryId", "Devices") // must be unique across a snappydata cluster .option("tableName", "devices") .option("checkpointLocation", checkpointDirectory) .start() - streamingQuery.awaitTermination(timeoutMs = 15000) + streamingQuery.awaitTermination( 15000) println("Data loaded in table: ") snappy.sql("select * from devices").show() @@ -111,6 +107,3 @@ object StructuredStreamingWithSnappySink { case class DeviceData(device: String, signal: Int) } - - - From 47ec289e946098922d9b4a17495c5a3b836a9576 Mon Sep 17 00:00:00 2001 From: Piyush Bisen <42795748+bisenpiyush@users.noreply.github.com> Date: Wed, 14 Aug 2019 19:51:11 +0530 Subject: [PATCH 10/26] Snap 3135 (#1424) This is an experimental utility to execute a given command on selected members of the cluster. The script relies on the entries you specify in locators, servers and leads files in conf directory to identify the members of the cluster. "-copyconf" This is a shortcut command which when specified copies log4j.properties, snappy-env.sh and spark-env.sh configuration files from local machine to all the members. --- cluster/sbin/cluster-util.sh | 231 +++++++++++++++++++++++++++++++++++ 1 file changed, 231 insertions(+) create mode 100755 cluster/sbin/cluster-util.sh diff --git a/cluster/sbin/cluster-util.sh b/cluster/sbin/cluster-util.sh new file mode 100755 index 0000000000..72cc1fcf3b --- /dev/null +++ b/cluster/sbin/cluster-util.sh @@ -0,0 +1,231 @@ +#!/bin/bash +# +# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); you +# may not use this file except in compliance with the License. You +# may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +# implied. See the License for the specific language governing +# permissions and limitations under the License. See accompanying +# LICENSE file. +# + + +function absPath() { + perl -MCwd -le 'print Cwd::abs_path(shift)' "$1" +} +sbin="$(dirname "$(absPath "$0")")" + +# Load the Spark configuration +. "$sbin/snappy-config.sh" +. "$sbin/spark-config.sh" + +if [[ ! -d ${SNAPPY_HOME} ]]; then + echo "${SNAPPY_HOME} does not exist. Exiting ..." + exit 1 +fi + +bold=$(tput bold) +normal=$(tput sgr0) +space= +usage() { + echo "${bold}Usage: ${normal}cluster-util.sh (--on-locators|--on-servers|--on-leads|--on-all) [-y] (--copy-conf | --run \"\")" + echo + echo "${bold}Description${normal}" + echo + echo -e ' \t '"This is an utility to execute a given command on selected members of the cluster." + echo -e ' \t '"The script relies on the entries you specify in locators, servers and leads files in conf directory to identify the members of the cluster." + echo + echo -e ' \t '"--on-locators" + echo -e ' \t ''\t'"If specified, the given command would be executed on locators." + echo + echo + echo -e ' \t '"--on-servers" + echo -e ' \t ''\t'"If specified, the given command would be executed on servers." + echo + echo + echo -e ' \t '"--on-leads" + echo -e ' \t ''\t'"If specified, the given command would be executed on leads." + echo + echo + echo -e ' \t '"--on-all" + echo -e ' \t ''\t'"If specified, the given command would be executed on all member of the cluster." + echo + echo -e ' \t '"-y" + echo -e ' \t ''\t'"If specified, the script doesn't ask for confirmation for execution of the command on each member node." + echo + echo -e ' \t '"--copy-conf" + echo -e ' \t ''\t'"This is a shortcut command which when specified copies log4j.properties, snappy-env.sh and " + echo -e ' \t ''\t'"spark-env.sh configuration files from local machine to all the members." + echo -e ' \t ''\t'"These files are copied only if a) these are absent in the destination member or b) their content is different. In " + echo -e ' \t ''\t'"latter case, a backup of the file is taken in conf/backup directory on destination member, before copy." + echo -e ' \t '"--run ''" + echo -e ' \t ''\t'"Will execute the given command(s) on specified members. Command to be executed specified after --run must be in double-quotes." + echo + exit 1 +} + +ISFORCE=0 +SPARK_CONF_DIR= +COPY_CONF=0 + +MEMBER_LIST= +MEMBER_TYPE= + +COMMAND= + +while [ "$1" != "" ]; do + + case $1 in + --run) + RUN="true" + COMMAND="$2" + shift + ;; + --copy-conf) + COPY_CONF=1 + SPARK_CONF_DIR=$SNAPPY_HOME/conf/ + shift + ;; + -y) + ISFORCE=1 + shift + ;; + --on-locators) + if [[ ! -e $SNAPPY_HOME/conf/locators ]]; then + echo "${SNAPPY_HOME}/conf/locators does not exist. Exiting ..." + exit 2 + fi + LOCATOR_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/locators | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + MEMBER_LIST=$LOCATOR_LIST + MEMBER_TYPE="locator" + shift + ;; + --on-servers) + if [[ ! -e $SNAPPY_HOME/conf/servers ]]; then + echo "${SNAPPY_HOME}/conf/servers does not exist. Exiting ..." + exit 3 + fi + SERVER_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/servers | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + MEMBER_LIST=$SERVER_LIST + MEMBER_TYPE="server" + shift + ;; + --on-leads) + if [[ ! -e $SNAPPY_HOME/conf/leads ]]; then + echo "${SNAPPY_HOME}/conf/leads does not exist. Exiting ..." + exit 4 + fi + LEAD_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/leads | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + MEMBER_LIST=$LEAD_LIST + MEMBER_TYPE="lead" + shift + ;; + --on-all) + MEMBER_LIST="all" + MEMBER_TYPE="all" + rm /tmp/snappy-nodes.txt + LOCATOR_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/locators | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + echo $LOCATOR_LIST >> /tmp/snappy-nodes.txt + SERVER_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/servers | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + echo $SERVER_LIST >> /tmp/snappy-nodes.txt + LEAD_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/leads | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') + echo $LEAD_LIST >> /tmp/snappy-nodes.txt + #cat /tmp/snappy-nodes.txt + ALL_MEMBER_LIST=$(awk '!seen[$0]++' /tmp/snappy-nodes.txt) + rm /tmp/snappy-nodes.txt + shift + ;; + -*) + echo "ERROR: Unknown option: $1" >&2 + echo + usage + ;; + *) # End of options + break + ;; + esac +done + +if [[ $COPY_CONF == 1 && $RUN == "true" ]]; then + echo + echo "Invalid operation: Specify either '--copy-conf' or '--run \"\"'" + echo + usage + exit +fi + +START_ALL_TIMESTAMP="$(date +"%Y_%m_%d_%H_%M_%S")" + +function copyConf() { + for entry in "${SPARK_CONF_DIR}"/*; do + if [ -f "$entry" ];then + fileName=$(basename $entry) + if [[ $fileName == "log4j.properties" || $fileName == "snappy-env.sh" || $fileName == "spark-env.sh" ]]; then + if ! ssh $node "test -e $entry"; then #"File does not exist." + scp ${SPARK_CONF_DIR}/$fileName $node:${SPARK_CONF_DIR} + else + backupDir="backup_"${START_ALL_TIMESTAMP} + if [[ ! -z $(ssh $node "cat $entry" | diff - "$entry") ]] ; then + ssh "$node" "mkdir -p \"${SPARK_CONF_DIR}/$backupDir\" " + ssh $node "mv ${SPARK_CONF_DIR}/$fileName ${SPARK_CONF_DIR}/$backupDir/$fileName" + echo "INFO:Copying $filename from this host to $node. Moved the original $filename on $node to $backupDir/$fileName." + scp ${SPARK_CONF_DIR}/$fileName $node:${SPARK_CONF_DIR} + fi + fi + fi # end of if, check the conf file name + fi # end of if to get each file + done #end of for loop +} + +function executeCommand() { + echo + echo "--------- Executing $@ on $MEMBER_TYPE $node ----------" + echo + if [[ $ISFORCE -eq 0 ]]; then + read -p "Are you sure to run $@ on $MEMBER_TYPE $node (y/n)? " userinput + echo + if [[ $userinput == "y" || $userinput == "yes" || $userinput == "Y" || $userinput == "YES" ]]; then + if [[ $COPY_CONF == 1 ]]; then + copyConf "$@" + elif [[ $RUN == "true" ]]; then + ssh $node "$COMMAND | column" + else + echo "Invalid operation" + echo + usage + exit + fi + fi + else + if [[ $COPY_CONF == 1 ]]; then + copyConf "$@" + elif [[ $RUN == "true" ]]; then + ssh $node "$COMMAND | column" + else + echo "Invalid operation" + echo + usage + exit + fi + fi #end if ISFORCE +} + + +if [[ $MEMBER_LIST != "all" ]]; then + for node in $MEMBER_LIST; do + executeCommand "$@" + done +else + for node in $ALL_MEMBER_LIST; do + executeCommand "$@" + done +fi + + From 633ffe1a79bb0683be131901bcca8d79375f45ea Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 14 Aug 2019 20:22:05 +0530 Subject: [PATCH 11/26] [SNAP-3123] encapsulate any jobs in RDD evaluation under same executionId (#1431) - moved executionId clear after RDD evaluation - use common method to create SQLExecutionUIData for SparkListenerSQLPlanExecutionStart and SparkListenerSQLExecutionStart - add to activeExecutions in SparkListenerSQLPlanExecutionStart too so that it shows up in GUI in case RDD evaluation takes long (e.g. for ORDER BY or subqueries) - handle case of Union(commands) and treat like other ExecuteCommandExec/ExecutePlans are - handle exception cases in CollectAggregateExec to clear any blocks in BlockManager - clear plan cache for session when connection is closed; updated tests to expect the same - make 2 arg SnappySession.setCurrentSchema as private[sql] for internal use only --- .../JDBCPreparedStatementDUnitTest.scala | 19 ++-- .../gemxd/SparkSQLExecuteImpl.scala | 33 ++++--- .../gemxd/SparkSQLPrepareImpl.scala | 3 +- .../PreparedQueryRoutingSingleNodeSuite.scala | 35 ++++--- .../cluster/QueryRoutingSingleNodeSuite.scala | 10 +- .../org/apache/spark/sql/store/BugTest.scala | 34 +++++++ .../org/apache/spark/sql/SnappyContext.scala | 7 +- .../org/apache/spark/sql/SnappySession.scala | 29 +++++- .../apache/spark/sql/collection/Utils.scala | 3 + .../aggregate/CollectAggregateExec.scala | 57 +++++++---- .../sql/execution/ui/SnappySQLListener.scala | 97 ++++++++++--------- store | 2 +- 12 files changed, 206 insertions(+), 123 deletions(-) diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala index 9d59decc96..52209e41a1 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala @@ -17,6 +17,7 @@ package io.snappydata.externalstore import java.sql.PreparedStatement +import java.util.concurrent.atomic.AtomicInteger import io.snappydata.cluster.ClusterManagerTestBase import io.snappydata.test.dunit.AvailablePortHelper @@ -363,12 +364,12 @@ class JDBCPreparedStatementDUnitTest(s: String) extends ClusterManagerTestBase(s } var thrCount6: Integer = 0 - var deletedRecords = 0 + val deletedRecords = new AtomicInteger(0) val colThread6 = new Thread(new Runnable {def run() { (1 to 5) foreach (i => { - var result = deleteRecords(1, 20) + val result = deleteRecords(1, 20) thrCount6 += result._1 - deletedRecords += result._2 + deletedRecords.getAndAdd(result._2) }) } }) @@ -377,9 +378,9 @@ class JDBCPreparedStatementDUnitTest(s: String) extends ClusterManagerTestBase(s var thrCount7: Integer = 0 val colThread7 = new Thread(new Runnable {def run() { (1 to 5) foreach (i => { - var result = deleteRecords(11, 20) + val result = deleteRecords(11, 20) thrCount7 += result._1 - deletedRecords += result._2 + deletedRecords.getAndAdd(result._2) }) } }) @@ -388,9 +389,9 @@ class JDBCPreparedStatementDUnitTest(s: String) extends ClusterManagerTestBase(s var thrCount8: Integer = 0 val colThread8 = new Thread(new Runnable {def run() { (1 to 5) foreach (i => { - var result = deleteRecords(21, 30) + val result = deleteRecords(21, 30) thrCount8 += result._1 - deletedRecords += result._2 + deletedRecords.getAndAdd(result._2) }) } }) @@ -403,6 +404,6 @@ class JDBCPreparedStatementDUnitTest(s: String) extends ClusterManagerTestBase(s rscnt = stmt.executeQuery("select count(*) from t3") rscnt.next() assertEquals(0, rscnt.getInt(1)) - assertEquals(100, deletedRecords) + assertEquals(100, deletedRecords.get) } -} \ No newline at end of file +} diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala index c7ff7fcb14..cea16e8db8 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala @@ -75,7 +75,7 @@ class SparkSQLExecuteImpl(val sql: String, session.conf.set(Attribute.PASSWORD_ATTR, ctx.getAuthToken) } - session.setCurrentSchema(schema, createIfNotExists = true) + Utils.setCurrentSchema(session, schema, createIfNotExists = true) session.setPreparedQuery(preparePhase = false, pvs) @@ -478,28 +478,29 @@ object SparkSQLExecuteImpl { object SnappySessionPerConnection { - private val connectionIdMap = + private[this] val connectionIdMap = new java.util.concurrent.ConcurrentHashMap[java.lang.Long, SnappySession]() def getSnappySessionForConnection(connId: Long): SnappySession = { - val connectionID = Long.box(connId) - val session = connectionIdMap.get(connectionID) - if (session != null) session - else { - val session = SnappyContext.globalSparkContext match { - // use a CancelException to force failover by client to another lead if available - case null => throw new CacheClosedException("No SparkContext ...") - case sc => new SnappySession(sc) - } - Property.PlanCaching.set(session.sessionState.conf, true) - val oldSession = connectionIdMap.putIfAbsent(connectionID, session) - if (oldSession == null) session else oldSession - } + connectionIdMap.computeIfAbsent(Long.box(connId), CreateNewSession) } def getAllSessions: Seq[SnappySession] = connectionIdMap.values().asScala.toSeq def removeSnappySession(connectionID: java.lang.Long): Unit = { - connectionIdMap.remove(connectionID) + val session = connectionIdMap.remove(connectionID) + if (session ne null) session.clear() + } +} + +object CreateNewSession extends java.util.function.Function[java.lang.Long, SnappySession] { + override def apply(connId: java.lang.Long): SnappySession = { + val session = SnappyContext.globalSparkContext match { + // use a CancelException to force failover by client to another lead if available + case null => throw new CacheClosedException("No SparkContext ...") + case sc => new SnappySession(sc) + } + Property.PlanCaching.set(session.sessionState.conf, true) + session } } diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala index 262628e0d9..659177a97c 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala @@ -34,6 +34,7 @@ import com.pivotal.gemfirexd.internal.snappy.{LeadNodeExecutionContext, SparkSQL import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.{BinaryComparison, CaseWhen, Cast, Exists, Expression, Like, ListQuery, ParamLiteral, PredicateSubquery, ScalarSubquery, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.PutIntoValuesColumnTable import org.apache.spark.sql.hive.QuestionMark import org.apache.spark.sql.types._ @@ -59,7 +60,7 @@ class SparkSQLPrepareImpl(val sql: String, session.conf.set(Attribute.PASSWORD_ATTR, ctx.getAuthToken) } - session.setCurrentSchema(schema, createIfNotExists = true) + Utils.setCurrentSchema(session, schema, createIfNotExists = true) session.setPreparedQuery(preparePhase = true, None) diff --git a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala index 14898876e4..5c4f2e0943 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala @@ -21,7 +21,7 @@ import java.sql.{Connection, DriverManager, PreparedStatement, ResultSet, SQLExc import com.pivotal.gemfirexd.TestUtil import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils import io.snappydata.{SnappyFunSuite, SnappyTableStatsProviderService} -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{Assertions, BeforeAndAfterAll} import org.apache.spark.sql.{SnappyContext, SnappySession} import org.apache.spark.{Logging, SparkConf} @@ -336,13 +336,13 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA prepStatement.setInt(2, 300) prepStatement.setInt(3, 200) PreparedQueryRoutingSingleNodeSuite.verifyResults("query2-1", prepStatement.executeQuery, - Array(400, 200, 300), 2) + Array(400, 200, 300), 1) prepStatement.setInt(1, 600) prepStatement.setInt(2, 800) prepStatement.setInt(3, 700) PreparedQueryRoutingSingleNodeSuite.verifyResults("query2-2", prepStatement.executeQuery, - Array(600, 700, 800), 2) + Array(600, 700, 800), 1) // Thread.sleep(1000000) } finally { @@ -372,14 +372,14 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA prepStatement.setInt(3, 200) prepStatement.setInt(4, 400) PreparedQueryRoutingSingleNodeSuite.verifyResults("query3-1", prepStatement.executeQuery, - Array(200, 400), 3) + Array(200, 400), 1) prepStatement.setInt(1, 900) prepStatement.setInt(2, 700) prepStatement.setInt(3, 600) prepStatement.setInt(4, 800) PreparedQueryRoutingSingleNodeSuite.verifyResults("query3-2", prepStatement.executeQuery, - Array(600, 800), 3) + Array(600, 800), 1) // Thread.sleep(1000000) } finally { @@ -410,14 +410,14 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA prepStatement.setInt(3, 200) prepStatement.setInt(4, 400) PreparedQueryRoutingSingleNodeSuite.verifyResults("query4-1", prepStatement.executeQuery, - Array(100, 200, 300, 400), 4) + Array(100, 200, 300, 400), 1) prepStatement.setInt(1, 900) prepStatement.setInt(2, 600) prepStatement.setInt(3, 700) prepStatement.setInt(4, 800) PreparedQueryRoutingSingleNodeSuite.verifyResults("query4-2", prepStatement.executeQuery, - Array(900, 600, 700, 800), 4) + Array(900, 600, 700, 800), 1) // Thread.sleep(1000000) } finally { @@ -449,14 +449,14 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA prepStatement.setInt(3, 200) prepStatement.setInt(4, 300) PreparedQueryRoutingSingleNodeSuite.verifyResults("query5-1", prepStatement.executeQuery, - Array(100, 200, 300), 4) + Array(100, 200, 300), 0) prepStatement.setInt(1, 900) prepStatement.setInt(2, 600) prepStatement.setInt(3, 700) prepStatement.setInt(4, 800) PreparedQueryRoutingSingleNodeSuite.verifyResults("query5-2", prepStatement.executeQuery, - Array(600, 700, 800), 4) + Array(600, 700, 800), 0) // Thread.sleep(1000000) } finally { @@ -1119,7 +1119,7 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA } } -object PreparedQueryRoutingSingleNodeSuite extends Logging { +object PreparedQueryRoutingSingleNodeSuite extends Assertions with Logging { def insertRows(tableName: String, numRows: Int, serverHostPort: String): Unit = { @@ -1154,7 +1154,6 @@ object PreparedQueryRoutingSingleNodeSuite extends Logging { def verifyResults(qry: String, rs: ResultSet, results: Array[Int], cacheMapSize: Int): Unit = { val cacheMap = SnappySession.getPlanCache.asMap() - var index = 0 while (rs.next()) { val i = rs.getInt(1) @@ -1169,8 +1168,12 @@ object PreparedQueryRoutingSingleNodeSuite extends Logging { assert(index == results.length) rs.close() + // for dunit tests, connection close will happen in background so need to retry this + for (i <- 0 until 100 if cacheMap.size() != cacheMapSize && -1 != cacheMapSize) { + Thread.sleep(100) + } logInfo(s"cachemapsize = $cacheMapSize and .size = ${cacheMap.size()}") - assert( cacheMap.size() == cacheMapSize || -1 == cacheMapSize) + assert(cacheMap.size() == cacheMapSize || -1 == cacheMapSize) } def update_delete_query1(tableName1: String, cacheMapSize: Int, serverHostPort: String): Unit = { @@ -1316,9 +1319,9 @@ object PreparedQueryRoutingSingleNodeSuite extends Logging { insertRows(tableName1, 1000, serverHostPort) insertRows(tableName2, 1000, serverHostPort) update_delete_query1(tableName1, 1, serverHostPort) - update_delete_query1(tableName2, 3, serverHostPort) - update_delete_query2(tableName1, 5, serverHostPort) - update_delete_query2(tableName2, 6, serverHostPort) + update_delete_query1(tableName2, 1, serverHostPort) + update_delete_query2(tableName1, 1, serverHostPort) + update_delete_query2(tableName2, 1, serverHostPort) } finally { SnappyTableStatsProviderService.TEST_SUSPEND_CACHE_INVALIDATION = false } @@ -1405,7 +1408,7 @@ object PreparedQueryRoutingSingleNodeSuite extends Logging { insertRows(tableName1, 1000, serverHostPort) insertRows(tableName2, 1000, serverHostPort) equalityOnStringColumn_query1(tableName1, 1, serverHostPort) - equalityOnStringColumn_query1(tableName2, 4, serverHostPort) + equalityOnStringColumn_query1(tableName2, 1, serverHostPort) } finally { SnappyTableStatsProviderService.TEST_SUSPEND_CACHE_INVALIDATION = false } diff --git a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala index 7a9a58065e..57c8b16722 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala @@ -584,12 +584,12 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll insertRows(tableName1, 1000) insertRows(tableName2, 1000) update_delete_query1(tableName1, 1) - update_delete_query2(tableName1, 2) - update_delete_query3(tableName1, 3, 2) + update_delete_query2(tableName1, 1) + update_delete_query3(tableName1, 1, 2) - update_delete_query1(tableName2, 4) - update_delete_query2(tableName2, 5) - update_delete_query3(tableName2, 6, 1) + update_delete_query1(tableName2, 1) + update_delete_query2(tableName2, 1) + update_delete_query3(tableName2, 1, 1) } finally { SnappyTableStatsProviderService.TEST_SUSPEND_CACHE_INVALIDATION = false } diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala index 9864d20371..6697d8520c 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala @@ -32,6 +32,8 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.catalog.Column import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.execution.columnar.ExternalStoreUtils +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.sql.{Row, SaveMode, SparkSession} @@ -1168,4 +1170,36 @@ class BugTest extends SnappyFunSuite with BeforeAndAfterAll { // scalastyle:on println } + + test("SNAP-3123: check for GUI plans") { + // TODO: new SHA code generation fails for query below + val session = snc.snappySession.newSession() + session.sql(s"set ${Property.UseOptimzedHashAggregate.name} = false") + session.sql(s"set ${Property.UseOptimizedHashAggregateForSingleKey.name} = false") + + val numRows = 1000000 + val sleepTime = 7000L + session.sql("create table test1 (id long, data string) using column " + + s"options (buckets '8') as select id, 'data_' || id from range($numRows)") + val ds = session.sql( + "select avg(id) average, id % 10 from test1 group by id % 10 order by average") + Thread.sleep(sleepTime) + ds.collect() + + // check UI timings and plan details + val listener = ExternalStoreUtils.getSQLListener.get + // last one should be the query above + val queryUIData = listener.getCompletedExecutions.last + val duration = queryUIData.completionTime.get - queryUIData.submissionTime + // never expect the query above to take more than 7 secs + assert(duration > 0L) + assert(duration < sleepTime) + assert(queryUIData.succeededJobs.length === 2) + + val metrics = listener.getExecutionMetrics(queryUIData.executionId) + val scanNode = queryUIData.physicalPlanGraph.allNodes.find(_.name == "ColumnTableScan").get + val numRowsMetric = scanNode.metrics.find(_.name == "number of output rows").get + assert(metrics(numRowsMetric.accumulatorId) === + SQLMetrics.stringValue(numRowsMetric.metricType, numRows :: Nil)) + } } diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala index 8862e18300..02ad52b3a6 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -609,11 +609,10 @@ class SnappyContext protected[spark](val snappySession: SnappySession) /** * Set current database/schema. * - * @param schemaName schema name which goes in the catalog - * @param createIfNotExists create the schema if it does not exist + * @param schemaName schema name which goes in the catalog */ - def setCurrentSchema(schemaName: String, createIfNotExists: Boolean = false): Unit = { - snappySession.setCurrentSchema(schemaName, createIfNotExists) + def setCurrentSchema(schemaName: String): Unit = { + snappySession.setCurrentSchema(schemaName) } /** diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index b8db3a82d6..dc1f9ed860 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -1496,13 +1496,20 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { } } + /** + * Set current schema for the session. + * + * @param schema schema name which goes in the catalog + */ + def setCurrentSchema(schema: String): Unit = setCurrentSchema(schema, createIfNotExists = false) + /** * Set current schema for the session. * * @param schema schema name which goes in the catalog * @param createIfNotExists create the schema if it does not exist */ - def setCurrentSchema(schema: String, createIfNotExists: Boolean = false): Unit = { + private[sql] def setCurrentSchema(schema: String, createIfNotExists: Boolean): Unit = { val schemaName = sessionCatalog.formatDatabaseName(schema) if (createIfNotExists) { sessionCatalog.createSchema(schemaName, ignoreIfExists = true, createInStore = false) @@ -2096,9 +2103,9 @@ object SnappySession extends Logging { context.listenerBus.post(SparkListenerSQLPlanExecutionStart( executionId, CachedDataFrame.queryStringShortForm(sqlText), sqlText, postQueryExecutionStr, postQueryPlanInfo, start)) + val rdd = f clearExecutionProperties(localProperties) propertiesSet = false - val rdd = f (rdd, queryExecutionStr, queryPlanInfo, postQueryExecutionStr, postQueryPlanInfo, executionId, start, System.currentTimeMillis()) } finally { @@ -2124,7 +2131,7 @@ object SnappySession extends Logging { val (cachedRDD, execution, origExecutionString, origPlanInfo, executionString, planInfo, rddId, noSideEffects, executionId, planStartTime: Long, planEndTime: Long) = executedPlan match { - case _: ExecutedCommandExec | _: ExecutePlan => + case _: ExecutedCommandExec | _: ExecutePlan | UnionCommands(_) => // TODO add caching for point updates/deletes; a bit of complication // because getPlan will have to do execution with all waits/cleanups // normally done in CachedDataFrame.collectWithHandler/withCallback @@ -2141,7 +2148,7 @@ object SnappySession extends Logging { val planInfo = PartitionedPhysicalScan.updatePlanInfo(origPlanInfo, paramLiterals, paramsId) // don't post separate plan for CTAS since it already has posted one for the insert - val (eagerToRDD, postGUIPlans) = executedPlan match { + val (eagerToRDD, postGUIPlans) = executedPlan.collectFirst { case ExecutedCommandExec(c: CreateTableUsingCommand) if c.query.isDefined => handleCTAS(SnappyContext.getProviderType(c.provider)) case ExecutedCommandExec(c: CreateDataSourceTableAsSelectCommand) => @@ -2150,7 +2157,9 @@ object SnappySession extends Logging { // other commands may have their own withNewExecutionId but still post GUI // plans to see the command with proper SQL string in the GUI case _: ExecutedCommandExec => true -> true - case _ => false -> true + } match { + case None => false -> true + case Some(p) => p } var rdd = if (eagerToRDD) qe.toRdd else null @@ -2486,3 +2495,13 @@ object CachedKey { new CachedKey(session, currschema, normalizedPlan, sqlText, session.queryHints.hashCode()) } } + +private object UnionCommands { + def unapply(plan: SparkPlan): Option[Boolean] = plan match { + case union: UnionExec if union.children.nonEmpty && union.children.forall { + case _: ExecutedCommandExec | _: ExecutePlan => true + case _ => false + } => Some(true) + case _ => None + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 62df949a10..4323960ba8 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -567,6 +567,9 @@ object Utils extends Logging { } } + def setCurrentSchema(session: SnappySession, schema: String, createIfNotExists: Boolean): Unit = + session.setCurrentSchema(schema, createIfNotExists) + def getDriverClassName(url: String): String = DriverManager.getDriver(url) match { case wrapper: DriverWrapper => wrapper.wrapped.getClass.getCanonicalName case driver => driver.getClass.getCanonicalName diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala index c6420844ff..a214bb7a0b 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala @@ -72,33 +72,52 @@ case class CollectAggregateExec(child: SparkPlan)( val numPartitions = childRDD.getNumPartitions val partitionBlocks = new Array[Any](numPartitions) val rddId = childRDD.id - - sc.runJob(childRDD, CachedDataFrame, 0 until numPartitions, - (index: Int, r: (Array[Byte], Int)) => - // store the partition results in BlockManager for large results - partitionBlocks(index) = CachedDataFrame.localBlockStoreResultHandler( - rddId, bm)(index, r._1)) - - partitionBlocks + var success = false + + try { + sc.runJob(childRDD, CachedDataFrame, 0 until numPartitions, + (index: Int, r: (Array[Byte], Int)) => + // store the partition results in BlockManager for large results + partitionBlocks(index) = CachedDataFrame.localBlockStoreResultHandler( + rddId, bm)(index, r._1)) + success = true + + partitionBlocks + } finally { + if (!success) { + // remove any cached results from block manager + bm.removeRdd(rddId) + } + } } override def executeCollect(): Array[InternalRow] = { val sc = sqlContext.sparkContext val bm = sc.env.blockManager + var success = false val partitionBlocks = executeCollectData() - // create an iterator over the blocks and pass to generated iterator - val numFields = child.schema.length - val results = partitionBlocks.iterator.flatMap( - CachedDataFrame.localBlockStoreDecoder(numFields, bm)) - val buffer = generatedClass.generate(generatedReferences) - .asInstanceOf[BufferedRowIterator] - buffer.init(0, Array(results)) - val processedResults = new ArrayBuffer[InternalRow] - while (buffer.hasNext) { - processedResults += buffer.next().copy() + try { + // create an iterator over the blocks and pass to generated iterator + val numFields = child.schema.length + val results = partitionBlocks.iterator.flatMap( + CachedDataFrame.localBlockStoreDecoder(numFields, bm)) + val buffer = generatedClass.generate(generatedReferences) + .asInstanceOf[BufferedRowIterator] + buffer.init(0, Array(results)) + val processedResults = new ArrayBuffer[InternalRow] + while (buffer.hasNext) { + processedResults += buffer.next().copy() + } + val result = processedResults.toArray + success = true + result + } finally { + if (!success) { + // remove any cached results from block manager + bm.removeRdd(this.childRDD.id) + } } - processedResults.toArray } override def doExecute(): RDD[InternalRow] = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala index b91089b25a..763bef2a43 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala @@ -27,13 +27,13 @@ import org.apache.spark.{JobExecutionStatus, SparkConf} * A new event that is fired when a plan is executed to get an RDD. */ case class SparkListenerSQLPlanExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - sparkPlanInfo: SparkPlanInfo, - time: Long) - extends SparkListenerEvent + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo, + time: Long) + extends SparkListenerEvent /** * Snappy's SQL Listener. @@ -41,21 +41,22 @@ case class SparkListenerSQLPlanExecutionStart( * @param conf SparkConf of active SparkContext */ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { + // base class variables that are private - private val baseStageIdToStageMetrics = { + private[this] val baseStageIdToStageMetrics = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_stageIdToStageMetrics"). - asInstanceOf[mutable.HashMap[Long, SQLStageMetrics]] + asInstanceOf[mutable.HashMap[Long, SQLStageMetrics]] } - private val baseJobIdToExecutionId = { + private[this] val baseJobIdToExecutionId = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_jobIdToExecutionId"). - asInstanceOf[mutable.HashMap[Long, Long]] + asInstanceOf[mutable.HashMap[Long, Long]] } - private val baseActiveExecutions = { + private[this] val baseActiveExecutions = { getInternalField("activeExecutions").asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] } - private val baseExecutionIdToData = { + private[this] val baseExecutionIdToData = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_executionIdToData"). - asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] + asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] } def getInternalField(fieldName: String): Any = { @@ -80,7 +81,7 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { // in the active executions. For such cases, we need to // look up the executionUIToData as well. val executionData = baseActiveExecutions.get(executionId). - orElse(baseExecutionIdToData.get(executionId)) + orElse(baseExecutionIdToData.get(executionId)) executionData.foreach { executionUIData => executionUIData.jobs(jobId) = JobExecutionStatus.RUNNING executionUIData.stages ++= stageIds @@ -91,6 +92,28 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { } } + private def newExecutionUIData(executionId: Long, description: String, details: String, + physicalPlanDescription: String, sparkPlanInfo: SparkPlanInfo, + time: Long): SQLExecutionUIData = { + val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) + val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => + node.metrics.map(metric => metric.accumulatorId -> metric) + } + // description and details strings being reference equals means + // trim off former here + val desc = if (description eq details) { + CachedDataFrame.queryStringShortForm(details) + } else description + new SQLExecutionUIData( + executionId, + desc, + details, + physicalPlanDescription, + physicalPlanGraph, + sqlPlanMetrics.toMap, + time) + } + /** * Snappy's execution happens in two phases. First phase the plan is executed * to create a rdd which is then used to create a CachedDataFrame. @@ -110,46 +133,26 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { case SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) => synchronized { - val executionUIData = baseExecutionIdToData.getOrElseUpdate(executionId, { - val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) - val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) + baseExecutionIdToData.get(executionId) match { + case None => + val executionUIData = newExecutionUIData(executionId, description, details, + physicalPlanDescription, sparkPlanInfo, time) + baseExecutionIdToData(executionId) = executionUIData + baseActiveExecutions(executionId) = executionUIData + case _ => } - // description and details strings being reference equals means - // trim off former here - val desc = if (description eq details) { - CachedDataFrame.queryStringShortForm(details) - } else description - new SQLExecutionUIData( - executionId, - desc, - details, - physicalPlanDescription, - physicalPlanGraph, - sqlPlanMetrics.toMap, - time) - }) - baseActiveExecutions(executionId) = executionUIData } + case SparkListenerSQLPlanExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) => - val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) - val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) - } - val executionUIData = new SQLExecutionUIData( - executionId, - description, - details, - physicalPlanDescription, - physicalPlanGraph, - sqlPlanMetrics.toMap, - time) + val executionUIData = newExecutionUIData(executionId, description, details, + physicalPlanDescription, sparkPlanInfo, time) synchronized { baseExecutionIdToData(executionId) = executionUIData + baseActiveExecutions(executionId) = executionUIData } + case _ => super.onOtherEvent(event) } - } } diff --git a/store b/store index be153871c3..e20da1537f 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit be153871c35ed6e6627323c39d0131a7c46b1e50 +Subproject commit e20da1537fd9f459d622548bec7a52fd50ad06e3 From 920c88e2204feee356b921262996150c584f97e4 Mon Sep 17 00:00:00 2001 From: suranjan kumar Date: Wed, 14 Aug 2019 21:19:35 +0530 Subject: [PATCH 12/26] Snap 3126 [Take Write lock only after executors are assigned] (#1428) * Added a listener to know if executors are added for an App This is to make sure that we take write lock only when resources are allocated and there is no deadlock * Added two properties To disable the serialized writes To provide the timeout for the write lock, after which op will fail --- .../main/scala/io/snappydata/Literals.scala | 14 ++++ .../org/apache/spark/sql/SnappyContext.scala | 12 ++- .../org/apache/spark/sql/SnappySession.scala | 84 +++++++++++++++---- .../columnar/impl/ColumnFormatRelation.scala | 36 +++++--- store | 2 +- 5 files changed, 117 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/io/snappydata/Literals.scala b/core/src/main/scala/io/snappydata/Literals.scala index 8ee24c3dee..6441749064 100644 --- a/core/src/main/scala/io/snappydata/Literals.scala +++ b/core/src/main/scala/io/snappydata/Literals.scala @@ -19,6 +19,7 @@ package io.snappydata import scala.reflect.ClassTag import com.gemstone.gemfire.internal.shared.unsafe.DirectBufferAllocator +import com.pivotal.gemfirexd.internal.engine.GfxdConstants import org.apache.spark.sql.execution.columnar.ExternalStoreUtils import org.apache.spark.sql.internal.{AltName, SQLAltName, SQLConfigEntry} @@ -188,6 +189,19 @@ object Property extends Enumeration { s"${Constant.PROPERTY_PREFIX}sql.planCaching", "Property to set/unset plan caching", Some(false)) + val SerializeWrites: SQLValue[Boolean] = SQLVal[Boolean]( + s"${Constant.PROPERTY_PREFIX}sql.serializeWrites", + "Property to set/unset serialized writes on column table." + + "There will be a global lock which will ensure that at a time only" + + "one write operation is active on the column table.", Some(true)) + + val SerializedWriteLockTimeOut: SQLValue[Int] = SQLVal[Int]( + s"${Constant.PROPERTY_PREFIX}sql.serializedWriteLockTimeOut", + "Property to specify the lock timeout for write ops in seconds. If the" + + " write operation doesn't get lock for write within this time period" + + s" then operation will fail. Default value is ${GfxdConstants.MAX_LOCKWAIT_DEFAULT/1000} sec", + Some(GfxdConstants.MAX_LOCKWAIT_DEFAULT/1000)) + val Tokenize: SQLValue[Boolean] = SQLVal[Boolean]( s"${Constant.PROPERTY_PREFIX}sql.tokenize", "Property to enable/disable tokenization", Some(true)) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala index 02ad52b3a6..d0edc41c91 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -47,7 +47,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.memory.MemoryManagerCallback import org.apache.spark.rdd.RDD -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerExecutorAdded} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.expressions.SortDirection import org.apache.spark.sql.collection.{ToolsCallbackInit, Utils} @@ -806,8 +806,11 @@ object SnappyContext extends Logging { @volatile private[this] var _globalContextInitialized: Boolean = false @volatile private[this] var _globalSNContextInitialized: Boolean = false + @volatile private[sql] var executorAssigned = false + private[this] var _globalClear: () => Unit = _ private[this] val contextLock = new AnyRef + private[sql] val resourceLock = new AnyRef @GuardedBy("contextLock") private var hiveSession: SparkSession = _ @@ -1204,6 +1207,13 @@ object SnappyContext extends Logging { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { stopSnappyContext() } + override def onExecutorAdded(execList: SparkListenerExecutorAdded): Unit = { + logDebug(s"SparkContextListener: onExecutorAdded: added $execList") + resourceLock.synchronized { + executorAssigned = true + resourceLock.notifyAll() + } + } } private def invokeServices(sc: SparkContext): Unit = { diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index dc1f9ed860..5fd8bbd95c 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -488,7 +488,10 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { // first acquire the global lock for putInto val (schemaName: String, _) = JdbcExtendedUtils.getTableWithSchema(table, conn = null, Some(sqlContext.sparkSession)) - val lock = grabLock(table, schemaName, defaultConnectionProps) + val lockOption = if (Property.SerializeWrites.get(sessionState.conf)) { + grabLock(table, schemaName, defaultConnectionProps) + } else None + var newUpdateSubQuery: Option[LogicalPlan] = None try { val cachedTable = if (doCache) { @@ -514,8 +517,13 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { addContextObject(SnappySession.CACHED_PUTINTO_LOGICAL_PLAN, cachedTable) newUpdateSubQuery } finally { - logDebug(s"Adding the lock object $lock to the context") - addContextObject(SnappySession.PUTINTO_LOCK, lock) + lockOption match { + case Some(lock) => { + logDebug(s"Adding the lock object $lock to the context") + addContextObject(SnappySession.PUTINTO_LOCK, lock) + } + case None => // do nothing + } } } @@ -546,9 +554,28 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { } private[sql] def grabLock(table: String, schemaName: String, - connProperties: ConnectionProperties): Any = { + connProperties: ConnectionProperties): Option[Any] = { SnappyContext.getClusterMode(sparkContext) match { case _: ThinClientConnectorMode => + if (!sparkContext.isLocal) { + SnappyContext.resourceLock.synchronized { + while (!SnappyContext.executorAssigned && sparkContext.getExecutorIds().isEmpty) { + if (!SnappyContext.executorAssigned) { + try { + SnappyContext.resourceLock.wait(100) + } + catch { + case _: InterruptedException => + logWarning("Interrupted while waiting for executor.") + } + } + // Don't expect this case usually unless lots of + // applications are submitted in parallel + logDebug(s"grabLock waiting for resources to be " + + s"allocated ${SnappyContext.executorAssigned}") + } + } + } val conn = ConnectionPool.getPoolConnection(table, connProperties.dialect, connProperties.poolProps, connProperties.connProps, connProperties.hikariCP) var locked = false @@ -556,55 +583,80 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { do { try { logDebug(s" Going to take lock on server for table $table," + - s" current Thread ${Thread.currentThread().getId}") - val ps = conn.prepareCall(s"VALUES sys.ACQUIRE_REGION_LOCK(?)") + s" current Thread ${Thread.currentThread().getId} and " + + s"app ${sqlContext.sparkContext.appName}") + + val ps = conn.prepareCall(s"VALUES sys.ACQUIRE_REGION_LOCK(?,?)") ps.setString(1, SnappySession.WRITE_LOCK_PREFIX + table) + ps.setInt(2, Property.SerializedWriteLockTimeOut.get(sessionState.conf) * 1000) val rs = ps.executeQuery() rs.next() locked = rs.getBoolean(1) ps.close() - logDebug("Took lock on server. ") + logDebug(s"Took lock on server. for string " + + s"${SnappySession.WRITE_LOCK_PREFIX + table} and " + + s"app ${sqlContext.sparkContext.appName}") } catch { - case e: Throwable => - logWarning("Got exception while taking lock", e) + case sqle: SQLException => { + logDebug("Got exception while taking lock", sqle) + if (sqle.getMessage.contains("Couldn't acquire lock")) { + throw sqle + } else { + if (retrycount == 2) { + throw sqle + } + } + } + case e: Throwable => { + logDebug("Got exception while taking lock", e) if (retrycount == 2) { throw e } + } } finally { retrycount = retrycount + 1 // conn.close() } } while (!locked) - (conn, new TableIdentifier(table, Some(schemaName))) + Some(conn, new TableIdentifier(table, Some(schemaName))) case _ => logDebug(s"Taking lock in " + - s" ${Thread.currentThread().getId} ") + s" ${Thread.currentThread().getId} and " + + s"app ${sqlContext.sparkContext.appName}") val regionLock = PartitionedRegion.getRegionLock(SnappySession.WRITE_LOCK_PREFIX + table, GemFireCacheImpl.getExisting) - regionLock.lock() - regionLock + regionLock.lock(Property.SerializedWriteLockTimeOut.get(sessionState.conf) * 1000) + Some(regionLock) } } private[sql] def releaseLock(lock: Any): Unit = { + logInfo(s"Releasing the lock : $lock") lock match { case lock: RegionLock => if (lock != null) { - logInfo(s"Going to unlock the lock object bulkOp $lock ") + logInfo(s"Going to unlock the lock object bulkOp $lock and " + + s"app ${sqlContext.sparkContext.appName}") lock.asInstanceOf[PartitionedRegion.RegionLock].unlock() } case (conn: Connection, id: TableIdentifier) => var unlocked = false try { - logDebug(s"releasing lock on the server. ${id.table}") + logDebug(s"Going to unlock the lock on the server. ${id.table} for " + + s"app ${sqlContext.sparkContext.appName}") val ps = conn.prepareStatement(s"VALUES sys.RELEASE_REGION_LOCK(?)") ps.setString(1, SnappySession.WRITE_LOCK_PREFIX + id.table) val rs = ps.executeQuery() rs.next() unlocked = rs.getBoolean(1) ps.close() + } catch { + case t: Throwable => { + logWarning(s"Caught exception while unlocking the $lock", t) + throw t + } } finally { conn.close() @@ -1994,6 +2046,7 @@ private class FinalizeSession(session: SnappySession) override protected def clearThis(): Unit = { sessionId = SnappySession.INVALID_ID } + } object SnappySession extends Logging { @@ -2006,7 +2059,6 @@ object SnappySession extends Logging { private[sql] val BULKWRITE_LOCK = "bulkwrite_lock" private[sql] val WRITE_LOCK_PREFIX = "BULKWRITE_" - private val unresolvedStarRegex = """(cannot resolve ')(\w+).(\w+).*(' given input columns.*)""".r private val unresolvedColRegex = diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala index 5370b7bdc2..2833f52a89 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala @@ -273,10 +273,11 @@ abstract class BaseColumnFormatRelation( // use bulk insert directly into column store for large number of rows val snc = sqlContext.sparkSession.asInstanceOf[SnappySession] - val lock = snc.getContextObject[(Option[TableIdentifier], PartitionedRegion.RegionLock)]( + val lockOption = snc.getContextObject[(Option[TableIdentifier], PartitionedRegion.RegionLock)]( SnappySession.PUTINTO_LOCK) match { - case None => snc.grabLock(table, schemaName, connProperties) - case Some(_) => null // Do nothing as putInto will release lock + case None if (Property.SerializeWrites.get(snc.sessionState.conf)) => + snc.grabLock(table, schemaName, connProperties) + case _ => None // Do nothing as putInto will release lock } try { if (numRows > (batchSize * numBuckets)) { @@ -300,28 +301,37 @@ abstract class BaseColumnFormatRelation( } } finally { - if (lock != null) { - logDebug(s"Releasing the $lock object in InsertRows") - snc.releaseLock(lock) + lockOption match { + case Some(lock) => { + logDebug(s"Releasing the $lock object in InsertRows") + snc.releaseLock(lock) + } + case None => // do Nothing } } } def withTableWriteLock()(f: () => SparkPlan): SparkPlan = { val snc = sqlContext.sparkSession.asInstanceOf[SnappySession] - val lock = snc.getContextObject[(Option[TableIdentifier], PartitionedRegion.RegionLock)]( + logDebug(s"WithTable WriteLock ${SnappyContext.executorAssigned}") + + val lockOption = snc.getContextObject[(Option[TableIdentifier], PartitionedRegion.RegionLock)]( SnappySession.PUTINTO_LOCK) match { - case None => snc.grabLock(table, schemaName, connProperties) - case Some(_) => null // Do nothing as putInto will release lock + case None if (Property.SerializeWrites.get(snc.sessionState.conf)) => + snc.grabLock(table, schemaName, connProperties) + case _ => None // Do nothing as putInto will release lock } try { f() } finally { - if (lock != null) { - logDebug(s"Added the $lock object to the context for $table") - snc.addContextObject( - SnappySession.BULKWRITE_LOCK, lock) + lockOption match { + case Some(lock) => { + logDebug(s"Added the $lock object to the context for $table") + snc.addContextObject( + SnappySession.BULKWRITE_LOCK, lock) + } + case None => // do nothing } } } diff --git a/store b/store index e20da1537f..f0aa8f74d1 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit e20da1537fd9f459d622548bec7a52fd50ad06e3 +Subproject commit f0aa8f74d1779a01ca9aced623c99fbfd4db0339 From e0030e781498485f8e16d8a8fa3c67516ddf476a Mon Sep 17 00:00:00 2001 From: ahshahid Date: Wed, 14 Aug 2019 17:46:32 -0700 Subject: [PATCH 13/26] =?UTF-8?q?Fix=20for=20SNAP-3141.=20This=20was=20a?= =?UTF-8?q?=20case=20where=20the=20input=20variables=20were=20eva=E2=80=A6?= =?UTF-8?q?=20(#1432)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Fix for SNAP-3141. This was a case where the input variables were evaluated inside a block to generate the group by key. However same base input variables were also needed by aggregate function. The fix is to first evaluate the input var code & then generate Key Expression code on the input var. * removed the redundant bug test --- .../test/scala/org/apache/spark/sql/store/BugTest.scala | 7 ++++--- .../org/apache/spark/sql/execution/SHAMapAccessor.scala | 8 ++++---- .../sql/execution/aggregate/SnappyHashAggregateExec.scala | 4 +++- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala index 6697d8520c..cae9b83cd8 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala @@ -1171,11 +1171,11 @@ class BugTest extends SnappyFunSuite with BeforeAndAfterAll { } - test("SNAP-3123: check for GUI plans") { + test("SNAP-3123: check for GUI plans and SNAP-3141: code gen failure") { // TODO: new SHA code generation fails for query below val session = snc.snappySession.newSession() - session.sql(s"set ${Property.UseOptimzedHashAggregate.name} = false") - session.sql(s"set ${Property.UseOptimizedHashAggregateForSingleKey.name} = false") + session.sql(s"set ${Property.UseOptimzedHashAggregate.name} = true") + session.sql(s"set ${Property.UseOptimizedHashAggregateForSingleKey.name} = true") val numRows = 1000000 val sleepTime = 7000L @@ -1202,4 +1202,5 @@ class BugTest extends SnappyFunSuite with BeforeAndAfterAll { assert(metrics(numRowsMetric.accumulatorId) === SQLMetrics.stringValue(numRowsMetric.metricType, numRows :: Nil)) } + } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala index 1d27c4ce57..250cdf062e 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala @@ -379,12 +379,13 @@ case class SHAMapAccessor(@transient session: SnappySession, }.mkString("\n") def declareNullVarsForAggBuffer(varNames: Seq[String]): String = - varNames.map(varName => s"boolean ${varName}${SHAMapAccessor.nullVarSuffix};").mkString("\n") + varNames.map(varName => s"boolean ${varName}${SHAMapAccessor.nullVarSuffix} = false;"). + mkString("\n") /** * Generate code to lookup the map or insert a new key, value if not found. */ def generateMapGetOrInsert(valueInitVars: Seq[ExprCode], - valueInitCode: String, input: Seq[ExprCode], keyVars: Seq[ExprCode], + valueInitCode: String, evaluatedInputCode: String, keyVars: Seq[ExprCode], keysDataType: Seq[DataType], aggregateDataTypes: Seq[DataType]): String = { val hashVar = Array(ctx.freshName("hash")) val tempValueData = ctx.freshName("tempValueData") @@ -396,13 +397,12 @@ case class SHAMapAccessor(@transient session: SnappySession, val insertDoneTerm = ctx.freshName("insertDone"); /* generateUpdate(objVar, Nil, valueInitVars, forKey = false, doCopy = false) */ - val inputEvals = evaluateVariables(input) s"""|$valueInitCode |${SHAMapAccessor.resetNullBitsetCode(nullKeysBitsetTerm, numBytesForNullKeyBits)} |${SHAMapAccessor.resetNullBitsetCode(nullAggsBitsetTerm, numBytesForNullAggBits)} // evaluate input row vars - |$inputEvals + |$evaluatedInputCode // evaluate key vars |${evaluateVariables(keyVars)} |${keyVars.zip(keysDataType).filter(_._2 match { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index 36d851747a..939fe91d8f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -1157,6 +1157,7 @@ case class SnappyHashAggregateExec( } } + val evaluatedInputCode = evaluateVariables(input) ctx.currentVars = input val keysExpr = ctx.generateExpressions( groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) @@ -1174,7 +1175,7 @@ case class SnappyHashAggregateExec( // to be materialized explicitly for the dictionary optimization case (AQP-292) val updateAttrs = AttributeSet(updateExpr) // evaluate map lookup code before updateEvals possibly modifies the keyVars - val mapCode = byteBufferAccessor.generateMapGetOrInsert(initVars, initCode, input, + val mapCode = byteBufferAccessor.generateMapGetOrInsert(initVars, initCode, evaluatedInputCode, keysExpr, keysDataType, aggBuffDataTypes) val bufferVars = byteBufferAccessor.getBufferVars(aggBuffDataTypes, @@ -1234,6 +1235,7 @@ case class SnappyHashAggregateExec( | |// evaluate aggregate functions |${evaluateVariables(updateEvals)} + |// generate update |${byteBufferAccessor.generateUpdate(updateEvals, aggBuffDataTypes)} """.stripMargin } From cc0754200951955635030259caf680e1c42e5835 Mon Sep 17 00:00:00 2001 From: Vatsal Mevada Date: Thu, 15 Aug 2019 10:22:11 +0530 Subject: [PATCH 14/26] - Enhancing docs for structured streaming CDC example --- .../structuredstreaming/CDCExample.scala | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala index 2431516e6e..09361ab01c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala @@ -25,12 +25,25 @@ import scala.language.postfixOps import scala.reflect.io.Path /** - * An example showing CDC usage with SnappyData streaming sink. + * An example explaining CDC (change data capture) use case with SnappyData streaming sink. * - * To run this on your local machine, you need to first run a Netcat server
+ * For CDC use case following two conditions should match: + * 1) The target table must be defined with key columns (for column tables) or primary keys ( for + * row table). + * 2) The input dataset must have an numeric column with name `_eventType` indicating type of the + * event. The value of this column is mapped with event type in the following manner: + * + * 0 - insert + * 1 - putInto + * 2 - delete + * + * Based on the key values in the incoming dataset and the value of `_eventType` column the sink + * will decide which operation need to be performed for each record. + * + * To run this on your local machine, you need to first run a Netcat server: * `$ nc -lk 9999` * - * Example input data: + * Example input data. Note that the last value from CSV record indicates the `_eventType`: * * 1,user1,23,0 * 2,user2,45,0 @@ -63,7 +76,7 @@ object CDCExample{ import spark.implicits._ val snappy = new SnappySession(spark.sparkContext) - + // The target table is created with key columns (for column table) for CDC use case snappy.sql("create table users (id long , name varchar(40), age int) using" + " column options(key_columns 'id')") @@ -75,9 +88,11 @@ object CDCExample{ .option("port", 9999) .load() - // Creating a typed User from raw string received on socket. + // Creating a typed User from raw string received on socket val structDF = socketDF.as[String].map(s => { val fields = s.split(",") + // Note that the fourth field of `User` class is `_eventType`. Value of this field indicates + // the type of the DML operation to be performed. User(fields(0).toLong, fields(1), fields(2).toInt, fields(3).toInt) }) From e271b7ac2ab631e0c791aa10acd8317246437152 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 15 Aug 2019 21:07:17 +0530 Subject: [PATCH 15/26] remove execution from active list at the end of "plan" phase when the "plan" phase of SQL ends i.e. sql() before a collect()/count(), then remove query from running list else it can remain in dangling phase in case collect() is never issued --- .../org/apache/spark/sql/store/BugTest.scala | 2 -- .../org/apache/spark/sql/SnappySession.scala | 17 ++++++++++++----- .../sql/execution/ui/SnappySQLListener.scala | 13 ++++++++++--- store | 2 +- 4 files changed, 23 insertions(+), 11 deletions(-) diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala index cae9b83cd8..3860fed5c4 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala @@ -1172,7 +1172,6 @@ class BugTest extends SnappyFunSuite with BeforeAndAfterAll { } test("SNAP-3123: check for GUI plans and SNAP-3141: code gen failure") { - // TODO: new SHA code generation fails for query below val session = snc.snappySession.newSession() session.sql(s"set ${Property.UseOptimzedHashAggregate.name} = true") session.sql(s"set ${Property.UseOptimizedHashAggregateForSingleKey.name} = true") @@ -1202,5 +1201,4 @@ class BugTest extends SnappyFunSuite with BeforeAndAfterAll { assert(metrics(numRowsMetric.accumulatorId) === SQLMetrics.stringValue(numRowsMetric.metricType, numRows :: Nil)) } - } diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index 5fd8bbd95c..0068830cc6 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -63,7 +63,7 @@ import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} -import org.apache.spark.sql.execution.ui.SparkListenerSQLPlanExecutionStart +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLPlanExecutionEnd, SparkListenerSQLPlanExecutionStart} import org.apache.spark.sql.hive.{HiveClientUtil, SnappySessionState} import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD import org.apache.spark.sql.internal.{BypassRowLevelSecurity, MarkerForCreateTableAsSelect, SnappySessionCatalog, SnappySharedState, StaticSQLConf} @@ -2141,7 +2141,7 @@ object SnappySession extends Logging { val context = session.sparkContext val localProperties = context.getLocalProperties setExecutionProperties(localProperties, executionIdStr, sqlText) - var propertiesSet = true + var success = false val start = System.currentTimeMillis() try { // get below two with original "ParamLiteral(" tokens that will be replaced @@ -2156,12 +2156,19 @@ object SnappySession extends Logging { executionId, CachedDataFrame.queryStringShortForm(sqlText), sqlText, postQueryExecutionStr, postQueryPlanInfo, start)) val rdd = f - clearExecutionProperties(localProperties) - propertiesSet = false + success = true (rdd, queryExecutionStr, queryPlanInfo, postQueryExecutionStr, postQueryPlanInfo, executionId, start, System.currentTimeMillis()) } finally { - if (propertiesSet) clearExecutionProperties(localProperties) + clearExecutionProperties(localProperties) + if (success) { + // post the end of "plan" phase which will remove this execution from active list + context.listenerBus.post(SparkListenerSQLPlanExecutionEnd(executionId)) + } else { + // post the end of SQL since body of `f` failed + context.listenerBus.post(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) + } } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala index 763bef2a43..832346e61e 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala @@ -35,6 +35,8 @@ case class SparkListenerSQLPlanExecutionStart( time: Long) extends SparkListenerEvent +case class SparkListenerSQLPlanExecutionEnd(executionId: Long) extends SparkListenerEvent + /** * Snappy's SQL Listener. * @@ -133,14 +135,15 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { case SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) => synchronized { - baseExecutionIdToData.get(executionId) match { + val executionUIData = baseExecutionIdToData.get(executionId) match { case None => val executionUIData = newExecutionUIData(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) baseExecutionIdToData(executionId) = executionUIData - baseActiveExecutions(executionId) = executionUIData - case _ => + executionUIData + case Some(d) => d } + baseActiveExecutions(executionId) = executionUIData } case SparkListenerSQLPlanExecutionStart(executionId, description, details, @@ -152,6 +155,10 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { baseActiveExecutions(executionId) = executionUIData } + case SparkListenerSQLPlanExecutionEnd(executionId) => synchronized { + baseActiveExecutions.remove(executionId) + } + case _ => super.onOtherEvent(event) } } diff --git a/store b/store index f0aa8f74d1..bc1cb03aee 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit f0aa8f74d1779a01ca9aced623c99fbfd4db0339 +Subproject commit bc1cb03aee0c4242be65fcd22ed0bd561327f651 From a1e5bb71e4a55fa4a5a2586ce80cfa474009b830 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 16 Aug 2019 00:24:33 +0530 Subject: [PATCH 16/26] update store link --- store | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store b/store index bc1cb03aee..efc0138414 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit bc1cb03aee0c4242be65fcd22ed0bd561327f651 +Subproject commit efc0138414d07814960e0f6f96a4b5d2aef5551f From a4e3d2eaa45826b75d4dc797ac2a4f9ca0fbc86d Mon Sep 17 00:00:00 2001 From: ahshahid Date: Thu, 15 Aug 2019 12:36:42 -0700 Subject: [PATCH 17/26] added reliable bug test for SNAP-3132 --- .../spark/sql/store/SHAByteBufferTest.scala | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala index ae2ede78db..0d58d5c87e 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala @@ -1289,27 +1289,22 @@ class SHAByteBufferTest extends SnappyFunSuite with BeforeAndAfterAll { test("SNAP-3132") { snc snc.sql("drop table if exists test1") - snc.sql("drop table if exists test2") - snc.sql("drop table if exists test3") - snc.sql("create table test1 (col1 int, col2 int, col3 Decimal(7,2)) using column") + snc.sql("create table test1 (col1 int, col2 int, col3 Decimal(35,5), name string) using column") val conn = getSqlConnection - val ps1 = conn.prepareStatement("insert into test1 values (?,?, ?)") - for(i <- 0 until 500) { + val ps1 = conn.prepareStatement("insert into test1 values (?,?, ?, ?)") + for (i <- 0 until 500) { ps1.setInt(1, i % 5) ps1.setInt(2, i % 10) - val bd = new BigDecimal(174.576d * i) + val bd = new BigDecimal(17456567.576d * i) ps1.setBigDecimal(3, bd) + ps1.setString(4, (i % 10).toString) ps1.addBatch } - ps1.executeBatch - - - snc.sql(s" select 'asif' as name, col1, col2, Cast(sum(col3) as Decimal(15,4)) from test1" + - s" group by name, col1, col2 ").collect + snc.sql(s" select name, col3, sum(col2) from test1" + + s" group by name, col3").collect snc.dropTable("test1") - } ignore("SNAP-3077 test if default max capacity nearing Integer.MAX_VALUE is reached." + From e3182a5aecfbe12ec772dab5511638602a409b5e Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 17 Aug 2019 12:36:44 +0530 Subject: [PATCH 18/26] [SNAP-3142] clear local properties of a new session for JDBC connection --- .../main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala | 1 + core/src/main/scala/org/apache/spark/sql/collection/Utils.scala | 2 ++ 2 files changed, 3 insertions(+) diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala index cea16e8db8..a2f7e54ad8 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala @@ -500,6 +500,7 @@ object CreateNewSession extends java.util.function.Function[java.lang.Long, Snap case null => throw new CacheClosedException("No SparkContext ...") case sc => new SnappySession(sc) } + Utils.getLocalProperties(session.sparkContext).clear() Property.PlanCaching.set(session.sessionState.conf, true) session } diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 4323960ba8..9cb4e15190 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -570,6 +570,8 @@ object Utils extends Logging { def setCurrentSchema(session: SnappySession, schema: String, createIfNotExists: Boolean): Unit = session.setCurrentSchema(schema, createIfNotExists) + def getLocalProperties(sc: SparkContext): java.util.Properties = sc.getLocalProperties + def getDriverClassName(url: String): String = DriverManager.getDriver(url) match { case wrapper: DriverWrapper => wrapper.wrapped.getClass.getCanonicalName case driver => driver.getClass.getCanonicalName From 759f67b00a3bab7f1348c84dbbb142176608948b Mon Sep 17 00:00:00 2001 From: Sachin Kapse Date: Mon, 19 Aug 2019 18:08:57 +0530 Subject: [PATCH 19/26] Linking latest spark submodule. --- spark | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark b/spark index d8abf0b3e6..bb5ca692d2 160000 --- a/spark +++ b/spark @@ -1 +1 @@ -Subproject commit d8abf0b3e60c640fda521bf40136d21d678b20da +Subproject commit bb5ca692d2fd8551c3f5b4a3dacbe917fc40ad54 From 6326d8f7b6391361549ce8c1db897b761d12f1af Mon Sep 17 00:00:00 2001 From: Amogh Shetkar Date: Tue, 20 Aug 2019 18:24:16 +0530 Subject: [PATCH 20/26] Header update 1.1.1 (#1434) * Copyright headers updated. --- README.md | 6 +++--- build.gradle | 10 +++++----- cluster/bin/load-snappy-env.sh | 2 +- cluster/bin/snappy-job.sh | 2 +- cluster/bin/snappy-sql | 2 +- cluster/build.gradle | 2 +- cluster/conf/leads.template | 2 +- cluster/conf/locators.template | 2 +- cluster/conf/log4j.properties.template | 2 +- cluster/conf/servers.template | 2 +- cluster/sbin/check-dir-option.sh | 2 +- cluster/sbin/cluster-util.sh | 4 ++-- cluster/sbin/collect-debug-artifacts.sh | 2 +- cluster/sbin/snappy-config.sh | 2 +- cluster/sbin/snappy-lead.sh | 2 +- cluster/sbin/snappy-leads.sh | 2 +- cluster/sbin/snappy-locator.sh | 2 +- cluster/sbin/snappy-locators.sh | 2 +- cluster/sbin/snappy-nodes.sh | 2 +- cluster/sbin/snappy-server.sh | 2 +- cluster/sbin/snappy-servers.sh | 2 +- cluster/sbin/snappy-start-all.sh | 2 +- cluster/sbin/snappy-status-all.sh | 2 +- cluster/sbin/snappy-stop-all.sh | 2 +- .../HiveMetastoreClientAccessDUnitTest.scala | 2 +- .../cluster/ClusterManagerLDAPTestBase.scala | 2 +- .../io/snappydata/cluster/ClusterManagerTestBase.scala | 2 +- .../io/snappydata/cluster/ClusterMgrDUnitTest.scala | 2 +- .../ConcurrentQueryRoutingDUnitSecurityTest.scala | 2 +- .../cluster/ConcurrentQueryRoutingDUnitTest.scala | 2 +- .../io/snappydata/cluster/DDLRoutingDUnitTest.scala | 2 +- .../snappydata/cluster/DistributedIndexDUnitTest.scala | 2 +- .../cluster/ExecutionEngineArbiterDUnitTest.scala | 2 +- .../cluster/PreparedQueryRoutingDUnitTest.scala | 2 +- .../cluster/QueryRoutingDUnitSecurityTest.scala | 2 +- .../io/snappydata/cluster/QueryRoutingDUnitTest.scala | 2 +- .../io/snappydata/cluster/SetIsolationDUnitTest.scala | 2 +- .../cluster/SnappyResourceEventsDUnitTest.scala | 2 +- .../snappydata/cluster/SnappyRowStoreModeDUnit.scala | 2 +- .../cluster/SnappyTableStatsProviderDUnitTest.scala | 2 +- .../cluster/SplitSnappyClusterDUnitTest.scala | 2 +- .../snappydata/cluster/StringAsVarcharDUnitTest.scala | 2 +- .../io/snappydata/cluster/ValidateMVCCDUnitTest.scala | 2 +- .../cluster/jobs/CassandraSnappyConnectionJob.scala | 2 +- .../io/snappydata/cluster/jobs/SnappySecureJob.scala | 2 +- .../externalstore/CatalogConsistencyDUnitTest.scala | 2 +- .../externalstore/ColumnTableDUnitTest.scala | 2 +- .../externalstore/ExecutorMessageDUnitTest.scala | 2 +- .../externalstore/JDBCPreparedStatementDUnitTest.scala | 2 +- .../snappydata/externalstore/RowTableDUnitTest.scala | 2 +- .../io/snappydata/streaming/StreamingDUnitTest.scala | 2 +- .../apache/spark/DynamicJarInstallationDUnitTest.scala | 2 +- .../apache/spark/jdbc/ConnectionConfDUnitTest.scala | 2 +- .../spark/memory/MemoryManagerRestartDUnitTest.scala | 2 +- .../memory/SnappyUnifiedMemoryManagerDUnitTest.scala | 2 +- .../sql/ColumnBatchAndExternalTableDUnitTest.scala | 2 +- .../org/apache/spark/sql/NorthWindDUnitTest.scala | 2 +- .../org/apache/spark/sql/SmartConnectorFunctions.scala | 2 +- .../scala/org/apache/spark/sql/TPCHDUnitTest.scala | 2 +- .../dunit/scala/org/apache/spark/sql/TPCHSuite.scala | 2 +- .../spark/sql/udf/UserDefinedFunctionsDUnitTest.scala | 2 +- .../spark/transaction/SnapshotGIIDUnitTest.scala | 2 +- .../java/io/snappydata/gemxd/SnappySystemAdmin.java | 2 +- .../apache/spark/streaming/JavaSnappyStreamingJob.java | 2 +- .../src/main/scala/io/snappydata/ServiceManager.scala | 2 +- .../main/scala/io/snappydata/ToolsCallbackImpl.scala | 2 +- .../io/snappydata/cluster/ExecutorInitiator.scala | 2 +- .../io/snappydata/gemxd/ClusterCallbacksImpl.scala | 2 +- .../scala/io/snappydata/gemxd/SnappyDataVersion.scala | 2 +- .../io/snappydata/gemxd/SparkSQLExecuteImpl.scala | 2 +- .../io/snappydata/gemxd/SparkSQLPrepareImpl.scala | 2 +- .../src/main/scala/io/snappydata/impl/LeadImpl.scala | 2 +- .../main/scala/io/snappydata/impl/LocatorImpl.scala | 2 +- .../io/snappydata/tools/GfxdLauncherOverrides.scala | 2 +- .../scala/io/snappydata/tools/LeaderLauncher.scala | 2 +- .../scala/io/snappydata/tools/SnappyUtilLauncher.scala | 2 +- .../main/scala/org/apache/spark/SparkCallbacks.scala | 2 +- .../org/apache/spark/deploy/PackageAndDepUtils.scala | 2 +- .../executor/SnappyCoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/executor/SnappyExecutor.scala | 2 +- .../spark/memory/MemoryManagerStatsWrapper.scala | 2 +- .../org/apache/spark/memory/SnappyMemoryUtils.scala | 2 +- .../org/apache/spark/memory/SnappyStorageEvictor.scala | 2 +- .../spark/memory/SnappyUnifiedMemoryManager.scala | 2 +- .../spark/scheduler/SnappyTaskSchedulerImpl.scala | 2 +- .../cluster/SnappyCoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/SnappyEmbeddedModeClusterManager.scala | 2 +- .../org/apache/spark/sql/SnappySessionFactory.scala | 2 +- .../hive/thriftserver/SnappyHiveThriftServer2.scala | 2 +- .../sql/streaming/SnappyStreamingContextFactory.scala | 2 +- .../status/api/v1/AllExternalTablesResource.scala | 2 +- .../spark/status/api/v1/AllMembersResource.scala | 2 +- .../apache/spark/status/api/v1/AllTablesResource.scala | 2 +- .../apache/spark/status/api/v1/ClusterDetails.scala | 2 +- .../spark/status/api/v1/ClusterInfoResource.scala | 2 +- .../org/apache/spark/status/api/v1/MemberDetails.scala | 2 +- .../spark/status/api/v1/MembersDetailsResource.scala | 2 +- .../spark/status/api/v1/SnappyApiRootResource.scala | 2 +- .../org/apache/spark/status/api/v1/TableDetails.scala | 2 +- .../org/apache/spark/status/api/v1/snappyapi.scala | 2 +- .../org/apache/spark/ui/SnappyBasicAuthenticator.scala | 2 +- .../org/apache/spark/ui/SnappyDashboardPage.scala | 2 +- .../scala/org/apache/spark/ui/SnappyDashboardTab.scala | 2 +- .../org/apache/spark/ui/SnappyMemberDetailsPage.scala | 2 +- .../scala/org/apache/spark/ui/SnappyStatsPage.scala | 2 +- .../scala/org/apache/spark/ui/SnappyStatsTab.scala | 2 +- .../main/scala/org/apache/spark/util/SnappyUtils.scala | 2 +- cluster/src/test/resources/log4j.properties | 2 +- cluster/src/test/scala/io/snappydata/QueryTest.scala | 2 +- cluster/src/test/scala/io/snappydata/Snap_213.scala | 2 +- .../benchmark/LoadPerformance/BulkLoad_Snappy.scala | 2 +- .../benchmark/LoadPerformance/BulkLoad_Spark.scala | 2 +- .../benchmark/LoadPerformance/ParquetLoad.scala | 2 +- .../benchmark/TPCHColumnPartitionedTable.scala | 2 +- .../io/snappydata/benchmark/TPCHReplicatedTable.scala | 2 +- .../snappydata/benchmark/TPCHRowPartitionedTable.scala | 2 +- .../io/snappydata/benchmark/TPCHTableSchema.scala | 2 +- .../scala/io/snappydata/benchmark/TPCH_Queries.scala | 2 +- .../snappydata/benchmark/kuduimpala/TPCH_Impala.scala | 2 +- .../benchmark/kuduimpala/TPCH_Impala_Query.scala | 2 +- .../benchmark/kuduimpala/TPCH_Impala_Tables.scala | 2 +- .../snappydata/benchmark/memsql/ConcurrentMemsql.scala | 2 +- .../io/snappydata/benchmark/memsql/TPCH_Memsql.scala | 2 +- .../benchmark/memsql/TPCH_Memsql_Query.scala | 2 +- .../memsql/TPCH_Memsql_Query_StreamExecution.scala | 2 +- .../benchmark/memsql/TPCH_Memsql_StreamExecution.scala | 2 +- .../benchmark/memsql/TPCH_Memsql_Tables.scala | 2 +- .../io/snappydata/benchmark/snappy/TPCDSSuite.scala | 2 +- .../scala/io/snappydata/benchmark/snappy/TPCH.scala | 2 +- .../snappydata/benchmark/snappy/TPCHPerfComparer.java | 2 +- .../benchmark/snappy/tpcds/QueryExecutionJob.scala | 2 +- .../snappydata/benchmark/snappy/tpcds/SparkApp.scala | 2 +- .../benchmark/snappy/tpch/DataValidationJob.scala | 2 +- .../benchmark/snappy/tpch/QueryExecutionJdbc.scala | 2 +- .../benchmark/snappy/tpch/QueryExecutionJob.scala | 2 +- .../snappy/tpch/QueryExecutionSmartConnector.scala | 2 +- .../benchmark/snappy/tpch/QueryExecutor.scala | 2 +- .../io/snappydata/benchmark/snappy/tpch/SparkApp.scala | 2 +- .../benchmark/snappy/tpch/SparkAppUsingJob.scala | 2 +- .../benchmark/snappy/tpch/TableCreationJob.scala | 2 +- .../snappy/tpch/TableCreationSmartConnector.scala | 2 +- .../io/snappydata/benchmark/snappy/tpchmodifiers.scala | 2 +- .../cluster/JDBCConnectionPoolTestSuite.scala | 2 +- .../cluster/PreparedQueryRoutingSingleNodeSuite.scala | 2 +- .../cluster/QueryRoutingSingleNodeSuite.scala | 2 +- .../io/snappydata/cluster/StringAsClobTestSuite.scala | 2 +- .../scala/io/snappydata/filodb/FiloDb_SnappyJob.scala | 2 +- .../scala/io/snappydata/tools/LeaderLauncherSpec.scala | 2 +- .../scala/org/apache/spark/memory/MemoryFunSuite.scala | 2 +- .../apache/spark/memory/MemoryManagerStatsSuite.scala | 2 +- .../spark/memory/SnappyLocalIndexAccountingSuite.scala | 2 +- .../spark/memory/SnappyMemoryAccountingSuite.scala | 2 +- .../spark/memory/SnappyStorageEvictorSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/IndexTest.scala | 2 +- .../spark/sql/LikeEscapeSimplificationSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/MiscTest.scala | 2 +- .../test/scala/org/apache/spark/sql/NWQueries.scala | 2 +- .../scala/org/apache/spark/sql/NorthWindTest.scala | 2 +- .../org/apache/spark/sql/SQLFunctionsTestSuite.scala | 2 +- .../scala/org/apache/spark/sql/SingleNodeTest.scala | 2 +- .../org/apache/spark/sql/SnappySQLQuerySuite.scala | 2 +- .../org/apache/spark/sql/execution/DataGenerator.scala | 2 +- .../sql/execution/SnappyTableMutableAPISuite.scala | 2 +- .../sql/execution/benchmark/ColumnCacheBenchmark.scala | 2 +- .../apache/spark/sql/execution/benchmark/MapTest.scala | 2 +- .../sql/execution/benchmark/StringBenchmark.scala | 2 +- .../apache/spark/sql/execution/benchmark/TAQTest.scala | 2 +- .../benchmark/TPCDSQuerySnappyBenchmark.scala | 2 +- .../sql/kafka010/SnappyStructuredKafkaSuite.scala | 2 +- .../policy/AlterTableRowLevelSecurityEnableTest.scala | 2 +- .../apache/spark/sql/policy/PolicyJdbcClientTest.scala | 2 +- .../scala/org/apache/spark/sql/policy/PolicyTest.scala | 2 +- .../org/apache/spark/sql/policy/PolicyTestBase.scala | 2 +- .../sql/policy/RestrictTableCreationPolicyTest.scala | 2 +- .../policy/SecurityEnabledJdbcClientPolicyTest.scala | 2 +- .../spark/sql/policy/SecurityEnabledPolicyTest.scala | 2 +- .../scala/org/apache/spark/sql/store/BitSetTest.scala | 2 +- .../scala/org/apache/spark/sql/store/BugTest.scala | 2 +- .../apache/spark/sql/store/ColumnEncodersTest.scala | 2 +- .../org/apache/spark/sql/store/ColumnMutableTest.scala | 2 +- .../apache/spark/sql/store/ColumnTablesTestBase.scala | 2 +- .../spark/sql/store/ColumnUpdateDeleteTest.scala | 2 +- .../org/apache/spark/sql/store/ComplexTypesTest.scala | 2 +- .../org/apache/spark/sql/store/RowMutableTest.scala | 2 +- .../org/apache/spark/sql/store/SHAByteBufferTest.scala | 2 +- .../org/apache/spark/sql/store/SQLMetadataTest.scala | 2 +- .../org/apache/spark/sql/store/SecurityBugTest.scala | 2 +- .../org/apache/spark/sql/store/SnappyUDFTest.scala | 2 +- .../unsafe/NativeUTF8StringPropertyCheckSuite.scala | 2 +- compatibilityTests/build.gradle | 2 +- compatibilityTests/src/test/resources/log4j.properties | 2 +- .../scala/com/fruit/eyephone/CameraInputFormat.scala | 2 +- .../scala/com/fruit/eyephone/CameraOutputFormat.scala | 2 +- .../apache/spark/sql/SnappyAggregateHashMapSuite.scala | 2 +- .../sql/SnappyApproximatePercentileQuerySuite.scala | 2 +- .../org/apache/spark/sql/SnappyCachedTableSuite.scala | 2 +- .../org/apache/spark/sql/SnappyCatalogSuite.scala | 2 +- .../spark/sql/SnappyCoGroupedIteratorSuite.scala | 2 +- .../apache/spark/sql/SnappyColumnExpressionSuite.scala | 2 +- .../spark/sql/SnappyCreateTableAsSelectSuite.scala | 2 +- .../apache/spark/sql/SnappyDDLSourceLoadSuite.scala | 2 +- .../org/apache/spark/sql/SnappyDDLTestSuite.scala | 2 +- .../spark/sql/SnappyDataFrameAggregateSuite.scala | 2 +- .../spark/sql/SnappyDataFrameCallbackSuite.scala | 2 +- .../spark/sql/SnappyDataFrameComplexTypeSuite.scala | 2 +- .../spark/sql/SnappyDataFrameFunctionsSuite.scala | 2 +- .../spark/sql/SnappyDataFrameImplicitsSuite.scala | 2 +- .../apache/spark/sql/SnappyDataFrameJoinSuite.scala | 2 +- .../spark/sql/SnappyDataFrameNaFunctionsSuite.scala | 2 +- .../apache/spark/sql/SnappyDataFramePivotSuite.scala | 2 +- .../spark/sql/SnappyDataFrameReaderWriterSuite.scala | 2 +- .../spark/sql/SnappyDataFrameStatPerfSuite.scala | 2 +- .../org/apache/spark/sql/SnappyDataFrameSuite.scala | 2 +- .../spark/sql/SnappyDataFrameTimeWindowingSuite.scala | 2 +- .../spark/sql/SnappyDataFrameTungstenSuite.scala | 2 +- .../sql/SnappyDataFrameWindowFunctionsSuite.scala | 2 +- .../spark/sql/SnappyDatasetAggregatorSuite.scala | 2 +- .../org/apache/spark/sql/SnappyDatasetCacheSuite.scala | 2 +- .../apache/spark/sql/SnappyDatasetPrimitiveSuite.scala | 2 +- .../sql/SnappyDatasetSerializerRegistratorSuite.scala | 2 +- .../org/apache/spark/sql/SnappyDatasetSuite.scala | 2 +- .../apache/spark/sql/SnappyDateFunctionsSuite.scala | 2 +- .../spark/sql/SnappyEventTimeWatermarkSuite.scala | 2 +- .../spark/sql/SnappyExchangeCoordinatorSuite.scala | 2 +- .../org/apache/spark/sql/SnappyExchangeSuite.scala | 2 +- .../apache/spark/sql/SnappyExtraStrategiesSuite.scala | 2 +- .../apache/spark/sql/SnappyFileStreamSinkSuite.scala | 2 +- .../apache/spark/sql/SnappyFileStreamSourceSuite.scala | 2 +- .../apache/spark/sql/SnappyFileStreamStressSuite.scala | 2 +- .../org/apache/spark/sql/SnappyFilteredScanSuite.scala | 2 +- .../org/apache/spark/sql/SnappyFiltersSuite.scala | 2 +- .../spark/sql/SnappyGeneratorFunctionSuite.scala | 2 +- .../apache/spark/sql/SnappyGlobalTempViewSuite.scala | 2 +- .../apache/spark/sql/SnappyGroupedIteratorSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyInsertSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyJDBCSuite.scala | 2 +- .../org/apache/spark/sql/SnappyJDBCWriteSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyJoinSuite.scala | 2 +- .../apache/spark/sql/SnappyJsonFunctionsSuite.scala | 2 +- .../apache/spark/sql/SnappyMathFunctionsSuite.scala | 2 +- .../spark/sql/SnappyMemorySourceStressSuite.scala | 2 +- .../apache/spark/sql/SnappyMetadataCacheSuite.scala | 2 +- .../apache/spark/sql/SnappyMiscFunctionsSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyOffsetSuite.scala | 2 +- .../sql/SnappyOptimizeMetadataOnlyQuerySuite.scala | 2 +- .../apache/spark/sql/SnappyPartitionedWriteSuite.scala | 2 +- .../org/apache/spark/sql/SnappyPathOptionSuite.scala | 2 +- .../org/apache/spark/sql/SnappyPlannerSuite.scala | 2 +- .../org/apache/spark/sql/SnappyPrunedScanSuite.scala | 2 +- .../apache/spark/sql/SnappyQueryExecutionSuite.scala | 2 +- .../org/apache/spark/sql/SnappyQueryTestSuite.scala | 2 +- .../apache/spark/sql/SnappyReduceAggregatorSuite.scala | 2 +- .../spark/sql/SnappyResolvedDataSourceSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyRowSuite.scala | 2 +- .../org/apache/spark/sql/SnappySQLConfEntrySuite.scala | 2 +- .../org/apache/spark/sql/SnappySQLConfSuite.scala | 2 +- .../org/apache/spark/sql/SnappySQLExecutionSuite.scala | 2 +- .../org/apache/spark/sql/SnappySQLQuerySuite.scala | 2 +- .../spark/sql/SnappySQLWindowFunctionSuite.scala | 2 +- .../org/apache/spark/sql/SnappySaveLoadSuite.scala | 2 +- .../spark/sql/SnappyScalaReflectionRelationSuite.scala | 2 +- .../apache/spark/sql/SnappySerializationSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappySortSuite.scala | 2 +- .../org/apache/spark/sql/SnappySparkPlannerSuite.scala | 2 +- .../apache/spark/sql/SnappySparkSqlParserSuite.scala | 2 +- .../spark/sql/SnappyStatisticsCollectionSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyStreamSuite.scala | 2 +- .../spark/sql/SnappyStreamingAggregationSuite.scala | 2 +- .../spark/sql/SnappyStreamingQueryListenerSuite.scala | 2 +- .../spark/sql/SnappyStreamingQueryManagerSuite.scala | 2 +- .../SnappyStreamingQueryStatusAndProgressSuite.scala | 2 +- .../apache/spark/sql/SnappyStreamingQuerySuite.scala | 2 +- .../apache/spark/sql/SnappyStringFunctionsSuite.scala | 2 +- .../org/apache/spark/sql/SnappySubquerySuite.scala | 2 +- .../org/apache/spark/sql/SnappyTableScanSuite.scala | 2 +- .../spark/sql/SnappyTakeOrderedAndProjectSuite.scala | 2 +- .../sql/SnappyTypedImperativeAggregateSuite.scala | 2 +- .../scala/org/apache/spark/sql/SnappyUDFSuite.scala | 2 +- .../SnappyUnsafeFixedWidthAggregationMapSuite.scala | 2 +- .../spark/sql/SnappyUnsafeKVExternalSorterSuite.scala | 2 +- .../apache/spark/sql/SnappyUserDefinedTypeSuite.scala | 2 +- .../spark/sql/SnappyVariableSubstitutionSuite.scala | 2 +- .../spark/sql/SnappyWholeStageCodegenSuite.scala | 2 +- .../apache/spark/sql/SnappyXPathFunctionsSuite.scala | 2 +- .../sql/catalyst/SnappyExpressionSQLBuilderSuite.scala | 2 +- .../execution/streaming/SnappyForeachSinkSuite.scala | 2 +- .../streaming/SnappyStreamMetadataSuite.scala | 2 +- .../streaming/SnappyTextSocketStreamSuite.scala | 2 +- .../apache/spark/sql/hive/TestHiveSnappySession.scala | 2 +- .../spark/sql/kafka010/SnappyKafkaSinkSuite.scala | 2 +- .../spark/sql/kafka010/SnappyKafkaSourceSuite.scala | 2 +- .../spark/sql/streaming/SnappyMemorySinkSuite.scala | 2 +- .../test/SnappyDataStreamReaderWriterSuite.scala | 2 +- .../spark/sql/test/SharedSnappySessionContext.scala | 2 +- .../apache/spark/sql/test/SnappySparkTestUtil.scala | 2 +- .../org/apache/spark/sql/test/TestSnappySession.scala | 2 +- core/build.gradle | 2 +- .../snappydata/cluster/CassandraSnappyDUnitTest.scala | 2 +- .../io/snappydata/cluster/SnappyJobTestSupport.scala | 2 +- .../cluster/SplitClusterDUnitSecurityTest.scala | 2 +- .../io/snappydata/cluster/SplitClusterDUnitTest.scala | 2 +- .../snappydata/cluster/SplitClusterDUnitTestBase.scala | 2 +- .../scala/org/apache/spark/SparkUtilsAccess.scala | 2 +- .../sql/streaming/SnappySinkProviderDUnitTest.scala | 2 +- .../main/java/io/snappydata/impl/SnappyHiveConf.java | 2 +- .../clearspring/analytics/stream/StreamSummary.java | 2 +- .../analytics/stream/membership/Filter.java | 2 +- .../main/java/org/apache/spark/SnappyJavaUtils.java | 2 +- .../apache/spark/sql/collection/BoundedSortedSet.java | 2 +- .../apache/spark/sql/hive/SnappyHiveCatalogBase.java | 2 +- .../apache/spark/sql/internal/SnappySharedState.java | 2 +- core/src/main/scala/io/snappydata/Literals.scala | 2 +- .../main/scala/io/snappydata/LocalizedMessages.scala | 2 +- core/src/main/scala/io/snappydata/ServerManager.scala | 2 +- core/src/main/scala/io/snappydata/Services.scala | 2 +- .../snappydata/SnappyTableStatsProviderService.scala | 2 +- .../SnappyThinConnectorTableStatsProvider.scala | 2 +- .../io/snappydata/TableStatsProviderService.scala | 2 +- core/src/main/scala/io/snappydata/ToolsCallback.scala | 2 +- core/src/main/scala/io/snappydata/functions.scala | 2 +- .../io/snappydata/impl/ComplexTypeSerializerImpl.scala | 2 +- .../src/main/scala/io/snappydata/impl/ServerImpl.scala | 2 +- .../io/snappydata/impl/SmartConnectorRDDHelper.scala | 2 +- .../sql/catalog/ConnectorExternalCatalog.scala | 2 +- .../snappydata/sql/catalog/SmartConnectorHelper.scala | 2 +- .../snappydata/sql/catalog/SnappyExternalCatalog.scala | 2 +- .../catalog/impl/SmartConnectorExternalCatalog.scala | 2 +- .../snappydata/sql/catalog/impl/StoreHiveCatalog.scala | 2 +- .../main/scala/io/snappydata/util/ServiceUtils.scala | 2 +- .../main/scala/org/apache/spark/RDDJavaFunctions.scala | 2 +- .../scala/org/apache/spark/SnappyJavaHelperUtils.scala | 2 +- .../scala/org/apache/spark/jdbc/ConnectionConf.scala | 2 +- .../scala/org/apache/spark/jdbc/ConnectionUtil.scala | 2 +- .../apache/spark/serializer/PooledKryoSerializer.scala | 2 +- .../spark/serializer/SnappyKryoSerializerFactory.scala | 2 +- .../org/apache/spark/serializer/serializers.scala | 2 +- .../scala/org/apache/spark/sql/CachedDataFrame.scala | 2 +- .../org/apache/spark/sql/DataFrameJavaFunctions.scala | 2 +- .../spark/sql/DataFrameWriterJavaFunctions.scala | 2 +- .../main/scala/org/apache/spark/sql/LockUtils.scala | 2 +- .../org/apache/spark/sql/SampleDataFrameContract.scala | 2 +- .../scala/org/apache/spark/sql/SnappyBaseParser.scala | 2 +- .../scala/org/apache/spark/sql/SnappyContext.scala | 2 +- .../org/apache/spark/sql/SnappyContextFunctions.scala | 2 +- .../scala/org/apache/spark/sql/SnappyDDLParser.scala | 2 +- .../scala/org/apache/spark/sql/SnappyImplicits.scala | 2 +- .../main/scala/org/apache/spark/sql/SnappyParser.scala | 2 +- .../scala/org/apache/spark/sql/SnappySession.scala | 2 +- .../scala/org/apache/spark/sql/SnappySqlParser.scala | 2 +- .../scala/org/apache/spark/sql/SnappyStrategies.scala | 2 +- .../main/scala/org/apache/spark/sql/TimeEpoch.scala | 2 +- .../sql/collection/ConcurrentSegmentedHashMap.scala | 2 +- .../spark/sql/collection/GenerateFlatIterator.scala | 2 +- .../spark/sql/collection/MultiColumnOpenHashMap.scala | 2 +- .../spark/sql/collection/MultiColumnOpenHashSet.scala | 2 +- .../org/apache/spark/sql/collection/ReusableRow.scala | 2 +- .../org/apache/spark/sql/collection/SegmentMap.scala | 2 +- .../scala/org/apache/spark/sql/collection/Utils.scala | 2 +- .../spark/sql/collection/WrappedInternalRow.scala | 2 +- .../main/scala/org/apache/spark/sql/dataFrames.scala | 2 +- .../spark/sql/execution/CatalogStaleException.scala | 2 +- .../spark/sql/execution/CodegenSparkFallback.scala | 2 +- .../spark/sql/execution/CodegenSupportOnExecutor.scala | 2 +- .../sql/execution/DictionaryOptimizedMapAccessor.scala | 2 +- .../apache/spark/sql/execution/EncoderScanExec.scala | 2 +- .../org/apache/spark/sql/execution/ExistingPlans.scala | 2 +- .../apache/spark/sql/execution/NonRecursivePlans.scala | 2 +- .../spark/sql/execution/ObjectHashMapAccessor.scala | 2 +- .../scala/org/apache/spark/sql/execution/RDDKryo.scala | 2 +- .../apache/spark/sql/execution/RefreshMetadata.scala | 2 +- .../apache/spark/sql/execution/SHAMapAccessor.scala | 2 +- .../org/apache/spark/sql/execution/SecurityUtils.scala | 2 +- .../apache/spark/sql/execution/SnappySortExec.scala | 2 +- .../org/apache/spark/sql/execution/TableExec.scala | 2 +- .../scala/org/apache/spark/sql/execution/TopK.scala | 2 +- .../sql/execution/aggregate/CollectAggregateExec.scala | 2 +- .../execution/aggregate/SnappyHashAggregateExec.scala | 2 +- .../sql/execution/columnar/ColumnBatchCreator.scala | 2 +- .../sql/execution/columnar/ColumnBatchIterator.scala | 2 +- .../sql/execution/columnar/ColumnDeleteExec.scala | 2 +- .../spark/sql/execution/columnar/ColumnExec.scala | 2 +- .../sql/execution/columnar/ColumnInsertExec.scala | 2 +- .../sql/execution/columnar/ColumnPutIntoExec.scala | 2 +- .../spark/sql/execution/columnar/ColumnTableScan.scala | 2 +- .../sql/execution/columnar/ColumnUpdateExec.scala | 2 +- .../spark/sql/execution/columnar/ExternalStore.scala | 2 +- .../sql/execution/columnar/ExternalStoreUtils.scala | 2 +- .../execution/columnar/JDBCAppendableRelation.scala | 2 +- .../columnar/impl/ClusteredColumnIterator.scala | 2 +- .../execution/columnar/impl/ColumnFormatEncoder.scala | 2 +- .../execution/columnar/impl/ColumnFormatIterator.scala | 2 +- .../execution/columnar/impl/ColumnFormatRelation.scala | 2 +- .../sql/execution/columnar/impl/DefaultSource.scala | 2 +- .../columnar/impl/JDBCSourceAsColumnarStore.scala | 2 +- .../columnar/impl/RemoteEntriesIterator.scala | 2 +- .../execution/columnar/impl/StoreCallbacksImpl.scala | 2 +- .../scala/org/apache/spark/sql/execution/ddl.scala | 2 +- .../apache/spark/sql/execution/externalRelations.scala | 2 +- .../spark/sql/execution/joins/HashJoinExec.scala | 2 +- .../spark/sql/execution/joins/SnappyJoinLike.scala | 2 +- .../sql/execution/joins/SnappySortMergeJoinExec.scala | 2 +- .../apache/spark/sql/execution/row/DefaultSource.scala | 2 +- .../spark/sql/execution/row/ResultSetDecoder.scala | 2 +- .../apache/spark/sql/execution/row/RowDeleteExec.scala | 2 +- .../org/apache/spark/sql/execution/row/RowExec.scala | 2 +- .../spark/sql/execution/row/RowFormatRelation.scala | 2 +- .../spark/sql/execution/row/RowFormatScanRDD.scala | 2 +- .../apache/spark/sql/execution/row/RowInsertExec.scala | 2 +- .../apache/spark/sql/execution/row/RowTableScan.scala | 2 +- .../apache/spark/sql/execution/row/RowUpdateExec.scala | 2 +- .../spark/sql/execution/row/UnsafeRowDecoder.scala | 2 +- .../execution/sources/StoreDataSourceStrategy.scala | 2 +- .../spark/sql/execution/ui/SnappySQLListener.scala | 2 +- .../org/apache/spark/sql/hive/HiveClientUtil.scala | 2 +- .../spark/sql/hive/SnappyHiveExternalCatalog.scala | 2 +- .../org/apache/spark/sql/hive/SnappySessionState.scala | 2 +- .../spark/sql/internal/CodeGenerationException.scala | 2 +- .../apache/spark/sql/internal/ColumnTableBulkOps.scala | 2 +- .../scala/org/apache/spark/sql/internal/JarUtils.scala | 2 +- .../apache/spark/sql/internal/JoinQueryPlanning.scala | 2 +- .../spark/sql/internal/SnappySessionCatalog.scala | 2 +- .../org/apache/spark/sql/internal/UDFFunction.scala | 2 +- .../scala/org/apache/spark/sql/internal/session.scala | 2 +- .../scala/org/apache/spark/sql/policy/policies.scala | 2 +- .../org/apache/spark/sql/policy/policyFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/sql/rdds.scala | 2 +- .../org/apache/spark/sql/row/JDBCMutableRelation.scala | 2 +- .../org/apache/spark/sql/row/SnappyStoreDialect.scala | 2 +- .../org/apache/spark/sql/sources/CastDouble.scala | 2 +- .../org/apache/spark/sql/sources/CastLongTime.scala | 2 +- .../scala/org/apache/spark/sql/sources/RuleUtils.scala | 2 +- .../apache/spark/sql/sources/SnappyOptimizations.scala | 2 +- .../apache/spark/sql/sources/StatVarianceCounter.scala | 2 +- .../org/apache/spark/sql/sources/StoreStrategy.scala | 2 +- .../org/apache/spark/sql/sources/interfaces.scala | 2 +- .../scala/org/apache/spark/sql/sources/subrules.scala | 2 +- .../org/apache/spark/sql/store/CodeGeneration.scala | 2 +- .../org/apache/spark/sql/store/ColumnPartitioner.scala | 2 +- .../apache/spark/sql/store/MembershipAccumulator.scala | 2 +- .../org/apache/spark/sql/store/StoreHashFunction.scala | 2 +- .../scala/org/apache/spark/sql/store/StoreUtils.scala | 2 +- .../spark/sql/streaming/DirectKafkaStreamSource.scala | 2 +- .../apache/spark/sql/streaming/FileStreamSource.scala | 2 +- .../spark/sql/streaming/FilteredSchemaDStream.scala | 2 +- .../apache/spark/sql/streaming/FlatMappedDStream.scala | 2 +- .../spark/sql/streaming/GlommedSchemaDStream.scala | 2 +- .../spark/sql/streaming/LogicalDStreamPlan.scala | 2 +- .../sql/streaming/MapPartitionedSchemaDStream.scala | 2 +- .../org/apache/spark/sql/streaming/MappedDStream.scala | 2 +- .../spark/sql/streaming/PhysicalDStreamPlan.scala | 2 +- .../spark/sql/streaming/RabbitMQStreamSource.scala | 2 +- .../org/apache/spark/sql/streaming/RabbitMQUtils.scala | 2 +- .../spark/sql/streaming/RawSocketStreamSource.scala | 2 +- .../org/apache/spark/sql/streaming/SchemaDStream.scala | 2 +- .../spark/sql/streaming/SnappySinkCallback.scala | 2 +- .../spark/sql/streaming/SocketStreamSource.scala | 2 +- .../spark/sql/streaming/StreamBaseRelation.scala | 2 +- .../apache/spark/sql/streaming/StreamConverter.scala | 2 +- .../apache/spark/sql/streaming/StreamSqlHelper.scala | 2 +- .../spark/sql/streaming/StreamToRowsConverter.scala | 2 +- .../spark/sql/streaming/TextSocketStreamSource.scala | 2 +- .../spark/sql/streaming/TransformedSchemaDStream.scala | 2 +- .../spark/sql/streaming/TwitterStreamSource.scala | 2 +- .../apache/spark/sql/streaming/WindowLogicalPlan.scala | 2 +- .../spark/sql/streaming/WindowPhysicalPlan.scala | 2 +- .../org/apache/spark/sql/types/TypeUtilities.scala | 2 +- .../spark/streaming/SnappyStreamingContext.scala | 2 +- .../scala/org/apache/spark/streaming/StreamUtils.scala | 2 +- .../api/java/JavaSnappyStreamingContext.scala | 2 +- .../io/snappydata/api/JavaCreateIndexTestSuite.java | 2 +- .../java/io/snappydata/util/BoundedSortedSetTest.java | 2 +- .../streaming/JavaSnappyStreamingContextSuite.java | 2 +- core/src/test/resources/log4j.properties | 2 +- .../scala/io/snappydata/ColumnUpdateDeleteTests.scala | 2 +- .../scala/io/snappydata/CommandLineToolsSuite.scala | 2 +- .../test/scala/io/snappydata/ConcurrentOpsTests.scala | 2 +- core/src/test/scala/io/snappydata/SnappyFunSuite.scala | 2 +- .../test/scala/io/snappydata/app/ColumnTableSpec.scala | 2 +- .../scala/io/snappydata/app/ExternalTableTest.scala | 2 +- .../io/snappydata/app/ExternalTableTestData.scala | 2 +- .../scala/io/snappydata/app/KMeansModelForDemo.scala | 2 +- .../scala/io/snappydata/app/SampleTableQuery.scala | 2 +- .../io/snappydata/app/SparkSQLMultiThreadingTest.scala | 2 +- .../io/snappydata/app/StreamingTupleGenerator.scala | 2 +- .../test/scala/io/snappydata/core/LocalTestData.scala | 2 +- .../scala/io/snappydata/gemxd/BasicStoreSuite.scala | 2 +- .../test/scala/io/snappydata/util/DebugUtilsSpec.scala | 2 +- .../scala/io/snappydata/util/NumberUtilsSpec.scala | 2 +- .../scala/io/snappydata/util/StringUtilsSpec.scala | 2 +- core/src/test/scala/io/snappydata/util/TestUtils.scala | 2 +- .../org/apache/spark/jdbc/ConnectionConfTest.scala | 2 +- .../spark/serializer/PooledKryoSerializerSuite.scala | 2 +- .../scala/org/apache/spark/sql/DiskStoreDDLTest.scala | 2 +- .../org/apache/spark/sql/SnappyTempTableTest.scala | 2 +- .../apache/spark/sql/execution/AggregationSuite.scala | 2 +- .../sql/internal/UpdateStatementTypeCastingSuite.scala | 2 +- .../org/apache/spark/sql/kafka010/KafkaTestUtils.scala | 2 +- .../spark/sql/kafka010/SnappyStreamingKafkaSuite.scala | 2 +- .../spark/sql/store/CatalogConsistencyTest.scala | 2 +- .../spark/sql/store/ColumnTableBatchInsertTest.scala | 2 +- .../sql/store/ColumnTableInternalValidationTest.scala | 2 +- .../org/apache/spark/sql/store/ColumnTableTest.scala | 2 +- .../org/apache/spark/sql/store/CreateIndexTest.scala | 2 +- .../spark/sql/store/CubeRollupGroupingSetsTest.scala | 2 +- .../spark/sql/store/DisableTokenizationTest.scala | 2 +- .../spark/sql/store/JDBCMutableRelationAPISuite.scala | 2 +- .../org/apache/spark/sql/store/MetadataTest.scala | 2 +- .../org/apache/spark/sql/store/PlanCachingTest.scala | 2 +- .../apache/spark/sql/store/RowRelationAPISuite.scala | 2 +- .../org/apache/spark/sql/store/RowTableTest.scala | 2 +- .../apache/spark/sql/store/SnappyCatalogSuite.scala | 2 +- .../org/apache/spark/sql/store/SnappyJoinSuite.scala | 2 +- .../scala/org/apache/spark/sql/store/StringTest.scala | 2 +- .../org/apache/spark/sql/store/TokenizationTest.scala | 2 +- .../spark/sql/store/UnifiedPartitionerTest.scala | 2 +- .../scala/org/apache/spark/sql/store/ViewTest.scala | 2 +- .../SnappyStoreSinkProviderSecuritySuite.scala | 2 +- .../sql/streaming/SnappyStoreSinkProviderSuite.scala | 2 +- .../spark/sql/streaming/SnappyStreamingAPISuite.scala | 2 +- .../spark/streaming/SnappyStreamingContextSuite.scala | 2 +- dtests/build.gradle | 2 +- .../hydra/adAnalytics/SnappyAdAnalyticsTest.java | 2 +- .../snappydata/hydra/cdcConnector/CDCPerfSparkJob.java | 2 +- .../hydra/cluster/SnappyAlterTablesTest.java | 2 +- .../java/io/snappydata/hydra/cluster/SnappyBB.java | 2 +- .../hydra/cluster/SnappyColocationAndEvictionTest.java | 2 +- .../hydra/cluster/SnappyConcurrencyTest.java | 2 +- .../snappydata/hydra/cluster/SnappyLocatorHATest.java | 2 +- .../hydra/cluster/SnappyNetworkServerBB.java | 2 +- .../java/io/snappydata/hydra/cluster/SnappyPrms.java | 2 +- .../hydra/cluster/SnappySparkSQLDriverTest.java | 2 +- .../io/snappydata/hydra/cluster/SnappyStartUpTest.java | 2 +- .../java/io/snappydata/hydra/cluster/SnappyTest.java | 2 +- ...alyticalWorkLoadWithDMLOpsUsingStreaming_airline.sh | 2 +- .../concurrency/concurrency_analyticalWorkLoad_tpch.sh | 2 +- .../concurrency/concurrency_mixedWorkLoad_tpch.sh | 2 +- .../java/io/snappydata/hydra/concurrency/stability.sh | 2 +- .../hydra/connectionPool/HikariConnectionPool.java | 2 +- .../hydra/connectionPool/SnappyConnectionPoolPrms.java | 2 +- .../hydra/connectionPool/TomcatConnectionPool.java | 2 +- .../hydra/consistency/SnappyConsistencyTest.java | 2 +- .../hydra/installJar/DynamicJarLoadingJob.java | 2 +- .../hydra/installJar/DynamicJarLoadingTest.java | 2 +- .../hydra/installJar/InstallJarTestUtils.java | 2 +- .../io/snappydata/hydra/longRun/longRunningJobs.sh | 2 +- .../io/snappydata/hydra/longRun/longRunningTest.sh | 2 +- .../test/java/io/snappydata/hydra/misc/MiscTest.java | 2 +- .../io/snappydata/hydra/preparedStmt/JdbcTestPS.java | 2 +- .../hydra/preparedStmt/SnappyPreparedStmtTest.java | 2 +- .../io/snappydata/hydra/putInto/LoadDataFromJson.scala | 2 +- .../io/snappydata/hydra/putInto/SnappyPutIntoTest.java | 2 +- .../snappydata/hydra/security/SnappySecurityPrms.java | 2 +- .../snappydata/hydra/security/SnappySecurityTest.java | 2 +- dtests/src/test/java/io/snappydata/hydra/smoke.sh | 2 +- dtests/src/test/java/io/snappydata/hydra/smokePerf.sh | 2 +- .../hydra/snapshotIsolation/SnapshotIsolationBB.java | 2 +- .../snapshotIsolation/SnapshotIsolationDMLOpsBB.java | 2 +- .../hydra/snapshotIsolation/SnapshotIsolationTest.java | 2 +- .../io/snappydata/hydra/testDMLOps/SnappyDMLOpsBB.java | 2 +- .../hydra/testDMLOps/SnappyDMLOpsUsingAPITest.java | 2 +- .../snappydata/hydra/testDMLOps/SnappyDMLOpsUtil.java | 2 +- .../snappydata/hydra/testDMLOps/SnappySchemaPrms.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF1.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF10.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF11.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF12.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF13.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF14.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF15.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF16.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF17.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF18.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF19.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF2.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF20.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF21.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF22.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF3.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF4.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF5.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF6.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF7.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF8.java | 2 +- .../test/java/io/snappydata/hydra/udfs/JavaUDF9.java | 2 +- .../java/io/snappydata/hydra/udfs/UDFInterface.java | 2 +- .../snappydata/hydra/AirlineCleanedParquertData.scala | 2 +- .../io/snappydata/hydra/AirlineDataQueriesJob.scala | 2 +- .../io/snappydata/hydra/AirlineDataSparkJobApp.scala | 2 +- .../io/snappydata/hydra/CatalogConsistencyTest.scala | 2 +- .../scala/io/snappydata/hydra/FileStreamingJob.scala | 2 +- .../scala/io/snappydata/hydra/InstallJarTest.scala | 2 +- .../io/snappydata/hydra/InstallJarTestSparkApp.scala | 2 +- .../scala/io/snappydata/hydra/LoadCSVDataJob.scala | 2 +- .../test/scala/io/snappydata/hydra/LoadODSData.scala | 2 +- .../snappydata/hydra/SnappyCustomSparkListener.scala | 2 +- .../scala/io/snappydata/hydra/SnappyHydraRunner.scala | 2 +- .../io/snappydata/hydra/SnappyHydraTestRunner.scala | 2 +- .../scala/io/snappydata/hydra/SnappyTestUtils.scala | 2 +- .../src/test/scala/io/snappydata/hydra/TestUtil.scala | 2 +- .../hydra/adAnalytics/SnappySQLLogAggregatorJob.scala | 2 +- .../hydra/adAnalytics/StopSnappyStreamingJob.scala | 2 +- .../snappydata/hydra/alterTable/AlterTablesJob.scala | 2 +- .../test/scala/io/snappydata/hydra/ao/LoadData.scala | 4 ++-- .../hydra/cdcConnector/CDCValidationApp.scala | 2 +- .../hydra/clusterUpgrade/ClusterUpgradeSnappyJob.scala | 2 +- .../hydra/clusterUpgrade/ClusterUpgradeSparkJob.scala | 2 +- .../hydra/complexdatatypes/AllMixedTypes.scala | 2 +- .../snappydata/hydra/complexdatatypes/ArrayType.scala | 2 +- .../hydra/complexdatatypes/ArrayTypeAPI.scala | 2 +- .../complexdatatypes/ArraysOfStringInMapAsValue.scala | 2 +- .../hydra/complexdatatypes/ArraysOfStructType.scala | 2 +- .../hydra/complexdatatypes/ComplexTypeUtils.scala | 2 +- .../io/snappydata/hydra/complexdatatypes/MapType.scala | 2 +- .../snappydata/hydra/complexdatatypes/MapTypeAPI.scala | 2 +- .../complexdatatypes/SmartConnectorAllMixedType.scala | 2 +- .../complexdatatypes/SmartConnectorArrayType.scala | 2 +- .../complexdatatypes/SmartConnectorArrayTypeAPI1.scala | 2 +- .../SmartConnectorArraysOfStringInMapAsValue.scala | 2 +- .../SmartConnectorArraysOfStructType.scala | 2 +- .../hydra/complexdatatypes/SmartConnectorMapType.scala | 2 +- .../complexdatatypes/SmartConnectorMapTypeAPI.scala | 2 +- .../complexdatatypes/SmartConnectorStructType.scala | 2 +- .../complexdatatypes/SmartConnectorStructTypeAPI.scala | 2 +- .../snappydata/hydra/complexdatatypes/StructType.scala | 2 +- .../hydra/complexdatatypes/StructTypeAPI.scala | 2 +- .../snappydata/hydra/concurrency/ConcTestUtils.scala | 2 +- .../io/snappydata/hydra/concurrency/ExportTable.scala | 2 +- .../hydra/concurrency/StreamingActivity.scala | 2 +- .../concurrency/ValidateConcurrentQueriesJob.scala | 2 +- .../snappydata/hydra/consistency/ConsistencyTest.scala | 2 +- .../consistency/ValidateConsistencyWithDMLOpsApp.scala | 2 +- .../consistency/ValidateConsistencyWithDMLOpsJob.scala | 2 +- .../hydra/ct/AddDataUsingPutIntoCTTablesJob.scala | 2 +- .../test/scala/io/snappydata/hydra/ct/CTQueries.scala | 2 +- .../test/scala/io/snappydata/hydra/ct/CTTestUtil.scala | 2 +- .../CreateAndLoadCTColumnTablesWithKeyColumnsJob.scala | 2 +- .../snappydata/hydra/ct/CreateAndLoadCTTablesApp.scala | 2 +- .../snappydata/hydra/ct/CreateAndLoadCTTablesJob.scala | 2 +- .../io/snappydata/hydra/ct/ValidateCTQueriesApp.scala | 2 +- .../io/snappydata/hydra/ct/ValidateCTQueriesJob.scala | 2 +- .../hydra/ct/ValidateCTQueriesWithPutIntoJob.scala | 2 +- .../deployPkgUDF/AccessDeployedJarSnappyJob.scala | 2 +- .../hydra/deployPkgUDF/AccessUdfSnappyJob.scala | 2 +- .../hydra/deployPkgUDF/AccessUdfSparkApp.scala | 2 +- .../hydra/deployPkgUDF/CreateDropUDFSnappyJob.scala | 2 +- .../hydra/deployPkgUDF/CreateDropUDFSparkApp.scala | 2 +- .../io/snappydata/hydra/distIndex/DistIndexJob.scala | 2 +- .../externaltables/ExternalTablesAPINorthWind.scala | 2 +- .../SmartConnectorExternalTableAPINW.scala | 2 +- .../hydra/hiveThriftServer/HiveThriftServer.scala | 2 +- .../HiveThriftServerConcurrentOps.scala | 2 +- .../ThriftServerInsertFromSnappy.scala | 2 +- .../hiveThriftServer/ValidateHiveThriftServer.scala | 2 +- .../scala/io/snappydata/hydra/misc/snap_3028Job.scala | 2 +- .../hydra/northwind/CreateAndLoadNWTablesJob.scala | 2 +- .../northwind/CreateAndLoadNWTablesSparkApp.scala | 2 +- .../hydra/northwind/EvictionTestNWQueriesApp.scala | 2 +- .../hydra/northwind/EvictionTestNWQueriesJob.scala | 2 +- .../io/snappydata/hydra/northwind/NWPLQueries.scala | 2 +- .../io/snappydata/hydra/northwind/NWQueries.scala | 2 +- .../hydra/northwind/NWSparkTablesAndQueriesApp.scala | 2 +- .../io/snappydata/hydra/northwind/NWTestJob.scala | 2 +- .../io/snappydata/hydra/northwind/NWTestSparkApp.scala | 2 +- .../io/snappydata/hydra/northwind/NWTestUtil.scala | 2 +- .../hydra/northwind/ValidateNWQueriesApp.scala | 2 +- .../hydra/northwind/ValidateNWQueriesJob.scala | 2 +- .../ValidateNWQueriesWithChangingConstantsApp.scala | 2 +- .../ValidateNWQueriesWithChangingConstantsJob.scala | 2 +- .../hydra/preparedStmt/PopulateDataJob.scala | 2 +- .../io/snappydata/hydra/putInto/ConcPutIntoTest.scala | 2 +- .../hydra/putInto/ConcPutIntoWith30Columns.scala | 2 +- .../snappydata/hydra/putInto/ConcurrentPutInto.scala | 2 +- .../hydra/putInto/GenrateJsonDataFilesJob.scala | 4 ++-- .../snappydata/hydra/putInto/InsertFromJsonFile.scala | 2 +- .../io/snappydata/hydra/putInto/PutIntoProducer.scala | 2 +- .../io/snappydata/hydra/putInto/PutIntoReceiver.scala | 2 +- .../hydra/security/CreateAndLoadTablesSnappyJob.scala | 2 +- .../hydra/security/CreateAndLoadTablesSparkApp.scala | 2 +- .../snappydata/hydra/security/SecurityTestUtil.scala | 2 +- .../hydra/snapshotIsolation/SnappyDummyJob.scala | 2 +- .../snapshotIsolation/SnapshotIsolationQueries.scala | 2 +- .../snapshotIsolation/SnapshotIsolationTestUtils.scala | 2 +- .../ValidateSnapshotSelectWithInsertJob.scala | 2 +- .../hydra/spva/CreateAnLoadSPVATablesJob.scala | 2 +- .../hydra/spva/CreateAndLoadSPVATablesSparkApp.scala | 2 +- .../scala/io/snappydata/hydra/spva/SPVAQueries.scala | 2 +- .../scala/io/snappydata/hydra/spva/SPVATestUtil.scala | 2 +- .../snappydata/hydra/spva/ValidateSPVAQueriesJob.scala | 2 +- .../hydra/streaming_sink/CreateTableForAvgJob.scala | 2 +- .../hydra/streaming_sink/CreateTableForCountJob.scala | 2 +- .../hydra/streaming_sink/CreateTableForJoinJob.scala | 2 +- .../hydra/streaming_sink/CreateTableForSumJob.scala | 2 +- .../hydra/streaming_sink/CreateTableInSnappyJob.scala | 2 +- .../SnappyStreamingSinkGenericColNameJob.scala | 2 +- .../hydra/streaming_sink/SnappyStreamingSinkJob.scala | 2 +- .../SnappyStreamingSinkJobWithAggregate.scala | 2 +- .../hydra/streaming_sink/StopSnappyStreamingJob.scala | 2 +- .../hydra/streaming_sink/StringMessageProducer.scala | 2 +- .../ConvertCsvToParquetAndLoadTablesJob.scala | 2 +- .../hydra/testDMLOps/RecreateTableInSnappyJob.scala | 2 +- .../hydra/testDMLOps/SnappyDMLTestUtil.scala | 2 +- .../snappydata/hydra/testDMLOps/ValidateDMLOpApp.scala | 2 +- .../snappydata/hydra/testDMLOps/ValidateDMLOpJob.scala | 2 +- .../hydra/testDMLOps/ValidateDeleteOpAppUsingAPI.scala | 2 +- .../testDMLOps/ValidateDeleteOpUsingAPI_Job.scala | 2 +- .../hydra/testDMLOps/ValidateInsertOpAppUsingAPI.scala | 2 +- .../testDMLOps/ValidateInsertOpUsingAPI_Job.scala | 2 +- .../hydra/testDMLOps/ValidateUpdateOpAppUsingAPI.scala | 2 +- .../testDMLOps/ValidateUpdateOpUsingAPI_Job.scala | 2 +- .../io/snappydata/hydra/udfs/BadCase_ScalaUDF13.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF1.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF10.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF11.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF12.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF13.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF14.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF15.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF16.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF17.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF18.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF19.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF2.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF20.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF21.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF22.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF3.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF4.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF5.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF6.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF7.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF8.scala | 2 +- .../scala/io/snappydata/hydra/udfs/ScalaUDF9.scala | 2 +- .../io/snappydata/hydra/udfs/Validate_SNAP2658.scala | 2 +- .../test/scala/org/apache/spark/SnappyTestUtils.scala | 2 +- .../org/apache/spark/sql/DistIndexTestUtils.scala | 2 +- dunit/build.gradle | 2 +- .../test/batterytest/greplogs/ExpectedStrings.java | 2 +- .../test/batterytest/greplogs/LogConsumer.java | 2 +- .../java/io/snappydata/test/dunit/AsyncInvocation.java | 2 +- .../io/snappydata/test/dunit/AvailablePortHelper.java | 2 +- .../java/io/snappydata/test/dunit/BounceResult.java | 2 +- .../main/java/io/snappydata/test/dunit/DUnitEnv.java | 2 +- .../io/snappydata/test/dunit/DistributedTestBase.java | 2 +- dunit/src/main/java/io/snappydata/test/dunit/Host.java | 2 +- .../java/io/snappydata/test/dunit/RMIException.java | 2 +- .../java/io/snappydata/test/dunit/RemoteDUnitVMIF.java | 2 +- .../io/snappydata/test/dunit/RepeatableRunnable.java | 2 +- .../io/snappydata/test/dunit/SerializableCallable.java | 2 +- .../io/snappydata/test/dunit/SerializableRunnable.java | 2 +- dunit/src/main/java/io/snappydata/test/dunit/VM.java | 2 +- .../test/dunit/logging/log4j/ConfigLocator.java | 2 +- .../io/snappydata/test/dunit/standalone/ChildVM.java | 2 +- .../test/dunit/standalone/DUnitLauncher.java | 2 +- .../test/dunit/standalone/ProcessManager.java | 2 +- .../test/dunit/standalone/RemoteDUnitVM.java | 2 +- .../test/dunit/standalone/StandAloneDUnitEnv.java | 2 +- .../io/snappydata/test/hydra/LogVersionHelper.java | 2 +- .../java/io/snappydata/test/hydra/MethExecutor.java | 2 +- .../io/snappydata/test/hydra/MethExecutorResult.java | 2 +- .../java/io/snappydata/test/hydra/SchedulingOrder.java | 2 +- .../io/snappydata/test/memscale/OffHeapHelper.java | 2 +- .../java/io/snappydata/test/util/TestException.java | 2 +- encoders/build.gradle | 2 +- .../collection/BufferSizeLimitExceededException.scala | 2 +- .../io/snappydata/collection/ByteBufferHashMap.scala | 2 +- .../scala/io/snappydata/collection/DictionaryMap.scala | 2 +- .../scala/io/snappydata/collection/ObjectHashSet.scala | 2 +- .../main/scala/io/snappydata/collection/SHAMap.scala | 2 +- .../src/main/scala/io/snappydata/util/DebugUtils.scala | 2 +- .../main/scala/io/snappydata/util/NumberUtils.scala | 2 +- .../main/scala/io/snappydata/util/StringUtils.scala | 2 +- .../org/apache/spark/memory/DefaultMemoryManager.scala | 2 +- .../org/apache/spark/memory/StoreUnifiedManager.scala | 2 +- .../apache/spark/sql/SnappyStoreClientDialect.scala | 2 +- .../spark/sql/catalyst/util/SerializedArray.scala | 2 +- .../apache/spark/sql/catalyst/util/SerializedMap.scala | 2 +- .../apache/spark/sql/catalyst/util/SerializedRow.scala | 2 +- .../org/apache/spark/sql/collection/SharedUtils.scala | 2 +- .../apache/spark/sql/execution/ConnectionPool.scala | 2 +- .../spark/sql/execution/columnar/ColumnBatch.scala | 2 +- .../spark/sql/execution/columnar/encoding/BitSet.scala | 2 +- .../columnar/encoding/BooleanBitSetEncoding.scala | 2 +- .../columnar/encoding/BooleanRunLengthEncoding.scala | 2 +- .../columnar/encoding/ColumnDeleteDecoder.scala | 2 +- .../columnar/encoding/ColumnDeleteEncoder.scala | 2 +- .../columnar/encoding/ColumnDeltaDecoder.scala | 2 +- .../columnar/encoding/ColumnDeltaEncoder.scala | 2 +- .../execution/columnar/encoding/ColumnEncoding.scala | 2 +- .../columnar/encoding/DictionaryEncoding.scala | 2 +- .../columnar/encoding/RunLengthEncoding.scala | 2 +- .../sql/execution/columnar/encoding/Uncompressed.scala | 2 +- .../columnar/encoding/UpdatedColumnDecoder.scala | 2 +- .../sql/execution/columnar/impl/ColumnDelta.scala | 2 +- .../execution/columnar/impl/ColumnFormatEntry.scala | 2 +- .../spark/sql/execution/row/ResultSetDecoder.scala | 2 +- .../spark/sql/execution/row/UnsafeRowDecoder.scala | 2 +- .../spark/sql/sources/ConnectionProperties.scala | 2 +- .../apache/spark/sql/store/CompressionCodecId.scala | 2 +- .../org/apache/spark/sql/store/CompressionUtils.scala | 2 +- encoders/src/test/resources/log4j.properties | 2 +- examples/build.gradle | 4 ++-- examples/quickstart/scripts/Quickstart.scala | 2 +- examples/src/main/python/KMeansWeather.py | 2 +- .../scala/io/snappydata/examples/AirlineDataJob.scala | 2 +- .../io/snappydata/examples/AirlineDataSparkApp.scala | 2 +- .../examples/CreateAndLoadAirlineDataJob.scala | 2 +- .../scala/io/snappydata/examples/DataUpdateJob.scala | 2 +- .../io/snappydata/examples/SnappyDataTestJob.scala | 2 +- .../scala/io/snappydata/examples/StreamingUtils.scala | 2 +- .../io/snappydata/examples/TwitterPopularTagsJob.scala | 2 +- .../examples/snappydata/CollocatedJoinExample.scala | 2 +- .../spark/examples/snappydata/CreateColumnTable.scala | 2 +- .../snappydata/CreatePartitionedRowTable.scala | 2 +- .../examples/snappydata/CreateReplicatedRowTable.scala | 2 +- .../apache/spark/examples/snappydata/JDBCExample.scala | 2 +- .../examples/snappydata/JDBCWithComplexTypes.scala | 2 +- .../examples/snappydata/SmartConnectorExample.scala | 2 +- .../spark/examples/snappydata/StreamingExample.scala | 2 +- .../spark/examples/snappydata/WorkingWithJson.scala | 2 +- .../spark/examples/snappydata/WorkingWithObjects.scala | 2 +- .../test/scala/io/snappydata/SnappyTestRunner.scala | 2 +- .../io/snappydata/quickstart/ExampleTestSuite.scala | 2 +- jdbc/build.gradle | 2 +- jdbc/src/main/scala/io/snappydata/Constant.scala | 2 +- jdbc/src/main/scala/io/snappydata/sql/implicits.scala | 2 +- .../main/scala/org/apache/spark/sql/JdbcExecute.scala | 2 +- .../main/scala/org/apache/spark/sql/JdbcWriter.scala | 2 +- .../org/apache/spark/sql/SnappyDataBaseDialect.scala | 2 +- .../org/apache/spark/sql/SnappyDataPoolDialect.scala | 2 +- .../org/apache/spark/sql/sources/jdbcExtensions.scala | 2 +- launcher/build.gradle | 2 +- .../main/java/io/snappydata/tools/QuickLauncher.java | 2 +- python/pyspark/sql/snappy/__init__.py | 2 +- python/pyspark/sql/snappy/context.py | 2 +- python/pyspark/sql/snappy/snappysession.py | 2 +- python/pyspark/sql/snappy/tests.py | 2 +- python/pyspark/streaming/snappy/__init__.py | 2 +- python/pyspark/streaming/snappy/snappydstream.py | 2 +- python/pyspark/streaming/snappy/tests.py | 2 +- python/run-snappy-tests.py | 2 +- release/filehdr-mod.txt | 2 +- scalastyle-config.xml | 2 +- settings.gradle | 2 +- spark | 2 +- spark-jobserver | 2 +- store | 2 +- .../src/main/resources/SnappySqlPoolTestCode.txt | 2 +- tests/common/src/main/resources/SparkSqlTestCode.txt | 2 +- 848 files changed, 858 insertions(+), 858 deletions(-) diff --git a/README.md b/README.md index 7be870e936..4986e2e210 100644 --- a/README.md +++ b/README.md @@ -47,18 +47,18 @@ SnappyData artifacts are hosted in Maven Central. You can add a Maven dependency ``` groupId: io.snappydata artifactId: snappydata-cluster_2.11 -version: 1.1.0 +version: 1.1.1 ``` **Using SBT Dependency** If you are using SBT, add this line to your **build.sbt** for core SnappyData artifacts: -`libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.0"` +`libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.1"` For additions related to SnappyData cluster, use: -`libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.0"` +`libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1"` You can find more specific SnappyData artifacts [here](http://mvnrepository.com/artifact/io.snappydata) diff --git a/build.gradle b/build.gradle index c8b4a710b7..8a48a62923 100644 --- a/build.gradle +++ b/build.gradle @@ -72,7 +72,7 @@ allprojects { apply plugin: "build-time-tracker" group = 'io.snappydata' - version = '1.1.0-HF-1' + version = '1.1.1' // apply compiler options tasks.withType(JavaCompile) { @@ -108,7 +108,7 @@ allprojects { scalaBinaryVersion = '2.11' scalaVersion = scalaBinaryVersion + '.8' sparkVersion = '2.1.1' - snappySparkVersion = '2.1.1.6' + snappySparkVersion = '2.1.1.7' sparkDistName = "spark-${sparkVersion}-bin-hadoop2.7" sparkCurrentVersion = '2.3.2' sparkCurrentDistName = "spark-${sparkCurrentVersion}-bin-hadoop2.7" @@ -151,10 +151,10 @@ allprojects { antlr2Version = '2.7.7' pegdownVersion = '1.6.0' - snappyStoreVersion = '1.6.3' + snappyStoreVersion = '1.6.4' snappydataVersion = version pulseVersion = '1.5.1' - zeppelinInterpreterVersion = '0.7.3.5' + zeppelinInterpreterVersion = '0.7.3.6' buildFlags = '' createdBy = System.getProperty('user.name') @@ -165,7 +165,7 @@ allprojects { buildDate = new Date().format('yyyy-MM-dd HH:mm:ss Z') buildNumber = new Date().format('MMddyy') jdkVersion = System.getProperty('java.version') - sparkJobServerVersion = '0.6.2.9' + sparkJobServerVersion = '0.6.2.10' eclipseCollectionsVersion = '9.2.0' fastutilVersion = '8.2.2' diff --git a/cluster/bin/load-snappy-env.sh b/cluster/bin/load-snappy-env.sh index 1b105c8e74..30d206898d 100644 --- a/cluster/bin/load-snappy-env.sh +++ b/cluster/bin/load-snappy-env.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/bin/snappy-job.sh b/cluster/bin/snappy-job.sh index 9697717c45..bafd87badb 100755 --- a/cluster/bin/snappy-job.sh +++ b/cluster/bin/snappy-job.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/bin/snappy-sql b/cluster/bin/snappy-sql index 8571d47b3a..061bdc2003 100755 --- a/cluster/bin/snappy-sql +++ b/cluster/bin/snappy-sql @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/build.gradle b/cluster/build.gradle index 5fd69f11df..be3718319b 100644 --- a/cluster/build.gradle +++ b/cluster/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/conf/leads.template b/cluster/conf/leads.template index 6ad62e6d21..37fd5fcc97 100644 --- a/cluster/conf/leads.template +++ b/cluster/conf/leads.template @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/conf/locators.template b/cluster/conf/locators.template index 396a2e1a26..f4a08f72f5 100644 --- a/cluster/conf/locators.template +++ b/cluster/conf/locators.template @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/conf/log4j.properties.template b/cluster/conf/log4j.properties.template index 0df4d6bfa0..3bc8aaa561 100644 --- a/cluster/conf/log4j.properties.template +++ b/cluster/conf/log4j.properties.template @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/conf/servers.template b/cluster/conf/servers.template index 01085567f8..63a410e481 100644 --- a/cluster/conf/servers.template +++ b/cluster/conf/servers.template @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/check-dir-option.sh b/cluster/sbin/check-dir-option.sh index 7f5261d37b..3ca808f693 100755 --- a/cluster/sbin/check-dir-option.sh +++ b/cluster/sbin/check-dir-option.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/cluster-util.sh b/cluster/sbin/cluster-util.sh index 72cc1fcf3b..11e3102475 100755 --- a/cluster/sbin/cluster-util.sh +++ b/cluster/sbin/cluster-util.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You @@ -130,7 +130,7 @@ while [ "$1" != "" ]; do --on-all) MEMBER_LIST="all" MEMBER_TYPE="all" - rm /tmp/snappy-nodes.txt + rm -f /tmp/snappy-nodes.txt LOCATOR_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/locators | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') echo $LOCATOR_LIST >> /tmp/snappy-nodes.txt SERVER_LIST=$(sed ':loop /^[^#].*[^\\]\\$/N; s/\\\n//; t loop' $SNAPPY_HOME/conf/servers | awk '!/^ *#/ && !/^[[:space:]]*$/ { print$1; }' | awk '!seen[$0]++') diff --git a/cluster/sbin/collect-debug-artifacts.sh b/cluster/sbin/collect-debug-artifacts.sh index 1890e47787..a94ea6e619 100755 --- a/cluster/sbin/collect-debug-artifacts.sh +++ b/cluster/sbin/collect-debug-artifacts.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-config.sh b/cluster/sbin/snappy-config.sh index 30cc4c4066..bf9680f7c5 100755 --- a/cluster/sbin/snappy-config.sh +++ b/cluster/sbin/snappy-config.sh @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-lead.sh b/cluster/sbin/snappy-lead.sh index 349fd67acd..1595023e0e 100755 --- a/cluster/sbin/snappy-lead.sh +++ b/cluster/sbin/snappy-lead.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-leads.sh b/cluster/sbin/snappy-leads.sh index d0a36f7cfb..2a7d0110a6 100755 --- a/cluster/sbin/snappy-leads.sh +++ b/cluster/sbin/snappy-leads.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-locator.sh b/cluster/sbin/snappy-locator.sh index 96498cfdc4..ab94985bb6 100755 --- a/cluster/sbin/snappy-locator.sh +++ b/cluster/sbin/snappy-locator.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-locators.sh b/cluster/sbin/snappy-locators.sh index 0898928bf8..b253249b0d 100755 --- a/cluster/sbin/snappy-locators.sh +++ b/cluster/sbin/snappy-locators.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-nodes.sh b/cluster/sbin/snappy-nodes.sh index 854efa5741..d287209835 100755 --- a/cluster/sbin/snappy-nodes.sh +++ b/cluster/sbin/snappy-nodes.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-server.sh b/cluster/sbin/snappy-server.sh index 96efbd12b6..9ed2811902 100755 --- a/cluster/sbin/snappy-server.sh +++ b/cluster/sbin/snappy-server.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-servers.sh b/cluster/sbin/snappy-servers.sh index 3f4278ded4..3bfacbb5d5 100755 --- a/cluster/sbin/snappy-servers.sh +++ b/cluster/sbin/snappy-servers.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-start-all.sh b/cluster/sbin/snappy-start-all.sh index 18f09bd710..53e57e4886 100755 --- a/cluster/sbin/snappy-start-all.sh +++ b/cluster/sbin/snappy-start-all.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-status-all.sh b/cluster/sbin/snappy-status-all.sh index c63c541b06..5f68896a6e 100755 --- a/cluster/sbin/snappy-status-all.sh +++ b/cluster/sbin/snappy-status-all.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/sbin/snappy-stop-all.sh b/cluster/sbin/snappy-stop-all.sh index dcca58f772..5b18ff2873 100755 --- a/cluster/sbin/snappy-stop-all.sh +++ b/cluster/sbin/snappy-stop-all.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/HiveMetastoreClientAccessDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/HiveMetastoreClientAccessDUnitTest.scala index 06937d62e5..2f861bffbf 100644 --- a/cluster/src/dunit/scala/io/snappydata/HiveMetastoreClientAccessDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/HiveMetastoreClientAccessDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerLDAPTestBase.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerLDAPTestBase.scala index c07590a05b..11e49eed44 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerLDAPTestBase.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerLDAPTestBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala index 58deca8651..ce51bcde94 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterManagerTestBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala index 4638eaf432..2eb43acc37 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ClusterMgrDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitSecurityTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitSecurityTest.scala index f45db2e70b..9f283b9829 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitSecurityTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitSecurityTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitTest.scala index 9fc39723d1..bd528d062d 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ConcurrentQueryRoutingDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala index 8d789bfec7..490a69df26 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala index 088165761f..6e5a2e6c07 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala index fb9a9d3436..04711259ff 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ExecutionEngineArbiterDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala index 79b233110a..2f389ea428 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitSecurityTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitSecurityTest.scala index 8f4c990978..04046c89a9 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitSecurityTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitSecurityTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala index f3cf387759..840e225688 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SetIsolationDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SetIsolationDUnitTest.scala index 27ce84bb9d..a4d56658a3 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SetIsolationDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SetIsolationDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyResourceEventsDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyResourceEventsDUnitTest.scala index 7a16853801..862a9bd2a3 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyResourceEventsDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyResourceEventsDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyRowStoreModeDUnit.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyRowStoreModeDUnit.scala index 8de3ccf4e0..1dd495511e 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyRowStoreModeDUnit.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyRowStoreModeDUnit.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala index cb3b79df4a..241c8987d3 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappyTableStatsProviderDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 7e49eb9452..bb96c7d76a 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/StringAsVarcharDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/StringAsVarcharDUnitTest.scala index db04858ced..603d03666a 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/StringAsVarcharDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/StringAsVarcharDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala index 45a9b7fa08..c9890396f6 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/ValidateMVCCDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/jobs/CassandraSnappyConnectionJob.scala b/cluster/src/dunit/scala/io/snappydata/cluster/jobs/CassandraSnappyConnectionJob.scala index d2879c3369..1f3ab2f81d 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/jobs/CassandraSnappyConnectionJob.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/jobs/CassandraSnappyConnectionJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/jobs/SnappySecureJob.scala b/cluster/src/dunit/scala/io/snappydata/cluster/jobs/SnappySecureJob.scala index 8dab562fb3..82161bab1c 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/jobs/SnappySecureJob.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/jobs/SnappySecureJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/CatalogConsistencyDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/CatalogConsistencyDUnitTest.scala index c2b1d566de..40de5aaff4 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/CatalogConsistencyDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/CatalogConsistencyDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/ColumnTableDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/ColumnTableDUnitTest.scala index 95abc273c4..627ed83e1a 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/ColumnTableDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/ColumnTableDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/ExecutorMessageDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/ExecutorMessageDUnitTest.scala index 30bd9b36b2..0431928dcf 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/ExecutorMessageDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/ExecutorMessageDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala index 52209e41a1..4c4784054d 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/JDBCPreparedStatementDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/externalstore/RowTableDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/externalstore/RowTableDUnitTest.scala index 42fa54ea0f..1f382e3101 100644 --- a/cluster/src/dunit/scala/io/snappydata/externalstore/RowTableDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/externalstore/RowTableDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/io/snappydata/streaming/StreamingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/streaming/StreamingDUnitTest.scala index 93783078a9..7c40d55a60 100644 --- a/cluster/src/dunit/scala/io/snappydata/streaming/StreamingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/streaming/StreamingDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala index 73307d28b4..58bc2c879e 100644 --- a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/jdbc/ConnectionConfDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/jdbc/ConnectionConfDUnitTest.scala index f96809ec93..04edd1c1bf 100644 --- a/cluster/src/dunit/scala/org/apache/spark/jdbc/ConnectionConfDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/jdbc/ConnectionConfDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/memory/MemoryManagerRestartDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/memory/MemoryManagerRestartDUnitTest.scala index 46aa9f48b6..10a88be3c7 100644 --- a/cluster/src/dunit/scala/org/apache/spark/memory/MemoryManagerRestartDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/memory/MemoryManagerRestartDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/memory/SnappyUnifiedMemoryManagerDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/memory/SnappyUnifiedMemoryManagerDUnitTest.scala index abf126018c..de1ac83d6c 100644 --- a/cluster/src/dunit/scala/org/apache/spark/memory/SnappyUnifiedMemoryManagerDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/memory/SnappyUnifiedMemoryManagerDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala index 7cab6930e2..2dd02d66a8 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/NorthWindDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/NorthWindDUnitTest.scala index 54d64cca85..f48a70ee9d 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/NorthWindDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/NorthWindDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala b/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala index a7e7e91b5f..ed7d3456d0 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/SmartConnectorFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala index 09181c76a1..5192fc1bc9 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHSuite.scala b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHSuite.scala index 264a43f176..5ce4a13804 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/TPCHSuite.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/TPCHSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/udf/UserDefinedFunctionsDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/udf/UserDefinedFunctionsDUnitTest.scala index 8daf5561e5..77c3cfdc51 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/udf/UserDefinedFunctionsDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/udf/UserDefinedFunctionsDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/dunit/scala/org/apache/spark/transaction/SnapshotGIIDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/transaction/SnapshotGIIDUnitTest.scala index ec5b52ba93..69e903bb79 100644 --- a/cluster/src/dunit/scala/org/apache/spark/transaction/SnapshotGIIDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/transaction/SnapshotGIIDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java b/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java index be595f1541..2d979bb1d3 100644 --- a/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java +++ b/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/java/org/apache/spark/streaming/JavaSnappyStreamingJob.java b/cluster/src/main/java/org/apache/spark/streaming/JavaSnappyStreamingJob.java index 001f60b737..8fa00d4a13 100644 --- a/cluster/src/main/java/org/apache/spark/streaming/JavaSnappyStreamingJob.java +++ b/cluster/src/main/java/org/apache/spark/streaming/JavaSnappyStreamingJob.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/ServiceManager.scala b/cluster/src/main/scala/io/snappydata/ServiceManager.scala index 89910d4eb3..c2807b5582 100644 --- a/cluster/src/main/scala/io/snappydata/ServiceManager.scala +++ b/cluster/src/main/scala/io/snappydata/ServiceManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala index 333e617317..367694233a 100644 --- a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala +++ b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/cluster/ExecutorInitiator.scala b/cluster/src/main/scala/io/snappydata/cluster/ExecutorInitiator.scala index a302009827..827456adc8 100644 --- a/cluster/src/main/scala/io/snappydata/cluster/ExecutorInitiator.scala +++ b/cluster/src/main/scala/io/snappydata/cluster/ExecutorInitiator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/gemxd/ClusterCallbacksImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/ClusterCallbacksImpl.scala index b5186a1be8..286c5469bb 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/ClusterCallbacksImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/ClusterCallbacksImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SnappyDataVersion.scala b/cluster/src/main/scala/io/snappydata/gemxd/SnappyDataVersion.scala index 0928cd9405..78bf9f29be 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SnappyDataVersion.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SnappyDataVersion.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala index a2f7e54ad8..c40ceb8f49 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLExecuteImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala index 659177a97c..069714c7c1 100644 --- a/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala +++ b/cluster/src/main/scala/io/snappydata/gemxd/SparkSQLPrepareImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala index d6013928ca..d6a3663809 100644 --- a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala +++ b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/impl/LocatorImpl.scala b/cluster/src/main/scala/io/snappydata/impl/LocatorImpl.scala index e65953ec76..51c295ccab 100644 --- a/cluster/src/main/scala/io/snappydata/impl/LocatorImpl.scala +++ b/cluster/src/main/scala/io/snappydata/impl/LocatorImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/tools/GfxdLauncherOverrides.scala b/cluster/src/main/scala/io/snappydata/tools/GfxdLauncherOverrides.scala index be73ce2672..244d4ab41d 100644 --- a/cluster/src/main/scala/io/snappydata/tools/GfxdLauncherOverrides.scala +++ b/cluster/src/main/scala/io/snappydata/tools/GfxdLauncherOverrides.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/tools/LeaderLauncher.scala b/cluster/src/main/scala/io/snappydata/tools/LeaderLauncher.scala index d6bf01398c..0e6cb4a49c 100644 --- a/cluster/src/main/scala/io/snappydata/tools/LeaderLauncher.scala +++ b/cluster/src/main/scala/io/snappydata/tools/LeaderLauncher.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/io/snappydata/tools/SnappyUtilLauncher.scala b/cluster/src/main/scala/io/snappydata/tools/SnappyUtilLauncher.scala index 3761fb8a6a..fa911bd42c 100644 --- a/cluster/src/main/scala/io/snappydata/tools/SnappyUtilLauncher.scala +++ b/cluster/src/main/scala/io/snappydata/tools/SnappyUtilLauncher.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala b/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala index 903e71c31e..8e5ab180c6 100644 --- a/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala +++ b/cluster/src/main/scala/org/apache/spark/SparkCallbacks.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/deploy/PackageAndDepUtils.scala b/cluster/src/main/scala/org/apache/spark/deploy/PackageAndDepUtils.scala index c92986e1a7..c0ddc8aee7 100644 --- a/cluster/src/main/scala/org/apache/spark/deploy/PackageAndDepUtils.scala +++ b/cluster/src/main/scala/org/apache/spark/deploy/PackageAndDepUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/executor/SnappyCoarseGrainedExecutorBackend.scala b/cluster/src/main/scala/org/apache/spark/executor/SnappyCoarseGrainedExecutorBackend.scala index df6de7c1d7..ca9d306a98 100644 --- a/cluster/src/main/scala/org/apache/spark/executor/SnappyCoarseGrainedExecutorBackend.scala +++ b/cluster/src/main/scala/org/apache/spark/executor/SnappyCoarseGrainedExecutorBackend.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/executor/SnappyExecutor.scala b/cluster/src/main/scala/org/apache/spark/executor/SnappyExecutor.scala index 368dd9f7be..dad2864835 100644 --- a/cluster/src/main/scala/org/apache/spark/executor/SnappyExecutor.scala +++ b/cluster/src/main/scala/org/apache/spark/executor/SnappyExecutor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/memory/MemoryManagerStatsWrapper.scala b/cluster/src/main/scala/org/apache/spark/memory/MemoryManagerStatsWrapper.scala index 5b90e8e0e6..2a8302696e 100644 --- a/cluster/src/main/scala/org/apache/spark/memory/MemoryManagerStatsWrapper.scala +++ b/cluster/src/main/scala/org/apache/spark/memory/MemoryManagerStatsWrapper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/memory/SnappyMemoryUtils.scala b/cluster/src/main/scala/org/apache/spark/memory/SnappyMemoryUtils.scala index 921e3e20be..0c9b649772 100644 --- a/cluster/src/main/scala/org/apache/spark/memory/SnappyMemoryUtils.scala +++ b/cluster/src/main/scala/org/apache/spark/memory/SnappyMemoryUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala b/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala index fb77cde5d5..5f34f5fc6a 100644 --- a/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala +++ b/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala b/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala index fbc795e0db..81c3a13011 100644 --- a/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala +++ b/cluster/src/main/scala/org/apache/spark/memory/SnappyUnifiedMemoryManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/scheduler/SnappyTaskSchedulerImpl.scala b/cluster/src/main/scala/org/apache/spark/scheduler/SnappyTaskSchedulerImpl.scala index cf792949f5..a2853b50c5 100644 --- a/cluster/src/main/scala/org/apache/spark/scheduler/SnappyTaskSchedulerImpl.scala +++ b/cluster/src/main/scala/org/apache/spark/scheduler/SnappyTaskSchedulerImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyCoarseGrainedSchedulerBackend.scala b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyCoarseGrainedSchedulerBackend.scala index d8388c79fd..7ffe324ba4 100644 --- a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyCoarseGrainedSchedulerBackend.scala +++ b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyCoarseGrainedSchedulerBackend.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala index f27634b2cb..3cc37c3432 100644 --- a/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala +++ b/cluster/src/main/scala/org/apache/spark/scheduler/cluster/SnappyEmbeddedModeClusterManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/sql/SnappySessionFactory.scala b/cluster/src/main/scala/org/apache/spark/sql/SnappySessionFactory.scala index 3055919073..cc15ddec9f 100644 --- a/cluster/src/main/scala/org/apache/spark/sql/SnappySessionFactory.scala +++ b/cluster/src/main/scala/org/apache/spark/sql/SnappySessionFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/sql/hive/thriftserver/SnappyHiveThriftServer2.scala b/cluster/src/main/scala/org/apache/spark/sql/hive/thriftserver/SnappyHiveThriftServer2.scala index 07cbcee7bb..87d81325fe 100644 --- a/cluster/src/main/scala/org/apache/spark/sql/hive/thriftserver/SnappyHiveThriftServer2.scala +++ b/cluster/src/main/scala/org/apache/spark/sql/hive/thriftserver/SnappyHiveThriftServer2.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/sql/streaming/SnappyStreamingContextFactory.scala b/cluster/src/main/scala/org/apache/spark/sql/streaming/SnappyStreamingContextFactory.scala index 580568d98f..d9830920a8 100644 --- a/cluster/src/main/scala/org/apache/spark/sql/streaming/SnappyStreamingContextFactory.scala +++ b/cluster/src/main/scala/org/apache/spark/sql/streaming/SnappyStreamingContextFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllExternalTablesResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllExternalTablesResource.scala index 69ae40a7f6..f9bd4ce789 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllExternalTablesResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllExternalTablesResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllMembersResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllMembersResource.scala index b86264b530..a05eedd0a8 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllMembersResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllMembersResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllTablesResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllTablesResource.scala index f50d74caf7..55a45794ee 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/AllTablesResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/AllTablesResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterDetails.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterDetails.scala index 07183236f6..d90a1d7252 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterDetails.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterDetails.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterInfoResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterInfoResource.scala index 3dd4f9117a..6a34ef7f68 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterInfoResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/ClusterInfoResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/MemberDetails.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/MemberDetails.scala index 85a08a1b12..0c6f6da9a5 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/MemberDetails.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/MemberDetails.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/MembersDetailsResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/MembersDetailsResource.scala index 132332f5c3..613fec8a80 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/MembersDetailsResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/MembersDetailsResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/SnappyApiRootResource.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/SnappyApiRootResource.scala index e7502c41b5..fb278fcdcd 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/SnappyApiRootResource.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/SnappyApiRootResource.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/TableDetails.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/TableDetails.scala index 744c4cec36..9c234ca562 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/TableDetails.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/TableDetails.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/status/api/v1/snappyapi.scala b/cluster/src/main/scala/org/apache/spark/status/api/v1/snappyapi.scala index 86ab23421e..2a77467c28 100644 --- a/cluster/src/main/scala/org/apache/spark/status/api/v1/snappyapi.scala +++ b/cluster/src/main/scala/org/apache/spark/status/api/v1/snappyapi.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyBasicAuthenticator.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyBasicAuthenticator.scala index db2a169495..efeead5b8d 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyBasicAuthenticator.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyBasicAuthenticator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardPage.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardPage.scala index e85be20f02..e4309159ea 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardPage.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardPage.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardTab.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardTab.scala index 1589d5b809..ef1632f73a 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardTab.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyDashboardTab.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyMemberDetailsPage.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyMemberDetailsPage.scala index 9937bc355d..8604b58a1d 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyMemberDetailsPage.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyMemberDetailsPage.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsPage.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsPage.scala index e04c48d4bc..2030cda21d 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsPage.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsPage.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsTab.scala b/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsTab.scala index 35cc22f1a3..bde35efab0 100644 --- a/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsTab.scala +++ b/cluster/src/main/scala/org/apache/spark/ui/SnappyStatsTab.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/main/scala/org/apache/spark/util/SnappyUtils.scala b/cluster/src/main/scala/org/apache/spark/util/SnappyUtils.scala index 93495725ef..de8b07dec2 100644 --- a/cluster/src/main/scala/org/apache/spark/util/SnappyUtils.scala +++ b/cluster/src/main/scala/org/apache/spark/util/SnappyUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/resources/log4j.properties b/cluster/src/test/resources/log4j.properties index 85513aca15..551c796164 100644 --- a/cluster/src/test/resources/log4j.properties +++ b/cluster/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/QueryTest.scala b/cluster/src/test/scala/io/snappydata/QueryTest.scala index 5ae4ffc395..2ba69be49b 100644 --- a/cluster/src/test/scala/io/snappydata/QueryTest.scala +++ b/cluster/src/test/scala/io/snappydata/QueryTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/Snap_213.scala b/cluster/src/test/scala/io/snappydata/Snap_213.scala index 6a7e18fd7f..71f900a17b 100644 --- a/cluster/src/test/scala/io/snappydata/Snap_213.scala +++ b/cluster/src/test/scala/io/snappydata/Snap_213.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Snappy.scala b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Snappy.scala index 0f09b38f4a..da179f84a6 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Snappy.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Snappy.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Spark.scala b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Spark.scala index 55bea852fa..c2384a997f 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Spark.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/BulkLoad_Spark.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/ParquetLoad.scala b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/ParquetLoad.scala index 353f533899..76b594c4e7 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/ParquetLoad.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/LoadPerformance/ParquetLoad.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala index dd9ebf9934..15de49a020 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala index 581cb37f3d..2ce698cdb8 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHRowPartitionedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHRowPartitionedTable.scala index 5151dd9bdf..bf718a982f 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHRowPartitionedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHRowPartitionedTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHTableSchema.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHTableSchema.scala index 286673ad30..3a3b855dff 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHTableSchema.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHTableSchema.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala index 513ecfabba..2ba19deab3 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala.scala b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala.scala index 1fdf1f5570..318cb10ba8 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Query.scala b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Query.scala index 3bb8b8dbc3..0051bd7d34 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Query.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Query.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Tables.scala b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Tables.scala index 44c25c1f85..7924db8afe 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Tables.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/kuduimpala/TPCH_Impala_Tables.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/ConcurrentMemsql.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/ConcurrentMemsql.scala index ecd968335e..485e256c44 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/ConcurrentMemsql.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/ConcurrentMemsql.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql.scala index 85b12601e0..f967cd2133 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query.scala index a9447d6283..a8cbba2471 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query_StreamExecution.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query_StreamExecution.scala index 0dc9e9d7b2..7639c2b9ef 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query_StreamExecution.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Query_StreamExecution.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_StreamExecution.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_StreamExecution.scala index 7f01f06fbb..9d02750a58 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_StreamExecution.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_StreamExecution.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Tables.scala b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Tables.scala index 9428bce52f..4838f90d11 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Tables.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/memsql/TPCH_Memsql_Tables.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala index 8176b0be3e..a930b8a980 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCDSSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala index b651bb0ebf..8e36e01a55 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java index d9de171a7a..82dcb0b15c 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCHPerfComparer.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala index 9caaa9dea1..a77dc54723 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/QueryExecutionJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala index 16fec34fed..8f6e2800de 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpcds/SparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/DataValidationJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/DataValidationJob.scala index 375778f674..98a18b168b 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/DataValidationJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/DataValidationJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJdbc.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJdbc.scala index e2cc0318f7..23576e92f4 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJdbc.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJdbc.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJob.scala index bd64c611fd..792d2918be 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionSmartConnector.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionSmartConnector.scala index b6273ae3b4..60325684ea 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionSmartConnector.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutionSmartConnector.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutor.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutor.scala index 581b949aed..d9315befaf 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutor.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/QueryExecutor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala index 58fa9b03a6..9491166ff5 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala index 3007637341..367fc132fc 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationJob.scala index c842d1ae36..61064963ed 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationSmartConnector.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationSmartConnector.scala index 8abf46b65d..561cdb70a3 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationSmartConnector.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/TableCreationSmartConnector.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpchmodifiers.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpchmodifiers.scala index de5a371e32..b55b0bf088 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpchmodifiers.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpchmodifiers.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/cluster/JDBCConnectionPoolTestSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/JDBCConnectionPoolTestSuite.scala index aff1571017..fb89561389 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/JDBCConnectionPoolTestSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/JDBCConnectionPoolTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala index 5c4f2e0943..ed7f3630fb 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala index 57c8b16722..9f9b542308 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala index 93e30f8311..bd951b8ca4 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/filodb/FiloDb_SnappyJob.scala b/cluster/src/test/scala/io/snappydata/filodb/FiloDb_SnappyJob.scala index 22912cfce4..fd57d2d6d7 100644 --- a/cluster/src/test/scala/io/snappydata/filodb/FiloDb_SnappyJob.scala +++ b/cluster/src/test/scala/io/snappydata/filodb/FiloDb_SnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/io/snappydata/tools/LeaderLauncherSpec.scala b/cluster/src/test/scala/io/snappydata/tools/LeaderLauncherSpec.scala index 360c5b11ba..5f5500ee78 100644 --- a/cluster/src/test/scala/io/snappydata/tools/LeaderLauncherSpec.scala +++ b/cluster/src/test/scala/io/snappydata/tools/LeaderLauncherSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/memory/MemoryFunSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/MemoryFunSuite.scala index c4f22fb532..f0ffa507ad 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/MemoryFunSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/MemoryFunSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/memory/MemoryManagerStatsSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/MemoryManagerStatsSuite.scala index ee14cc245c..2c668a8251 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/MemoryManagerStatsSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/MemoryManagerStatsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/memory/SnappyLocalIndexAccountingSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/SnappyLocalIndexAccountingSuite.scala index 6d874d90a8..cf596604e9 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/SnappyLocalIndexAccountingSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/SnappyLocalIndexAccountingSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala index df997aa904..25f1faa26d 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala index b9f960895b..5298ae59e9 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala b/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala index 4501ede1c9..4762da4917 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/LikeEscapeSimplificationSuite.scala b/cluster/src/test/scala/org/apache/spark/sql/LikeEscapeSimplificationSuite.scala index 8cffb6e57c..d2724f4825 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/LikeEscapeSimplificationSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/LikeEscapeSimplificationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala b/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala index 84ff19dfaa..86b078e831 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/NWQueries.scala b/cluster/src/test/scala/org/apache/spark/sql/NWQueries.scala index 52290215d2..3f93f39a60 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/NWQueries.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/NWQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/NorthWindTest.scala b/cluster/src/test/scala/org/apache/spark/sql/NorthWindTest.scala index 69ad197a5a..43caad1961 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/NorthWindTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/NorthWindTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala b/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala index abe80a4603..0d32c64746 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/SQLFunctionsTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala b/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala index ab55f944dd..2e0debf8d7 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala b/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala index b676cf311e..893d98e248 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/DataGenerator.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/DataGenerator.scala index 36f20728e0..55a883d0fb 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/DataGenerator.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/DataGenerator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/SnappyTableMutableAPISuite.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/SnappyTableMutableAPISuite.scala index 4ced7654dd..8571a81e30 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/SnappyTableMutableAPISuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/SnappyTableMutableAPISuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala index 8a01a308ba..74a88012ba 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnCacheBenchmark.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/MapTest.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/MapTest.scala index 1c357974af..2bf4b586f0 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/MapTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/MapTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/StringBenchmark.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/StringBenchmark.scala index 2bba333a41..253ab389a8 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/StringBenchmark.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/StringBenchmark.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala index 2039f11800..00b565bdcc 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TAQTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala index c263479d4a..d4773c4562 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQuerySnappyBenchmark.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala b/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala index 0615195772..d0aa40167c 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/kafka010/SnappyStructuredKafkaSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/AlterTableRowLevelSecurityEnableTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/AlterTableRowLevelSecurityEnableTest.scala index 67e9832874..b90e97b19c 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/AlterTableRowLevelSecurityEnableTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/AlterTableRowLevelSecurityEnableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyJdbcClientTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyJdbcClientTest.scala index 02936cc9ef..847dc9e19a 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyJdbcClientTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyJdbcClientTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTest.scala index 00c43c9d39..a79e60c1a4 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTestBase.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTestBase.scala index 26d0075025..691839bf27 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTestBase.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/PolicyTestBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/RestrictTableCreationPolicyTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/RestrictTableCreationPolicyTest.scala index 9f5e48c2cc..601911b0df 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/RestrictTableCreationPolicyTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/RestrictTableCreationPolicyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledJdbcClientPolicyTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledJdbcClientPolicyTest.scala index 5646f16978..f3d5d1b905 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledJdbcClientPolicyTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledJdbcClientPolicyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledPolicyTest.scala b/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledPolicyTest.scala index bb684f2950..d6e336922a 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledPolicyTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/policy/SecurityEnabledPolicyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/BitSetTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/BitSetTest.scala index 648844b47e..b10d1751de 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/BitSetTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/BitSetTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala index 3860fed5c4..a58df9f5ca 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/BugTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnEncodersTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnEncodersTest.scala index 4342dbe4d1..ae019655f6 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnEncodersTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnEncodersTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnMutableTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnMutableTest.scala index c1bffd2402..1c88b904ea 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnMutableTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnMutableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnTablesTestBase.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnTablesTestBase.scala index 61a8dc3b9d..c9b6716127 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnTablesTestBase.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnTablesTestBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala index a563809310..55ad1a51e1 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ComplexTypesTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ComplexTypesTest.scala index 1095620833..7fa85e45fa 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ComplexTypesTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ComplexTypesTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/RowMutableTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/RowMutableTest.scala index 382a59d22b..6d1ea5c982 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/RowMutableTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/RowMutableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala index 0d58d5c87e..7bb0959fc3 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SHAByteBufferTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SQLMetadataTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SQLMetadataTest.scala index 8fac8798a9..80f43876e1 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SQLMetadataTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SQLMetadataTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SecurityBugTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SecurityBugTest.scala index 9077b3ffc4..9735496507 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SecurityBugTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SecurityBugTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/SnappyUDFTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/SnappyUDFTest.scala index 9ca0e2061a..03cbf91a21 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/SnappyUDFTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/SnappyUDFTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/cluster/src/test/scala/org/apache/spark/unsafe/NativeUTF8StringPropertyCheckSuite.scala b/cluster/src/test/scala/org/apache/spark/unsafe/NativeUTF8StringPropertyCheckSuite.scala index 7d34f1510c..559c79355d 100644 --- a/cluster/src/test/scala/org/apache/spark/unsafe/NativeUTF8StringPropertyCheckSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/unsafe/NativeUTF8StringPropertyCheckSuite.scala @@ -17,7 +17,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/build.gradle b/compatibilityTests/build.gradle index 0e6c39d425..a7d579c52d 100644 --- a/compatibilityTests/build.gradle +++ b/compatibilityTests/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/resources/log4j.properties b/compatibilityTests/src/test/resources/log4j.properties index 222d083979..e923051db3 100644 --- a/compatibilityTests/src/test/resources/log4j.properties +++ b/compatibilityTests/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraInputFormat.scala b/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraInputFormat.scala index c8d5dc4eb4..c3929c1b0c 100644 --- a/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraInputFormat.scala +++ b/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraInputFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraOutputFormat.scala b/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraOutputFormat.scala index 3e9377e320..2898e16ecb 100644 --- a/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraOutputFormat.scala +++ b/compatibilityTests/src/test/scala/com/fruit/eyephone/CameraOutputFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyAggregateHashMapSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyAggregateHashMapSuite.scala index d05653c4a3..fc311ecb24 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyAggregateHashMapSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyAggregateHashMapSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyApproximatePercentileQuerySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyApproximatePercentileQuerySuite.scala index 43c783a0be..00670c2186 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyApproximatePercentileQuerySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyApproximatePercentileQuerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCachedTableSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCachedTableSuite.scala index 8ba58f55ba..2aa2950963 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCachedTableSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCachedTableSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCatalogSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCatalogSuite.scala index a5bf8f42e4..3c508bfab3 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCatalogSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCatalogSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCoGroupedIteratorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCoGroupedIteratorSuite.scala index cf54274449..acaff4e1a2 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCoGroupedIteratorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCoGroupedIteratorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyColumnExpressionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyColumnExpressionSuite.scala index cda60b3c32..14bec2463f 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyColumnExpressionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyColumnExpressionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCreateTableAsSelectSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCreateTableAsSelectSuite.scala index e612ed0a40..f770ca6316 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCreateTableAsSelectSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyCreateTableAsSelectSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLSourceLoadSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLSourceLoadSuite.scala index 06b9cf89ca..1101584c19 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLSourceLoadSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLSourceLoadSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLTestSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLTestSuite.scala index aaf3c2c2f0..08d17d3f72 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLTestSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDDLTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameAggregateSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameAggregateSuite.scala index 531dc7a3f4..06727542af 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameAggregateSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameAggregateSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameCallbackSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameCallbackSuite.scala index 0276e8a7e8..dce1b0f45f 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameCallbackSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameCallbackSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameComplexTypeSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameComplexTypeSuite.scala index 4094c5b7ff..2ed0ba45e7 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameComplexTypeSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameComplexTypeSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameFunctionsSuite.scala index bff1c601fa..af3fd04eb4 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameImplicitsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameImplicitsSuite.scala index 7cde6d90a4..71a43d69ae 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameImplicitsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameImplicitsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameJoinSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameJoinSuite.scala index bc38c2a776..4279034692 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameJoinSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameJoinSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameNaFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameNaFunctionsSuite.scala index 26664e57ad..29c0efb237 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameNaFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameNaFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFramePivotSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFramePivotSuite.scala index a89a129ea2..f949a66756 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFramePivotSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFramePivotSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameReaderWriterSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameReaderWriterSuite.scala index c1f3ce7453..8b523b30fc 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameReaderWriterSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameReaderWriterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameStatPerfSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameStatPerfSuite.scala index 90e446a4b8..1a96685016 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameStatPerfSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameStatPerfSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameSuite.scala index 97816c9455..5d7b1bee6c 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTimeWindowingSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTimeWindowingSuite.scala index ff75cc6a5e..e128aaf207 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTimeWindowingSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTimeWindowingSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTungstenSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTungstenSuite.scala index 4128bd20ec..9a977d99a8 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTungstenSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameTungstenSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameWindowFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameWindowFunctionsSuite.scala index 70c580cb49..04c05cdd54 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameWindowFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDataFrameWindowFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetAggregatorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetAggregatorSuite.scala index 76584b07e9..e102152032 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetAggregatorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetAggregatorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetCacheSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetCacheSuite.scala index 51b99beb4e..c76fc0dd84 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetCacheSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetCacheSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetPrimitiveSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetPrimitiveSuite.scala index 2de5f7982f..2d2e4ce9af 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetPrimitiveSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetPrimitiveSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSerializerRegistratorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSerializerRegistratorSuite.scala index 9ebc681c51..a334e7f145 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSerializerRegistratorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSerializerRegistratorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSuite.scala index 9bf0931a4f..c85c905631 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDatasetSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDateFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDateFunctionsSuite.scala index a7845f7cb0..c55ac123bc 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDateFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyDateFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyEventTimeWatermarkSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyEventTimeWatermarkSuite.scala index 055764ca2b..bbfdec0f82 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyEventTimeWatermarkSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyEventTimeWatermarkSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeCoordinatorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeCoordinatorSuite.scala index a339101be3..8e6a9dda0d 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeCoordinatorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeCoordinatorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeSuite.scala index 3688e2dbb7..17a603a89a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExchangeSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExtraStrategiesSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExtraStrategiesSuite.scala index 8509bdd83d..261f9b1ea4 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExtraStrategiesSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyExtraStrategiesSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSinkSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSinkSuite.scala index 634711a0f4..42e3ee1cd1 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSinkSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSinkSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSourceSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSourceSuite.scala index 7bb138575b..58f06aec86 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSourceSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamSourceSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamStressSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamStressSuite.scala index 88a602758b..2bc463c94a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamStressSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFileStreamStressSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFilteredScanSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFilteredScanSuite.scala index 04af8d49d6..e8c9508722 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFilteredScanSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFilteredScanSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFiltersSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFiltersSuite.scala index c78f91407b..4070b9c741 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFiltersSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyFiltersSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGeneratorFunctionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGeneratorFunctionSuite.scala index a9ff228537..1b3fd4bd01 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGeneratorFunctionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGeneratorFunctionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGlobalTempViewSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGlobalTempViewSuite.scala index 56c1b9910e..80cb8856dd 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGlobalTempViewSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGlobalTempViewSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGroupedIteratorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGroupedIteratorSuite.scala index c18f0e56a3..da1dd48bfd 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGroupedIteratorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyGroupedIteratorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyInsertSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyInsertSuite.scala index 332781afcb..2da0598b47 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyInsertSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyInsertSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCSuite.scala index d892bf2ae4..ad6591550c 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCWriteSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCWriteSuite.scala index 8c6439f58b..c015afeb84 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCWriteSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJDBCWriteSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJoinSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJoinSuite.scala index 5cccea0c45..dfc81ef8ca 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJoinSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJoinSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJsonFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJsonFunctionsSuite.scala index f99065b42a..e53b50d149 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJsonFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyJsonFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMathFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMathFunctionsSuite.scala index d60e01238a..bee6ef9615 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMathFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMathFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMemorySourceStressSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMemorySourceStressSuite.scala index f71d829f6e..0890f26fea 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMemorySourceStressSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMemorySourceStressSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMetadataCacheSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMetadataCacheSuite.scala index 41bf72adfa..7a3526e1e5 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMetadataCacheSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMetadataCacheSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMiscFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMiscFunctionsSuite.scala index 5b3dff970c..aac54c20f9 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMiscFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyMiscFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOffsetSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOffsetSuite.scala index 8771f1ddbd..edfa7c617a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOffsetSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOffsetSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOptimizeMetadataOnlyQuerySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOptimizeMetadataOnlyQuerySuite.scala index cd585cc8d1..2c7abcda51 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOptimizeMetadataOnlyQuerySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyOptimizeMetadataOnlyQuerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPartitionedWriteSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPartitionedWriteSuite.scala index b8df089b39..1f1bf28b13 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPartitionedWriteSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPartitionedWriteSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPathOptionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPathOptionSuite.scala index 6808bd404c..46627bded8 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPathOptionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPathOptionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPlannerSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPlannerSuite.scala index 501c602cb8..56fd096294 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPlannerSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPlannerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPrunedScanSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPrunedScanSuite.scala index 268e1f7f85..c929cc897c 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPrunedScanSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyPrunedScanSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryExecutionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryExecutionSuite.scala index e1c534351d..0f7514a26f 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryExecutionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryExecutionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryTestSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryTestSuite.scala index b4bbc40383..4dd101be7a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryTestSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyQueryTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyReduceAggregatorSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyReduceAggregatorSuite.scala index 25fa6f880e..04386dcbea 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyReduceAggregatorSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyReduceAggregatorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyResolvedDataSourceSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyResolvedDataSourceSuite.scala index 17d023c691..ed532281da 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyResolvedDataSourceSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyResolvedDataSourceSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyRowSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyRowSuite.scala index 0c545a2f0e..2bf92a1f2a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyRowSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyRowSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfEntrySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfEntrySuite.scala index 601d5a2a22..db8b792435 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfEntrySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfEntrySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfSuite.scala index 7e4742fbc0..ed4c850980 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLConfSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLExecutionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLExecutionSuite.scala index eae16248eb..4060042b6f 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLExecutionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLExecutionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala index 5a5fd5ecdb..f134a84b64 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLQuerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLWindowFunctionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLWindowFunctionSuite.scala index d14e8451b6..9ff815cd8a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLWindowFunctionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySQLWindowFunctionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySaveLoadSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySaveLoadSuite.scala index 253cb2aad8..fbf977d117 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySaveLoadSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySaveLoadSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyScalaReflectionRelationSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyScalaReflectionRelationSuite.scala index 9592546c6b..70f9b91c73 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyScalaReflectionRelationSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyScalaReflectionRelationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySerializationSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySerializationSuite.scala index 619e80011e..5ea1ed9bcd 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySerializationSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySerializationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySortSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySortSuite.scala index d9e1d53f99..a046ec79df 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySortSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySortSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkPlannerSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkPlannerSuite.scala index aa3316a7dd..2188381220 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkPlannerSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkPlannerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkSqlParserSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkSqlParserSuite.scala index 321fe0e2ee..380da2b335 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkSqlParserSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySparkSqlParserSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStatisticsCollectionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStatisticsCollectionSuite.scala index 803cb0429d..b326f19e12 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStatisticsCollectionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStatisticsCollectionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamSuite.scala index 25e76f47f5..85bae63ae5 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingAggregationSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingAggregationSuite.scala index 2a620455e8..5e6d7e0b9a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingAggregationSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingAggregationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryListenerSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryListenerSuite.scala index a273398ab9..751f13a3a1 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryListenerSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryListenerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryManagerSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryManagerSuite.scala index 1f6ba76e95..962e2850e3 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryManagerSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryManagerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryStatusAndProgressSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryStatusAndProgressSuite.scala index 699cb5cbb5..27a33b9fc7 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryStatusAndProgressSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQueryStatusAndProgressSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQuerySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQuerySuite.scala index c464d0f658..995736bee0 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQuerySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStreamingQuerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStringFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStringFunctionsSuite.scala index d4cf3d320d..cfb3e23aaf 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStringFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyStringFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySubquerySuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySubquerySuite.scala index ab6e9993d7..d6ac1bd3eb 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySubquerySuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappySubquerySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTableScanSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTableScanSuite.scala index 177eb44166..6f0d596b7e 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTableScanSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTableScanSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTakeOrderedAndProjectSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTakeOrderedAndProjectSuite.scala index 4034897caa..009178616b 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTakeOrderedAndProjectSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTakeOrderedAndProjectSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTypedImperativeAggregateSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTypedImperativeAggregateSuite.scala index 5599043b6b..72a48a9aa9 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTypedImperativeAggregateSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyTypedImperativeAggregateSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUDFSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUDFSuite.scala index 980df98b87..985db1e106 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUDFSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUDFSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeFixedWidthAggregationMapSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeFixedWidthAggregationMapSuite.scala index b1ce91df9f..87498e98ae 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeFixedWidthAggregationMapSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeFixedWidthAggregationMapSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeKVExternalSorterSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeKVExternalSorterSuite.scala index fd6fbd6b5b..010b39950a 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeKVExternalSorterSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUnsafeKVExternalSorterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUserDefinedTypeSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUserDefinedTypeSuite.scala index 0c36076bfe..cd2165a7ea 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUserDefinedTypeSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyUserDefinedTypeSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyVariableSubstitutionSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyVariableSubstitutionSuite.scala index a241c20332..dc9b480725 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyVariableSubstitutionSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyVariableSubstitutionSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyWholeStageCodegenSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyWholeStageCodegenSuite.scala index 79cce42eb5..4e42753a2e 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyWholeStageCodegenSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyWholeStageCodegenSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyXPathFunctionsSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyXPathFunctionsSuite.scala index 02edf97abb..3f109837cf 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyXPathFunctionsSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/SnappyXPathFunctionsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/catalyst/SnappyExpressionSQLBuilderSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/catalyst/SnappyExpressionSQLBuilderSuite.scala index 7280e79a2f..a2f51d66b2 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/catalyst/SnappyExpressionSQLBuilderSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/catalyst/SnappyExpressionSQLBuilderSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyForeachSinkSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyForeachSinkSuite.scala index b65f30432a..f5f7b3f1f9 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyForeachSinkSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyForeachSinkSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyStreamMetadataSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyStreamMetadataSuite.scala index 4ffd031031..e19e392fa2 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyStreamMetadataSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyStreamMetadataSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyTextSocketStreamSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyTextSocketStreamSuite.scala index c0824f74dd..23011e9123 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyTextSocketStreamSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/execution/streaming/SnappyTextSocketStreamSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/hive/TestHiveSnappySession.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/hive/TestHiveSnappySession.scala index 29be39cc97..61bed2d283 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/hive/TestHiveSnappySession.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/hive/TestHiveSnappySession.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSinkSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSinkSuite.scala index 1890b0b49a..0242d1f7c9 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSinkSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSinkSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSourceSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSourceSuite.scala index 27f0129463..39c65bfb78 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSourceSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/kafka010/SnappyKafkaSourceSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/SnappyMemorySinkSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/SnappyMemorySinkSuite.scala index 7144ca173f..2fce98f75d 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/SnappyMemorySinkSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/SnappyMemorySinkSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/test/SnappyDataStreamReaderWriterSuite.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/test/SnappyDataStreamReaderWriterSuite.scala index d72cbd62aa..86436c5234 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/test/SnappyDataStreamReaderWriterSuite.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/streaming/test/SnappyDataStreamReaderWriterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala index ee20eee69f..8d05482bc0 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SharedSnappySessionContext.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SnappySparkTestUtil.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SnappySparkTestUtil.scala index 87edc69b6f..23cdea8de3 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SnappySparkTestUtil.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/SnappySparkTestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/TestSnappySession.scala b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/TestSnappySession.scala index 748e8c6f66..49e560c0ce 100644 --- a/compatibilityTests/src/test/scala/org/apache/spark/sql/test/TestSnappySession.scala +++ b/compatibilityTests/src/test/scala/org/apache/spark/sql/test/TestSnappySession.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/build.gradle b/core/build.gradle index f69a3c04d7..8c7c8f511a 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala b/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala index a88da0cdad..ae361828ba 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/CassandraSnappyDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/io/snappydata/cluster/SnappyJobTestSupport.scala b/core/src/dunit/scala/io/snappydata/cluster/SnappyJobTestSupport.scala index 29e765b577..736306261f 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SnappyJobTestSupport.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SnappyJobTestSupport.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala index 75b0168eb5..a9a012d525 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala index c992381a72..1fce790917 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index 00fe5dfe4d..c38f8cb7eb 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/org/apache/spark/SparkUtilsAccess.scala b/core/src/dunit/scala/org/apache/spark/SparkUtilsAccess.scala index 530be06ee2..456f5e9560 100644 --- a/core/src/dunit/scala/org/apache/spark/SparkUtilsAccess.scala +++ b/core/src/dunit/scala/org/apache/spark/SparkUtilsAccess.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala b/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala index 532684a471..e7e4a99cf1 100644 --- a/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala +++ b/core/src/dunit/scala/org/apache/spark/sql/streaming/SnappySinkProviderDUnitTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/io/snappydata/impl/SnappyHiveConf.java b/core/src/main/java/io/snappydata/impl/SnappyHiveConf.java index 7f21a9be00..94cf387621 100644 --- a/core/src/main/java/io/snappydata/impl/SnappyHiveConf.java +++ b/core/src/main/java/io/snappydata/impl/SnappyHiveConf.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/StreamSummary.java b/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/StreamSummary.java index b8d3a8fd2f..14797a4c6d 100644 --- a/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/StreamSummary.java +++ b/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/StreamSummary.java @@ -19,7 +19,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/membership/Filter.java b/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/membership/Filter.java index a9f0b650bc..dcaa7d13cf 100644 --- a/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/membership/Filter.java +++ b/core/src/main/java/io/snappydata/util/com/clearspring/analytics/stream/membership/Filter.java @@ -20,7 +20,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/org/apache/spark/SnappyJavaUtils.java b/core/src/main/java/org/apache/spark/SnappyJavaUtils.java index 09aedfcd7a..345681ef52 100644 --- a/core/src/main/java/org/apache/spark/SnappyJavaUtils.java +++ b/core/src/main/java/org/apache/spark/SnappyJavaUtils.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/org/apache/spark/sql/collection/BoundedSortedSet.java b/core/src/main/java/org/apache/spark/sql/collection/BoundedSortedSet.java index 058d06c62b..bf61725118 100644 --- a/core/src/main/java/org/apache/spark/sql/collection/BoundedSortedSet.java +++ b/core/src/main/java/org/apache/spark/sql/collection/BoundedSortedSet.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/org/apache/spark/sql/hive/SnappyHiveCatalogBase.java b/core/src/main/java/org/apache/spark/sql/hive/SnappyHiveCatalogBase.java index 5eb62b0681..bf77b7cfcb 100644 --- a/core/src/main/java/org/apache/spark/sql/hive/SnappyHiveCatalogBase.java +++ b/core/src/main/java/org/apache/spark/sql/hive/SnappyHiveCatalogBase.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/java/org/apache/spark/sql/internal/SnappySharedState.java b/core/src/main/java/org/apache/spark/sql/internal/SnappySharedState.java index bbe0bdbc31..ca9355e29c 100644 --- a/core/src/main/java/org/apache/spark/sql/internal/SnappySharedState.java +++ b/core/src/main/java/org/apache/spark/sql/internal/SnappySharedState.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/Literals.scala b/core/src/main/scala/io/snappydata/Literals.scala index 6441749064..adfb070a41 100644 --- a/core/src/main/scala/io/snappydata/Literals.scala +++ b/core/src/main/scala/io/snappydata/Literals.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/LocalizedMessages.scala b/core/src/main/scala/io/snappydata/LocalizedMessages.scala index f4ddbf87bc..b1a8dd537b 100644 --- a/core/src/main/scala/io/snappydata/LocalizedMessages.scala +++ b/core/src/main/scala/io/snappydata/LocalizedMessages.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/ServerManager.scala b/core/src/main/scala/io/snappydata/ServerManager.scala index 30ff9b9d2f..72474211cb 100644 --- a/core/src/main/scala/io/snappydata/ServerManager.scala +++ b/core/src/main/scala/io/snappydata/ServerManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/Services.scala b/core/src/main/scala/io/snappydata/Services.scala index 19a864e20a..36d271ea89 100644 --- a/core/src/main/scala/io/snappydata/Services.scala +++ b/core/src/main/scala/io/snappydata/Services.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index 15c4a342d6..ada76b6145 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala b/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala index f38a6b72c7..ae54a30720 100644 --- a/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala +++ b/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala @@ -1,7 +1,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/TableStatsProviderService.scala b/core/src/main/scala/io/snappydata/TableStatsProviderService.scala index 886d72d38f..bcb9f14e3d 100644 --- a/core/src/main/scala/io/snappydata/TableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/TableStatsProviderService.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/ToolsCallback.scala b/core/src/main/scala/io/snappydata/ToolsCallback.scala index b225e752b2..540a3736de 100644 --- a/core/src/main/scala/io/snappydata/ToolsCallback.scala +++ b/core/src/main/scala/io/snappydata/ToolsCallback.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/functions.scala b/core/src/main/scala/io/snappydata/functions.scala index 9ada8bc070..64f94db1a6 100644 --- a/core/src/main/scala/io/snappydata/functions.scala +++ b/core/src/main/scala/io/snappydata/functions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/impl/ComplexTypeSerializerImpl.scala b/core/src/main/scala/io/snappydata/impl/ComplexTypeSerializerImpl.scala index aed2197586..ea3985194d 100644 --- a/core/src/main/scala/io/snappydata/impl/ComplexTypeSerializerImpl.scala +++ b/core/src/main/scala/io/snappydata/impl/ComplexTypeSerializerImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/impl/ServerImpl.scala b/core/src/main/scala/io/snappydata/impl/ServerImpl.scala index 32c3810a14..68d93a525d 100644 --- a/core/src/main/scala/io/snappydata/impl/ServerImpl.scala +++ b/core/src/main/scala/io/snappydata/impl/ServerImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala index 95045f7643..0442195265 100644 --- a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala +++ b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/sql/catalog/ConnectorExternalCatalog.scala b/core/src/main/scala/io/snappydata/sql/catalog/ConnectorExternalCatalog.scala index ce0705fb47..a06883cb2e 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/ConnectorExternalCatalog.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/ConnectorExternalCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala b/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala index 9e6d1f2ce6..663b59ac07 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala b/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala index df50e9ec71..aa61ffae9f 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/SnappyExternalCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/sql/catalog/impl/SmartConnectorExternalCatalog.scala b/core/src/main/scala/io/snappydata/sql/catalog/impl/SmartConnectorExternalCatalog.scala index a8f6e4b3cc..7f8f767c50 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/impl/SmartConnectorExternalCatalog.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/impl/SmartConnectorExternalCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/sql/catalog/impl/StoreHiveCatalog.scala b/core/src/main/scala/io/snappydata/sql/catalog/impl/StoreHiveCatalog.scala index 8f1a516764..a3cde08cb2 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/impl/StoreHiveCatalog.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/impl/StoreHiveCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/io/snappydata/util/ServiceUtils.scala b/core/src/main/scala/io/snappydata/util/ServiceUtils.scala index c846c644b2..e6bec12456 100644 --- a/core/src/main/scala/io/snappydata/util/ServiceUtils.scala +++ b/core/src/main/scala/io/snappydata/util/ServiceUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/RDDJavaFunctions.scala b/core/src/main/scala/org/apache/spark/RDDJavaFunctions.scala index 88e467b643..abdf90a65b 100644 --- a/core/src/main/scala/org/apache/spark/RDDJavaFunctions.scala +++ b/core/src/main/scala/org/apache/spark/RDDJavaFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/SnappyJavaHelperUtils.scala b/core/src/main/scala/org/apache/spark/SnappyJavaHelperUtils.scala index dbcd5adc70..b8519d6a1d 100644 --- a/core/src/main/scala/org/apache/spark/SnappyJavaHelperUtils.scala +++ b/core/src/main/scala/org/apache/spark/SnappyJavaHelperUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/jdbc/ConnectionConf.scala b/core/src/main/scala/org/apache/spark/jdbc/ConnectionConf.scala index 8f1fb99486..a65bb7577a 100644 --- a/core/src/main/scala/org/apache/spark/jdbc/ConnectionConf.scala +++ b/core/src/main/scala/org/apache/spark/jdbc/ConnectionConf.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/jdbc/ConnectionUtil.scala b/core/src/main/scala/org/apache/spark/jdbc/ConnectionUtil.scala index 93fdb89249..6e940ff5f9 100644 --- a/core/src/main/scala/org/apache/spark/jdbc/ConnectionUtil.scala +++ b/core/src/main/scala/org/apache/spark/jdbc/ConnectionUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/serializer/PooledKryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/PooledKryoSerializer.scala index e6489609ca..1b386b676d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/PooledKryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/PooledKryoSerializer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/serializer/SnappyKryoSerializerFactory.scala b/core/src/main/scala/org/apache/spark/serializer/SnappyKryoSerializerFactory.scala index bb6828c3b8..afebb31ae2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SnappyKryoSerializerFactory.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SnappyKryoSerializerFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/serializer/serializers.scala b/core/src/main/scala/org/apache/spark/serializer/serializers.scala index 89b3667745..105c4a90a5 100644 --- a/core/src/main/scala/org/apache/spark/serializer/serializers.scala +++ b/core/src/main/scala/org/apache/spark/serializer/serializers.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala index 18e42747c5..895142ef7d 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/DataFrameJavaFunctions.scala b/core/src/main/scala/org/apache/spark/sql/DataFrameJavaFunctions.scala index 249137b270..82252ae9f9 100644 --- a/core/src/main/scala/org/apache/spark/sql/DataFrameJavaFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/DataFrameJavaFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/DataFrameWriterJavaFunctions.scala b/core/src/main/scala/org/apache/spark/sql/DataFrameWriterJavaFunctions.scala index b3051e39db..efa55d217a 100644 --- a/core/src/main/scala/org/apache/spark/sql/DataFrameWriterJavaFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/DataFrameWriterJavaFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/LockUtils.scala b/core/src/main/scala/org/apache/spark/sql/LockUtils.scala index 3d1585db4e..2f7b6f24f5 100644 --- a/core/src/main/scala/org/apache/spark/sql/LockUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/LockUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SampleDataFrameContract.scala b/core/src/main/scala/org/apache/spark/sql/SampleDataFrameContract.scala index 27f36b0210..a492d28399 100644 --- a/core/src/main/scala/org/apache/spark/sql/SampleDataFrameContract.scala +++ b/core/src/main/scala/org/apache/spark/sql/SampleDataFrameContract.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala index 7b1665a5c2..c589d65420 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyBaseParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala index d0edc41c91..6773e93620 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala index d4352f6e1c..690b1f54e7 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala index 714d4c8c20..13c8430c8a 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyImplicits.scala b/core/src/main/scala/org/apache/spark/sql/SnappyImplicits.scala index 507bafe37e..b30a074694 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyImplicits.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyImplicits.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala index 7d997b1845..1d35f92671 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index 0068830cc6..6f563e26a3 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySqlParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappySqlParser.scala index 36beeb82ea..ac43d2937a 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySqlParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySqlParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala index 51d4b75a74..8976cb2f1c 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/TimeEpoch.scala b/core/src/main/scala/org/apache/spark/sql/TimeEpoch.scala index c37dffd378..53e8bd5960 100644 --- a/core/src/main/scala/org/apache/spark/sql/TimeEpoch.scala +++ b/core/src/main/scala/org/apache/spark/sql/TimeEpoch.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/ConcurrentSegmentedHashMap.scala b/core/src/main/scala/org/apache/spark/sql/collection/ConcurrentSegmentedHashMap.scala index 7dededd1ae..31eacd4e19 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/ConcurrentSegmentedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/ConcurrentSegmentedHashMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/GenerateFlatIterator.scala b/core/src/main/scala/org/apache/spark/sql/collection/GenerateFlatIterator.scala index 06b614e310..7045f61b7a 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/GenerateFlatIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/GenerateFlatIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashMap.scala b/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashMap.scala index ffa6ac1a38..5d2641c9d2 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashSet.scala b/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashSet.scala index 64887014d4..a4cf0bdf1e 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/MultiColumnOpenHashSet.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/ReusableRow.scala b/core/src/main/scala/org/apache/spark/sql/collection/ReusableRow.scala index 2abd465c9b..19c1f10fe3 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/ReusableRow.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/ReusableRow.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/SegmentMap.scala b/core/src/main/scala/org/apache/spark/sql/collection/SegmentMap.scala index 57bb07f22d..c72d392e7a 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/SegmentMap.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/SegmentMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 9cb4e15190..e6477a455b 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/collection/WrappedInternalRow.scala b/core/src/main/scala/org/apache/spark/sql/collection/WrappedInternalRow.scala index 33acab1ca3..2ac35b4fa7 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/WrappedInternalRow.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/WrappedInternalRow.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/dataFrames.scala b/core/src/main/scala/org/apache/spark/sql/dataFrames.scala index e8a243698a..942422e951 100644 --- a/core/src/main/scala/org/apache/spark/sql/dataFrames.scala +++ b/core/src/main/scala/org/apache/spark/sql/dataFrames.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/CatalogStaleException.scala b/core/src/main/scala/org/apache/spark/sql/execution/CatalogStaleException.scala index 049e65a1ad..1d2e4926ab 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/CatalogStaleException.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/CatalogStaleException.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/CodegenSparkFallback.scala b/core/src/main/scala/org/apache/spark/sql/execution/CodegenSparkFallback.scala index fd82076f76..3892040e24 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/CodegenSparkFallback.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/CodegenSparkFallback.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/CodegenSupportOnExecutor.scala b/core/src/main/scala/org/apache/spark/sql/execution/CodegenSupportOnExecutor.scala index 766f0f3c78..a9ba2d11bd 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/CodegenSupportOnExecutor.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/CodegenSupportOnExecutor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/DictionaryOptimizedMapAccessor.scala b/core/src/main/scala/org/apache/spark/sql/execution/DictionaryOptimizedMapAccessor.scala index 3fcc21fae7..9c4768f761 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/DictionaryOptimizedMapAccessor.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/DictionaryOptimizedMapAccessor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/EncoderScanExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/EncoderScanExec.scala index 6309e37f5d..2079e3e287 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/EncoderScanExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/EncoderScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala index 3462b343db..308dcf479f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/NonRecursivePlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/NonRecursivePlans.scala index c17e938a2c..c14644c922 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/NonRecursivePlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/NonRecursivePlans.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ObjectHashMapAccessor.scala b/core/src/main/scala/org/apache/spark/sql/execution/ObjectHashMapAccessor.scala index b885e0755b..b9ccf6b8a3 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ObjectHashMapAccessor.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ObjectHashMapAccessor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/RDDKryo.scala b/core/src/main/scala/org/apache/spark/sql/execution/RDDKryo.scala index 6fda6ce967..1ff0bd19fb 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/RDDKryo.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/RDDKryo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/RefreshMetadata.scala b/core/src/main/scala/org/apache/spark/sql/execution/RefreshMetadata.scala index bbceb54482..c3c67f622c 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/RefreshMetadata.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/RefreshMetadata.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala index 250cdf062e..3ed64ae89a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SHAMapAccessor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SecurityUtils.scala b/core/src/main/scala/org/apache/spark/sql/execution/SecurityUtils.scala index 3034072635..a214361cb1 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SecurityUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SecurityUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SnappySortExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/SnappySortExec.scala index 989a92879f..0eb5943b56 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SnappySortExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SnappySortExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala index 5b8a4a36fa..542769c2c2 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/TopK.scala b/core/src/main/scala/org/apache/spark/sql/execution/TopK.scala index b5c0ecb08a..7372c0d0c6 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/TopK.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/TopK.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala index a214bb7a0b..e0f3793739 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/CollectAggregateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index 939fe91d8f..5972fd9195 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala index 65cbe4e1d0..722769f43c 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchIterator.scala index b6fbd3d327..1e38f4de7c 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnDeleteExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnDeleteExec.scala index 9a51962bd9..fa07563fc1 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnDeleteExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnDeleteExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala index e1f35b9fcd..886b65beac 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala index e0b0622762..e3eaea041c 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnPutIntoExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnPutIntoExec.scala index a0ea26ee4d..b5007da061 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnPutIntoExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnPutIntoExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala index b86ba85491..899b5b1f06 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala index d05717b13a..384b193936 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnUpdateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala index 1f12fc7a6a..f8eb960b40 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStore.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala index a2a293fc80..fb12f4b4c0 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala index 496802e88b..955b9807dc 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala index 2118116918..72a29c4d80 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala index 0ac2813b75..a263a4e0be 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala index 06e782da79..6e3539f339 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala index 2833f52a89..507233bc17 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/DefaultSource.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/DefaultSource.scala index b2ae8dc10e..312cd86b79 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/DefaultSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/DefaultSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index 29f6cd963d..36d594eeb2 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala index 74ed5f3b1a..25c54592b3 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala index 897d48b064..1e35ad48f5 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala index 2355f75724..a11ef58d84 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ddl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/externalRelations.scala b/core/src/main/scala/org/apache/spark/sql/execution/externalRelations.scala index 11c53946b8..07b94d4579 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/externalRelations.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/externalRelations.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoinExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoinExec.scala index 8d5bc8f37d..9c5d7c76d8 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoinExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappyJoinLike.scala b/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappyJoinLike.scala index fa7a15fceb..7963b22ab7 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappyJoinLike.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappyJoinLike.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappySortMergeJoinExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappySortMergeJoinExec.scala index a86ca024f3..a9af6b373b 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappySortMergeJoinExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/joins/SnappySortMergeJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/DefaultSource.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/DefaultSource.scala index 5b8a0bb343..3df538f809 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/DefaultSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/DefaultSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala index 980809cda3..08ec8a434b 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowDeleteExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowDeleteExec.scala index 8245fb7c02..1336956ddc 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowDeleteExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowDeleteExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowExec.scala index beadd678dc..1e328c083b 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala index d7e80f0c76..db35e4f3f3 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatRelation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 18039633f0..8277c19e8b 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowInsertExec.scala index 009b0f9136..a9bed020ed 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowInsertExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowInsertExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowTableScan.scala index adec652421..3e321dfce1 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowTableScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowUpdateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowUpdateExec.scala index bb516298d2..5825ce4e87 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowUpdateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowUpdateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala index 9ba3cd6cba..0fd90c94d4 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala b/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala index 62448ec198..e5b701600e 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala index 832346e61e..4a20156e52 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/hive/HiveClientUtil.scala b/core/src/main/scala/org/apache/spark/sql/hive/HiveClientUtil.scala index e4c79c8ee6..4fe2594db2 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/HiveClientUtil.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/HiveClientUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala index eaae0c76ff..138acee53c 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala index d638ddffd2..15d16ed41d 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappySessionState.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/CodeGenerationException.scala b/core/src/main/scala/org/apache/spark/sql/internal/CodeGenerationException.scala index a09bf82f8b..b1064a4cd8 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/CodeGenerationException.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/CodeGenerationException.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala index e6fc888fc2..e5987238c2 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/JarUtils.scala b/core/src/main/scala/org/apache/spark/sql/internal/JarUtils.scala index e8f4823b99..0d24ef8c76 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/JarUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/JarUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/JoinQueryPlanning.scala b/core/src/main/scala/org/apache/spark/sql/internal/JoinQueryPlanning.scala index df888336f3..7dc547e94f 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/JoinQueryPlanning.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/JoinQueryPlanning.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala index ad720b3f19..3ba0401fda 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/UDFFunction.scala b/core/src/main/scala/org/apache/spark/sql/internal/UDFFunction.scala index ed39acf00c..04aed81d7e 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/UDFFunction.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/UDFFunction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/internal/session.scala b/core/src/main/scala/org/apache/spark/sql/internal/session.scala index 1c526c4217..7aa26fb3db 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/session.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/session.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/policy/policies.scala b/core/src/main/scala/org/apache/spark/sql/policy/policies.scala index 5e4c84ba3e..ad33581ac8 100644 --- a/core/src/main/scala/org/apache/spark/sql/policy/policies.scala +++ b/core/src/main/scala/org/apache/spark/sql/policy/policies.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/policy/policyFunctions.scala b/core/src/main/scala/org/apache/spark/sql/policy/policyFunctions.scala index 521ce62abf..5036402391 100644 --- a/core/src/main/scala/org/apache/spark/sql/policy/policyFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/policy/policyFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/rdds.scala b/core/src/main/scala/org/apache/spark/sql/rdds.scala index 85c649a5b7..f9e82ca54d 100644 --- a/core/src/main/scala/org/apache/spark/sql/rdds.scala +++ b/core/src/main/scala/org/apache/spark/sql/rdds.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala b/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala index 7f3aa5fd04..f5aa5315a4 100644 --- a/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/row/SnappyStoreDialect.scala b/core/src/main/scala/org/apache/spark/sql/row/SnappyStoreDialect.scala index 5fb8f0e283..231088e47a 100644 --- a/core/src/main/scala/org/apache/spark/sql/row/SnappyStoreDialect.scala +++ b/core/src/main/scala/org/apache/spark/sql/row/SnappyStoreDialect.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/CastDouble.scala b/core/src/main/scala/org/apache/spark/sql/sources/CastDouble.scala index 439f120b1a..ed04d87d00 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/CastDouble.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/CastDouble.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/CastLongTime.scala b/core/src/main/scala/org/apache/spark/sql/sources/CastLongTime.scala index 0fc706e3c2..29eb00d5fe 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/CastLongTime.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/CastLongTime.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/RuleUtils.scala b/core/src/main/scala/org/apache/spark/sql/sources/RuleUtils.scala index 897c8fd70c..5bf3247018 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/RuleUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/RuleUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/SnappyOptimizations.scala b/core/src/main/scala/org/apache/spark/sql/sources/SnappyOptimizations.scala index d282ffc8f0..8366b208d3 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/SnappyOptimizations.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/SnappyOptimizations.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/StatVarianceCounter.scala b/core/src/main/scala/org/apache/spark/sql/sources/StatVarianceCounter.scala index b9a1c4c6ae..d1b86e64af 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/StatVarianceCounter.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/StatVarianceCounter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/StoreStrategy.scala b/core/src/main/scala/org/apache/spark/sql/sources/StoreStrategy.scala index 6484986125..56c724f7a7 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/StoreStrategy.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/StoreStrategy.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 7fa4052fd0..bac7e2c1a3 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/sources/subrules.scala b/core/src/main/scala/org/apache/spark/sql/sources/subrules.scala index 56ec831e32..c24fd6ef10 100644 --- a/core/src/main/scala/org/apache/spark/sql/sources/subrules.scala +++ b/core/src/main/scala/org/apache/spark/sql/sources/subrules.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/store/CodeGeneration.scala b/core/src/main/scala/org/apache/spark/sql/store/CodeGeneration.scala index 6031ddc799..f26b54ccdb 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/CodeGeneration.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/CodeGeneration.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/store/ColumnPartitioner.scala b/core/src/main/scala/org/apache/spark/sql/store/ColumnPartitioner.scala index 20edcc32c5..ee69239194 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/ColumnPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/ColumnPartitioner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/store/MembershipAccumulator.scala b/core/src/main/scala/org/apache/spark/sql/store/MembershipAccumulator.scala index add9d7ba12..9d8968ef63 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/MembershipAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/MembershipAccumulator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreHashFunction.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreHashFunction.scala index 67b1616028..bfc57f4a61 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/StoreHashFunction.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/StoreHashFunction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala index 97bd23dd52..aa7f6ed39d 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/DirectKafkaStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/DirectKafkaStreamSource.scala index ab15c17bcb..135315b70a 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/DirectKafkaStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/DirectKafkaStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/FileStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/FileStreamSource.scala index 2799731553..bfe82df683 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/FileStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/FileStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/FilteredSchemaDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/FilteredSchemaDStream.scala index 8d70e88cfa..cc8d7ef80b 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/FilteredSchemaDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/FilteredSchemaDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/FlatMappedDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/FlatMappedDStream.scala index 48e6488f8b..4d389ffb7e 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/FlatMappedDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/FlatMappedDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/GlommedSchemaDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/GlommedSchemaDStream.scala index f9f1d97158..4a2f91de42 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/GlommedSchemaDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/GlommedSchemaDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/LogicalDStreamPlan.scala b/core/src/main/scala/org/apache/spark/sql/streaming/LogicalDStreamPlan.scala index 740415cc71..61f91ff5d1 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/LogicalDStreamPlan.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/LogicalDStreamPlan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/MapPartitionedSchemaDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/MapPartitionedSchemaDStream.scala index 2fd0b44e94..fc0a4dde2e 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/MapPartitionedSchemaDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/MapPartitionedSchemaDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/MappedDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/MappedDStream.scala index 3a2e6f0256..f452e5bd20 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/MappedDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/MappedDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/PhysicalDStreamPlan.scala b/core/src/main/scala/org/apache/spark/sql/streaming/PhysicalDStreamPlan.scala index 91bffbe365..28054aaa10 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/PhysicalDStreamPlan.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/PhysicalDStreamPlan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQStreamSource.scala index 5bc4edd5f6..9a1973aa82 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQUtils.scala b/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQUtils.scala index eb0fb3f625..9182580776 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/RabbitMQUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/RawSocketStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/RawSocketStreamSource.scala index 7410cb22d6..2feb50f6b3 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/RawSocketStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/RawSocketStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/SchemaDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/SchemaDStream.scala index 7ff315d958..5d57de35a2 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/SchemaDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/SchemaDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/SnappySinkCallback.scala b/core/src/main/scala/org/apache/spark/sql/streaming/SnappySinkCallback.scala index 482217eb16..9894531ae7 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/SnappySinkCallback.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/SnappySinkCallback.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/SocketStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/SocketStreamSource.scala index 8428af1aeb..720779bde7 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/SocketStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/SocketStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/StreamBaseRelation.scala b/core/src/main/scala/org/apache/spark/sql/streaming/StreamBaseRelation.scala index 9d975189ba..ed067c6665 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/StreamBaseRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/StreamBaseRelation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/StreamConverter.scala b/core/src/main/scala/org/apache/spark/sql/streaming/StreamConverter.scala index f2ce06385e..cb9e785bbd 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/StreamConverter.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/StreamConverter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/StreamSqlHelper.scala b/core/src/main/scala/org/apache/spark/sql/streaming/StreamSqlHelper.scala index 51a637a8c4..31dd9c8005 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/StreamSqlHelper.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/StreamSqlHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/StreamToRowsConverter.scala b/core/src/main/scala/org/apache/spark/sql/streaming/StreamToRowsConverter.scala index e91b7e891f..68fdabed57 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/StreamToRowsConverter.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/StreamToRowsConverter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/TextSocketStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/TextSocketStreamSource.scala index d7c915697d..e78675a10f 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/TextSocketStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/TextSocketStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/TransformedSchemaDStream.scala b/core/src/main/scala/org/apache/spark/sql/streaming/TransformedSchemaDStream.scala index 782499f596..7f839e4d50 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/TransformedSchemaDStream.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/TransformedSchemaDStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/TwitterStreamSource.scala b/core/src/main/scala/org/apache/spark/sql/streaming/TwitterStreamSource.scala index 529eec1211..cac4b793bd 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/TwitterStreamSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/TwitterStreamSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/WindowLogicalPlan.scala b/core/src/main/scala/org/apache/spark/sql/streaming/WindowLogicalPlan.scala index e137671178..bcbc258bc9 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/WindowLogicalPlan.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/WindowLogicalPlan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/streaming/WindowPhysicalPlan.scala b/core/src/main/scala/org/apache/spark/sql/streaming/WindowPhysicalPlan.scala index e2fce9c1da..f7759af3a9 100644 --- a/core/src/main/scala/org/apache/spark/sql/streaming/WindowPhysicalPlan.scala +++ b/core/src/main/scala/org/apache/spark/sql/streaming/WindowPhysicalPlan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala b/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala index 1b8eb9d60f..2478cd6ad3 100644 --- a/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala +++ b/core/src/main/scala/org/apache/spark/sql/types/TypeUtilities.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/streaming/SnappyStreamingContext.scala b/core/src/main/scala/org/apache/spark/streaming/SnappyStreamingContext.scala index 679ae46eae..f6d143a9f3 100644 --- a/core/src/main/scala/org/apache/spark/streaming/SnappyStreamingContext.scala +++ b/core/src/main/scala/org/apache/spark/streaming/SnappyStreamingContext.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/streaming/StreamUtils.scala b/core/src/main/scala/org/apache/spark/streaming/StreamUtils.scala index 3985cfc40b..430b5b08e8 100644 --- a/core/src/main/scala/org/apache/spark/streaming/StreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/streaming/StreamUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/main/scala/org/apache/spark/streaming/api/java/JavaSnappyStreamingContext.scala b/core/src/main/scala/org/apache/spark/streaming/api/java/JavaSnappyStreamingContext.scala index 6101c88660..2e57965aa3 100644 --- a/core/src/main/scala/org/apache/spark/streaming/api/java/JavaSnappyStreamingContext.scala +++ b/core/src/main/scala/org/apache/spark/streaming/api/java/JavaSnappyStreamingContext.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java b/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java index f80ba1d028..dddf55cec9 100644 --- a/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java +++ b/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/java/io/snappydata/util/BoundedSortedSetTest.java b/core/src/test/java/io/snappydata/util/BoundedSortedSetTest.java index 2191f0e53b..4e407d98bf 100644 --- a/core/src/test/java/io/snappydata/util/BoundedSortedSetTest.java +++ b/core/src/test/java/io/snappydata/util/BoundedSortedSetTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/java/org/apache/spark/streaming/JavaSnappyStreamingContextSuite.java b/core/src/test/java/org/apache/spark/streaming/JavaSnappyStreamingContextSuite.java index f7bf3841b2..bb2f331c63 100644 --- a/core/src/test/java/org/apache/spark/streaming/JavaSnappyStreamingContextSuite.java +++ b/core/src/test/java/org/apache/spark/streaming/JavaSnappyStreamingContextSuite.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 0df4d6bfa0..3bc8aaa561 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index 6bb728b9d0..78555c753b 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/CommandLineToolsSuite.scala b/core/src/test/scala/io/snappydata/CommandLineToolsSuite.scala index 6446154a3c..51caf3ba06 100644 --- a/core/src/test/scala/io/snappydata/CommandLineToolsSuite.scala +++ b/core/src/test/scala/io/snappydata/CommandLineToolsSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/ConcurrentOpsTests.scala b/core/src/test/scala/io/snappydata/ConcurrentOpsTests.scala index 2a5602d42c..ed730247ee 100644 --- a/core/src/test/scala/io/snappydata/ConcurrentOpsTests.scala +++ b/core/src/test/scala/io/snappydata/ConcurrentOpsTests.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala index 97d2b28b4a..1f488bd3ed 100644 --- a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala +++ b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/ColumnTableSpec.scala b/core/src/test/scala/io/snappydata/app/ColumnTableSpec.scala index 3478c1b87f..812eb075b7 100644 --- a/core/src/test/scala/io/snappydata/app/ColumnTableSpec.scala +++ b/core/src/test/scala/io/snappydata/app/ColumnTableSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/ExternalTableTest.scala b/core/src/test/scala/io/snappydata/app/ExternalTableTest.scala index 09ee980aad..e5ff37a47e 100644 --- a/core/src/test/scala/io/snappydata/app/ExternalTableTest.scala +++ b/core/src/test/scala/io/snappydata/app/ExternalTableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/ExternalTableTestData.scala b/core/src/test/scala/io/snappydata/app/ExternalTableTestData.scala index c52d170f95..06346d98ab 100644 --- a/core/src/test/scala/io/snappydata/app/ExternalTableTestData.scala +++ b/core/src/test/scala/io/snappydata/app/ExternalTableTestData.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/KMeansModelForDemo.scala b/core/src/test/scala/io/snappydata/app/KMeansModelForDemo.scala index c4244509d3..905d1326e7 100644 --- a/core/src/test/scala/io/snappydata/app/KMeansModelForDemo.scala +++ b/core/src/test/scala/io/snappydata/app/KMeansModelForDemo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala b/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala index 5c1e2f3bd0..fca4bb0e34 100644 --- a/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala +++ b/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/SparkSQLMultiThreadingTest.scala b/core/src/test/scala/io/snappydata/app/SparkSQLMultiThreadingTest.scala index 276f042874..133d14ee58 100644 --- a/core/src/test/scala/io/snappydata/app/SparkSQLMultiThreadingTest.scala +++ b/core/src/test/scala/io/snappydata/app/SparkSQLMultiThreadingTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/app/StreamingTupleGenerator.scala b/core/src/test/scala/io/snappydata/app/StreamingTupleGenerator.scala index 51d4fac2cc..d76243b475 100644 --- a/core/src/test/scala/io/snappydata/app/StreamingTupleGenerator.scala +++ b/core/src/test/scala/io/snappydata/app/StreamingTupleGenerator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/core/LocalTestData.scala b/core/src/test/scala/io/snappydata/core/LocalTestData.scala index 93863ee380..8c45270cac 100644 --- a/core/src/test/scala/io/snappydata/core/LocalTestData.scala +++ b/core/src/test/scala/io/snappydata/core/LocalTestData.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/gemxd/BasicStoreSuite.scala b/core/src/test/scala/io/snappydata/gemxd/BasicStoreSuite.scala index 7a55aa52d3..7262a59b76 100644 --- a/core/src/test/scala/io/snappydata/gemxd/BasicStoreSuite.scala +++ b/core/src/test/scala/io/snappydata/gemxd/BasicStoreSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/util/DebugUtilsSpec.scala b/core/src/test/scala/io/snappydata/util/DebugUtilsSpec.scala index c97631288a..6b1cd43454 100644 --- a/core/src/test/scala/io/snappydata/util/DebugUtilsSpec.scala +++ b/core/src/test/scala/io/snappydata/util/DebugUtilsSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/util/NumberUtilsSpec.scala b/core/src/test/scala/io/snappydata/util/NumberUtilsSpec.scala index f1278208e3..c7582d6193 100644 --- a/core/src/test/scala/io/snappydata/util/NumberUtilsSpec.scala +++ b/core/src/test/scala/io/snappydata/util/NumberUtilsSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/util/StringUtilsSpec.scala b/core/src/test/scala/io/snappydata/util/StringUtilsSpec.scala index 3abaf43e44..3ac78ac63a 100644 --- a/core/src/test/scala/io/snappydata/util/StringUtilsSpec.scala +++ b/core/src/test/scala/io/snappydata/util/StringUtilsSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/io/snappydata/util/TestUtils.scala b/core/src/test/scala/io/snappydata/util/TestUtils.scala index 1232d9240c..6a5de125b4 100644 --- a/core/src/test/scala/io/snappydata/util/TestUtils.scala +++ b/core/src/test/scala/io/snappydata/util/TestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala b/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala index 73687362c3..d31c9a2eeb 100644 --- a/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala +++ b/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/serializer/PooledKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/PooledKryoSerializerSuite.scala index 1cc138dd35..58a6943ffd 100644 --- a/core/src/test/scala/org/apache/spark/serializer/PooledKryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/PooledKryoSerializerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/DiskStoreDDLTest.scala b/core/src/test/scala/org/apache/spark/sql/DiskStoreDDLTest.scala index 87b67e1865..485d7a897d 100644 --- a/core/src/test/scala/org/apache/spark/sql/DiskStoreDDLTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/DiskStoreDDLTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/SnappyTempTableTest.scala b/core/src/test/scala/org/apache/spark/sql/SnappyTempTableTest.scala index 246b405b9c..1576bcffb4 100644 --- a/core/src/test/scala/org/apache/spark/sql/SnappyTempTableTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/SnappyTempTableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala b/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala index 6399b860ee..5e794c6c76 100644 --- a/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/execution/AggregationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/internal/UpdateStatementTypeCastingSuite.scala b/core/src/test/scala/org/apache/spark/sql/internal/UpdateStatementTypeCastingSuite.scala index 9d82e10281..69eef94eae 100644 --- a/core/src/test/scala/org/apache/spark/sql/internal/UpdateStatementTypeCastingSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/internal/UpdateStatementTypeCastingSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/core/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 1b314beeef..c75c309972 100644 --- a/core/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/kafka010/SnappyStreamingKafkaSuite.scala b/core/src/test/scala/org/apache/spark/sql/kafka010/SnappyStreamingKafkaSuite.scala index 08aad08055..4f11eea2e7 100644 --- a/core/src/test/scala/org/apache/spark/sql/kafka010/SnappyStreamingKafkaSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/kafka010/SnappyStreamingKafkaSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/CatalogConsistencyTest.scala b/core/src/test/scala/org/apache/spark/sql/store/CatalogConsistencyTest.scala index a8f83a623d..1a90416d8c 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/CatalogConsistencyTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/CatalogConsistencyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableBatchInsertTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableBatchInsertTest.scala index 04b870bf6a..0863e87fd3 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableBatchInsertTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableBatchInsertTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala index 49904a49be..e0d2606872 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableTest.scala index 5a22390d5a..eb2337f0c0 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala b/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala index 067310f85c..34bbf52b0d 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/CreateIndexTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/CubeRollupGroupingSetsTest.scala b/core/src/test/scala/org/apache/spark/sql/store/CubeRollupGroupingSetsTest.scala index abaebd6500..54e0d03f06 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/CubeRollupGroupingSetsTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/CubeRollupGroupingSetsTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/DisableTokenizationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/DisableTokenizationTest.scala index 3565cdea04..a85818a522 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/DisableTokenizationTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/DisableTokenizationTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/JDBCMutableRelationAPISuite.scala b/core/src/test/scala/org/apache/spark/sql/store/JDBCMutableRelationAPISuite.scala index 9268a30a59..fd70aca987 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/JDBCMutableRelationAPISuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/JDBCMutableRelationAPISuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/MetadataTest.scala b/core/src/test/scala/org/apache/spark/sql/store/MetadataTest.scala index da7c7dd627..664c154e17 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/MetadataTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/MetadataTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/PlanCachingTest.scala b/core/src/test/scala/org/apache/spark/sql/store/PlanCachingTest.scala index d20cd7cc4e..e522489fa4 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/PlanCachingTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/PlanCachingTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/RowRelationAPISuite.scala b/core/src/test/scala/org/apache/spark/sql/store/RowRelationAPISuite.scala index 3c54c1cefc..97bdd8cebd 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/RowRelationAPISuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/RowRelationAPISuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/RowTableTest.scala b/core/src/test/scala/org/apache/spark/sql/store/RowTableTest.scala index 9a900cf99e..cb99ea9aa5 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/RowTableTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/RowTableTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/SnappyCatalogSuite.scala b/core/src/test/scala/org/apache/spark/sql/store/SnappyCatalogSuite.scala index f214a75236..d1cc1818a5 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/SnappyCatalogSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/SnappyCatalogSuite.scala @@ -16,7 +16,7 @@ */ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/SnappyJoinSuite.scala b/core/src/test/scala/org/apache/spark/sql/store/SnappyJoinSuite.scala index 3d4aabe08e..8e3512fec2 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/SnappyJoinSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/SnappyJoinSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/StringTest.scala b/core/src/test/scala/org/apache/spark/sql/store/StringTest.scala index 50d5445819..9e001a3e12 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/StringTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/StringTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala index b11a1d1e9e..59bcd60e68 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/TokenizationTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/UnifiedPartitionerTest.scala b/core/src/test/scala/org/apache/spark/sql/store/UnifiedPartitionerTest.scala index eb22814cb4..41ab7cb4cf 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/UnifiedPartitionerTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/UnifiedPartitionerTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/store/ViewTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ViewTest.scala index c37c2b71b2..3a8e2fbcbf 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/ViewTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/ViewTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSecuritySuite.scala b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSecuritySuite.scala index 0dd99412f0..4c163d5ab1 100644 --- a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSecuritySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSuite.scala b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSuite.scala index ed73c43a23..208bc97469 100644 --- a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStoreSinkProviderSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingAPISuite.scala b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingAPISuite.scala index 6afa1badf8..2a3d427307 100644 --- a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingAPISuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingAPISuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/core/src/test/scala/org/apache/spark/streaming/SnappyStreamingContextSuite.scala b/core/src/test/scala/org/apache/spark/streaming/SnappyStreamingContextSuite.scala index 495074a26b..7b6feff020 100644 --- a/core/src/test/scala/org/apache/spark/streaming/SnappyStreamingContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/streaming/SnappyStreamingContextSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/build.gradle b/dtests/build.gradle index c026d451ea..e3d4c3b101 100644 --- a/dtests/build.gradle +++ b/dtests/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/adAnalytics/SnappyAdAnalyticsTest.java b/dtests/src/test/java/io/snappydata/hydra/adAnalytics/SnappyAdAnalyticsTest.java index 3767b5a23d..1f0bfff09d 100644 --- a/dtests/src/test/java/io/snappydata/hydra/adAnalytics/SnappyAdAnalyticsTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/adAnalytics/SnappyAdAnalyticsTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cdcConnector/CDCPerfSparkJob.java b/dtests/src/test/java/io/snappydata/hydra/cdcConnector/CDCPerfSparkJob.java index 390d0a949a..19f2c423c9 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cdcConnector/CDCPerfSparkJob.java +++ b/dtests/src/test/java/io/snappydata/hydra/cdcConnector/CDCPerfSparkJob.java @@ -1,7 +1,7 @@ package io.snappydata.hydra.cdcConnector; /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyAlterTablesTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyAlterTablesTest.java index 3097b8b0fe..e27e293e12 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyAlterTablesTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyAlterTablesTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyBB.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyBB.java index e8c1db2560..cdfe9a9205 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyBB.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyBB.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyColocationAndEvictionTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyColocationAndEvictionTest.java index 1117cbbe96..c26aa26603 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyColocationAndEvictionTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyColocationAndEvictionTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyConcurrencyTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyConcurrencyTest.java index 47a7b2a693..2975bdcffd 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyConcurrencyTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyConcurrencyTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyLocatorHATest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyLocatorHATest.java index 1ee512ad05..980b203ea9 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyLocatorHATest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyLocatorHATest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyNetworkServerBB.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyNetworkServerBB.java index 2fbf5f8973..329fbbdaf2 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyNetworkServerBB.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyNetworkServerBB.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyPrms.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyPrms.java index 0b71ed5ace..da593b2fc1 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyPrms.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyPrms.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappySparkSQLDriverTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappySparkSQLDriverTest.java index 145edf722f..80e51f9b77 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappySparkSQLDriverTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappySparkSQLDriverTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyStartUpTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyStartUpTest.java index cb3f9ae564..020aab67f9 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyStartUpTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyStartUpTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyTest.java b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyTest.java index 3b10634b43..924c17a019 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/cluster/SnappyTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/concurrency/airline/concurrency_analyticalWorkLoadWithDMLOpsUsingStreaming_airline.sh b/dtests/src/test/java/io/snappydata/hydra/concurrency/airline/concurrency_analyticalWorkLoadWithDMLOpsUsingStreaming_airline.sh index 38eea939e9..ef22bcb2e1 100755 --- a/dtests/src/test/java/io/snappydata/hydra/concurrency/airline/concurrency_analyticalWorkLoadWithDMLOpsUsingStreaming_airline.sh +++ b/dtests/src/test/java/io/snappydata/hydra/concurrency/airline/concurrency_analyticalWorkLoadWithDMLOpsUsingStreaming_airline.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_analyticalWorkLoad_tpch.sh b/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_analyticalWorkLoad_tpch.sh index fd52c798a5..1635262b6b 100644 --- a/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_analyticalWorkLoad_tpch.sh +++ b/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_analyticalWorkLoad_tpch.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_mixedWorkLoad_tpch.sh b/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_mixedWorkLoad_tpch.sh index f8d44aea97..238de67d2e 100755 --- a/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_mixedWorkLoad_tpch.sh +++ b/dtests/src/test/java/io/snappydata/hydra/concurrency/concurrency_mixedWorkLoad_tpch.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/concurrency/stability.sh b/dtests/src/test/java/io/snappydata/hydra/concurrency/stability.sh index 3379fa6b56..5de8a15d7c 100644 --- a/dtests/src/test/java/io/snappydata/hydra/concurrency/stability.sh +++ b/dtests/src/test/java/io/snappydata/hydra/concurrency/stability.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/connectionPool/HikariConnectionPool.java b/dtests/src/test/java/io/snappydata/hydra/connectionPool/HikariConnectionPool.java index 94a9453ea4..e459e2c712 100644 --- a/dtests/src/test/java/io/snappydata/hydra/connectionPool/HikariConnectionPool.java +++ b/dtests/src/test/java/io/snappydata/hydra/connectionPool/HikariConnectionPool.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/connectionPool/SnappyConnectionPoolPrms.java b/dtests/src/test/java/io/snappydata/hydra/connectionPool/SnappyConnectionPoolPrms.java index af5eacd7d8..76af2df9c1 100644 --- a/dtests/src/test/java/io/snappydata/hydra/connectionPool/SnappyConnectionPoolPrms.java +++ b/dtests/src/test/java/io/snappydata/hydra/connectionPool/SnappyConnectionPoolPrms.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/connectionPool/TomcatConnectionPool.java b/dtests/src/test/java/io/snappydata/hydra/connectionPool/TomcatConnectionPool.java index 1fcd985909..658465c72c 100644 --- a/dtests/src/test/java/io/snappydata/hydra/connectionPool/TomcatConnectionPool.java +++ b/dtests/src/test/java/io/snappydata/hydra/connectionPool/TomcatConnectionPool.java @@ -1,6 +1,6 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/consistency/SnappyConsistencyTest.java b/dtests/src/test/java/io/snappydata/hydra/consistency/SnappyConsistencyTest.java index 639129e1e7..2bb42d7977 100644 --- a/dtests/src/test/java/io/snappydata/hydra/consistency/SnappyConsistencyTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/consistency/SnappyConsistencyTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingJob.java b/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingJob.java index 4836020b49..351e8a3674 100644 --- a/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingJob.java +++ b/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingJob.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingTest.java b/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingTest.java index 9a6fa15171..8534cc3a92 100644 --- a/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/installJar/DynamicJarLoadingTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/installJar/InstallJarTestUtils.java b/dtests/src/test/java/io/snappydata/hydra/installJar/InstallJarTestUtils.java index 992486ce06..8d4214c8ed 100644 --- a/dtests/src/test/java/io/snappydata/hydra/installJar/InstallJarTestUtils.java +++ b/dtests/src/test/java/io/snappydata/hydra/installJar/InstallJarTestUtils.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningJobs.sh b/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningJobs.sh index e8781deabc..41cfa42e30 100755 --- a/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningJobs.sh +++ b/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningJobs.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningTest.sh b/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningTest.sh index faad93e4f9..b5ee727c26 100755 --- a/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningTest.sh +++ b/dtests/src/test/java/io/snappydata/hydra/longRun/longRunningTest.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/misc/MiscTest.java b/dtests/src/test/java/io/snappydata/hydra/misc/MiscTest.java index 1d27cacafe..ac1d13e6ed 100644 --- a/dtests/src/test/java/io/snappydata/hydra/misc/MiscTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/misc/MiscTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/preparedStmt/JdbcTestPS.java b/dtests/src/test/java/io/snappydata/hydra/preparedStmt/JdbcTestPS.java index 119debacbf..2ac6d0e3c5 100644 --- a/dtests/src/test/java/io/snappydata/hydra/preparedStmt/JdbcTestPS.java +++ b/dtests/src/test/java/io/snappydata/hydra/preparedStmt/JdbcTestPS.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/preparedStmt/SnappyPreparedStmtTest.java b/dtests/src/test/java/io/snappydata/hydra/preparedStmt/SnappyPreparedStmtTest.java index 43cfaf9201..6caff8ea69 100644 --- a/dtests/src/test/java/io/snappydata/hydra/preparedStmt/SnappyPreparedStmtTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/preparedStmt/SnappyPreparedStmtTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/putInto/LoadDataFromJson.scala b/dtests/src/test/java/io/snappydata/hydra/putInto/LoadDataFromJson.scala index de1ab6b51b..cdb154de84 100644 --- a/dtests/src/test/java/io/snappydata/hydra/putInto/LoadDataFromJson.scala +++ b/dtests/src/test/java/io/snappydata/hydra/putInto/LoadDataFromJson.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/putInto/SnappyPutIntoTest.java b/dtests/src/test/java/io/snappydata/hydra/putInto/SnappyPutIntoTest.java index 4c52c12bc4..b94d1b9423 100644 --- a/dtests/src/test/java/io/snappydata/hydra/putInto/SnappyPutIntoTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/putInto/SnappyPutIntoTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityPrms.java b/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityPrms.java index 211c657c26..be0fada2de 100644 --- a/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityPrms.java +++ b/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityPrms.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityTest.java b/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityTest.java index eb08e00ccb..2b14ffcc0d 100644 --- a/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/security/SnappySecurityTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/smoke.sh b/dtests/src/test/java/io/snappydata/hydra/smoke.sh index 5847d56c52..55b0345673 100755 --- a/dtests/src/test/java/io/snappydata/hydra/smoke.sh +++ b/dtests/src/test/java/io/snappydata/hydra/smoke.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/smokePerf.sh b/dtests/src/test/java/io/snappydata/hydra/smokePerf.sh index 129c204e16..a03e0fd1b6 100755 --- a/dtests/src/test/java/io/snappydata/hydra/smokePerf.sh +++ b/dtests/src/test/java/io/snappydata/hydra/smokePerf.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationBB.java b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationBB.java index 04771006f9..455da13d5c 100644 --- a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationBB.java +++ b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationBB.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationDMLOpsBB.java b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationDMLOpsBB.java index 0e4212471f..fd5aa468e5 100644 --- a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationDMLOpsBB.java +++ b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationDMLOpsBB.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTest.java b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTest.java index 73e383386f..ed48144818 100644 --- a/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTest.java +++ b/dtests/src/test/java/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsBB.java b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsBB.java index ca2f11be92..01b8c1d646 100644 --- a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsBB.java +++ b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsBB.java @@ -2,7 +2,7 @@ import hydra.blackboard.Blackboard; /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUsingAPITest.java b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUsingAPITest.java index 7883eaa962..4959ab4ae0 100644 --- a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUsingAPITest.java +++ b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUsingAPITest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUtil.java b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUtil.java index 43f8ae6a7e..2e1cc24520 100644 --- a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUtil.java +++ b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappyDMLOpsUtil.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappySchemaPrms.java b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappySchemaPrms.java index cb31c50b22..73d803b792 100644 --- a/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappySchemaPrms.java +++ b/dtests/src/test/java/io/snappydata/hydra/testDMLOps/SnappySchemaPrms.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF1.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF1.java index a955ca060a..433ace4929 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF1.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF1.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF10.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF10.java index 0591aa9a07..2a3344fc85 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF10.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF10.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF11.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF11.java index f793f2e8e1..9f98f55260 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF11.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF11.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF12.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF12.java index 4b8b1b068c..4d80af95c0 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF12.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF12.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF13.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF13.java index cd6914209a..525839a340 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF13.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF13.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF14.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF14.java index 13e321c91d..c21a8a4f73 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF14.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF14.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF15.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF15.java index fcd4cc796d..6f90856e99 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF15.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF15.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF16.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF16.java index ff14e4e7d8..57e4abdc66 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF16.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF16.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF17.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF17.java index 66aa5fdc9f..8fd46e09b2 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF17.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF17.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF18.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF18.java index 542bad9a50..173d4e861f 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF18.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF18.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF19.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF19.java index d61a97b134..60d2cb86bd 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF19.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF19.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF2.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF2.java index 55ad4ff223..2d719ce7fb 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF2.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF2.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF20.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF20.java index 9b1686f823..64d4a5cf3e 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF20.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF20.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF21.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF21.java index 573a7bbc57..b65259fc9b 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF21.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF21.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF22.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF22.java index 740a0dcaff..2bf678b245 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF22.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF22.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF3.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF3.java index a06dffa6d8..3826bad7e9 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF3.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF3.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF4.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF4.java index d18fa76839..4e60ce3ed9 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF4.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF4.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF5.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF5.java index e2339e4ebe..d02fddefc1 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF5.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF5.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF6.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF6.java index 58b5e79117..037052db3f 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF6.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF6.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF7.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF7.java index e56851dfb8..c729da8b13 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF7.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF7.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF8.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF8.java index 14106047fc..d8451135f3 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF8.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF8.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF9.java b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF9.java index c87dd68f29..fb8de05678 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF9.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/JavaUDF9.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/java/io/snappydata/hydra/udfs/UDFInterface.java b/dtests/src/test/java/io/snappydata/hydra/udfs/UDFInterface.java index 291561e40a..c9f91d194b 100644 --- a/dtests/src/test/java/io/snappydata/hydra/udfs/UDFInterface.java +++ b/dtests/src/test/java/io/snappydata/hydra/udfs/UDFInterface.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/AirlineCleanedParquertData.scala b/dtests/src/test/scala/io/snappydata/hydra/AirlineCleanedParquertData.scala index 7bd14c7f61..82dcff8e1d 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/AirlineCleanedParquertData.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/AirlineCleanedParquertData.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/AirlineDataQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/AirlineDataQueriesJob.scala index 92de0ed5ed..8890a0ebda 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/AirlineDataQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/AirlineDataQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/AirlineDataSparkJobApp.scala b/dtests/src/test/scala/io/snappydata/hydra/AirlineDataSparkJobApp.scala index abe6f5c837..46439a7b53 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/AirlineDataSparkJobApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/AirlineDataSparkJobApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/CatalogConsistencyTest.scala b/dtests/src/test/scala/io/snappydata/hydra/CatalogConsistencyTest.scala index 2f9f07fe98..8b8238a42b 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/CatalogConsistencyTest.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/CatalogConsistencyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/FileStreamingJob.scala b/dtests/src/test/scala/io/snappydata/hydra/FileStreamingJob.scala index 259f368359..7c3f384d79 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/FileStreamingJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/FileStreamingJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/InstallJarTest.scala b/dtests/src/test/scala/io/snappydata/hydra/InstallJarTest.scala index 92b7babd1c..224ff68339 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/InstallJarTest.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/InstallJarTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/InstallJarTestSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/InstallJarTestSparkApp.scala index e00486edd2..4470c93598 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/InstallJarTestSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/InstallJarTestSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/LoadCSVDataJob.scala b/dtests/src/test/scala/io/snappydata/hydra/LoadCSVDataJob.scala index 5628e14a6d..2972f5a0ef 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/LoadCSVDataJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/LoadCSVDataJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/LoadODSData.scala b/dtests/src/test/scala/io/snappydata/hydra/LoadODSData.scala index 989a688def..81bac4dcda 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/LoadODSData.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/LoadODSData.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/SnappyCustomSparkListener.scala b/dtests/src/test/scala/io/snappydata/hydra/SnappyCustomSparkListener.scala index 6b8ee5c588..c0b603bad4 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/SnappyCustomSparkListener.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/SnappyCustomSparkListener.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraRunner.scala b/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraRunner.scala index 914f100a57..c598706ca9 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraRunner.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraTestRunner.scala b/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraTestRunner.scala index faf8f8d907..d9580f1a14 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraTestRunner.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/SnappyHydraTestRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/SnappyTestUtils.scala b/dtests/src/test/scala/io/snappydata/hydra/SnappyTestUtils.scala index cb4c949307..a9693de565 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/SnappyTestUtils.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/SnappyTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/TestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/TestUtil.scala index 951b525db4..b243441929 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/TestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/TestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/SnappySQLLogAggregatorJob.scala b/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/SnappySQLLogAggregatorJob.scala index 434562428d..5dea091674 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/SnappySQLLogAggregatorJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/SnappySQLLogAggregatorJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/StopSnappyStreamingJob.scala b/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/StopSnappyStreamingJob.scala index e61b80f978..4d853ce046 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/StopSnappyStreamingJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/adAnalytics/StopSnappyStreamingJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/alterTable/AlterTablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/alterTable/AlterTablesJob.scala index 139cc2ee07..4285f909ba 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/alterTable/AlterTablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/alterTable/AlterTablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ao/LoadData.scala b/dtests/src/test/scala/io/snappydata/hydra/ao/LoadData.scala index 02498ecf8c..5279449392 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ao/LoadData.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ao/LoadData.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -149,4 +149,4 @@ object LoadData extends SnappySQLJob { */ override def isValidJob(sc: SnappySession, config: Config): SnappyJobValidation = SnappyJobValid() -} \ No newline at end of file +} diff --git a/dtests/src/test/scala/io/snappydata/hydra/cdcConnector/CDCValidationApp.scala b/dtests/src/test/scala/io/snappydata/hydra/cdcConnector/CDCValidationApp.scala index 46fd9faa70..c59e384c11 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/cdcConnector/CDCValidationApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/cdcConnector/CDCValidationApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSnappyJob.scala index 8081d9ac24..540d521b9e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSparkJob.scala b/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSparkJob.scala index 39d6d383e3..e222cf06b9 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSparkJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/clusterUpgrade/ClusterUpgradeSparkJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/AllMixedTypes.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/AllMixedTypes.scala index df5baee487..17d0dd91b9 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/AllMixedTypes.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/AllMixedTypes.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayType.scala index 21b5d9469c..cf6ebf813d 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayTypeAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayTypeAPI.scala index 97c507bb8f..1009d9ed81 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayTypeAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArrayTypeAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStringInMapAsValue.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStringInMapAsValue.scala index 1961ad8777..cac94aa74c 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStringInMapAsValue.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStringInMapAsValue.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStructType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStructType.scala index aff70e603c..67c2dd58bb 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStructType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ArraysOfStructType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ComplexTypeUtils.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ComplexTypeUtils.scala index f019f7e247..aed453beed 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ComplexTypeUtils.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/ComplexTypeUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapType.scala index 7b69323e0f..a7e0ebb83c 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapTypeAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapTypeAPI.scala index 104ffee31c..fe5b6ff51f 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapTypeAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/MapTypeAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorAllMixedType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorAllMixedType.scala index 505fd7edff..f7d40ce089 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorAllMixedType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorAllMixedType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayType.scala index bee7de7cf6..9b72c0e283 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayTypeAPI1.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayTypeAPI1.scala index 5e61ac5b8d..a8d5d6e811 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayTypeAPI1.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArrayTypeAPI1.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStringInMapAsValue.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStringInMapAsValue.scala index 178b7a8b04..d46120b53e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStringInMapAsValue.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStringInMapAsValue.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStructType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStructType.scala index a0aab06113..0917807591 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStructType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorArraysOfStructType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapType.scala index c549614a83..00bc1b9a21 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapTypeAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapTypeAPI.scala index 70b132ae28..7c6805bbf7 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapTypeAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorMapTypeAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructType.scala index dac6c84f9b..5a8d81fced 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructTypeAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructTypeAPI.scala index ee08f8be60..8945c56963 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructTypeAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/SmartConnectorStructTypeAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructType.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructType.scala index f236eb35b2..dfd00b00c2 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructType.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructType.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructTypeAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructTypeAPI.scala index 23f00f8c6c..19bfc3477a 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructTypeAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/complexdatatypes/StructTypeAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ConcTestUtils.scala b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ConcTestUtils.scala index 26789c6a05..a894d07ffb 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ConcTestUtils.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ConcTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ExportTable.scala b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ExportTable.scala index 5798173de7..7675e72e7e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ExportTable.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ExportTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/concurrency/StreamingActivity.scala b/dtests/src/test/scala/io/snappydata/hydra/concurrency/StreamingActivity.scala index e3956b6749..b733ebf825 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/concurrency/StreamingActivity.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/concurrency/StreamingActivity.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ValidateConcurrentQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ValidateConcurrentQueriesJob.scala index e5c244ff9d..c2bde9ce3c 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/concurrency/ValidateConcurrentQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/concurrency/ValidateConcurrentQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/consistency/ConsistencyTest.scala b/dtests/src/test/scala/io/snappydata/hydra/consistency/ConsistencyTest.scala index c762183526..4870bea9dd 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/consistency/ConsistencyTest.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/consistency/ConsistencyTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsApp.scala b/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsApp.scala index 1af8bbae0b..efe179dd77 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsJob.scala b/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsJob.scala index 9207ba6c66..d21a9288cc 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/consistency/ValidateConsistencyWithDMLOpsJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/AddDataUsingPutIntoCTTablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/AddDataUsingPutIntoCTTablesJob.scala index 5d9ed1b9e0..6e18faf7fa 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/AddDataUsingPutIntoCTTablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/AddDataUsingPutIntoCTTablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/CTQueries.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/CTQueries.scala index 3d43505855..a5b2a2fc5d 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/CTQueries.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/CTQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/CTTestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/CTTestUtil.scala index 0e6d7bf1e8..6753768d3f 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/CTTestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/CTTestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTColumnTablesWithKeyColumnsJob.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTColumnTablesWithKeyColumnsJob.scala index 7e3b95b523..31165a6258 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTColumnTablesWithKeyColumnsJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTColumnTablesWithKeyColumnsJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesApp.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesApp.scala index e7350ed0b7..1be6e28eab 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesJob.scala index 84f82e1f96..96c5682b9e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/CreateAndLoadCTTablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesApp.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesApp.scala index 113321478e..5f9f1006c3 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesJob.scala index 762d68770e..4681af33bc 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesWithPutIntoJob.scala b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesWithPutIntoJob.scala index 9446808092..c13762b3d8 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesWithPutIntoJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/ct/ValidateCTQueriesWithPutIntoJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessDeployedJarSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessDeployedJarSnappyJob.scala index 2e9e52010e..8053ceb32b 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessDeployedJarSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessDeployedJarSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSnappyJob.scala index 3ac454cc4e..b23b392029 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSparkApp.scala index cbdf515ea8..3d078e4038 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/AccessUdfSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSnappyJob.scala index 19955a7f5b..46fb4ea867 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSparkApp.scala index b7095b1a8b..c74707ed1e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/deployPkgUDF/CreateDropUDFSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/distIndex/DistIndexJob.scala b/dtests/src/test/scala/io/snappydata/hydra/distIndex/DistIndexJob.scala index 7f5b6b290a..9961a698f8 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/distIndex/DistIndexJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/distIndex/DistIndexJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/externaltables/ExternalTablesAPINorthWind.scala b/dtests/src/test/scala/io/snappydata/hydra/externaltables/ExternalTablesAPINorthWind.scala index 8d9328ffff..419fb17672 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/externaltables/ExternalTablesAPINorthWind.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/externaltables/ExternalTablesAPINorthWind.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/externaltables/SmartConnectorExternalTableAPINW.scala b/dtests/src/test/scala/io/snappydata/hydra/externaltables/SmartConnectorExternalTableAPINW.scala index cfae971074..0192e0e88d 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/externaltables/SmartConnectorExternalTableAPINW.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/externaltables/SmartConnectorExternalTableAPINW.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServer.scala b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServer.scala index 569ed2bfc5..f3a10f20e9 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServer.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServerConcurrentOps.scala b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServerConcurrentOps.scala index 3770538453..fe81581e4d 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServerConcurrentOps.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/HiveThriftServerConcurrentOps.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ThriftServerInsertFromSnappy.scala b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ThriftServerInsertFromSnappy.scala index dc221e0124..49c943736e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ThriftServerInsertFromSnappy.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ThriftServerInsertFromSnappy.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ValidateHiveThriftServer.scala b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ValidateHiveThriftServer.scala index da4d79e1de..b845be04de 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ValidateHiveThriftServer.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/hiveThriftServer/ValidateHiveThriftServer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/misc/snap_3028Job.scala b/dtests/src/test/scala/io/snappydata/hydra/misc/snap_3028Job.scala index 28d6ac4b82..d738136876 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/misc/snap_3028Job.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/misc/snap_3028Job.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesJob.scala index 0e105d6803..37cfb4f6ef 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesSparkApp.scala index 90e90e5899..d3956f2769 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/CreateAndLoadNWTablesSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesApp.scala index 3b8ce12c1e..4efb259482 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesJob.scala index c34c45b8cd..0a1e9350ab 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/EvictionTestNWQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWPLQueries.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWPLQueries.scala index f79f6a01c2..611219544f 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWPLQueries.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWPLQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWQueries.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWQueries.scala index 93024c3e90..13d2efa1e7 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWQueries.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWSparkTablesAndQueriesApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWSparkTablesAndQueriesApp.scala index 4b45530d47..1ec75cc98c 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWSparkTablesAndQueriesApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWSparkTablesAndQueriesApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestJob.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestJob.scala index ae39964764..bd5abb7cf4 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestSparkApp.scala index 0f25af39e1..09403d2cdb 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestUtil.scala index 60e7497844..f7b1d44954 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/NWTestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesApp.scala index 03ea8bead0..ba6204e85a 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesJob.scala index 81a7a482f2..fbf51f38b3 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsApp.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsApp.scala index a7281444bc..c9e823f711 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsJob.scala b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsJob.scala index cf7f0abb28..d6e7676290 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/northwind/ValidateNWQueriesWithChangingConstantsJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/preparedStmt/PopulateDataJob.scala b/dtests/src/test/scala/io/snappydata/hydra/preparedStmt/PopulateDataJob.scala index 32382ba601..6556499d05 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/preparedStmt/PopulateDataJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/preparedStmt/PopulateDataJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoTest.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoTest.scala index d5ec3a2e96..578550cee0 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoTest.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoTest.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoWith30Columns.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoWith30Columns.scala index 1f92b250ba..ec9dd61104 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoWith30Columns.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcPutIntoWith30Columns.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcurrentPutInto.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcurrentPutInto.scala index c9cedb35b3..4c755370b4 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcurrentPutInto.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/ConcurrentPutInto.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/GenrateJsonDataFilesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/GenrateJsonDataFilesJob.scala index 5bbf13eaac..993a721290 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/GenrateJsonDataFilesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/GenrateJsonDataFilesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -98,4 +98,4 @@ object GenerateJsonDataFilesJob extends SnappySQLJob { SnappyJobValid() } -} \ No newline at end of file +} diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/InsertFromJsonFile.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/InsertFromJsonFile.scala index 04a7ff6f0e..6bcde36c72 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/InsertFromJsonFile.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/InsertFromJsonFile.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoProducer.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoProducer.scala index 8519e48eed..9ce2e76220 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoProducer.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoProducer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoReceiver.scala b/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoReceiver.scala index 60517bce3a..e7bd9dd0a6 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoReceiver.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/putInto/PutIntoReceiver.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSnappyJob.scala index b4c76a0aff..3f33dace12 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSparkApp.scala index cd529cfd61..fe68fc4175 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/security/CreateAndLoadTablesSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/security/SecurityTestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/security/SecurityTestUtil.scala index 5b2a498047..b72744ea3e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/security/SecurityTestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/security/SecurityTestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnappyDummyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnappyDummyJob.scala index fd73468ecf..3667cc5899 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnappyDummyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnappyDummyJob.scala @@ -1,5 +1,5 @@ /* -* Copyright (c) 2018 SnappyData, Inc. All rights reserved. +* Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationQueries.scala b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationQueries.scala index 846ca6a6cc..bf4b3e3259 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationQueries.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTestUtils.scala b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTestUtils.scala index 2a712e68de..05c6fdab64 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTestUtils.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/SnapshotIsolationTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/ValidateSnapshotSelectWithInsertJob.scala b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/ValidateSnapshotSelectWithInsertJob.scala index 9aa773e289..5249fcdfb2 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/ValidateSnapshotSelectWithInsertJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/snapshotIsolation/ValidateSnapshotSelectWithInsertJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAnLoadSPVATablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAnLoadSPVATablesJob.scala index 9d230c8e43..a40ef55f00 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAnLoadSPVATablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAnLoadSPVATablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAndLoadSPVATablesSparkApp.scala b/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAndLoadSPVATablesSparkApp.scala index 949d34f07f..1e2a77ddf7 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAndLoadSPVATablesSparkApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/spva/CreateAndLoadSPVATablesSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/spva/SPVAQueries.scala b/dtests/src/test/scala/io/snappydata/hydra/spva/SPVAQueries.scala index bed0a0b168..f1a1356c54 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/spva/SPVAQueries.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/spva/SPVAQueries.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/spva/SPVATestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/spva/SPVATestUtil.scala index c572020761..9d9f97d59b 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/spva/SPVATestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/spva/SPVATestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/spva/ValidateSPVAQueriesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/spva/ValidateSPVAQueriesJob.scala index 37202a16b9..38e1b09b91 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/spva/ValidateSPVAQueriesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/spva/ValidateSPVAQueriesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForAvgJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForAvgJob.scala index 22783e8e93..f82f9967ec 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForAvgJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForAvgJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForCountJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForCountJob.scala index e849493277..0ca3c2cafd 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForCountJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForCountJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForJoinJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForJoinJob.scala index 273ef4095a..0fd003f64f 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForJoinJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForJoinJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForSumJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForSumJob.scala index bd10779d0b..474b6c0c60 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForSumJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableForSumJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableInSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableInSnappyJob.scala index 4716c4b680..0aed772eb1 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableInSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/CreateTableInSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkGenericColNameJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkGenericColNameJob.scala index 829bcc56f9..706064b0d4 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkGenericColNameJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkGenericColNameJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJob.scala index 0408fd0c97..d1c3e7c683 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJobWithAggregate.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJobWithAggregate.scala index fbe4734d46..6c91255951 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJobWithAggregate.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/SnappyStreamingSinkJobWithAggregate.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StopSnappyStreamingJob.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StopSnappyStreamingJob.scala index effc25c87e..664aeed271 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StopSnappyStreamingJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StopSnappyStreamingJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StringMessageProducer.scala b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StringMessageProducer.scala index af40e385f5..4adab1c3b3 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StringMessageProducer.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/streaming_sink/StringMessageProducer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ConvertCsvToParquetAndLoadTablesJob.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ConvertCsvToParquetAndLoadTablesJob.scala index 2c27df2a91..37174264f1 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ConvertCsvToParquetAndLoadTablesJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ConvertCsvToParquetAndLoadTablesJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/RecreateTableInSnappyJob.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/RecreateTableInSnappyJob.scala index 7c776e1cda..25805f4c9a 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/RecreateTableInSnappyJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/RecreateTableInSnappyJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/SnappyDMLTestUtil.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/SnappyDMLTestUtil.scala index df1b500590..1678798b7f 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/SnappyDMLTestUtil.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/SnappyDMLTestUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpApp.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpApp.scala index d9d31e7f9f..f2c2d14566 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpApp.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpJob.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpJob.scala index 8a4988ae21..9ef5cad5d5 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpJob.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDMLOpJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpAppUsingAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpAppUsingAPI.scala index ee45345265..36b091c593 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpAppUsingAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpAppUsingAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpUsingAPI_Job.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpUsingAPI_Job.scala index 23a455a85a..3f39e203b9 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpUsingAPI_Job.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateDeleteOpUsingAPI_Job.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpAppUsingAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpAppUsingAPI.scala index ece7bf3109..18ab4ad737 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpAppUsingAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpAppUsingAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpUsingAPI_Job.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpUsingAPI_Job.scala index d70c2c351c..7a94f5a91e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpUsingAPI_Job.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateInsertOpUsingAPI_Job.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpAppUsingAPI.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpAppUsingAPI.scala index bd479e001b..0afae7ccc8 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpAppUsingAPI.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpAppUsingAPI.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpUsingAPI_Job.scala b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpUsingAPI_Job.scala index 071eeec236..943cd15caf 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpUsingAPI_Job.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/testDMLOps/ValidateUpdateOpUsingAPI_Job.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/BadCase_ScalaUDF13.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/BadCase_ScalaUDF13.scala index de16132623..352c5f82db 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/BadCase_ScalaUDF13.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/BadCase_ScalaUDF13.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF1.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF1.scala index 197c3daa6e..8ab923b611 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF1.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF1.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF10.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF10.scala index b86f522eee..0209bf2f7b 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF10.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF10.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF11.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF11.scala index 603bbfceca..2650270caa 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF11.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF11.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF12.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF12.scala index 272db1a6c5..2f5cd28cb6 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF12.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF12.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF13.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF13.scala index 3317431455..fc71b34f50 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF13.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF13.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF14.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF14.scala index 55fde93f4e..b095b84c41 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF14.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF14.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF15.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF15.scala index f13be6d0b5..95c283a185 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF15.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF15.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF16.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF16.scala index 7577414c18..8b4ba33004 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF16.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF16.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF17.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF17.scala index fe37c9e451..c8367d3b2e 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF17.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF17.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF18.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF18.scala index 1ca1df2a89..8699b89173 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF18.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF18.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF19.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF19.scala index d5daf15201..8de70043f1 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF19.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF19.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF2.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF2.scala index 512b866027..6e515eaa26 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF2.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF2.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF20.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF20.scala index 599bba8be5..88df688949 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF20.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF20.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF21.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF21.scala index 8c500d79af..c23723ecdf 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF21.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF21.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF22.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF22.scala index 464a8b64b4..6157dbdb60 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF22.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF22.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF3.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF3.scala index 263117a0d1..7a579f310a 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF3.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF3.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF4.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF4.scala index e6f326406e..776a980789 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF4.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF4.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF5.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF5.scala index 321aec95c7..e831ca72a3 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF5.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF5.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF6.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF6.scala index 512da97e8e..d31a9927e3 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF6.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF6.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF7.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF7.scala index 1b440586c3..dad70398e4 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF7.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF7.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF8.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF8.scala index 905b36247e..8148eb734a 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF8.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF8.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF9.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF9.scala index 1e88ceafbb..a040c55067 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF9.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/ScalaUDF9.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/io/snappydata/hydra/udfs/Validate_SNAP2658.scala b/dtests/src/test/scala/io/snappydata/hydra/udfs/Validate_SNAP2658.scala index 6f87579f84..7d9bbc1b23 100644 --- a/dtests/src/test/scala/io/snappydata/hydra/udfs/Validate_SNAP2658.scala +++ b/dtests/src/test/scala/io/snappydata/hydra/udfs/Validate_SNAP2658.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/org/apache/spark/SnappyTestUtils.scala b/dtests/src/test/scala/org/apache/spark/SnappyTestUtils.scala index e53a94c81c..cff3418543 100644 --- a/dtests/src/test/scala/org/apache/spark/SnappyTestUtils.scala +++ b/dtests/src/test/scala/org/apache/spark/SnappyTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dtests/src/test/scala/org/apache/spark/sql/DistIndexTestUtils.scala b/dtests/src/test/scala/org/apache/spark/sql/DistIndexTestUtils.scala index 2413e1d2fc..52c2d0b1f0 100644 --- a/dtests/src/test/scala/org/apache/spark/sql/DistIndexTestUtils.scala +++ b/dtests/src/test/scala/org/apache/spark/sql/DistIndexTestUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/build.gradle b/dunit/build.gradle index 02dbdae414..1ffb8dcdbd 100644 --- a/dunit/build.gradle +++ b/dunit/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/ExpectedStrings.java b/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/ExpectedStrings.java index c58d5020c9..01da79c381 100644 --- a/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/ExpectedStrings.java +++ b/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/ExpectedStrings.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/LogConsumer.java b/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/LogConsumer.java index bf8caebfe3..d8427e1cdd 100644 --- a/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/LogConsumer.java +++ b/dunit/src/main/java/io/snappydata/test/batterytest/greplogs/LogConsumer.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/AsyncInvocation.java b/dunit/src/main/java/io/snappydata/test/dunit/AsyncInvocation.java index 1a6cabb67b..db081440d0 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/AsyncInvocation.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/AsyncInvocation.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/AvailablePortHelper.java b/dunit/src/main/java/io/snappydata/test/dunit/AvailablePortHelper.java index 39b8a7d003..24b4eb225c 100755 --- a/dunit/src/main/java/io/snappydata/test/dunit/AvailablePortHelper.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/AvailablePortHelper.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/BounceResult.java b/dunit/src/main/java/io/snappydata/test/dunit/BounceResult.java index 3a3d1ee33a..033bf400b5 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/BounceResult.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/BounceResult.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/DUnitEnv.java b/dunit/src/main/java/io/snappydata/test/dunit/DUnitEnv.java index bed3166ff6..38f070f18a 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/DUnitEnv.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/DUnitEnv.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java b/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java index 2e367ac631..9c30e6d4b5 100755 --- a/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/DistributedTestBase.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/Host.java b/dunit/src/main/java/io/snappydata/test/dunit/Host.java index 1b1b4a3789..7075e9ee98 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/Host.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/Host.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/RMIException.java b/dunit/src/main/java/io/snappydata/test/dunit/RMIException.java index ba259fcb8e..4528fe3e19 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/RMIException.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/RMIException.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/RemoteDUnitVMIF.java b/dunit/src/main/java/io/snappydata/test/dunit/RemoteDUnitVMIF.java index 83f34f4a38..ab0359ecbb 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/RemoteDUnitVMIF.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/RemoteDUnitVMIF.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/RepeatableRunnable.java b/dunit/src/main/java/io/snappydata/test/dunit/RepeatableRunnable.java index c335e18cf7..c0e1cc40f3 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/RepeatableRunnable.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/RepeatableRunnable.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/SerializableCallable.java b/dunit/src/main/java/io/snappydata/test/dunit/SerializableCallable.java index 589f722b48..2e72634d9d 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/SerializableCallable.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/SerializableCallable.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/SerializableRunnable.java b/dunit/src/main/java/io/snappydata/test/dunit/SerializableRunnable.java index 23f1e924ce..9e529b404d 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/SerializableRunnable.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/SerializableRunnable.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/VM.java b/dunit/src/main/java/io/snappydata/test/dunit/VM.java index 7827987705..79da423afb 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/VM.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/VM.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/logging/log4j/ConfigLocator.java b/dunit/src/main/java/io/snappydata/test/dunit/logging/log4j/ConfigLocator.java index 1b893bf56b..f778c9f50c 100755 --- a/dunit/src/main/java/io/snappydata/test/dunit/logging/log4j/ConfigLocator.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/logging/log4j/ConfigLocator.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ChildVM.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ChildVM.java index 8d8f3c309b..a4f62257e6 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ChildVM.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ChildVM.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java index c811f45954..a8d706842b 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java index f2e72a95be..df1e74548f 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/RemoteDUnitVM.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/RemoteDUnitVM.java index ae2b615cf0..775f23c93c 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/RemoteDUnitVM.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/RemoteDUnitVM.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/StandAloneDUnitEnv.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/StandAloneDUnitEnv.java index 5ea0b4a19b..8d92948fd4 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/StandAloneDUnitEnv.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/StandAloneDUnitEnv.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/hydra/LogVersionHelper.java b/dunit/src/main/java/io/snappydata/test/hydra/LogVersionHelper.java index fedc65d5c5..ff265e552a 100644 --- a/dunit/src/main/java/io/snappydata/test/hydra/LogVersionHelper.java +++ b/dunit/src/main/java/io/snappydata/test/hydra/LogVersionHelper.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/hydra/MethExecutor.java b/dunit/src/main/java/io/snappydata/test/hydra/MethExecutor.java index 831ee9a320..5771774248 100644 --- a/dunit/src/main/java/io/snappydata/test/hydra/MethExecutor.java +++ b/dunit/src/main/java/io/snappydata/test/hydra/MethExecutor.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/hydra/MethExecutorResult.java b/dunit/src/main/java/io/snappydata/test/hydra/MethExecutorResult.java index d52365dcd0..0f18369b40 100644 --- a/dunit/src/main/java/io/snappydata/test/hydra/MethExecutorResult.java +++ b/dunit/src/main/java/io/snappydata/test/hydra/MethExecutorResult.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/hydra/SchedulingOrder.java b/dunit/src/main/java/io/snappydata/test/hydra/SchedulingOrder.java index 43864bd233..16c56b2261 100644 --- a/dunit/src/main/java/io/snappydata/test/hydra/SchedulingOrder.java +++ b/dunit/src/main/java/io/snappydata/test/hydra/SchedulingOrder.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/memscale/OffHeapHelper.java b/dunit/src/main/java/io/snappydata/test/memscale/OffHeapHelper.java index 2ddfd2e79b..95071a82e4 100644 --- a/dunit/src/main/java/io/snappydata/test/memscale/OffHeapHelper.java +++ b/dunit/src/main/java/io/snappydata/test/memscale/OffHeapHelper.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/dunit/src/main/java/io/snappydata/test/util/TestException.java b/dunit/src/main/java/io/snappydata/test/util/TestException.java index ddaa0d76cd..8e01f8d9ad 100644 --- a/dunit/src/main/java/io/snappydata/test/util/TestException.java +++ b/dunit/src/main/java/io/snappydata/test/util/TestException.java @@ -18,7 +18,7 @@ /* * Changes for SnappyData data platform. * - * Portions Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/build.gradle b/encoders/build.gradle index 9008278931..f55506666d 100644 --- a/encoders/build.gradle +++ b/encoders/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala b/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala index b72cad72ea..ebc9653c8e 100644 --- a/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala +++ b/encoders/src/main/scala/io/snappydata/collection/BufferSizeLimitExceededException.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala b/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala index 2374dc9cee..22d7648b42 100644 --- a/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala +++ b/encoders/src/main/scala/io/snappydata/collection/ByteBufferHashMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala b/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala index b89389646d..62b6af7f66 100644 --- a/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala +++ b/encoders/src/main/scala/io/snappydata/collection/DictionaryMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/collection/ObjectHashSet.scala b/encoders/src/main/scala/io/snappydata/collection/ObjectHashSet.scala index c4e2c57377..90ae03c665 100644 --- a/encoders/src/main/scala/io/snappydata/collection/ObjectHashSet.scala +++ b/encoders/src/main/scala/io/snappydata/collection/ObjectHashSet.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala b/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala index daee315ec9..2b72d93b55 100644 --- a/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala +++ b/encoders/src/main/scala/io/snappydata/collection/SHAMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/util/DebugUtils.scala b/encoders/src/main/scala/io/snappydata/util/DebugUtils.scala index d1134790b8..f017509ca8 100644 --- a/encoders/src/main/scala/io/snappydata/util/DebugUtils.scala +++ b/encoders/src/main/scala/io/snappydata/util/DebugUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/util/NumberUtils.scala b/encoders/src/main/scala/io/snappydata/util/NumberUtils.scala index 043a419351..7b539fa1a7 100644 --- a/encoders/src/main/scala/io/snappydata/util/NumberUtils.scala +++ b/encoders/src/main/scala/io/snappydata/util/NumberUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/io/snappydata/util/StringUtils.scala b/encoders/src/main/scala/io/snappydata/util/StringUtils.scala index ca3e1f32a9..3b2ac63511 100644 --- a/encoders/src/main/scala/io/snappydata/util/StringUtils.scala +++ b/encoders/src/main/scala/io/snappydata/util/StringUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala b/encoders/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala index 6b0ff7f457..6ce1dd77be 100644 --- a/encoders/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala +++ b/encoders/src/main/scala/org/apache/spark/memory/DefaultMemoryManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala b/encoders/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala index 6dc557eafe..dbd8e9350d 100644 --- a/encoders/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala +++ b/encoders/src/main/scala/org/apache/spark/memory/StoreUnifiedManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/SnappyStoreClientDialect.scala b/encoders/src/main/scala/org/apache/spark/sql/SnappyStoreClientDialect.scala index c66be6011c..4728b3c86c 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/SnappyStoreClientDialect.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/SnappyStoreClientDialect.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedArray.scala b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedArray.scala index 3057d40dbf..7507a9763e 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedArray.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedArray.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedMap.scala b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedMap.scala index def6e56ecb..00b3689517 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedMap.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedMap.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedRow.scala b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedRow.scala index 9738206c90..0bd9ac826b 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedRow.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/catalyst/util/SerializedRow.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/collection/SharedUtils.scala b/encoders/src/main/scala/org/apache/spark/sql/collection/SharedUtils.scala index 6db531b96f..ae7a75d222 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/collection/SharedUtils.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/collection/SharedUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/ConnectionPool.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/ConnectionPool.scala index 8f43d8a68b..cf74baccae 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/ConnectionPool.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/ConnectionPool.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala index f1aa4d5594..f3bb294999 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BitSet.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BitSet.scala index 6a2e183461..0b89378763 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BitSet.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BitSet.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanBitSetEncoding.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanBitSetEncoding.scala index c4a849196a..f317152abc 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanBitSetEncoding.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanBitSetEncoding.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanRunLengthEncoding.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanRunLengthEncoding.scala index 50e802e85d..bc64fa1c1a 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanRunLengthEncoding.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/BooleanRunLengthEncoding.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteDecoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteDecoder.scala index e71a5277d3..a9014a528c 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteDecoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteEncoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteEncoder.scala index a92db39fb9..1c7c1e8201 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteEncoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeleteEncoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaDecoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaDecoder.scala index b0a092273a..caa45dd520 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaDecoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala index a24dfec57b..134eb81fd8 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala index 63b649edd9..3ca91a235d 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnEncoding.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala index 70ce000502..be2c2b2ef0 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/RunLengthEncoding.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/RunLengthEncoding.scala index c9f1ca8e03..5938fd547c 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/RunLengthEncoding.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/RunLengthEncoding.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/Uncompressed.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/Uncompressed.scala index 823950b8ba..d9b4606019 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/Uncompressed.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/Uncompressed.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/UpdatedColumnDecoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/UpdatedColumnDecoder.scala index f7c09a6674..af3a8009c8 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/UpdatedColumnDecoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/UpdatedColumnDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala index c648f0bac2..5d58556cbf 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index 28c2990235..540e29244e 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala index 980809cda3..08ec8a434b 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/row/ResultSetDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala b/encoders/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala index 9ba3cd6cba..0fd90c94d4 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/execution/row/UnsafeRowDecoder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/sources/ConnectionProperties.scala b/encoders/src/main/scala/org/apache/spark/sql/sources/ConnectionProperties.scala index 445a9c93a3..c810ae285d 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/sources/ConnectionProperties.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/sources/ConnectionProperties.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala b/encoders/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala index f8d40b63b4..87efd6df93 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/main/scala/org/apache/spark/sql/store/CompressionUtils.scala b/encoders/src/main/scala/org/apache/spark/sql/store/CompressionUtils.scala index 5a40f3e742..09f49da831 100644 --- a/encoders/src/main/scala/org/apache/spark/sql/store/CompressionUtils.scala +++ b/encoders/src/main/scala/org/apache/spark/sql/store/CompressionUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/encoders/src/test/resources/log4j.properties b/encoders/src/test/resources/log4j.properties index 0df4d6bfa0..3bc8aaa561 100644 --- a/encoders/src/test/resources/log4j.properties +++ b/encoders/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/examples/build.gradle b/examples/build.gradle index 8f5ccd8038..bb1829f774 100644 --- a/examples/build.gradle +++ b/examples/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You @@ -104,4 +104,4 @@ scalaTest { } check.dependsOn scalaTest -} \ No newline at end of file +} diff --git a/examples/quickstart/scripts/Quickstart.scala b/examples/quickstart/scripts/Quickstart.scala index 2dfbab1692..9f941a1937 100644 --- a/examples/quickstart/scripts/Quickstart.scala +++ b/examples/quickstart/scripts/Quickstart.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/python/KMeansWeather.py b/examples/src/main/python/KMeansWeather.py index a3167df562..efbaad942f 100644 --- a/examples/src/main/python/KMeansWeather.py +++ b/examples/src/main/python/KMeansWeather.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/AirlineDataJob.scala b/examples/src/main/scala/io/snappydata/examples/AirlineDataJob.scala index 0df3abb344..9f80ec04fe 100644 --- a/examples/src/main/scala/io/snappydata/examples/AirlineDataJob.scala +++ b/examples/src/main/scala/io/snappydata/examples/AirlineDataJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/AirlineDataSparkApp.scala b/examples/src/main/scala/io/snappydata/examples/AirlineDataSparkApp.scala index db01ee43b7..e421f5a991 100644 --- a/examples/src/main/scala/io/snappydata/examples/AirlineDataSparkApp.scala +++ b/examples/src/main/scala/io/snappydata/examples/AirlineDataSparkApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/CreateAndLoadAirlineDataJob.scala b/examples/src/main/scala/io/snappydata/examples/CreateAndLoadAirlineDataJob.scala index 5b67fb212b..754c07ca43 100644 --- a/examples/src/main/scala/io/snappydata/examples/CreateAndLoadAirlineDataJob.scala +++ b/examples/src/main/scala/io/snappydata/examples/CreateAndLoadAirlineDataJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/DataUpdateJob.scala b/examples/src/main/scala/io/snappydata/examples/DataUpdateJob.scala index 1eef09bd4d..aa8173a00b 100644 --- a/examples/src/main/scala/io/snappydata/examples/DataUpdateJob.scala +++ b/examples/src/main/scala/io/snappydata/examples/DataUpdateJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/SnappyDataTestJob.scala b/examples/src/main/scala/io/snappydata/examples/SnappyDataTestJob.scala index e179caf08f..937df2ec07 100644 --- a/examples/src/main/scala/io/snappydata/examples/SnappyDataTestJob.scala +++ b/examples/src/main/scala/io/snappydata/examples/SnappyDataTestJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/StreamingUtils.scala b/examples/src/main/scala/io/snappydata/examples/StreamingUtils.scala index 37d0dfd659..fb7660ecee 100644 --- a/examples/src/main/scala/io/snappydata/examples/StreamingUtils.scala +++ b/examples/src/main/scala/io/snappydata/examples/StreamingUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/io/snappydata/examples/TwitterPopularTagsJob.scala b/examples/src/main/scala/io/snappydata/examples/TwitterPopularTagsJob.scala index 5c52bed18e..3792e1d899 100644 --- a/examples/src/main/scala/io/snappydata/examples/TwitterPopularTagsJob.scala +++ b/examples/src/main/scala/io/snappydata/examples/TwitterPopularTagsJob.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/CollocatedJoinExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/CollocatedJoinExample.scala index 4b9b468049..43b91b140a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/CollocatedJoinExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/CollocatedJoinExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateColumnTable.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateColumnTable.scala index c22829ab7f..b7535f7b63 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateColumnTable.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateColumnTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreatePartitionedRowTable.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreatePartitionedRowTable.scala index 675bc72d69..140f683cda 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreatePartitionedRowTable.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreatePartitionedRowTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateReplicatedRowTable.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateReplicatedRowTable.scala index 02e5a87dce..e7257a0fce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateReplicatedRowTable.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/CreateReplicatedRowTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCExample.scala index ef1da52a77..536cf494c8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCWithComplexTypes.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCWithComplexTypes.scala index 02ae46c810..50df2f4315 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCWithComplexTypes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/JDBCWithComplexTypes.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/SmartConnectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/SmartConnectorExample.scala index 45df2c783c..499e5019c7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/SmartConnectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/SmartConnectorExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/StreamingExample.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/StreamingExample.scala index f146f0032c..3889b0bbdf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/StreamingExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/StreamingExample.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithJson.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithJson.scala index 4a7cca0898..bdc2fa0e55 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithJson.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithJson.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithObjects.scala b/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithObjects.scala index fb5c263747..2ffdd2f09c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithObjects.scala +++ b/examples/src/main/scala/org/apache/spark/examples/snappydata/WorkingWithObjects.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/test/scala/io/snappydata/SnappyTestRunner.scala b/examples/src/test/scala/io/snappydata/SnappyTestRunner.scala index 1c5367d906..2d34d34012 100644 --- a/examples/src/test/scala/io/snappydata/SnappyTestRunner.scala +++ b/examples/src/test/scala/io/snappydata/SnappyTestRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/examples/src/test/scala/io/snappydata/quickstart/ExampleTestSuite.scala b/examples/src/test/scala/io/snappydata/quickstart/ExampleTestSuite.scala index 2885dad746..583d05e158 100644 --- a/examples/src/test/scala/io/snappydata/quickstart/ExampleTestSuite.scala +++ b/examples/src/test/scala/io/snappydata/quickstart/ExampleTestSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/build.gradle b/jdbc/build.gradle index 4bd6758ff5..2031fb6e69 100644 --- a/jdbc/build.gradle +++ b/jdbc/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/io/snappydata/Constant.scala b/jdbc/src/main/scala/io/snappydata/Constant.scala index d94c6be719..3301075265 100644 --- a/jdbc/src/main/scala/io/snappydata/Constant.scala +++ b/jdbc/src/main/scala/io/snappydata/Constant.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/io/snappydata/sql/implicits.scala b/jdbc/src/main/scala/io/snappydata/sql/implicits.scala index fd0ae7d53a..8d50ed9cbd 100644 --- a/jdbc/src/main/scala/io/snappydata/sql/implicits.scala +++ b/jdbc/src/main/scala/io/snappydata/sql/implicits.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/org/apache/spark/sql/JdbcExecute.scala b/jdbc/src/main/scala/org/apache/spark/sql/JdbcExecute.scala index 99100aff96..52241ba7a8 100644 --- a/jdbc/src/main/scala/org/apache/spark/sql/JdbcExecute.scala +++ b/jdbc/src/main/scala/org/apache/spark/sql/JdbcExecute.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/org/apache/spark/sql/JdbcWriter.scala b/jdbc/src/main/scala/org/apache/spark/sql/JdbcWriter.scala index 72344d886c..f3e763d51b 100644 --- a/jdbc/src/main/scala/org/apache/spark/sql/JdbcWriter.scala +++ b/jdbc/src/main/scala/org/apache/spark/sql/JdbcWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataBaseDialect.scala b/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataBaseDialect.scala index 35d3a06a40..aa3dfb9bbf 100644 --- a/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataBaseDialect.scala +++ b/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataBaseDialect.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataPoolDialect.scala b/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataPoolDialect.scala index b695dbc994..1e4234574d 100644 --- a/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataPoolDialect.scala +++ b/jdbc/src/main/scala/org/apache/spark/sql/SnappyDataPoolDialect.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/jdbc/src/main/scala/org/apache/spark/sql/sources/jdbcExtensions.scala b/jdbc/src/main/scala/org/apache/spark/sql/sources/jdbcExtensions.scala index f5e9528029..34afea4c5b 100644 --- a/jdbc/src/main/scala/org/apache/spark/sql/sources/jdbcExtensions.scala +++ b/jdbc/src/main/scala/org/apache/spark/sql/sources/jdbcExtensions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/launcher/build.gradle b/launcher/build.gradle index 9c14b5e591..dae8a61f8d 100644 --- a/launcher/build.gradle +++ b/launcher/build.gradle @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/launcher/src/main/java/io/snappydata/tools/QuickLauncher.java b/launcher/src/main/java/io/snappydata/tools/QuickLauncher.java index 6db5bb5f40..0f129b54a9 100644 --- a/launcher/src/main/java/io/snappydata/tools/QuickLauncher.java +++ b/launcher/src/main/java/io/snappydata/tools/QuickLauncher.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 SnappyData, Inc. All rights reserved. + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); you * may not use this file except in compliance with the License. You diff --git a/python/pyspark/sql/snappy/__init__.py b/python/pyspark/sql/snappy/__init__.py index d46e545cc5..1979d8d154 100644 --- a/python/pyspark/sql/snappy/__init__.py +++ b/python/pyspark/sql/snappy/__init__.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/sql/snappy/context.py b/python/pyspark/sql/snappy/context.py index d5be26d4f1..8de5c1b177 100644 --- a/python/pyspark/sql/snappy/context.py +++ b/python/pyspark/sql/snappy/context.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/sql/snappy/snappysession.py b/python/pyspark/sql/snappy/snappysession.py index 1d2eeb3ed5..5777baf54c 100644 --- a/python/pyspark/sql/snappy/snappysession.py +++ b/python/pyspark/sql/snappy/snappysession.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/sql/snappy/tests.py b/python/pyspark/sql/snappy/tests.py index c6edc70bde..9d1eef777c 100644 --- a/python/pyspark/sql/snappy/tests.py +++ b/python/pyspark/sql/snappy/tests.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/streaming/snappy/__init__.py b/python/pyspark/streaming/snappy/__init__.py index 415b252539..8be2cb2b0a 100644 --- a/python/pyspark/streaming/snappy/__init__.py +++ b/python/pyspark/streaming/snappy/__init__.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/streaming/snappy/snappydstream.py b/python/pyspark/streaming/snappy/snappydstream.py index 50893ff562..b72763b725 100644 --- a/python/pyspark/streaming/snappy/snappydstream.py +++ b/python/pyspark/streaming/snappy/snappydstream.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/pyspark/streaming/snappy/tests.py b/python/pyspark/streaming/snappy/tests.py index c98b22ec95..846c1e215c 100644 --- a/python/pyspark/streaming/snappy/tests.py +++ b/python/pyspark/streaming/snappy/tests.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/python/run-snappy-tests.py b/python/run-snappy-tests.py index 7c211e2b7e..620781dc16 100755 --- a/python/run-snappy-tests.py +++ b/python/run-snappy-tests.py @@ -18,7 +18,7 @@ # # -# Copyright (c) 2018 SnappyData, Inc. All rights reserved. +# Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); you # diff --git a/release/filehdr-mod.txt b/release/filehdr-mod.txt index 2347c88770..45fdd561b5 100644 --- a/release/filehdr-mod.txt +++ b/release/filehdr-mod.txt @@ -1,5 +1,5 @@ /* - * Changes for TIBCO ComputeDB data platform. + * Changes for TIBCO Project SnappyData data platform. * * Portions Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. * diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 8250e52ca0..837532be37 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -49,7 +49,7 @@ This file is divided into 3 sections: Date: Wed, 21 Aug 2019 12:23:43 +0530 Subject: [PATCH 21/26] Community docv1.1.1 (#1435) Changes to Documentation. --- README.md | 124 +++++++++--- docs/GettingStarted.md | 176 ++++++++++++------ .../SnappyData-UI-Dashboard-Tables.png | Bin 25379 -> 40755 bytes docs/Images/Snappy_intro.1.png | Bin 0 -> 314872 bytes docs/affinity_modes/connector_mode.md | 6 +- docs/affinity_modes/local_mode.md | 6 +- docs/best_practices/setup_cluster.md | 10 +- docs/configuration.md | 16 +- .../configure_launch_cluster.md | 175 +++++++++++++++++ .../configure_launch_cluster_multinode.md | 115 ++++++++++++ .../configuring_cluster.md | 24 ++- .../property_description.md | 103 ++++++++-- docs/connectors/deployment_dependency_jar.md | 2 +- docs/connectors/jdbc_streaming_connector.md | 2 +- ...he_zeppelin_access_to_secure_snappydata.md | 4 +- docs/howto/connect_oss_vis_client_tools.md | 6 +- docs/howto/connect_using_jdbc_driver.md | 68 +------ docs/howto/connect_using_odbc_driver.md | 40 ++-- docs/howto/connecttibcospotfire.md | 28 ++- docs/howto/export_hdfs.md | 3 +- docs/howto/import_from_hive_table.md | 90 +++++++++ docs/howto/run_spark_job_inside_cluster.md | 4 +- ...park_installation_using_smart_connector.md | 8 +- docs/howto/start_snappy_cluster.md | 52 +++++- docs/howto/tableauconnect.md | 2 +- .../use_stream_processing_with_snappydata.md | 40 ++-- .../using_snappydata_for_any_spark_dist.md | 6 +- docs/index.md | 176 ++++++++++++------ docs/install.md | 4 +- docs/install/concepts.md | 11 ++ ...tting_up_cluster_on_amazon_web_services.md | 6 +- docs/install/system_requirements.md | 3 + docs/install/upgrade.md | 6 +- docs/monitoring/configure_logging.md | 6 +- docs/monitoring/monitoring.md | 31 ++- docs/programming_guide/snappydata_jobs.md | 2 +- .../programming_guide/spark_jdbc_connector.md | 8 +- .../using_snappydata_shell.md | 2 +- ...ted_by_installing_snappydata_on-premise.md | 4 +- .../getting_started_with_docker_image.md | 6 +- ...ng_started_with_your_spark_distribution.md | 2 +- docs/quickstart/performance_apache_spark.md | 2 +- .../structucture_streamingquickstart.md | 5 + .../API_Reference/apireference_guide.md | 12 +- .../API_Reference/odbc_supported_apis.md | 143 ++++++++++++++ .../command_line_utilities/store-version.md | 4 +- .../allow-explicit-commit.md | 28 +++ .../config_parameters.md | 1 + .../configuration_parameters/load-balance.md | 6 +- .../inbuilt_system_procedures/dsid.md | 2 + .../inbuilt_system_procedures/dump-stacks.md | 3 + .../rebalance-all-buckets.md | 2 + .../set-trace-flag.md | 2 + .../set_critical_heap_percentage.md | 2 + .../sys_remove_metastore_entry.md | 46 +++++ .../system-procedures.md | 1 + .../interactive_commands/connect_client.md | 2 +- docs/reference/sql_functions/sql_functions.md | 4 + docs/reference/sql_general_limitations.md | 105 +++++++++++ docs/reference/sql_reference/create-table.md | 7 +- docs/reference/sql_reference/put-into.md | 14 +- docs/reference/sql_reference/undeploy.md | 4 - docs/reference/sql_reference/update.md | 55 +++++- .../specify_encrypt_passwords_conf_client.md | 2 +- .../setting_up_odbc_driver-tableau_desktop.md | 4 +- .../troubleshooting_error_messages.md | 75 ++++++++ docs/vsd/running_vsd.md | 2 + docs/vsd/system_performance.md | 2 + docs/vsd/vsd-connection-stats.md | 4 +- docs/vsd/vsd_cpu.md | 2 + docs/vsd/vsd_memory.md | 2 + docs/vsd/vsd_overview.md | 2 + docs/vsd/vsd_statements.md | 2 + docs/vsd/vsd_tables.md | 2 + docs/vsd/vsd_transactions.md | 2 + mkdocs.yml | 76 ++++---- 76 files changed, 1611 insertions(+), 393 deletions(-) create mode 100644 docs/Images/Snappy_intro.1.png create mode 100644 docs/configuring_cluster/configure_launch_cluster.md create mode 100644 docs/configuring_cluster/configure_launch_cluster_multinode.md create mode 100644 docs/install/concepts.md create mode 100644 docs/reference/API_Reference/odbc_supported_apis.md create mode 100644 docs/reference/configuration_parameters/allow-explicit-commit.md create mode 100644 docs/reference/inbuilt_system_procedures/sys_remove_metastore_entry.md create mode 100644 docs/reference/sql_functions/sql_functions.md create mode 100644 docs/reference/sql_general_limitations.md diff --git a/README.md b/README.md index 4986e2e210..a52ceeb894 100644 --- a/README.md +++ b/README.md @@ -1,46 +1,92 @@ -### SnappyData fuses Apache Spark with an in-memory database to deliver a data engine capable of processing streams, transactions and interactive analytics in a single cluster. +# Introduction +SnappyData (aka TIBCO ComputeDB community edition) is a distributed, in-memory optimized analytics database. SnappyData delivers high throughput, low latency, and high concurrency for unified analytics workloads. -### The Challenge with Spark and Remote Data Sources -Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and is capable of working with disparate data sources. While this provides rich unified access to data, this can also be quite inefficient and expensive. Analytic processing requires massive data sets to be repeatedly copied and data to be reformatted to suit Spark. In many cases, it ultimately fails to deliver the promise of interactive analytic performance. -For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Spark to do the aggregation. Caching within Spark is immutable and results in stale insight. +By fusing an in-memory hybrid database inside Apache Spark, it provides analytic query processing, mutability/transactions, access to virtually all big data sources and stream processing all in one unified cluster. -### The SnappyData Approach -At SnappyData, we take a very different approach. SnappyData fuses a low latency, highly available in-memory transactional database (GemFireXD) into Spark with shared memory management and optimizations. Data in the highly available in-memory store is laid out using the same columnar format as Spark (Tungsten). All query engine operators are significantly more optimized through better vectorization and code generation.
-The net effect is, an order of magnitude performance improvement when compared to native Spark caching, and more than two orders of magnitude better Spark performance when working with external data sources. +The primary use case of SnappyData is to provide analytics at interactive speeds over large volumes of data with minimal or no pre-processing of the dataset. For instance, often, there is no need to pre-aggregate/reduce or generate cubes over your large data sets for ad-hoc visual analytics. This is made possible by smartly managing data in-memory, dynamically generating code using vectorization optimizations and maximizing the potential of modern multi-core CPUs. +SnappyData enables complex processing on large data sets in sub-second timeframes. -Essentially, we turn Spark into an in-memory operational database capable of transactions, point reads, writes, working with Streams (Spark) and running analytic SQL queries. Or, it is an in-memory scale out Hybrid Database that can execute Spark code, SQL or even Objects. +![SnappyData Positioning](docs/Images/Snappy_intro.1.png) +!!!Note + SnappyData is not another Enterprise Data Warehouse (EDW) platform, but rather a nimble computational cluster that augments traditional EDWs and data lakes. -If you are already using Spark, experience 20x speed up for your query performance. Try out this [test](https://github.com/SnappyDataInc/snappydata/blob/master/examples/quickstart/scripts/Quickstart.scala). +### Important Capabilities -##### Snappy Architecture -![SnappyData Architecture](docs/Images/SnappyArchitecture.png) + + +* **Easily discover and catalog big data sets:** + You can connect and discover datasets in SQL DBs, Hadoop, NoSQL stores, file systems, or even cloud data stores such as S3 by using SQL, infer schemas automatically and register them in a secure catalog. A wide variety of data formats are supported out of the box such as JSON, CSV, text, Objects, Parquet, ORC, SQL, XML, and more. + +* **Rich connectivity:** + SnappyData is built with Apache Spark inside. Therefore, any data store that has a Spark connector can be accessed using SQL or by using the Spark RDD/Dataset API. Virtually all modern data stores do have Spark connector. see [SparkPackages](https://spark-packages.org/)). You can also dynamically deploy connectors to a running SnappyData cluster. + +* **Virtual or in-memory data:** + You can decide which datasets need to be provisioned into distributed memory or left at the source. When the data is left at source, after being modeled as a virtual/external tables, the analytic query processing is parallelized, and the query fragments are pushed down wherever possible and executed at high speed. +When speed is essential, applications can selectively copy the external data into memory using a single SQL command. + +* **In-memory Columnar + Row store:** + You can choose in-memory data to be stored in any of the following forms: + * **Columnar**: The form that is compressed and designed for scanning/aggregating large data sets. + * **Row store**: The form that has an extremely fast key access or highly selective access. + The columnar store is automatically indexed using a skipping index. Applications can explicitly add indexes for the row store. + +* **High performance:** + When data is loaded, the engine parallelizes all the accesses by carefully taking into account the available distributed cores, the available memory, and whether the source data can be partitioned to deliver extremely high-speed loading. Therefore, unlike a traditional warehouse, you can bring up SnappyData whenever required, load, process, and tear it down. Query processing uses code generation and vectorization techniques to shift the processing to the modern-day multi-core processor and L1/L2/L3 caches to the possible extent. + +* **Flexible rich data transformations:** + External data sets when discovered automatically through schema inference will have the schema of the source. Users can cleanse, blend, reshape data using a SQL function library (Spark SQL+) or even submit Apache Spark jobs and use custom logic. The entire rich Spark API is at your disposal. This logic can be written in SQL, Java, Scala, or even Python.* + +* **Prepares data for data science:** + Through the use of Spark API for statistics and machine learning, raw or curated datasets can be easily prepared for machine learning. You can understand the statistical characteristics such as correlation, independence of different variables and so on. You can generate distributed feature vectors from your data that is by using processes such as one-hot encoder, binarizer, and a range of functions built into the Spark ML library. These features can be stored back into column tables and shared across a group of users with security and avoid dumping copies to disk, which is slow and error-prone. + +* **Stream ingestion and liveness:** + While it is common to see query service engines today, most resort to periodic refreshing of data sets from the source as the managed data cannot be mutated — for example query engines such as Presto, HDFS formats like parquet, etc. Moreover, when updates can be applied pre-processing, re-shaping of the data is not necessarily simple. + In SnappyData, operational systems can feed data updates through Kafka to SnappyData. The incoming data can be CDC events (insert, updates, or deletes) and can be easily ingested into respective in-memory tables with ease, consistency, and exactly-once semantics. The Application can apply smart logic to reduce incoming streams, apply transformations, etc. by using APIS for Spark structured streaming.* + +* **Approximate Query Processing(AQP):** + When dealing with huge data sets, for example, IoT sensor streaming time-series data, it may not be possible to provision the data in-memory, and if left at the source (say Hadoop or S3) your analytic query processing can take too long. In SnappyData, you can create one or more stratified data samples on the full data set. The query engine automatically uses these samples for aggregation queries, and a nearly accurate answer returned to clients. This can be immensely valuable when visualizing a trend, plotting a graph or bar chart. + +* **Access from anywhere:** + You can use JDBC, ODBC, REST, or any of the Spark APIs. The product is fully compatible with Spark 2.1.1. SnappyData natively supports modern visualization tools such as TIBCO Spotfire, Tableau, and Qlikview. -## Getting Started -We provide multiple options to get going with SnappyData. The easiest option is, if you are already using Spark 2.1.1. -You can simply get started by adding SnappyData as a package dependency. You can find more information on options for running SnappyData [here](docs/quickstart.md). ## Downloading and Installing SnappyData -You can download and install the latest version of SnappyData from the [SnappyData Download Page](https://www.snappydata.io/download). +You can download and install the latest version of SnappyData from [github](https://github.com/SnappyDataInc/snappydata/releases) or you can download the enterprise version that is TIBCO ComputeDB from [here](https://edelivery.tibco.com/storefront/index.ep). Refer to the [documentation](docs/install.md) for installation steps. -If you would like to build SnappyData from source, refer to the [documentation on building from source](docs/install/building_from_source.md). +## Getting Started +Multiple options are provided to get started with SnappyData. You can run SnappyData on your laptop using any of the following options: +* On-premise clusters +* AWS +* Docker +* Kubernetes + +You can find more information on options for running SnappyData [here](docs/quickstart.md). -## SnappyData in 5 Minutes! -Refer to the [5 minutes guide](docs/quickstart.md) which is intended for both first time and experienced SnappyData users. It provides you with references and common examples to help you get started quickly! +## Quick Test to Measure Performance of SnappyData vs Apache Spark +If you are already using Spark, experience upto 20x speedup for your query performance with SnappyData. Try out this [test](https://github.com/SnappyDataInc/snappydata/blob/master/examples/quickstart/scripts/Quickstart.scala) using the Spark Shell. ## Documentation To understand SnappyData and its features refer to the [documentation](http://snappydatainc.github.io/snappydata/). +### Other Relevant content +- [Paper](http://cidrdb.org/cidr2017/papers/p28-mozafari-cidr17.pdf) on Snappydata at Conference on Innovative Data Systems Research (CIDR) - Info on key concepts and motivating problems. +- [Another early Paper](https://www.snappydata.io/snappy-industrial) that focuses on overall architecture, use cases, and benchmarks. ACM Sigmod 2016. +- [TPC-H benchmark](https://www.snappydata.io/whitepapers/snappydata-tpch) comparing Apache Spark with SnappyData +- Checkout the [SnappyData blog](https://www.snappydata.io/blog) for developer content +- [TIBCO community page](https://community.tibco.com/products/tibco-computedb) for the latest info. + ## Community Support -We monitor channels listed below for comments/questions. +We monitor the following channels comments/questions: + -[Stackoverflow](http://stackoverflow.com/questions/tagged/snappydata) ![Stackoverflow](http://i.imgur.com/LPIdp12.png) [Slack](http://snappydata-slackin.herokuapp.com/)![Slack](http://i.imgur.com/h3sc6GM.png) [Gitter](https://gitter.im/SnappyDataInc/snappydata) ![Gitter](http://i.imgur.com/jNAJeOn.jpg) [Mailing List](https://groups.google.com/forum/#!forum/snappydata-user) ![Mailing List](http://i.imgur.com/YomdH4s.png) [Reddit](https://www.reddit.com/r/snappydata) ![Reddit](http://i.imgur.com/AB3cVtj.png) [JIRA](https://jira.snappydata.io/projects/SNAP/issues) ![JIRA](http://i.imgur.com/E92zntA.png) +[Stackoverflow](http://stackoverflow.com/questions/tagged/snappydata) ![Stackoverflow](http://i.imgur.com/LPIdp12.png) [Slack](http://snappydata-slackin.herokuapp.com/)![Slack](http://i.imgur.com/h3sc6GM.png) [Gitter](https://gitter.im/SnappyDataInc/snappydata) ![Gitter](http://i.imgur.com/jNAJeOn.jpg) [Mailing List](https://groups.google.com/forum/#!forum/snappydata-user) ![Mailing List](http://i.imgur.com/YomdH4s.png) [Reddit](https://www.reddit.com/r/snappydata) ![Reddit](http://i.imgur.com/AB3cVtj.png) [JIRA](https://jira.snappydata.io/projects/SNAP/issues) ![JIRA](http://i.imgur.com/E92zntA.png) ## Link with SnappyData Distribution -**Using Maven Dependency** +### Using Maven Dependency SnappyData artifacts are hosted in Maven Central. You can add a Maven dependency with the following coordinates: @@ -50,19 +96,24 @@ artifactId: snappydata-cluster_2.11 version: 1.1.1 ``` -**Using SBT Dependency** +### Using SBT Dependency If you are using SBT, add this line to your **build.sbt** for core SnappyData artifacts: -`libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.1"` +``` +libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.1" +``` For additions related to SnappyData cluster, use: -`libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1"` +``` +libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1" +``` You can find more specific SnappyData artifacts [here](http://mvnrepository.com/artifact/io.snappydata) -**Note:** If your project fails when resolving the above dependency (that is, it fails to download javax.ws.rs#javax.ws.rs-api;2.1), it may be due an issue with its pom file.
As a workaround, you can add the below code to your **build.sbt**: +!!!Note + If your project fails when resolving the above dependency (that is, it fails to download `javax.ws.rs#javax.ws.rs-api;2.1`), it may be due an issue with its pom file.
As a workaround, you can add the below code to your **build.sbt**: ``` val workaround = { @@ -73,7 +124,28 @@ val workaround = { For more details, refer [https://github.com/sbt/sbt/issues/3618](https://github.com/sbt/sbt/issues/3618). -## Ad Analytics using SnappyData + +## Building from Source +If you would like to build SnappyData from source, refer to the [documentation on building from source](docs/install/building_from_source.md). + + +## What is the Delta between SnappyData and Apache Spark? + +Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and is capable of working with disparate data sources. While this provides rich unified access to data, this can also be quite inefficient and expensive. Analytic processing requires massive data sets to be repeatedly copied and data to be reformatted to suit Spark. In many cases, it ultimately fails to deliver the promise of interactive analytic performance. +For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Spark to do the aggregation. Caching within Spark is immutable and results in stale insight. + +### The SnappyData Approach + +##### Snappy Architecture +![SnappyData Architecture](docs/Images/SnappyArchitecture.png) + +At SnappyData, we take a very different approach. SnappyData fuses a low latency, highly available in-memory transactional database (GemFireXD) into Spark with shared memory management and optimizations. Data in the highly available in-memory store is laid out using the same columnar format as Spark (Tungsten). All query engine operators are significantly more optimized through better vectorization and code generation.
+The net effect is, an order of magnitude performance improvement when compared to native Spark caching, and more than two orders of magnitude better Spark performance when working with external data sources. + +Essentially, we turn Spark into an in-memory operational database capable of transactions, point reads, writes, working with Streams (Spark) and running analytic SQL queries. Or, it is an in-memory scale out Hybrid Database that can execute Spark code, SQL, or even Objects. + + +## Streaming Example - Ad Analytics Here is a stream + Transactions + Analytics use case example to illustrate the SQL as well as the Spark programming approaches in SnappyData - [Ad Analytics code example](https://github.com/SnappyDataInc/snappy-poc). Here is a [screencast](https://www.youtube.com/watch?v=bXofwFtmHjE) that showcases many useful features of SnappyData. The example also goes through a benchmark comparing SnappyData to a Hybrid in-memory database and Cassandra. ## Contributing to SnappyData diff --git a/docs/GettingStarted.md b/docs/GettingStarted.md index 646d0c436b..396e7d90c7 100644 --- a/docs/GettingStarted.md +++ b/docs/GettingStarted.md @@ -1,94 +1,164 @@ -## Introduction -SnappyData™ fuses Apache Spark with an in-memory database to deliver a compute+data engine capable of stream processing, transactions, interactive analytics and prediction in a single cluster. +## Introduction +SnappyData (aka TIBCO ComputeDB community edition) is a distributed, in-memory optimized analytics database. SnappyData delivers high throughput, low latency, and high concurrency for unified analytics workload. By fusing an in-memory hybrid database inside Apache Spark, it provides analytic query processing, mutability/transactions, access to virtually all big data sources and stream processing all in one unified cluster. -!!! Attention - This document assumes that you have familiarity with Apache Spark and its concepts. If you are new to Apache Spark, refer to the [Apache Spark documentation](https://spark.apache.org/docs/2.1.1/) to learn more about using Apache Spark.
-The current release of SnappyData is fully compatible with Spark 2.1.1 +The primary use case of SnappyData is to provide analytics at interactive speeds over large volumes of data with minimal or no pre-processing of the dataset. For instance, there is no need to often pre-aggregate/reduce or generate cubes over your large data sets for ad-hoc visual analytics. This is made possible by smartly managing data in-memory, dynamically generating code using vectorization optimizations and maximizing the potential of modern multi-core CPUs. +SnappyData enables complex processing on large data sets in sub-second timeframes. -## The Challenge with Apache Spark and Remote Data Sources -Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and can access disparate data sources in a highly parallelized manner for its distributed computations. Typically, data is fetched lazily as a result of SQL query or a Dataset (RDD) getting materialized. This can be quite inefficient and expensive since most workloads require the data set to be repeatedly processed. +![SnappyData Positioning](./Images/Snappy_intro.1.png) -Analytic processing requires large datasets to be repeatedly copied from an external data source like HDFS, into Apache Spark. Copying data, reformatting it (into a columnar format, depending on where the data is being copied from) and moving it across process and machine boundaries can be very expensive. As a result, we see that in several cases, applications using Apache Spark with an external data source fail to deliver the promise of interactive analytic performance. For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Apache Spark to do the aggregation. The alternative to working with a stateful store is to cache the data in Apache Spark. This, of course, suffers from the problems associated with stale data. +!!!Note + SnappyData is not another Enterprise Data Warehouse (EDW) platform, but rather a nimble computational cluster that augments traditional EDWs and data lakes. -## The SnappyData Approach -SnappyData fuses a low latency, highly available in-memory transactional database (Pivotal GemFire/Apache Geode) into Apache Spark with shared memory management and several optimizations that deliver performance and concurrency for mixed workloads. Data in the highly available in-memory store is laid out using a custom columnar format somewhat similar to the layout used by Apache Spark caching. Query engine operators are optimized through better vectorization and code generation. The net effect of these changes is, an order of magnitude performance improvement when compared to native Apache Spark caching, and more than two orders of magnitude better Apache Spark performance when working with external data sources. +### Important Capabilities -Essentially, Apache Spark is turned into an in-memory operational database capable of transactions, point reads, writes, working with streams and running analytic SQL queries. + -![SnappyData Architecture](Images/SnappyArchitecture.png) +* **Easily discover and catalog big data sets** + You can connect and discover datasets in SQL DBs, Hadoop, NoSQL stores, file systems, or even cloud data stores such as S3 by using SQL, infer schemas automatically and register them in a secure catalog. A wide variety of data formats are supported out of the box such as JSON, CSV, text, Objects, Parquet, ORC, SQL, XML, and more. -SnappyData is an in-memory database that runs Apache Spark’s compute engine directly in the database, and offers **Apache Spark's API and SQL as its interface and computational engine**. The fusion with Apache Spark allows SnappyData to work with a large number of data sources like HDFS, NoSQL etc. through bespoke Apache Spark connectors.
-While the SnappyData engine (that builds on Apache Spark Catalyst SQL engine) is primarily designed for SQL processing, applications can also work with Objects through Apache Spark RDDs and the Apache Spark Datasets API. +* **Rich connectivity** + SnappyData is built with Apache Spark inside. Therefore, any data store that has a Spark connector can be accessed using SQL or by using the Spark RDD/Dataset API. Virtually all modern data stores do have Spark connector. see [SparkPackages](https://spark-packages.org/)). You can also dynamically deploy connectors to a running SnappyData cluster. -Any Apache Spark DataFrame can be easily managed as a SnappyData table or conversely any table can be accessed as a DataFrame. +* **Virtual or in-memory data** + You can decide which datasets need to be provisioned into distributed memory or left at the source. When the data is left at source, after being modeled as a virtual/external tables, the analytic query processing is parallelized, and the query fragments are pushed down wherever possible and executed at high speed. +When speed is essential, applications can selectively copy the external data into memory using a single SQL command. -By default, when the cluster is started, the data store is bootstrapped and when any Apache Spark Jobs/OLAP queries are submitted, Apache Spark executors are automatically launched within the SnappyData process space (JVMs). There is no need to connect and manage external data store clusters. The SnappyData store can synchronously replicate for high availability (HA) with strong consistency and store/recover from disk for additional reliability. +* **In-memory Columnar + Row store** + You can choose in-memory data to be stored in any of the following forms: + * **Columnar**: The form that is compressed and designed for scanning/aggregating large data sets. + * **Row store**: The form that has an extremely fast key access or highly selective access. + The columnar store is automatically indexed using a skipping index. Applications can explicitly add indexes for the row store. +* **High performance** + When data is loaded, the engine parallelizes all the accesses by carefully taking into account the available distributed cores, the available memory, and whether the source data can be partitioned to deliver extremely high-speed loading. Therefore, unlike a traditional warehouse, you can bring up SnappyData whenever required, load, process, and tear it down. Query processing uses code generation and vectorization techniques to shift the processing to the modern-day multi-core processor and L1/L2/L3 caches to the possible extent. -## Key Features +* **Flexible rich data transformations** + External data sets when discovered automatically through schema inference will have the schema of the source. Users can cleanse, blend, reshape data using a SQL function library (Spark SQL+) or even submit Apache Spark jobs and use custom logic. The entire rich Spark API is at your disposal. This logic can be written in SQL, Java, Scala, or even Python.* -* **100% compatible with Apache Spark**- Use SnappyData as a database, and additionally use any of the Apache Spark APIs - ML, Graph, etc. +* **Prepares data for data science** + Through the use of Spark API for statistics and machine learning, raw or curated datasets can be easily prepared for machine learning. You can understand the statistical characteristics such as correlation, independence of different variables and so on. You can generate distributed feature vectors from your data that is by using processes such as one-hot encoder, binarizer, and a range of functions built into the Spark ML library. These features can be stored back into column tables and shared across a group of users with security and avoid dumping copies to disk, which is slow and error-prone. + +* **Stream ingestion and liveness** + While it is common to see query service engines today, most resort to periodic refreshing of data sets from the source as the managed data cannot be mutated — for example query engines such as Presto, HDFS formats like parquet, etc. Moreover, when updates can be applied pre-processing, re-shaping of the data is not necessarily simple. + In SnappyData, operational systems can feed data updates through Kafka to SnappyData. The incoming data can be CDC events (insert, updates, or deletes) and can be easily ingested into respective in-memory tables with ease, consistency, and exactly-once semantics. The Application can apply smart logic to reduce incoming streams, apply transformations, etc. by using APIS for Spark structured streaming.* -* **In-memory row and column stores**: Run the store colocated in Apache Spark executors or in its own process space (that is, a computational cluster and a data cluster) +* **Approximate Query Processing(AQP)** + When dealing with huge data sets, for example, IoT sensor streaming time-series data, it may not be possible to provision the data in-memory, and if left at the source (say Hadoop or S3) your analytic query processing can take too long. In SnappyData, you can create one or more stratified data samples on the full data set. The query engine automatically uses these samples for aggregation queries, and a nearly accurate answer returned to clients. This can be immensely valuable when visualizing a trend, plotting a graph or bar chart. -* **SQL standard compliance**: Apache Spark SQL + several SQL extensions: DML, DDL, indexing, constraints. +* **Access from anywhere** + You can use JDBC, ODBC, REST, or any of the Spark APIs. The product is fully compatible with Spark 2.1.1. SnappyData natively supports modern visualization tools such as TIBCO Spotfire, Tableau, and Qlikview. -* **SQL based extensions for streaming processing**: Use native Apache Spark streaming, DataFrame APIs or declaratively specify your streams and how you want it processed. You do not need to learn Apache Spark APIs to get going with stream processing or its subtleties when processing in parallel. -* **Not-Only SQL**: Use either as a SQL database or work with JSON or even arbitrary Application Objects. Essentially, any Apache Spark RDD/DataSet can also be persisted into SnappyData tables (type system same as Apache Spark Datasets). +## Downloading and Installing SnappyData +You can download and install the latest version of SnappyData from [github](https://github.com/SnappyDataInc/snappydata/releases) or you can download the enterprise version that is TIBCO ComputeDB from [here](https://edelivery.tibco.com/storefront/index.ep). +Refer to the [documentation](/install.md) for installation steps. -* **Mutate, transact on data in Apache Spark**: You can use SQL to insert, update, delete data in tables as one would expect. Extensions to Apache Spark’s context are also provided so you can mutate data in your Apache Spark programs. Tables defined in SnappyData are automatically visible as DataFrames. By eliminating the need to store data separately in a data store and then cache it in Apache Spark for computing, SnappyData simplifies system architecture and reduces the overall cost of ownership while simultaneously offering much better performance. +## Getting Started +Multiple options are provided to get started with SnappyData. You can run SnappyData on your laptop using any of the following options: -* **Optimizations - Indexing**: From version 1.0, you can add indexes to your row format tables and the query optimizer automatically uses in-memory indexes when available, to provide better performance. +* On-premise clusters -* **[Optimizations - colocation](howto/perform_a_colocated_join.md)**: SnappyData implements several optimizations to improve data locality and avoid shuffling data for queries on partitioned data sets. All related data can be colocated using declarative custom partitioning strategies (for example, common shared business key). Reference data tables can be modeled as replicated tables when tables cannot share a common key. Replicas are always consistent. +* AWS -* **High availability not just Fault tolerance**: Data can be instantly replicated (one at a time or batch at a time) to other nodes in the cluster. It is deeply integrated with a membership-based distributed system to detect and handle failures, instantaneously providing applications continuous HA. +* Docker +* Kubernetes -* **Durability and recovery:** Tables can be configured to be persisted to disk (the default) and recovered upon startup. Utilities for backup, restore and import/export are provided with the system. +You can find more information on options for running SnappyData [here](/quickstart.md). -* **[Interactive analytics using Synopsis Data Engine (SDE)](aqp.md)**: Multiple synopses techniques are introduced through data structures like count-min-sketch and stratified sampling to dramatically reduce the in-memory space requirements and provide true interactive speeds for analytic queries. These structures can be created and managed by developers with little to no statistical background and are completely transparent to the SQL developer running queries. Error estimators are also integrated with simple mechanisms to get to the errors through built-in SQL functions or SparkSession API extensions. +## Quick Test to Measure Performance of SnappyData vs Apache Spark +If you are already using Spark, experience upto 20x speedup for your query performance with SnappyData. Try out this [test](https://github.com/SnappyDataInc/snappydata/blob/master/examples/quickstart/scripts/Quickstart.scala) using the Spark Shell. - +## Documentation +To understand SnappyData and its features refer to the [documentation](http://snappydatainc.github.io/snappydata/). -## Extensions to the Apache Spark Runtime +### Other Relevant content +- [Paper](http://cidrdb.org/cidr2017/papers/p28-mozafari-cidr17.pdf) on Snappydata at Conference on Innovative Data Systems Research (CIDR) - Info on key concepts and motivating problems. +- [Another early Paper](https://www.snappydata.io/snappy-industrial) that focuses on overall architecture, use cases, and benchmarks. ACM Sigmod 2016. +- [TPC-H benchmark](https://www.snappydata.io/whitepapers/snappydata-tpch) comparing Apache Spark with SnappyData +- Checkout the [SnappyData blog](https://www.snappydata.io/blog) for developer content +- [TIBCO community page](https://community.tibco.com/products/tibco-computedb) for the latest info. -SnappyData makes the following contributions to deliver a unified and optimized runtime. +## Community Support -* **Integrating an operational in-memory data store with Apache Spark’s computational model**: A number of extensions are introduced to fuse our runtime with that of Apache Spark. Apache Spark executors run in the same process space as our store’s execution threads, sharing the same pool of memory. When Apache Spark executes tasks in a partitioned manner, it is designed to keep all the available CPU cores busy.
This design is extended by allowing low latency and fine-grained operations to interleave and get higher priority, without involving the scheduler. Furthermore, to support high concurrency, the runtime is extended with a “Job Server” that decouples applications from data servers, operating much in the same way as a traditional database, whereby the state is shared across many clients and applications.
+We monitor the following channels comments/questions: -* **Unified API for OLAP, OLTP, and Streaming**: Apache Spark builds on a common set of abstractions to provide a rich API for a diverse range of applications, such as MapReduce, Machine learning, stream processing, and SQL. -SnappyData extends Apache Spark’s unified API: - - * Allow for OLTP operations, for example, transactions and mutations (inserts/updates/deletions) on tables - - * Conform with SQL standards, for example, allowing tables alterations, constraints, and indexes +* [Stackoverflow](http://stackoverflow.com/questions/tagged/snappydata) ![Stackoverflow](http://i.imgur.com/LPIdp12.png) + +* [Slack](http://snappydata-slackin.herokuapp.com/) ![Slack](http://i.imgur.com/h3sc6GM.png) + +* [Gitter](https://gitter.im/SnappyDataInc/snappydata) ![Gitter](http://i.imgur.com/jNAJeOn.jpg) + +* [Mailing List](https://groups.google.com/forum/#!forum/snappydata-user) ![Mailing List](http://i.imgur.com/YomdH4s.png) + +* [Reddit](https://www.reddit.com/r/snappydata) ![Reddit](http://i.imgur.com/AB3cVtj.png) + +* [JIRA](https://jira.snappydata.io/projects/SNAP/issues) ![JIRA](http://i.imgur.com/E92zntA.png) + +## Link with SnappyData Distribution + +### Using Maven Dependency + +SnappyData artifacts are hosted in Maven Central. You can add a Maven dependency with the following coordinates: + +``` +groupId: io.snappydata +artifactId: snappydata-cluster_2.11 +version: 1.1.1 +``` + +### Using SBT Dependency + +If you are using SBT, add this line to your **build.sbt** for core SnappyData artifacts: + +``` +libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.1" +``` + +For additions related to SnappyData cluster, use: + +``` +libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1" +``` + +You can find more specific SnappyData artifacts [here](http://mvnrepository.com/artifact/io.snappydata) + +!!!Note + If your project fails when resolving the above dependency (that is, it fails to download `javax.ws.rs#javax.ws.rs-api;2.1`), it may be due an issue with its pom file.
As a workaround, you can add the below code to your **build.sbt**: + +``` +val workaround = { + sys.props += "packaging.type" -> "jar" + () +} +``` + +For more details, refer [https://github.com/sbt/sbt/issues/3618](https://github.com/sbt/sbt/issues/3618). - * Support declarative stream processing in SQL - * A unique addition of SnappyData is ability to mutate (all of inserts/updates/deletes) even column format tables efficiently without much change in query performance profile. +## Building from Source +If you would like to build SnappyData from source, refer to the [documentation on building from source](/install/building_from_source.md). -* **Optimizing Apache Spark application execution times**: Our goal is to eliminate the need for yet another external store (for example, a KV store) for Apache Spark applications. With a deeply integrated store, SnappyData improves overall performance by minimizing network traffic and serialization costs. In addition, by promoting colocated schema designs (tables and streams) where related data is colocated in the same process space, SnappyData eliminates the need for shuffling altogether in several scenarios. -* **Synopsis Data Engine support built into Apache Spark**: The SnappyData [Synopsis Data Engine (SDE)](aqp.md) offers a novel and scalable system to analyze large data sets. SDE uses statistical sampling techniques and probabilistic data structures to answer analytic queries with sub-second latency. There is no need to store or process the entire data set. The approach trades off query accuracy for fast response time.
The SDE engine enables you to: +## What is the Delta between SnappyData and Apache Spark? - - Intelligently sample the data set on frequently accessed dimensions to have a good representation across the entire data set (stratified sampling). Queries can execute on samples and return answers instantly. +Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and is capable of working with disparate data sources. While this provides rich unified access to data, this can also be quite inefficient and expensive. Analytic processing requires massive data sets to be repeatedly copied and data to be reformatted to suit Spark. In many cases, it ultimately fails to deliver the promise of interactive analytic performance. +For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Spark to do the aggregation. Caching within Spark is immutable and results in stale insight. - - Compute estimates for any ad hoc query from the sample(s). It can also provide error estimates for arbitrarily complex queries on streams. +### The SnappyData Approach - - Provide simple knobs for the user to trade off speed for accuracy, that is, simple SQL extensions so the user can specify the error tolerance for all queries. When query error is higher than tolerance level, the system automatically delegates the query to the source. +##### Snappy Architecture - - Express their accuracy requirements as high-level accuracy contracts (HAC), without overwhelming them with numerous statistical concepts. +![SnappyData Architecture](./Images/SnappyArchitecture.png) -## Morphing Apache Spark to support mixed workloads (OLTP, OLAP) -Apache Spark is designed as a computational engine for processing batch jobs. Each Apache Spark application (for example, a Map-reduce job) runs as an independent set of processes (that is, executor JVMs) in the cluster. These JVMs are reused for the lifetime of the application. While, data can be cached and reused in these JVMs for a single application, sharing data across applications or clients require an external storage tier, such as HDFS. SnappyData, on the other hand, targets a real-time, “always-on”, operational design center— clients can connect at will, and share data across any number of concurrent connections. This is similar to any operational database in the market today. Thus, to manage data in the same JVM, our first challenge is to alter the life cycle of these executors so that they are long-lived and decoupled from individual applications. +At SnappyData, there is a different approach. SnappyData fuses a low latency, highly available in-memory transactional database (GemFireXD) into Spark with shared memory management and optimizations. Data in the highly available in-memory store is laid out using the same columnar format as Spark (Tungsten). All query engine operators are significantly more optimized through better vectorization and code generation.
+The net effect is, an order of magnitude performance improvement when compared to native Spark caching, and more than two orders of magnitude better Spark performance when working with external data sources. +Spark is turned into an in-memory operational database capable of transactions, point reads, writes, working with Streams (Spark) and running analytic SQL queries or it is an in-memory scale out Hybrid Database that can execute Spark code, SQL, or even Objects. -A second but related challenge is Apache Spark’s design for how user requests (jobs) are handled. A single driver orchestrates all the work done on the executors. Given our need for high concurrency and a hybrid OLTP-OLAP workload, this driver introduces: -1. A single point of contention for all requests, and +## Streaming Example - Ad Analytics +Here is a stream + Transactions + Analytics use case example to illustrate the SQL as well as the Spark programming approaches in SnappyData - [Ad Analytics code example](https://github.com/SnappyDataInc/snappy-poc). Here is a [screencast](https://www.youtube.com/watch?v=bXofwFtmHjE) that showcases many useful features of SnappyData. The example also goes through a benchmark comparing SnappyData to a Hybrid in-memory database and Cassandra. -2. A barrier for achieving high availability (HA). Executors are shut down if the driver fails, requiring a full refresh of any cached state. +## Contributing to SnappyData -Apache Spark’s primary usage of memory is for caching RDDs and for shuffling blocks to other nodes. Data is managed in blocks and is immutable. On the other hand, more complex data structure needs to be managed (along with indexes) for point access and updates. Therefore, another challenge is merging these two disparate storage systems with little impedance to the application. This challenge is exacerbated by current limitations of Apache Spark SQL—mostly related to mutability characteristics and conformance to SQL. +If you are interested in contributing, please visit the [community page](http://www.snappydata.io/community) for ways in which you can help. -Finally, Apache Spark’s strong and growing community has zero tolerance for incompatible forks. This means that no changes can be made to Apache Spark’s execution model or its semantics for existing APIs. In other words, our changes have to be an extension. diff --git a/docs/Images/MonitoringUI/SnappyData-UI-Dashboard-Tables.png b/docs/Images/MonitoringUI/SnappyData-UI-Dashboard-Tables.png index 344d0724b9af37ae08e993b3be87e4620dd3d154..f09b1d98053c6f8aa3ca8f220a83dc3c2689939d 100644 GIT binary patch literal 40755 zcmd42cT`i$|1Rv&qaH=TAVs7dLl*?;9aKV>-n-IU2)!2-kTF@d@DU*IkyIWsJN%PhC%; z;^6K$!&d_Wi7Bx?;FsvnPDyV@>Hpkj_KW?9`eZ}*#>{ewD#8+lYoStSGGULsk%Qjk zzQ^aaLEVq`PGq=CN<#cb0tZ5w-XnD3?~B=2Gl%2fjIftH*R5ZfURf9FUr9B=*uG!M z_vJB0cYdY6M6U7vN`7--g8lX@ZOK^o;8&83rpNME^5yjZimJu@I3OmwlPW`2M-Ez1 z#(EEIZ!a|Rlll3%=o^*5tb7{kJmf>f80bc2?iuOPa%w&N_4c>vhySImH79H3&4K$U zYZ*}csC0B{lNnUP>D@+yDgXdT-kT`8`>RyGY|R%VN_dIoU*M=JD)J7vNI{D7VB}|( zO>+fCb9RrsqJ_x`)hj32Ps93FNj==EuIwS? zeDq0QCm$6=bb^>TG>${~&^ZvXS18@0^7iAIA@K97F$8J~bBCKePN%;m(n17kzE zSs*h5rD=8ILXXlyjRKCoYPKuBiC#oDw+-j*rfws3If$>jzS z17eEZHFu^;QyV7+g~WcS+W%x>%KdPXBq-mNNl+HQUyD<$HcP^YXOl= zAWXfuql|`*-+8Uz{E~K~Byw_L*>ls-^xrw_#Bs`7$5~!3%u9Y>4#o!rhRvthGlaCk z;qnz@>kkz|EtB+Pk^@PH`FRtd@@6OFSy{XKlQT@RUdvY}8`L}l`5hf1_ED7$l!PjD>WBn)+#S&AC55x3mR%1HbHGTfVJ z{q>3)H5R0kMuS`(u?bMGAmdko{w0ivhDso?e4X{QcjnN`2bvjh*8jUU z5sswkM%Bn7ddsJd+pCR1xmMu=L!8+l3sfeRM+ndSH2_eBr>yZrXG ze9za)YiIA=HS~A);0v(y1De&&ri&Lm1L}yAEuNf|*Q~DS!OC>rD8Uiq8^R5V>=2Y}%<9(-Rv0Dxv+? zFutGnw0XO8$ZCKu97X;+w6sjV7ymH=OM){0x0tsO4&x7VoP9pHi%rvj3(LOpXU4Gk z47T1FM!EfFe1MGlGvlB858;}No#plpzLodnC=&VAsG1YyyLe>+2Jd}P>M&{asroTt zKUwOJ_FeQnXt_%|q41AEep`WDL(VRoR$uKS(4~m|LN|A(5FadNQhBV{p$YyZu5U;R z??p;ZWUQy^|LuvF`I;o)7ZNgx(8TV+}Pko528f4n-oM3pf}`ki|W zm@eO_BNQN)DR}#LG;@*&pkbb))H>Q+U2Su#) zMn+iq``S3X*fPPY{v0W>{0ppslC=#_W7CkLMYh<~8sx@0+nF0Xd z+4Q-N1Zx)Idt@&wm%25)k<|ey*M}o_1F*Ub2Ah)4fJZAr&F9K8Vi)ISxYpVE<`G;M z8gZ6vin|P7k2kt!>z|pow-IfR@uf~>Ty7am7+`>i&Rr191%<>JbfQd2bZ#3>qU=^< zW1#eNFXWURK$ z-^_uTgQTXW$-`aYj=i^sXaOctIWkpyY`JNqqZn#oo-x>@e=jZF$LHDjegugd$@8R@ z(}KZvvwm`b0%$dXDXRz{%DqFP&2b#9%8lESD{CCU5*eDR1PFPb-6E}@#GMh5iaRk~ zG-hj?j9fX|uDGe{eGJl6HKj)#wujV*P);V0m7g3$WUY?Qt<1^)NkjkGxL;BEU}aNv zg1zZDP;9a+45~Mh2+d<97o9B>4NIuwUl3|qL8W_L9*uja1NLd6cIPu?{`uCH9DjxO z#_hkL3Q>t>l8|@=!=G*|VI}$Ir0Ubi7g7X!xFX%o1`P&+G+}wKpYzW8I%1x7r%XRr z7iU%R_Nucl`G4OT|mf=Rz9Fa8(`eS?T;?Q{-+GPHX%ldEupy=-El z*HtP3)^t(l%HSB-fNk59lt}ke0IlUvnfmQ5WXZ*2=A-Bv&#DO7x`uMk4yvYu;4Go5 zR;s4IT!)G!B^p=I)DH^xXJ&j#HXXwIRm&`46nPUP852|O%fxrYt$?=BZo9JFsR>^p z1mSi;Ve^7|WqvET7!Zp464w8`w{-u)t z9J$+Vrm?ef7*Qi&gNf$k3RRkn{Cv9CtgJf~E_Hd>cQi9kDv6Uriya+#;?swePo;9i z-Bli;1nRE#l`lqp(Fv2!j+TzD7V!aq?cH4?EP85cijc!`2ONgap7AHd%Can+$y;D= z_m%IEag45!0}#G@0B3btK>$Zr9a|O66*;#i80_guN*ib~o^|Q>A>r+}G^rnd?*i|i z#qm8t#&FN8qrtY$oP#Q#63BYib3hK$E_Ww9roRX(uxjQOwKU=!OiCu^p95_rtUZWc zhK4c2V6FafE-o+hxPfLx1(P$E64h(wtFHI99|DpYyL=rTgJ|xlC;Rl4^Bz?I{=kHFv z#B`^J?l^5KW%OgjB_Dd){-z9`Ft8`f*uN+zN#sM^{>Idi(DA*5b0E5md+d8J?{m1= z(T0PuV@>S%DvEl;;k7ChvA_%@(JoSzWu+vmDx6eb)t*<1IlN%5DchYZhqBoxS^@y; zR67e?+>PhubG|2AF)$N9Z$`xRN;vntKOuXhw8)GO#9xJ5C8X{fFju8$*iC+o85x-d zWi-c73<+;?jE%QtIJFiCR$(3x)}g&S3#D7Y*grd@R^)YJzTb9K{Ka-YS;GsSsWEzo zBVP;V_+!Ie0Y?~P{F_p$2C{vJ?Zl_gZ_Z_!^;mk^pYbH_fzkJZH-X{duFGnkXXjxx z5?{}#&K;Vn-!qc!B#*{%aKI0KEEU_7e0Tzgd4kj}8j>x(gM7ZCL*pa6TpUKi6`Vl9s;UkO z8WABfeFJ^{BpTYs#{8-7!DQWGV=O&3Ak_^lPm`bJvvTr#n2?QxQ7h4cA=KxJ!8V4? zznAArB;ix_!uFknY0cVC^hE7*VOF2-SQGbb80c4KR83RR?UAB-;wk*?Z2Sr)$|lxI za8e8d2@-PIV;Qy!S47Mo2JUe>tss zHQHS@5qE69Sye@zJDVpe^W>Ty0EljW1%MUS_mx&2{V0=ST1>f~^IMVQzTQHZUXE{4 zQtbKTd}a2rn6pdM%icVxGt63aY@pO(B6rgXs`+X^%aW?O*uPp*FO;&aW~dRwqEJ=bZ<(YO)mQFca_l<_^a7py}g1o`1`kATT5am z7mWfg_Ge77qzsqCz7l8q{=q^6(4V#b0KZY~kpuQB}TXx;M+}07$|6s~@ zvefvhEAYBYah-6CM12_#qp2pL)Kd&g8!7cal=D-#FFRV%v9Ka0Q&zQMxl_W z&TrD#vk`ZRA|TDQ+RZdy0xooW{!ew^T|MXBCutxJ>e~i4fIvcPirzLII^~cUy>DAS zpA5uK&dkgZ055A=AkN!(@)+ER$6ZC!ij=2gegWQn7hziNhJ#kIg~KQUcKypu)c7a3 zm8|o_W6zooT8Rp@ddrIyCv(Ja^Y)$1xvKj6VLtW0=8SH;-9C;)5+F_{8>!^^5&>wZtK^XyJp$XK>kGlv|uUa`fCLB+{m^g;KBHHtrxb)5eq-*3JDdsv!1rxA%#V?=^V z-t)eCkG&a5tv_z;TL>mUPae>zr@Y~mj39gx6Yur-pDlHC4a?L^&?bD0AK?9i<+iZQ z@$qq5Nh9$A1Snw`WEOj{*Jx2HZgi+N(?KS*$6S zOSgmI)v*-*_)3%F*UWhtO2Qy{JyFx74I+%LOnI#gy=)WV-KTlfoUJ3=f7_8|8mOef5L{5=+^(s{W$t=J8fo3Uk;S9hmCEY+OYc8 zC(Tv!_Vo?AOSy2_v40>651Mh*c8OMM+B^7nfxK7z&dC@LRY^zfCM)UIb4~sBACB%c z9DThQ;E$YaIz7>Y*--(^mwRtcuit|Hr>uV_)XZ?zX}qw6wIK{D@6(|-@5mK=_a|x| zK|}0NBmf^;*uyT<$WXEAGa>l-rl=i`u4rgz2wsHmQ2UA=_r zon4%o+Jcal6{BHk+b2STf_4%BS6#_ftn|PB`ODiDWAFoqegfsuIQbknJte|8LjoE4(DJXH*L z*hJy9O!4^}i;}-iS_#c?wtJOszR6%E^Vh#j=y%oPtvb0UlMPoRnr$Ny(6Py}jA_tl zMb_9orQN0qP1T`Kn`uoCofKIlMR~0i1w}<-zJrWD5UCCgeA?XLO3UZC8Xnw;=`#<) zW+(*>r~r6QIxMUs1kwUMi0uG`xOeB5%@#gYh{!jj)$&~|IxioWgOi?7TpqLe4FtE|rya{rFxk0y0;8GU@s{FvWH>v|vK zb4NhWPB@Pe=+I6Qk*Jc_;rga3T<`KVXY#N)HK2efW!w$~qQ#wl%j>^2i)2Bv0L@%| zjex)e%TRKz=?&u3kj`A$kAq+66B8e&D~Ca$i361Eg483#gtC>F_8^d*;T`KHTuFET zTg}J;ng`>d@@5{A?B(Sz9#w#LZoYyfZff5CFAHIxJPdt3Cm$D^LbFBv)pU?4Q$0(d zw!WwTaRR~F$ykjh0sAO*$|vU9hC#>2MJG+&MN`AN6XJR9m+J326|+`RBrAj3d&A`6 zvAV#F4{Ca<`aqxu!hhC)wk}*iC){0FNT>e0h#>u&Qqns#EFr6*iQO766GIJwX$c9J zuieWum|!t5%nZ#j9<54`rQ2LakZ9L)y}aFU*hwRG(W9E8vY8>x|J&R2;r+iz;UD~< z#B0(Hf3n`pDs&Pl9BVSBN=Deeuqdu1?uy(}L;q!8xQKq3Jf8v83-fU~$S!o4nO*i^ zQwWil9p#{oc|!1>`8KM>%PdQz)Eu6O9QB8lV+EW=@5eXrkIsuVa(Yx@{N&WYPR~i{ z&?y3IxgYOS*k?b8F#rG*WLtTheZ`3}Rn;>uhlQR*a}B+TnppuVO21ed#M^2^eBbFm zSi|ft;&*8lucvEH(zWY>FD(h%Hf9G|By~ltd}B(gQtW5r8O&~7Jf!Ik{gMC7C~Bv& zY_em1ViEZbDsSiUXQfS;aD!wtRIf*CV)g6_TPU6WAX4>n)@L)st&0*pVQ(OvYCZo=&%?*?2zkDCx2###RH6M3?%R$1`u z7&tj%%ouGiISn|OEDiL2N=Fp^*=$CyS*HV4T=u4lj=^M)DBhfE z>D~YpBNpdEWMa>!xYWAxP3QZ~y{?ee6}v|MG1mi1;&M(DVJr#sMmRUUh|3y*!766? zOHOnSx25KH%n>%fetP+lZvGY8;C;ONyppY=;r)nU0~O`G?B)KkN8sCc7hO~ zsv_y0{ir4Ty;j$a?EnFI-fZcDZVFQCr)i zj?VPzCSvnOMOfPPvvZf-Q>~J5n&qQC;~uec{6ITlvQ!NV78`c%4$JqgOIGDvfoiXM zkFM)hFKcJh6u4*-rtA&G1BRlN}@ z87~)30m`z+f4?C5vYW3vPo}IFqXuEy?vdTJUL~ym#5Zu-3gx}6BbhsyovlC2qaUyr z5%Gjq)NyQ40stZ0`68Rv!~po!$D%%oO&#U=NgdC%xIRL=plQ#W7E@6Kh7xanxHHs0 zDYM2Z)kul$L=q@V-Fl5HR*A`DugwDo{&yBSzC?@}LnP^vNU@6Y`ZhDXdERHMN6fg? z!Y6t@Ox5&99Q7tUru+Lw1hO;mu!G7p5bF-8BOo7Sk*Cfhwvc!Q=1kqsDpi<)oW6i# zoKQ~Cq!*g0MgdXrYqOqHNMi0?6O9xdTCMNL9ZrrKN{Rk7tffOj*p0e3sjoWie+sW6a({KGb3iXjcl}y z!vLxD=}3yj>5AV7CgAywOtKQVFSvk+HCmu{*G%8cQ{eY{E|dDh_M(!f-)3Y$DF(x^ zGoF}d=%)RARv1Lyn3@)xxN~^4rss%H@xvhSi%K+jB)-MLt9u{UA*B0V@rq4EqUkN+ z45758;uSLIYtPpC%fe5qG0o z*1hXpYbnIpf-jl?B=74K6tXoPtD62H++N-vKXrXReDM02fE2&m8)5;ZJoun>EQvX+ zO~3dnS+d6VZOSSv6rp#nd>v$Zb?N<++w^6igNE^^)QMM13c-2K*IeGurV_A~$sfWX z5NDtKr@KC~mm9)h#f(Oi9^k z*RW-ed-6^fFxj;9uo@Fxwa(e1vp6a>QNDc}U*qHG;3kBL6u^LIYNz;X10t#mAoE&l z6ZPcJ>U2=lV?Kgl`+KD@!^$ax60vOeO>t@kxmF)pP)e7<{eF(Z$}0m;ImtRsU^y*) zl1XnJ^Zys>?J!hQUR8B=b|$pcb<3+7#x)bT*vhjZ&K#K4oxf4$BKa=8e#2oGw<~)? zb<5)T1YMH8=%1uY))kQJc|IwB3{ptY1YV4au@3OrGoZj~TX0ib$%5Lq)wOHdtSXS> z_7c{Jq~u$_r5hHz)gI3`5)n)UQ6t*XVaF7o5Lw6nXvY?!-?-E=h;wp~pft-!=A<;ul8;e^ko@jGxYAci?Iyjq(z~zDQxWRf~2=gD7&@ zNRkJvT!rm+u;8F=@rb&AwE#gSgp*eNuO{;0-HNqm=ZC9Zs*zV~3y@gu>C!ud4Omr77t5m=2_PMJdeCVuwVkLEXl|Z0@rVZ| z`|;=9wC_zLcPlY(w(bU6u zr&{w@eWT z00R2xW}4c6E{!?JCUvV~JcJEC?I6g`>)mHgC3{fvMuPmOyC7J7!tGe}($Xxq_cD@h zYC30VOopm{o9$cu&t)_7L(TD;>FdDqgaHAtK;x@GVZ_tGSY!Q^Eqlnwwf&^01y5Jj z`s|GDAV;j&C&k_Sy;bS3tSPOMmwuGBfQ3VcLYto3N#GGP*Ulgj-6K8Is)4mra-ym# ze%r30=Z}GkZ9UZub?ZCLT%Id&eJas%3QQWIhZ*eP%mM%7&J!cZ=1YFYu!_OeN*%I- z^H!U(XAkgeN_nvZ7+F zj}i!a0i_6#o6nz=e5}=UW%oOp&qndr>1FgysND3l>lUfzqxnFeZQX`bMWh{rfQ{s4 zyuP`2{c-KS80GMFp)RHxQ&z%cm7D6WVmbXCM4s2T?eYu=+*o<*i;lNc0+GwWBP{(r zwzS+1dz3PAF}F*3kK1l#vW3Z)Oxkliw(Tj{*K9jPGO`tPJ8iDy4mIu$mwRhNfb_~B zj2i;M()4%;I$lu7M-7W?R-`Hp^svpWaOH;^F!(;>_45hPJiIGBe%xchTh?^BlVED< zzaFp4n;6jsCk;=e7Z!xPbXVb~i5*^@{^oHk>*}0;JXid5P!zeT9q}pcr+dp!2lL&n zONxa$*mKr3Tal;lMsldY`kwa5n!P)5GqZFkRDJH|?{E=aTh}BrTfUW{GQUk2S3`@Z z0D{L?Qk%Zdae6RwCxqa|WgRCH+&Jbb9S-1j@buRz-1GdeP$&hlyBwV(5~XeCWA?66 z%r@c;Cck?2dmXqIJYy_KM>N#Q34l-f&I4};^-|gV~T&dhsCAo1rd1EwwI1^&y4%D(Xr`-9i-WA-&>M)nH_r`T#3*xqq+ z&HPYG`IsQadNHs#QK`Ct{^F?P6;&Cz;xw~|Iz#BNnD^M_D2gtwoH7e4im&@&Pq~ml zm?30UjIIUZ7o!U*?{R_*os=^Vns0VpVrS`pD}K|jW!(%nJTmnOEaj@-+5<}>kqe$O z$UC3iSF1UbO_E6EhyjPck88{v^}azDe8_*AIc3bnqj4DAGv>UyvOZvc0kO}!D3w== zdibn_W9oyVaTHjy>gJ*Txoj>MYZ-xXJ7jWLNz= zE=f`(W=e~})%iR2Bz~Q2r{XS^wdwWT5ccn%c0J|Yu(3UIo_5{{aOe_<$qqUdok~q4 zfP{X%1lbIpOr!9K{0ij7P zl$y~c&s&qn-76X=#xa(JDHH`(o_NgOMyB6*ySJ$FL&1Uf`XBCco`)WleT962z592= zqB@u(@|cxh^`*y6Wxvd;lvx%TooJ*}4kGA--h4*PhGZtNrNv?CFfxNR<0N2Hv#fE= z!FdSPlXyqjH~;c{Ee8om}ir+t#mV5nE)A zkmLN)pH7R=i_BZm{3H$mEfiiyukG)}^%YA>J`n&J5&TBXmA!>!ZXWpU)1o1+Tn-#3cl`*}+E(j6-x~W8mDym7G zOQGzCEJ^cLa`jY03r<+n3ayAi(M;GaMQUSH^6RO3SaNN8j4>A9bVQ#mY-Da*y_+*dtTfVByQ&3E>T%v65R7O^rcrVf0*?CfZl%(Xmb9Az5vSn@G41TB)Up;u5f3;OD+ap zyW~}0t;(*crm$u#47#6B1ct3teaDb zeR@SkPtSX#9#Gn@5{EWq=#`Vn9<-nAd>W=82s$aeq2sp=beo0LW_g7leHU>6mgBlxPdy9{!txDC;aNR? z?5AfgYpI14m;Cn^{(!NaeR`_?eT^&$yl+O-5%fiPg5jH$i(CQ0=vuSDWN2p+`I8u|vgu6(W2{$H6j2!UDihWxJ1Ty8b<;(_yo zJs4w<47Jcb+0JMh$%oUXeMPGoR{P(bGV-6wn($wxE;rCE0Xp~fKF%#(>_#q}m{WN( z4)7}j0CIbs#2qunS=}nqeS5m8aebd9`D~d$?I?(v#j$4|(L`X0q<~PNsh$6vccJFq zt_Q+4>8IjogKF`5Luhu-v$gz%m|6B40H9;v4La9gq6a>>UeZ_`c9`qtL=p8}w#N57 zMdZ03m<^ql4LVMZ>bbZG(+S~hbwQl=$LQ$YU8>UX%5}`D$b{H$mQ|}Yy~ZA;l8+s} z(nb;J6Q$oQSsX*X3=9ONQp+!Pb-EsNnBz{W>Get65So6Qo3cq?*5+Og{kV2v0YpA2 zK@I&$e(%}yZ%AboE>z4`CPhVisYhk^^B%MAR7xX!6WC_4@kFStOB`7bll+TB&h| zM(#JUD^LFS*D%f}9d?8|;id&57GfW&$V*26n6ss|u-k#5Mz z>pg3tmG;5iQ}$qd_uS!1Z}9Frjz@gRX~=HW=*F{TipIu)tbdWPpXy*_an#ndgD3?Z zdK*#R@M2tO;bGGV@mfQD=S=EqB)yG9T>#w=D_&sP^3mzM8 zo#R4j_<~-M7FAT8IJaDgub(UEv5ZN|t(e!H`RvA6|J@c?^*<&s9aaVkWMo`8g={Ls z$N@^~sw8PzLJ&`KubaPVKScWllu4IUKfSQbz|hq}`-8lzFfEXpC-!sq4%7S#4OCmt;OoALC5T!n=VPMd z&rmNvyU6GIF4)u}j6#{kL`{;=eu874nv~ntu;Jq6gs!Q-ySe+dP~*1p*}Ugs@`76z z?xr0;!I(fvdTFUfO*@oK2xVZhVJ9~Hg{bhDEf*FZ3I1vEvDHD{CFMxw=)*NgLs*oB9)5x2beJwA z_y36Ug>-FHFALgu3VM#c?bfrUiJ}?FSa>?A)x4-Ko^QwBZ0{mML6`m|g00CsAbmN; z#?RSJ)H2lTrie@5%E6_0CaZ-AZ^igX-)FcHoD0h#J?C|r8(VXI$ovtuJ7#K6$j9Q(?K#WR!OVj(r674wPH z_H1hF6q0&CiXiepd)>I5QaE+bS`>YttCv4L03KUV+=V{7(hiZNevl#$15 zMZs?a1m2kg>#rH!Q)!zU=i~L3slI-rd6wDC5KDxJ=TYvll4&owNCCzs-HY6^1@ORH zGkR*FQ&kCbi;p?TZRyh|xd_Dd;ZNc^IpvpBAV9 z^rJqR>jPqTN8QeKP@0$pf=PqmJbD_>7Opmgn(IDd3Pcku368nL1i!ACYmooyJpi1+ zp4dStC};C>)*z`e_k&kSb53Jkphb4pE5dn(l?#84%j4rLO@i<)8*^At3uQA&?_xPc zZgxa~{A55(jgOMD;W=#9VMvuYm? zVu2c4*I6gAN-x8+P!^QG0XV12*;LnyIaUw4h;)a8_I z3*2k6bBylPTt+mfijPfFu3o{z)(rDh7U8q8(kUH*@M-d;qsd8k84N%)*y{=hRI~YI z5Rq4jF`hqBwO3U`ZP|K+K0an#g@Ywt-!|X)aYD2HY_zk+V??Z}v844UoZ zrAE9NVgZ46RbbNhNv9o^-NWSV0?aUj&Xz3WQvAcSA}^ZZLw|3n4#XK6^&jbqeymRk zYIB^}k~aXjTx`R%vebISh7>76Lt;C93MQ;RJ|lSZ`*D3h;C^)lT}7VkBbq><%XpDv zC`++?zR;Hwi>%`cXC+Yao7{d{SFqAqsMA^e`h{y3#<@l!&Pb3GK3VDIxL3qDbyqNO zGSPg*l}Og^-T6uDLGE*pDuhx+D70^zTp?8Qz{Z@->7ApRrr6(Dvf$ieGixlYygDh4 z!|AlYnpIebB8)$|IV_E3srv?E&qx&Pb$-WR3j?mD@`%YfACf97chG zav@KsM`3wl4B-Wi3Omlyf`p^Td`XV=fK%k^aE3I*yU)*(jOC)~=gS*L z=mx_`#4q=g1KdF=A1bVqtL5HB?zrk#+%s=FgEJ#FIb?we57os7TJH3Wd{Hd>xKjC? zSn9MpTaza;f;n^iw%APpVEL1s1WAJWd%aSlZ^g;+G7}QRNNGzXW@yhS^#2g^I+K#~ zem=2Z#WC_zjzbo_KtgH6O#w5+fLy|C)_mRUC4zG1vwPoHx+k`Eb9t0>e?Cd_8|0`U zK%@e@5@<13+MhsogFjVWdNMGiX}kN05i}z`3tMZE#9^Bjfv`yyU~n{wZBFC<>_pV5 zxQLT)yDS0Q4SN|3?nKF4!W-^;Dx!j^(iy>Kva(&P)PR@>$?J23kjI&uahjbcS|y}< z6Rv*5zPVp>=Ezk!M%1Xp=;Z=tl>xvD=;)d*K5V%N^UsRHEx{4J9sPTw5C_fOQOI7V zr=PE(KLOck*}etcrJo*sAw#ZukZtz}PVuB1Y$pff=6xtYd57ND!p2|wowcj(aN`Jm z%_c9nw0q*=GvUN$8HzgE*<8B|DPlwA44P%kJAc*+cEb$A-BjzG`yD~@q2~Tb!8q~J z%kiyt2mS;?=$ zE_|)5xpfR1Ogr#4@{j?6;0W=i4L+*83GTPpp0r2ADl#C#8G9HrP+(?hJmxq#E zoU=neFHP8Yd-QEo;$_mum<>l7YOzN)P6W8w=DX4p<5&g+3L=y}Xbl{zTG)Y^Uj8EV zM$e4!GfN%tXc9K7s%Xs&GzI*Iw*2q*??f6E8xp~pz+Ty|&MX6Hs?o?cfiSGni$M(GxYHE}| z_A&?Ie$eyCF4ef_F}0yIssqlK+`Oc70=2AJr>hY7T?+w)Y7$*mv)r63q2j76B+-{Z zlwv9n;N;oQvzdgcIe3=uLBIsmw2_wKcaV{4YFG>qNQ^_`5!VK|yHs>`XH)BNJ{sxx zm150X##FgnU_U_J!9SKUV|{-P44?__S~0d?65H*%@1ZDkpIcCFrhlfMbKMHz@zW48 zr@5pB_>#^ay?0k)JRk6_KB@Et6`#wqmG31KeB^QUJ)xPUXgV=OY#G-6GJldU$O&UE zm_BU-4i4F!Ng&dsQYj4%nldK-P#O*+P(t;G$uqVY9OVL^6w&uRvdLiPeLweE^#j@S z1S`#8#in2@LneAf8r1p?{y2f{y0k8_5bLaaj1@`mg~%><54JTM`RoIY^mhYy$A=v< zE8R@=#dJXerZDMSn$c1tm%8z#cis~#@nuYBk7=V0g|6lWV;nfgJ?wn+pA3pijCjyk z#uP3R0t1NK@(bf5oUY`DFyaA@Ee|(C;dxhrqi>WLTe?_blWE5V7MN!t9wC0v$i{et zrs(5mPiRhNwIc&=i2xpJ2lv4Pd#O>Rt?mj@zcVBoQ6gppyae;>$@!@17`70x-`l6f zPu-gB`ec3Rw~fDlF{f&xe0KT5h0xeHBdDVwhU+C*z&7l(PRAvWJN!2-#*p|wRCf}( zx4cqkZTWP`3L>D!;2sP@W$I@@Lt#OkG9~RxiqPPh%MINKOngA~nJA31#+`n&92bY0r}+LrY9hBD0pks|mFoiXo~} zqS@j9oK3h1Mh{?$fYUjN!w2oJIW6@FOWAw;ZP&C%^Cz(ybV`biuLJ*6cVk#vQJ|Db z;PTpL+dL-r?}#%-xWzL<#2IY1D0O$+&IP^4UuaiG)MNQ=X8aC!go2HDaaU~LJ!x`8 z#h_<#V-X^Fj@L=dsh%!35l(KMIy2|mG+?}ad84*%g#EW0PQD0OzaYvj!iRxoK zr4^&H;39EPOBQI?&%100iy3}2vhA9hyXq2UX}kYB6YRsz>cO2WxsUOsx8F;s)6j;m z*)L+|mm%(Gd+?39!~5=vY+(U3H`FkWL-2(SbltG!Nt7HhBixA@?j!>uht13_An8s- zX_bRCtC8_se%9P&rEW3GmMMih@Tk=0VI4@^$(yso5Bc%KkD^-k#HbRWgt)RRIG=e& z?h}A!`X2*{^vD>Br~e5@)E-y{GkDi~vx`d6gzoro#`Nm3V|&%|(r6QRdf=xa12q4{ zsAbcKc2HW!0fA;#6_yM$|0BNRt|J&e|N1||npZIbW1rtaD@^Q*CTBoQh)h*dOr&h& z`a!#2$V{v`I6Pk7=1=ko%>*cP-mykeI4Tj|u z&<6@|g*I#!o?0{f62UF8Om}1?$VE$?dTm0RaC7hy9Wb#V8_=xev24^CCqcZgf?}5e z`@8HO5p-W%*&;Ha``q6h^ik`{og3n=1P6xatg0D%-EnMOGSDQ7kpJnz)VUS0dc63L zPSEMa)cE@i@|f{>TvI`Z#SH(Lf`s@BvMNx!3ECrcs-2+yv3#Rht`GQsi8>l$o-fo_ zK3aLC(u--3Ws#^6={uKev#h5x?F8U9`Oud#ns0Jc1^=uxKV%mFa=?_l% zk@S>+g2VA#{3|?;sBCWCP2gIm55I#|PVait-jsfw2SG{sUEl&Iu^+Bh@vB#5Up=wO zZ^yVN9j7k>W=8_ECK}JNXer3Gu zZzU7k_fz2cUlf~k=gXvldNKcJ`!yTD1RE&x<6oMTJQpfZF7v;7G=NjC{1Hkjc^F)) zZ=pG2R$+RPPkDY`0R)mY*U>hc+T*a%ejEDc?nBn;!73B@1m;#TkJ|yIr>en(=2+akbkL8@iH(7uXNc zfrg+k8R73kVQ#p(%j=t1zthBMyV)z7=H`pl#RO$z|4;;2EAyph&a36O3KGZTasg7` zE~+--RIioIBw{Ou#GN(crbPSO(?u_?R6AT@U-6A~(ah8Bi{j4Z-k5bGJ_#+AG$pHm zD{iUFtKSC7ZU^AoR3+kiF@p{7)UF$I-WaDf7PysMtjXbHheci}+8$k;vg1t5vdXvk z_vNL$Pmxk!#HBBO7J0g_OPNpxX~2~YH5}!Ebpz3h##qv5%g@c%_1Z1Z%pPqRQ>cy7 znU!A61#KQIFtqxe_AROhEa2v-n(MF8Qff7}ZMX1;gitF94*-B8&gYaU9Xp;Fy-0;C zYu0=lm-WR=$ahI9GVKa{wbRUpKZWHHAsMZ=QK6c4;jrV0cR1qSH^jBZG2o|IjmH9IXAACM za3vn*uu7iZM%@Y&yqM)$P0UO4S}`B&=>fYQh`&=C_0=U}6Lr_;Ug&*1pW*`MN53cP&uCAEDAS~!ElZ2Ym+Ez!W^Gn8zy(V zmf>vi_Gn>$^cIBPXIpG~nTR`bVN>|HR64$NpDx;4w}TuZ&BDrl-D5F?ek)hfMlnJS8k7CXWir0$8uaPOp;}%sEVUt-|KDHxG+p ztB2x$Mb?O4Agu?^nrhZ$-4=)X0`cv6Y1k_KQ7J7%otBhe{Q7&*c)3#uZ}U;8YCzjX z$m%M?L9wh;TN~;8*~r9k`sK z9(aMBz~NtCu9zD=G1~k()79?KvI~Em-m#%#XEEY@co_Fei01>Hf)v@z6unN!7fs1w zQH{3SGo8;~4eO=jni)dT5HlJRzXKvr;KeiF<2qGG+|?#dGzzC$bdIVGv8$F~V+3Ao zp3YrL4>Vtgk_i=`;z~UqdzPf#E`7+nE;#w#GM%!`a&OCeC64&>r$`1jkKeuiBAf}u z@H~iF6k)#Rj5R8O$Z)Fv2nXVs%_>N9HM$ZzU-f(Z7S5z6<`{a9+fmbdJ@n%FmFhU{ z3kN1LA8x~WXWwH#M~#iR<8pIW(Xx|8h{l(hR+F2$De3tL!eNdw_e71K-H z5*s1(n>jIVg)5&v3Ye`;JdCq}rtT!vu4PGixx&JrHfC;GU*xXptj)~2pE1!(1_nU* z`2v^6pW*uM%uLAR|Hz6;4*PEdhZ%ot1sUKo3H+?z;Tg@Z#b*IN9Ulb6y_&;3;>GIow`U3da9ZcFq$R@(b5}79F{neVBI~#s;V+2Y@M_8 zoc3+-1u2fq9A~!LKu)Hrouc-gk^m)z7@m5h`B{djL*DpqPhko!cqFooc!J4HCZoNOnXN7R@y$>C6{TX#_MPFl=lKPR4!hkK0V|b z7Sy@*vD2qx0e}0#AH;M&u*rMIe75!}rU(Rzxc<>90}%5iXVPp$VvW(u%9k)(fUweJ=^vKq#dE zXlGlOynPvQVZ9yYkqPqI(i41u{6f=92j$S+?`RLu9El6XTeg26g1m*7Cid|g1q&Mx zn)(tTDIixU#fb79cBoxzDrz1-Pt*R1{F>h|Gde=Z{A0;i?A zkdf1GAuj{hQi}!^IqFbeTSlII+1t*x@31#+293tE9;4-~`?*N>a29Hp;R};wY3jHb zk51Y3`U6%re88ejF4UI8{bJjF{NN6{WEY?X!YDP?F zOp{|>@&;bsOqV(JKIcDepB9yrPRD&rjmnQazwUxo|Cq=L-LG+e8zZqn*La#TM$KDN#-!y&ZeTP%ZsGLJQ>*o9E=J%N_gwrKLi`h`)15Ij7@8vdG4CNAGhE<9e5W{&XuZ(eDH#*e}Dno7A ze%_toT5d{YtFV63kv#2a&f`2=0!Uz<(rQxf!)FPq*o}R9lBCH z#bj**8=wAi8)%OxPE^v^TmE5hHN1!Xs||%NE^kd` z>v&CLD0bf#LX`u78hWPUeN0kZFY!O;1c)@mlAIlJjUr>ufkUz0bP^B}lQKm{%IJA| zh>53%Ym*3PE*YAeie!Q$jy$ngG7Q9YgMRgW32J-^0!J&ul(kW?uDjWCqHg zAL35f8U=WHLBSama-p+$D@t5n*rOGv!@6^Z7#BQiktF`O0}$??f2{~~o5 zJd%lW6vA2r?t2ji_n#Hj%aUpt?+Z>T*)67P%a2&6iwCPR@MNu8-jK zzU6#QYsTXa>CU}8>*i8gyre24=xntpuVuMzka}Mx0W8So^#k5wlT#sMTaaZ;G z2aiKagWYvtRi-sIW}JUi?R|~iw<%o3;Vj&(r0961DGxQf&%$_dMJ6;&%p?HAT0U{) za#Eqmf&4?6uOSh#ku-ZLN;62}8F^kwQR(|eiq&Wx%%}pFd$u|?k(h|ggkQC~sgBqh zw4$;+#{@0vGXc)y8O=i#NJkS5xQG@cubUJpytO^n9hQH99FK2L;xfYZ)yTz{^3Fev zklATG`MuR?MixEN5p*cIxTFr*p;VWBE`I~N#Ihp>|1nJh+%(cx4^!;uOE)JpoWwr2 zweZ+i%Ll`UDB7JPF5tU|QxBheJ<Xqhn%%ItML?;Z2=>N7$>e zg)(Z(r?O$J?hm&T-*Y~WZl!8vHnJaD-PBsB*n0VExv<-Au z*r!jcYg-1yLxKa#i0B*Hs+AKp4FTzRHJ%RoE9@;B^(I0$*|p6-rc5(}tuGFc*9yFu z)M?hW9;&1k4cFb-l>6b=p~0N-Eh%t=3Y7uuc9jhkHt7`V=$J3I z)9xM~oY{s1ex1lskY;6+ywc`suM3vu|Dxo2?`myRm{PC@~ z>R(CM)U8RquUD|-`72N7bw&Yy<&^uKxv|r<0FlUeLj7AqD#0D%X8J4Cd7*3e`k{QS z@UJ9u-!4#H{*}G%*|Sgm|BBE2bED{g%fRr%5C4iT{UZW?4*VxF_RrCG)PE(dgCD)Z z|5scs{OBFqzw_L|=kZ4WR=WPz8w%igo&=G4H?ZCPBYnXyVA=m-V1p*p)xh}j9}N$F z7H8$*YFRJgz(Dh#Fy#NMJO4j+_)(IWXVx)u&H#tQhD zUVvMM*GrTe0|V`P`s{qNF+SX{@$h7Tvq{>i(T_0Gt=4_!E99q8DT^85mF|*C@nn)S z76w}KWLBnW)#&fEw~BTR%!W&`@x{J94^3TjSj`*nFk(I~sH#LKeuJ zxrZkFLBS11gjYKl?(2pbx?0;3mOa-Yic9o+SA=6l-j83W|BQGQhr|n3})_OTE`wK-4l_ zsWKyEKo&-zeC^$!AA%^aD|?b8;%{6V89C%X)b={3X|Z<@;1Sjn6O(tYnLWsFJplp& zN%TvKpsj`3ezxpIFEzR)!P99LS5SvMtJ9~&(M)QPmwn1<(AES6)lo}RQEQ{L?KH?c ziQburMfL<7u2!MQMchD^C&54`8R5BkNKU{SR8n0^>Ih4U)|j#gkc?uKg3-cLda5`*!) zZ};C+L7rJA_l+xRE?AkeAuzh|66G&Fi22$SLxHeY)(!p-S7!CU}7w7`VzG8Bi63_=R#z-vvPJn-Px zI8aO-PKtHynEs;hV8Q8A`?+siUE1EBFUhjiC?1VFK@}&N!W43DTT*&HMG#bXe?ad= z>4Y^>!xr_?#uj2rOS@4%9q_R(NQK-{tyPmdQPpXOn||FaKb`h(QFrGRixyVxcyel!#y7Hrg&8sbKaeTsRDio!-J~5XvT>Jm|7}t zV-G%T?y&=Mm=M0JX##0q2-KYr8B<~a4iIwbr755X2qhJwK_j1ZkSO%?IwN*>Nd%t` z>Olt7e0UM6BWb#{?Da-Esk`%APTc8=5EV>}*K0(Ry%0yugY<{jL@UpyX~*oejKR<} zS3robDt>j;2|LOHRQG={JNFS1_Eh$Yupr zbiN=)$$@}JtU1(-&D3*$#QJ5XZ|WnP8|K#p`Qn5rrOn6ZYy{I5H)qon(36IKV)vT2gIaEd#+&b*&OP%R!^Q8?A=n8To4)mSE zU39h$CdRP0Twss z(7Q-N+#a znxsD4I`ogcd5K}HyX$h&VIy~0O78{YzE`dgZXywyI}$i|{F(e^nR;FaLd$}b#jGXT z6i+2@M8_I6i_?N0rlm+&K3MTe2rEQ?p(#ObW$M$f4jasOb0HBouXVVp7!mF+u4VP} zY8M7(`eyq6ptM#Fm+N}#jE0E;vv!ut)2eRs@8HFzWa5hIS*s<8Nzg4^WB!wlIR3pTEH1;hd3u*ns2ZT}N$}kg8n*YF%(=W|juUShCCA zDN{-sg@S)CGJ-Cs43!dXHS8-kZ^~y}z1s2QUDS4PmmDknE^lqUTD(ns4X&mVUl<2_ zpjxW`PNZl+Ty40a_z=0(&(sh?tC&W~Js%iARFG%+O~sTpub|Y(yxwO78?v`-WbWlhrnz+>raaVER}pj(K^vFpa*<5v4pelX_wmT#`J(sLFy6wu0-D3OxFxJu zS#(OMSDyYDrm)pkDt`9bheUxg@MMtFp3x0xz9&H^4TSg9Z-o)a_Fu~AGHt?3eb^md z>S@{})ip2)4^Or_%W%n46KUzB?6h4PRKjdk*SXK+v++DyBs5KyYw80n>~X$ko8g-;Ya87MQMDJx}NMJnK`x-0GvLa`qru-Z9SA0Rb+Qi6qWENxsCap(X#z9!_ zZW{WlX)NU&6(gi`*3LuZq18(!u8p27y@D+eIL<)9=o)OC|EJDV!W|2|&cBPirzyvA zQ9zL1S`uO&O4TOWt3r*HFiy*2PP-YNTH?%%C6NT87Acx@c(ZK#Gip|7F1yDUCnrnf zCgVB1H$GrsPP{t^#VH)79k7Ba=}jgaDGB8%n{zzZ{fKQfDf|W#^6RDM%e_y(6Z0mD zlbz?kMQ@}52nlNd#;gQX7#QFh3;DBei_>etl4kv~2}d0=2Q^@+me)7qlS zT_}^biH=T3GhjzOv4GZE*t^j}bvm*R~K zVIE9rErJ^J41G^&>mI&;Y{@fJ2?@ z>egNZJ`W>KycDJeZy!L|<&M4t*IlNqLB23S21@&*(Wlk43ABEx!jcyzIC>Q)zoOh+ zD7rM6tCw%q5XmKhx7-PKt(HK;5iV2QsHeT*`2#1IMdS3ub%}@i#uJXkOkzd)K}$eyqI7w| zy*A44lbaBk+2~^p%^M)|*m+sr^KquxrpdqNo=uu|N}Ylj)e}^yc=scBIEkETu&{gX z==R$>8`Z*?v~3mvN0c!|P4ogSHI^oCZ4};xY}}6x0dCBVq(bNhCUrFy4_rbQ!78lP zrq@d5Dyw!;hPy0%?cd(zsZ{>y?zo=SL>T+SYh~RgRk@u47S7?xT2;6UGhZX4bIM}` z>d1bL#9%tAlZR}WF(bt<#w62gF%k5Ft$bR4)ijL!H>z=#Qq|3KUlvVDq@yUw74Y_V zGIvtdxugnIkZv>*rf`zOxx<(^=CMOMk3YH8cX`0WKMGW&W@~EvDn@VWh~(51ZHDQX z`|XW4jke=U`o^agej3CXPh$`BN8uDq9bYgBcxpyEG-9g6&Z)peB{!1o@Ta4x-lU=Q zy2)kPP8@<4>iZI!=JBvZ*Xj}j14&?!jp=v}9EdwMRWFuPh|JVxR@Y>ob+`r26Sy~= z6Sc%nZPzMWl-aC+;&{~5bZP@6A?yx7?e|+hIMSv(%L0)Iq|$n78dC0U3;OPA- z7yj}0kvgzoYnw$mJ~d$@MQVKdYu(44tyn+eQrHSF8n?2Z9WDEiieG6{T5zzsZb3xI zukBg`dmUrKtn!U?0W=ky2{opVcjiX!Ap@~08XDHknguk=?aVxLgqrE0G8EoI=pN~U zlj?ff8U@T_io9CIX$!ZZ17P=-RM~8&7l|1N)`nDnDm3fzy2?)?&FC-!b&IQWdyUrh zs=U&XQ9{`{0I^(QU291}j{<>#b?(C=3L2ZN$s5E{E(){T@W~Bafq4~Gy~$EIzDi)l z>|N_-0%kmjD0`#x^Yk`1*R*NK-9!lAtA2mV#cO)SCA~?3_bKViku1m;1<4KazC<2A z_Up563@xmWBOqYri05ho2CD6ZZzMS=iD(zK{KEn?>sjW=One++jZyoQ%L<(`GibHCKeuyfxv}P{xa!NV9M=?n!Cy@@6sa%;H*>YZPr%SU_`J`6~oF19DJOR_{PRu@+i5y8? zqWmluWnAiV-cg7}EJCKmWgm|$?J>$(6SIb9UOrHLHEAewRBzm;R0O4{y_BPWnrU>D zQ*$Wm)?WN_+^iH&$FZ}qb$WpkDW5wb{bo+@M_Cz{#biHU1v?v5G0lE1{*Ri}z9Y5t zQAKrn-%a@Idr4@KbYbMZRkc1s8P@h06h7d7=ZEx3{Y*+1QGNum>4z_s3p#1i9X*TJ zp2Q{}q=gW!hdis84%cDh>`6YE|7-Vxb_)-9qP;BF)sTecLx<77DZeYms8XnSN+cle zFVSVX+&CTd*ER-+IwjM}L1~G9pJre97ZeD;@c$1r=>Me2Tz)<7If7>i9NhnB=K=Ub zm3%;9(;%1mfnwr5z_~ocP*@}_F1o!zzA(vw{~-zS7Cp8>GyC8t5z!!$+JN8$tSh;>D7}~2n5MWjbnJBC=%4wvvC-BehvNLnBWfG{wK2Eu za_8+dk1HFKkE88S+Po>`L|soB&`EN@n<~@N7cNSCD9gh2@Y{IG3V)?#W@5-^IDcB} zZC@VvQ?drx_1&Hn5(4tX@5K%otEjlTx%1b)vnUm2nH}uVzU_AOOGuHBz2;cDCnWC) zAqIw*(R|hJTT0KRj#$xoA27b%yomyPL#}qD)i+vmxm*~mF0$0o$rO4MG-@ai>+l0q z=epoqH}GMov(M|Wzt6BuJ6>;3hTr~?RtdtAS{fe`EH~@)+pg|N^u>3EsE4_ewTNkq zdPZKrUHd{<)9*b-?YgU1bG;TKKh=viuZ|SuoYk4mqGkkskG1VQ++4L?kqVLCT)(YU zus>}Z3F8E^R}TfCkFea&kG+4I*$HG3kfYS}(4)6sN>Lli8+%Bq4kE?n^U&k1&DjpT z1ZQ*djkc=YM$Xs2xU?NBI8=#qL8GYU9xV6dHx(hoY;|tk3~S?=vM-rpe}Fxp+j^dZ zGb&+CBtBLHj~c4eZzz)jmLk1P@~rY}L5?1EotDDqqgVk0Y~e7z#*5q zml_D#+a)9Pt_>>PWhC<|FA<7;tHXKo+r6>i2Kb28P$~ zBnq66tzNYt8L+hHx`~w&100RN!vJ?N52d!GmGp1qwI$XAT6+zPLgh|Ig0SpQzemBY zC4lRP0v`Ks@G-eCW6tJsM2M?eXqfSgF2lqleVP(&1@VNqVIU93cmutX0qJxCr~dc+ z;|7sUXSXrAbK?zFXX0^gyHu%Q6~`vAvMg&xUd)v<2+4Q4oGaIH4b$Cm6dH0d7E&SS z?fB-K?S(_Oe-tk+_`-0%{1fxV_W+gOTds=;a6a}SZbyh@9_J4I>G+spmF8*b)`xlX z@lh`B5xQg98)XymnC^DH5~Ft!?!DuN$ZB6%KIN4X+<$N{JW$ScZ)hO4{{XD<_2I}* z<+hqTM@Qxo5Nx_dh+^oEk{!H9&^wpJKU+{v_c$FA4LA)7k!U_kdqmfK?0fQ9_kU9K z`h6mP_(*EJ!Q|thQQ>^fJ`eyFshugGuWqr)H8L8nJYm1Gq3#W(*s0BU$YiQ&4~vB- z3|wi6E3!FpLqmfp327Fu8B*vDF_v2|I(^W>AT1{&$JAyk^pBDYM|W+jiu$A_6MadA zfE`6<&|v1|*}ULG2mM@-%3Po*>VY&F<#PE(tKLC`csk^JCkk@CoY+eDy4kJCVeQ_f z4h!V_l2RWpUF~LP2iQ@}b2V|jo1zJwn9tHo@JHr7O~|Z-kf6pqVGyLo?pe96Cw<2CXXA{+seofQr*^%`bN}(m~OXnB!mjcGHx+5R@~E*Q+`KxNjh@4=mnijXwFEgij1#<&!r*g!iE_q z%WcvE`uYVoLb_LedX79kS=nI0?BwAA!GwESqU@T9KOy;5%C&f>=Uosu_-s7 z_m$_8VYj_rMsqe6KRP=pKh8ryIa0$j?QE0H@$z1w$A+ieYA(AdQDk*%gsbhilgWB| z8Zm)_Ysr%nU($p7w55eTp8poe^Vxdcbxg=hL=8 z@(fw&NyL2PZUsKY&#=f-l)R zV4k|%={?`Fm0--S*V^~2Y(7?A`oX*{9ijlog@BIw(6<^lEu5V55B#6yf$(re>I+s8 zbsdTaif~@N-%foe5 zeVIZhG97g(A?WGTkI1dJ!#;9Sp0luV!EgZ2bf?ySdNUr~I;1 z>%DdGRZrTfC^)0rWiOPZ> z%8ye*rj^w8KRhtBv%P{BILS{@{fz;n9vNpCyU{@$4bAYwYb3ab4SiB4s9{Lqq={yx z>(>3&6Knf4x_1|c4JtEIhf0+u3+AQ6GY`=nrtl0yi4tF0u8EpzJ6{gBCvR*xLKk^^7vD;S)Q=&b#jx5R0|g_euAa!D!0k^GRLo*H3eDUBq&)NJcXG$Z;1Z ziA9$;!c?Usdb-G0trWk5w!h ziilYsdc!ktii&ZX;?P^$BrE^|3pqJ?JuO?+E7-)LSvHf@eF)PRd)ycv<+MuZw53k3 zUGARQsWRAc(TdK*aG?%sa#^YJVZ@}THmsC*I;>A?3=<0@P1Ci(F3iuVX^}E+nIziWhcq+2I$$Fh! zG$Zra$0wYQ5QU;Nek<*zaa`lW$1obpv#5`)ap_1xhpRtfx!z(JgshIxO71Ds;twa& zC+8!5MNb9>X9Te)HBQ%9e>gRUqK-<+(7zP!suwj8$KSFm%OnN$pC^W|Nt%)L zwzQmaWisvShyoO6D&ZG9vBGd`;e1|Bxfdy|Dc$V8tmSE0aQ|K7j4FzEy4m6p_A7wt zNt1{5m+Sr)Zw&T}i5>j^if!9jUceDNw4ts%9mxx!5NAUthHzdtw=!Jz+;$oT1_SFs z`FST+-hv?S(ajJ0AK@C}(aX5s9|He!aB=Xa^}?_Bmipm4qWb6iwZP6kv;fLR@2l!j zy*kgM<;Q`pX%g>rdvCLYqH^WOBJtp0@>opL$K{N+TktVr#yOPiU5?(;GqX_I+ZFPi0PxW@E))#MxiNS;u%J!mSJwq4O! zXSER$93v_YnRpu;CGJGQ`=Me-AK2bp9kM?_NeA_l<6 zs5``-uPjtVD$&d9Y^iU)ziz8^%qgS%LZ+iR9^CRwdUVR;G(jX|VKg;0Dao6eP@Ukj z-c~RptNei}#!wYyln;QVmc6Zd1}eS6H)nhUGaQ6NbHBDQgV!k!8rWaEA=mYXtOo>) z871Bsh2m*AmS_q*gsuajAo0Tu%S)X*mFLBN740X@!vZUpe>BON?`bP;Dfv$M@l@nB z5g&cCD@Xroa{WiVQL}V+Ph3DO4Ay%?`J)MXyqs$f(#VOmC=H=|L@%k0nQ9XjN}2z5f_a!B=}xqW=`FmEa!%Rfba0_!}vQ{#DpT;O{_Cnh5%cqW~P1b;=auOBUu+*`Iza2Cl2j8N@E znqrOX3C1qnSRDTa%su!fs~iX<4?0eBU9Kp^t6(+v1Ui3eu~vp8&p&v{l^gtT7&`E$ zhS;Tq;n(@nD3!QhL)?&->3M$*9F&5^O#tC;fi@9etkSQ}e`GGj@i`M!%0*egx6Gq7 z86~XZZ|R$uh#YQ?>HV9mlglBx8X&k~LPo*rjx;Rt4D11^>BQAvm#~l9kLR|y+>!*1 z=LowvYI87H!!=-pPxI>buL$;*bkZxHg`Kq9iaq9SoR$ln-flf353u|F)B~o?$D-1GA!*z6FtO^ zafT#$vU5GQRav;bUeRd1>-$6D^_RjaygV5Agr6bYSr_Wc(D{Yk1(VgufgC%-I$~Cp z^!}x=l5fR(_(u6hNiL1(D|cr!SYiN=M=J&WWN;cOD)g!NDQHk!q!| z`Im<(Sb2H5o!O_64pvqU&92*}Ycmw5s{m8mwE;OP!PMcnE4vIhuUAB2DyI`Ro7N94 zvCV+t~jMrcbyGHO`5MT0Nf8ghqGKX7jQpFz5sG_u#`N)xJt@o2MRLZopaZV z`OLz}KIGPXqK7C@+w8I9Qc^B9d-?CR%;ihBJ;JtKd+!}+z;Yy3g9&-i`a=`Ny-ohr zjG=Cp)mIYLZV=E>Jo)-+DTw3)y!W!b75ACteca|+TrL}*xn$IHa67-fhU#4lXWk~(I$UlPGBA%_c-7+h zQ&acX?wMqF-R-6S*%{h@MEkJfqB6^&4s|DUur;*>ax4sDQ=Ttb^(un3t1D+{uOlLk zI7pyuAJ{O)LjtQ%JnDyiSUQzk2H-yFJaJj`)D|fg(Y6xv?e@M_P?RBBNY!*J(7&x{(lZH0Oik$Oxu*plh2$~}j-)SX$r zQVPn61c&k^rkwrWJ*_U#$9iw5n9h3)jFmI|B-h^~2(>hXVQi6MRkD>}W7|zC5Kyqo zi5Xh5*$!5U5KSop!U`LH<=Gxanc{jZY-pixIvHj1x(&Wdl3Ma77pqgis6Xg_Y>~h! zeXhOgo7%b7X9;3o94`FcLi1Lpjvx!t{AusyttT_wZ+^>ut|+hmjp|hWbS5&@=f?l& zVHdg2=cdy2DNkOEn)sS!S;os~e}CJo+h;V=qupbpPbuqu@aejKRCNX5!?GNw4l=vD z^37XXKfa;-?AE{(8N7|l9(L&CsMM|WbUn=NB3d$~6?X_}J};$6IeMwdxZ0EXk68Ka znf`tkx{&8;2#u5x}(r82d*Z*El4(8v1>jRfz8LCPD^{~$>0vV&Vz5l$biO|2#e#Kbw zd3sQRF&Fmy`{aKEjQD?)pMSrx5^2zvV%bwj)V3ZBPJJ9!O{hB3nHi=JmnoYtPL-Iu^W^Sb$oktWrScrIIolT5dB|3&SxoL6e zs20F{e1)Ru*`D571Ou3QQSV&0O&82U6fa#9$PHWrxJHz9v%@IK$;Z@P{GlBN)qJ9> z33I0GrzmU&CXcVbuhcc(Zl)o;6<&8um`;vMgQ!(0-6`cP&lJ$L!La~8P38R02X~@Xn-V=my?MALSm=_1mNAYU->c9R`rGOOVr%d`G* zEerJ##4dP-1B*y8yF+q6ndOq5_)tW z&HP9>=nelthtT~D{|SRQxm^Cp(%D^H)pB~i_|pwf^)Eed#F9@A8rwGYl@6mz2KGkl zv@JM4*yPjBiT5| zEKqhw;d`dN)^X}#vU=RnMw14jmTP`xL9kNTWkoyHdCwP_R0wo0A7&_=OaCwF#{on3 zBaf-hBU;*l1W@kK^u*UC!z*ozK}F1%pE@y0Rv5MTP;6={_QP%W`Wy}jP2OR;B;Nze z`XRXda3bwuqU+jDN7y`R!l*UA8>KEVbXN^>1HtBXM3APi3)_&Gc&@`}Dj$_cCI)yY z>T{l?#&qM}`^B3Tn!*6IgGwO+OHO?lYJ|Q)=C%ss@msTwaT2qtU z<~Xpm9;m`aNE#CS{jf3AgpGPYaxPn8yUerJhOu;a!hC65TSGyc*r`@(3?riUJ#IY+ zx+yINvr{rR_63|;fVsA(4%D@FAe?WlZ{}5QC3dOFFAlliI5NVExxRji`+fYWVB4;6 z@|$0j`6Uzuj2M55`%(Oni4&_Pd~t9c`4?R4!NC*+0@1yzFJsDCfvcuim-4M1k5_%y zDGP@P%lvn^lV}%{dRLzJ(`ju&W$Pp6S6DGkTj+?${08@1kB_gaLmtlk)3Q9Dj%}Z; z`|>u)#kgz<>*t@ooZe<-@}Ayw-QJmh3nr1^qn;o3D0g_$lPk*c(PX(_hl#g&vk(Ps z$TOqp9B^G~D9vRq5@S2B{^mc_unO1-du z)0Dj2v(A$VMvcm;g0Ox^cWmQ*b92H``rv=?Nw6b*M$UiYCrwVXz9&tNSb*wP9cu8* zDtT(5viO>j!MTGd?uztt$Pz(K2Z^7LaTwsz{+qn%;(eV(L%<+hudURuQm{x^T&Y#; zmqq-FkNEacxH0yVud=2~yW|09BF*KC60OV*X2LY(q&y`|y6V%25f68yUQm3)96AO2 zNz6ZtDCTVJJL@l(D3e%I*2r)D>mT17!=zr0zK-DH`N4=_mAO(sWhQJhkA zFO=!Bj4Q#fL?NhhNF&)Y&3KnSv<9yhVi0bExb?imOW$^eDz?B6r;N=K+ZIO{an7c$ z%Uz^@z8A&BeL#P<1Q((tksV{MO)u!13QRqB<)cOn;s`GsjtG-P zZ0VmdnEp$q)hztevMY(kpjOs8h9tT6T1_KVYGAt!{4%1yfy*39a&v#NChO#Jvm(;x z3Kkp-6(ffq6iqUl+<65Ci>y9%ep-a|=zI?|!Fu+LBvBG5tn6vCNQi$%>Yvr^P<3(f zGh1=VS%V3CF7z zg*2u`lK0~rRW$_h>%NG;3P)f5ICBv__+ukjA}#@f-&C=6!>FX9q;=Jv9K=&VO&v<* zFuR}2X1-J-s-RiAAYcWOc^aXCV-|ZBU>lOex+g2;a4~?+j=Bek+dK>Hu^B@G8@~F% z4`2Nd+ld%V*{?FK7^0q`8F_e3{j2arMag~NM7GXA3BcD;l1T}!Gqu$~srVYoR=3(7 z&nK5>XL>i%&J1ywUqXbpEzAfeH}^A1U{ZJ-Zr+eZ)-*GL(^}+Xf}UC)5l#-^EOWIwlXe>1b3yrbYq$i``tzLQAr&{3$iN$L8ZY9ND zhupR7mTXr-wQJ)D_~`G(#XC*A7lnkfDAtZBR;r^4L!Wk<0$lJm{If{!*RDKBc$3$i z2+OaFs^@#h+_4ubHdr1vu4TxYe9r1g&3(po=7*<^1wUENo>C|EFCYlCDa3)@cAoKUgAj9M4Ioip%;E;M=7ivfS)8M?hCg_yz4N6_;7VSz zR6^0oy0dz!g{F3I>>72(Oj77CljItF>eF!aEA>=Dg2Bs>7@?A{#zYaXUyLu{erZD!{#RO)$LKG2Zu1AU4I4`8p0T+QLoe6G$kPmV94FBp?n16b&7W z;c`1}{3`82$*|*guD1Thy*bv_LN}WolfX(DstbGBAyP7P4K%!6k(Vj!2$Y&^ePOa2 zFx9#ces~R~&uBRhSe2?C-^vi#UEGp!hikj~Rt+sV-phi81A5r+Q!NE= zN@Exp49J6ngI5kIwu)?W0R#B;d+e)+9!9(6W66}Gp6NHnFrZt3$Jx_uSoug`HnMa2 z`gt{7#~t313sT6BF?cGzf!4E5w6#1-x6KV>S>`Aw4J66@suB655zkGFBLaCzIMN3n z#c&X`0ltxi1;<-p*P^1LdQlo`UXS)lqxI&fQaDWsCr{zl zMmR+w7FJc%TF9U`SdVv9Hm7E_!nU{jCh;-m)QQnGZ0h?T;g?9HGg*E@)2=C4=_Y*L z0KN>kESr{1v|X4_Pjw^995e@?>x3+;H-1sWq>W}~nmX;#{dQsI`SWmfik+AOqC5m% z+)2HfcPNxk~4ppDcO*pYR!sIk9B8UAHLVc0G+g)<3+fa_>?r%6)#uc~i7J zch~29*TsSbA9?5%lX%9`C=Rre7kRn0u_XDlGM@>P-#Xm;6V4soYiCCYKAFLHcU{gR zRj{7rP&o^Ha^B8~01}27HMU{NO$XbKHRnY^k62U)TM5oXre*mW!d-ms7750Dv`)*L z9TGvK#z6&M(USD;ce~+}eD8BhA2zQ$js1T+?zk6jeV(kCkxg6UCP=kjLv1r5wbSQ> zV&k87+=faU!V}WYRutqw=RGXQ72ZkqvUcyVU1-aTWQ=&MxT<+|k=YJQq~0O062@tz z_NrSDSfL1nuHygjsGaTHn9k5D_cejNw5`k`{$lGOi@H_=>unB&HSXvCqV-gT$58O3 z6_mCFo%fQ7(C)xpsj=jd)aJ{Hf?9330F|-oiVq-;+`$@LtKkbtIujmhr$0tLM~%~j zWS*_~`HR7c*tb_ag^v!XsIOQxv86K)i_gy|%68zhv-dy)z*BvVtT#c5qJh+Djhd

LKWmiF;VZ>F@Ri~tW;u0rIf?6Qm*so2b~6${#hPzc8(mE}2F5@v z$z!9;{bO40ReAxI4yl*V@#6?zv){n_(+TmU#(FS&3qjDP0EBl&pK= z%MxHJ5C7}A3d%6tqPhlEMk=bRgypI`11?|ApXyznSQVDtNcxs(Gdc!o556Kcuh#E7 zw=;tv_)pQ`5WPtvJ1y3TNy~4AYBHDKtnydVmk{lgR!A!Vy@tdN3><)>MW*D%&*}vN zzpG|g%{%gk%F;eCgzNCly-~!+V|hUEGo;k zeMWPcqk?NoS4gn0^W9-$@`yFo6KpT8;{jWA^Y3W1yu!K4oO{TyC{QWocg(J= zYaP%S=fRx7mrXpMXw=bo%`MO9LM#O-_o;6!87GCGpFVFvyTP9nn*Y?DeYn(80Fs|+ z;f2ktTuMn0z_0GQr#`n7m7#$GlEcmLSgx#g74aZgZeA-5S#oaIvB7SEd2fHK+`ZI^ zHG!0kw%uiIg7M7*HxFyh!GHxF6A*!h)BQ{4H6=8hS<0F$~4iXNT#s(ZVZW zGP6l2Ka9KR$Z2jM5tpoOj74lMpK4>J&Aj<+-pRlA{98+|!P96GLSwD@%~U}#JX!fy zrcwqc)v@|-f>)oUu08HASuGxF!Qd-38FwhWXV+F(Bqi7F@q-r76uV07l3?P$&{*jhf zU@nfLoA~ag(~1kchuLFfk5-IxLO60`|04a|&+`nQAAIhyM>T&S;0WH6b^n>6Q@VHE z7VY)-&SYL)_@nU1UTOSyh3dc6?7X|rp`lHCMdKFekwjStQBiw)?~W@E`z9nTym1GG z{>^S@Jl>R<{ZBPIZ~u$uCfPya#?*hyq_i-3jDr{|2(%Nf9w4J zdI}}*kDC=Q@8OmAx!(RiI_X}Hv z#2koebuRi?9|!mGuP*EDVX%)f0PSpw6kLb)HUYk$-um+R%x(B>=e5+X;{%mmW2gbb zUg+3_HdT27sxBCZSlYu@Ua%+-SjR-KP3Le5CuZuN#*zng%pFLG&21fv?x!&S^(OJ)AdakMx$}bfOF)hfP$6|;3{|r?`aS`H$kgdjG^TzFwi=c% z^@_KT02trvt1LR#B#j#QW|5v6m$)_T{1OR}q==+Fica)Oo1ivU>cOt#5?iHASI?hY(T5X+K{7K>#6lc)=f>sLsOBJ;CAsp*s-Jq4JGKi8 zJ)^3Q@L;#i`wI2Rf4z|BW~JjylD#IT1;Q!ea~$;b_s{vxkO#vSWnZ%8yO^`2qNg=H zhZgh|tzT<;~7~Vi?h)$;{Hm!^KSh zXb1vIxv;T$ps@7*_q4j7*zohKcF#`n3aHrvx2(*3W9D(GNwb!ybBkAx(ez zZPzTwg`}X_d*3_P-P{1Ai!S)|NgdF0|CzfJk$tsS#Ds(mg&Y@wYb@-{z@Nf`2Kt+% z9}?Ap-|!Wbyk5UtvX^r<-m&=ELKXWygvGv`u6)MWM(vi5@w89*LxdpMxhC!;Yc>mw z%>W})pdnXd@9OJ90Q9S*8grDRT(flnzT#_zcHZFA@mFExu$`>@Z8pi}XT0zY#H#EJ z^TgxzYFV+n*K^3HM|LWy9FrJLv$}?QJ{5d_VQ|c*iZ=Q7$%{PtW<`Y@wZ(pO)b7(- z6%?%RRJavEU1{-?uoJ4YTGL6>ge>|IZZLWYNce5W+(A6iwaXq>ATG&_4o%mf0|B%{Rax6JG$9!on=#)iJ1a zL}XV~mHS7 z>I+ozEHF%L_(}TIu-WP-hTv{T5r?tNIzsx-v|(v7UDpQC8$B(Rb z^OCl2NJa|P^?w#%F@&s2>g1*AYXffOIUl`$?VJ}KGISztBpW4q&7=y$S<-3&ud^R< zyQ2p)9=ojFDHGIKvH0=MNy!Mv#R-V@IFG-yMPk?2uYYESE{!aFi-7&m7vYv_Oh-9ul%umu`~C zG*uXR%<#FvTphGGgPS-7;zZn4H&*K|%mH>pI3uII)(^4{Z>}hPD3`>8`rjGoFMn@z z^CTF~2N5dhk{e7F&ija;FjV_GW-($Hs)CDcy3i-mN~ql>O{)0QzC%>z=0;tXj@ zh<0caj-lMvHL?;UDX(I6Pb7yj=>KF*0L0D&EVo!sYPr4koQrXDK|iklm)^^Ux3?O$ z?m1XC2jEthJxd0Vv(^Bhjw6kI2!s>NI`G4jLRsHuFpfP2`efW{h@4r7m2&yFTSgUe z@S#$ZJm0y9s1DhvI7w~@x962)j|hXHE6gosUUq*S@8K2+zVk=4*f_428UE+&JMYBo#U zPi3Iv9&?i>;#&~>#$xHync(n1Q+N52q)R~u^x8bLJ~VCaRK#qO4KE0U@Bb}QITWHI zg1@Hj;78RTJiqK4WMCL`R^XL^@#<4TOu#8AsNUDUjAg=%#^!s)yX$~{dREezH0oH| zwHeMj)VsaVQ=(vm6v#adUO7B(s`||_uXQ<)&i!B$Z#it!*=|SsB0hDf^*9ecc&ro- z2ba^d5nim)>Z2fqHu|{h(s^lN!YYM$lJz*6;YhzlJNdWYA9xeIzhYdH$4r|pCJLTN&bVtU3N`Opc)nr4c`H`^t^RBw=CI z;8kKo^s0lqFgPylNstX|YMO(4&|D>&tMF#?eJv1o_RZs^;HJV@{R`T=OI;6bQ27>T zOXP1@BFW{KJ!$vgc9p-3c59Z){&t@sSK-Ue@;W7xA=?~bsumH`f{mN6y1d&_?p zRF!}9eO6zdM#7Gt_JAQdEoIN8Rp^3&Gx?txuU`9v|mA9ZgL-78A>712lO zlei{zjD0YK&35V%tjH}vDT!YzLD%)*rzY-I_m78OhplUd-I_Cl0%`B!&o~SZjw!N> zlWWJYU*;4O*xjZ!qoPo|_J`;BEiEOGo&dd@MEM~%k;NxZxNmZ@`VQ>;2zcw?L&_Gz z4_hX@j?lkN&WpD2yz>6;)9aE5oSde%s`|OIR0yX&vd8lPe%jRT-E3D_nTz}Gzntwy!Fhx0!PqgR-pHDc%S+rt@ti+u}Aq? zEL6ErlspN|lJ<3h0Rud$n*R05u%$17H6u6Jv-< zxA?jg;V9-s97yFkW0s5yth-zpR^em!`8?eAHFgdNR4~Cj=8#THm}!CgLBCfE{aD|j zpcmEsmgV4PHffC|O89uO>@9&q5X3|@Oik0gTR>}PuAYsN{U9pRtQCS;s8dz4aFR0f zZ6+SLt3+uyA^ECvRKbXVRx`hdQQZJKFzY*&>tIzfXaJxZT7T&o}Wh(94Sk z&*54{HLMh$)ZF!#V@LH^)u)!5u^oyLdQaX*M87#*Ii~r&ah7q@fqNtt5}d)hdu$IJs0U*8v3IH+$Mi(^edx@m1sDI+qpS{Y(`ftY!$NQ@{a) z@dl9eE1jX?y6V{0qJx($Jv~=Qx43;XIMXc*^;}nDAM}0)%YEBxF;1sMdrHIaH#}XI ziffG#inLG2zvaJ9c#=!Aq2l_S|xpU1r6WHO2DoB=T$Q zoy>V8k2R`kEKXePoGEL9oSbF5{nC;bt&Hv6Y!4f2ny_-yOi?j!ygY9 zLz|3DoO3=`R!mJaHxK6X>&0L;86^E5i?}aIOrFU6aOw1I{ybliwcQLPwLzdCjH!Ai+SvSP2T@& fdFcPgz}wntD0XnwpB<`y&DhYuL?5U7;N^b+3p_1y literal 25379 zcmce71yEaC)GoKwixensZE!CX*S3TL#S6i$P~3_JPg@EEcPL(@KyeQcAjKVuO9&K~ zK=9!3(tCTK%=~ZOdo%z0Gn1T?J?osk_u6Z(^)1;U>Z~6l=j<@LgXjYTdni7rCJJ3-_1WO-|QM)5+2e{MN++&&tus!Gg!t+{MDe z(bd|?Z3iDAjfeLbPZ9V+%QI~Y1$l3Disfk zkUaZ7@L}K^3FU-kvS?dJMzzP+3Z;-XX5qLg?)Eum;itlF?F{R}Gw9z~yIjp7{>U&lIJie7O{H=01E*XX3yVg?U<2VJGb*MLSLv-|GEE4E@tuP_Jj6v{J$jnF`BpkG+WF4=MtCb zb%zDp();)@-P7hoMiB{#$2NNBPM^LHHc{?i^C@&$a(KAKi-5QAyuKRFI(v8P-!UcL zIp)CsGpK6)pMJgAnP!u1N)nQB)X9&zXHSOL7`MgZ^Py0vDr|T2-=_q+Ke_Y*8ur>N z2A?)R*C42VX`$QOjCs3yXM8vKQ+h#OW!THv}i?;K;mUusUWUS0?91&wWnW3c7-~_!cW@2hkw=Znka*NREUel~OM@G(ial~W zKD?}KEmW{B|2{xZPO|fX>oH)Wass>BxjKQtGIa!YSg_RJcE30O_)m|%J>2gxVYzO! zX*Ch5G1fmtBn1Vx7~BGZ!0gTH@QS&LKAVXU=sU{zRTzQCt+BUBI`{2IAB1_|o-Kra zIiSyb|CEZtJfm%7aqqJszH!A$3b&=)jl%p5Y563f$;4EvJ}#cBR5F$9f7k6x@V9=B zuy7(F5kU|51qlg>Tv-$>qQ&$}+J|Xt0q9L2VQ>y$6tc)i7g_)S%q610u=dis5kc7i zJ-OgWZ(>0=IDX8Bu5q;Uk%y2Fm4k(RR3d9y`oD7jR75FR6I-LB1NH=Uys1NEi}Ha3 zMoj#{0S0cPRUPJH3l(x%&vk+Arh@C16qY~HFmL%lNd*%!ukPVyRsd_!SGUG~4LVXlm>EHoJIo><2t*&-I@@5i`F*R}cUiD*P zO2WFFJYH);uy$gVRu0(8Nyho7uODW5t>^LXwDuGhKXV$8h3GaS)O*N~>2sjvpnaCe z=Z4+~-r=`}9=(wRRwRPN7nOBljC3_-jpi2IMG_vlrB;n=I6VfK2_~&l0SScVYiH0Q zpJ`r)dbhts1Y)R+5#FdrgySlQ;Efd#=B84H0v6ZkX+zhzs^u)gvI?^_DfOT-8Yv)!XSN zCFyY)%d|)7SoaMFHt{AQX@&J+l#Dhq3-9J0{1HH1L6I?cY2We_%8k;0b(85W-QM&6 zuC$6C{8#I;|5!_or?}}nmIi_5z1GsBWUh2U<8G4{&g`KseLS#P|MTt8>|8kSKeN(} z9#3j)4UOycHD_Io2Z1iPogk^{=}n6Q<&3_&FJfyZ%S^6uZ9ZM};NoJDTYo?GLD#&e z$jSd86+KY`-|M6uHKO-b^utbDExZ@yIqg*L=Rh0`N=#2r&!0`eMvP+QuzOvMko_LE zoyLQ~W5o5P+vGnl!aFz|l-Th_8_es&J+@PX>>TSBf@mGanXmE#m&dNNg74|k4*fy2yA;B*ONQCQKv|~;l30hQfdIh zAg<7eL@z}039VlyZiu$q(aM$oh-~@D?;+bw6zi3oauq)one)uaX79^%>TgdI_=h>s zJ%mA2h?ZEQlT{r`x;|6-+Vx;f})WRzHS>Kn)*Z;7F`xC(c@GoeBK>pSSHNgtVec}-{5ZI0>hq{8n zLkfx(`mRY3n#+LV>v_A^S7&(@*G`vMeUg}{tECR6bcl=O-t5rC)xzupKI!XyktnG> zSP^#--44+Te*@vIZ?uUOtp6}4jLVrdH0@3_|C>R-EOx+?`E@OTtE`Q=UDK!}R?W?= zpnhbd97!*?5+U0X;_&KD3Rn6q9N*+My!r}*?XiK9uaQV_C^Ri}f>S)&la&EAS;PG3$GJyDDbasYHmfh|) zAd6n-K@81_P2xqslSz~v{*5h|{a}fULi7)#NPXB%c;9wH#CC!lk-WzCb5RdWSe?ad z&$_C--0rt9>|Z9f9In0fEEe@S^za?7?}bh~)HDncp)G0f zCqDLbx5np@E3RYJ9<>$E6+59jc}L(cUDxN>vm@AvKU#B5`>>MM zX5L_&XYg!LL=~p3CrOw;h++bqfwAX~o!i7WKbGzsa11eiqKHRTk}*&s~^dDva`waP>ule*U& zf2Kc=N}L*)woE-xcgQ*f6Wp{2U>f@6?3Dvxl5O)g$ml`uhjw>I8`u-|&Dxek8ehiw z9-@T>1s(MS__x7~KjS4|Iv zTa2N`g>1_T!885S)3kJ;^RQfmo#gjoDtZfxTJCfzm-N%b@<~e4m3LNrjfTXAp>1$HBqgxnJx& z)de{@S$nN!Bdi&va~~W`{6YrO9BLCV^CtG=MS#630|0KPC49TJ>gIR%d61G=W+e^* z4Cq<7YAXtgE^hAF+DWLFS~h83DvBFX8%pC342R zw*sF?;~F4>dKAanbi{h9r0=Ndzn&Eg%x!G+-I|}6mT6)=TT|u+ifHrj}pC#zz8si?(Imq?ugq7bW?knFzVZ8w|*s_HLCzl-adSoi< zMzmpJ+P(rAIE~e?W#I? zZlY9>3w~1u$-!4X*iVYG_;=0jJ6XcP#Z4d*5`LMhrQysm#8HuqI2~+`84{ubfE-fR za~OdaRr3ogpfwZf}fxK~8h9i%#_Z`%QGUMNq=%xyJrV%nipL1!% z+u>E+)Lb~GXA0}KevEmB*0YRz^6Nn5k>re#vDUb({>?9$T4ENi&MO`pCA=lJTNqj@ znoYE0aj`w^0FOKCg@%-ov10SvuN8Tt+qCrbn9`yha(QR}M4VLf@F7tWU-ca9Je&+WMDKlng#wQwWIQUfIqj<4WrMRUXtU=ajGcnu^}!D!52TMtJ_MDmuQU zhpZQnT}4ir9F$5dNE{v>8F@H9Swj#QA=b0s8LGi1J^2Bdw0nF+n()25`?oD{P6@`!6FC{ql1YEX{z z>jm(tEC*(!W8Z7tsC5MsSD#F$jXC!6>~9wl&D`>rJ!BTNFYyQ5$9;|`YU(>G5Jy0b zSjqB1?NR+5b$HQhjL+A~XZA8<-rf8e^@3~D;Q(#89I)LC#%WeL-S1jjU3ujs+*>AL zj5)qqh=SDZukIxUKkB-~8pt_bruu9%HXXJ0A@(m@j(4{CGA`GhFsCguVw8wRw|dZk ze0uNmHq1E&iCX5@5uBuCiL?g9I)G<#5v7%0?9Ok*{FB8ER)z}+=kieCd3O_=fXPZk zy>00;y?+1_cUF6H!OaaKS&i{5-wjU0`JG$DTF0C$n8kf0T zN!*g4V~I+*+J>=eA0kd#WiDr~_f)1KmynB-u48>iqqF;JXTJi7V~gp}BczMQQ}$8+ z;F-tf^(YMDtO%iB#X3%$@R!og*%A8sH1oQ?|Fkn<=8(Kh&(}YV{O=XT z!}}2UTI~1_@WjJg>i!?jJRqOo56#cef59K#b1s+OKdpEr0{r>DI7I1>LyQ}x9kF$% ztEuF2ieR392%bW4=y7Q+6Ow_^#{&RxnAzmYcPYhUE|cPRzyjoI4N_TEHEI-d>baH< zo^=Fg*TCle&v9V9GVSOOcwH2rB7j^hF(MAqcXHu?4>to^R)$0@7yWvFlQLePbUlCk zwa~T!Eg+&u+AA{edk{oO>)3p{igZHwCENW0=5lW5+ps$UEeT&5TWzxoZ680pdpqE7 zOnV+rE?2)4DKUrh2~3~XEqdpT+eW|*HR25%>sPQyCo9Z9e$EGS2RXK~a@+6~+MT*3 zqSa=qeD>adF#De<12kZnyt!Y%`OgD*c+V~W&;0v;V)lPbY^&1CQ)+@>Ah-Df)>svhvJ z`p1fSV7}SCqN(r!)m=nzn0^iTRxW z3PS|WWjKg%@|6XCYWd}2>8v^H?RzjF!lx$nFNLecQ>sL-hO9?o7OU_-jUU2mEC*67 zu2>}BbpA_M@*mIieT*jEC#q ziLmDU3C+9l`|`;nHd$c1E?ToQyZ&T{vH22B3|YE*igT#SKl$yp+`)N(Ey0YWig?AA z?c&xz&5y4H$$wucs^j|OZMrN}8n~kGwMwmk$Lr<71KCt3X9$;+a{Lm&^zXlSrkD5$ z)mBe9$(<}^YE2>;eXr^-HW%-puVDt^Xvf#!h^LxAQPU%HXc}xFpfFn`MP!*A!|6Ix z8B_%|Lp8tl zh{~23=?lTkCC@kfR2cZlGq=9?rlVD(V+#v$llTb5fxB}U&RHU)2>2s5Tw zLg(HM4!Gi?5w2YlJ85U}iBE-cek^omqm2Hl!1&}eG5at$y**<{;=DmHK6Q*O+MDF? zlE12{e;lwRu=>`bv%wU6iXy~p^gZgZ7BrxP7joY3SDJkt(1xPlmcnOnVrMoJ)U%IV zH2QXnmf*Wd{aqhbBpwG!#p3U1BP7GTh=s29o7%{No49;P-?Y{nBTZ)hC8pO`JIZ0+ zrG~RldOg)DwtR9l+f137O4v~v79`xoEY`#u-=pL@Z+{N#YxE!mKg$#ka~?D4Zs z=^VZH(k$zv#f7o5U%fO2JB`kv#-(!1X$i)q%i&p*w5oPIYe08m0=2_Q4VXmqM6W6T zZ^s;mpRcSk{aAmiyX#;0fS%2qS2v6&lY>w!LVhh`2+z~DvEB#ay7#m}#hT;lKhT@v z>e89gB}Q&_O&1?gghl}w&lNy&AX$GkNY_UcOAWqKy#a}$9i6U`iiHpX@(oe`sD)VS z>rVf>4{EXO!!DUOw)+;hM=FRUiU6F0v^m%{y?fAq)Z@)`p&4s-`O5J2!N2 z7qV)6qN9{5ftbh(D_po~g0(D#B3HyR**GqmMPAyvfN*FGOP z2IiIk6w(n*S0)g51t75XavVHa4=g@cg^8<}+qs9o)=na4MDG(_9t`^iK%s>}jVN~z z4@SXyx8Jn3_-P!28#%N{2IQT!yO;Lo*ke6Cv; z?#Sj9j1%$h#{g2obHfPGc+-_S#OGSNi!|aztS*M+r6mI;Ml+*j9`#d*6s+Qwo8LT( zLxE1yfBVHHwv06Aw_OV+DlGaEv(K78t`^4LIH!Q;6jaw*F^i69nqb_fU{ep$5(Z>7 z=RJbt8@hC+9=ztH^?9XzQZhWezIN@hFqG+kcDhUox-|RMnptL|b`56z^6m?;j_Fq4K5_psb5spSnK1N z{e)PA!T~oxjnQdMgGF2bV(=`hfO&%zK+3RD%ERL_wT=BWk}|lEbyh(d#5y^cWwNYW z;vMq=xlKkA;(hy-7D?E+xOr9k5oOHod#mlK1{}32*m_k-M@MGB7pGuPYwRYauYEaV zfA0&()j{U71*8sld^VMvc>17e_lzmu_cFdgl9+DONmF&;D|pck9A{%ZU=dREJQ^k@ z<`#-E=8~cDTEf`VY=Co{*6ee^svpizoYeeIb#HPehblK?V|ff}?VW4S-<^G0jj8hq z*YsQzeKw<4iQ_X*0;~Y)U(GFa5$wg4;O?fue+^=7`8rz=dGl%v^&+;DPG;wXthfJDM!lT@(D=~hqQNg<>Se;bcaEn0!aYmEVGMMw@?6 zU#>>BiZgmpJ3dCk!hdy=l=SoAJIZp!ch)$0CU9aS3pND0-LA^%wtPgQkyEHpINAgS z$_LNb{SMmgnWH2ht>x|O8Hr+sL*~;2Uxj#ePyJRSn&A{-i$}K`9SGASv7`rw2`$Z= z`{FvlN9K_o1(DwP87+yM(F<;HSKo$&AIbPatf@?VYPWAUWz038E<>rfiQpY>A+3!|0e`h(gviLF{g14qT}lYZ5Oul!vFC$fc5GeK&VS8_mR zNi7q7zs|ByXg0fT_YTHkYMVyfrLw}*EH+C4qM6Pf3W!ZlEmE3M@l8*Fr%IapsNcX> zS&`+BFdw{pnPk^@ZH9zddunQ``%w1!Q?F-^g7=BxjfGI%(6kCQ z(oK6IloY>#&3brG2f8ppfAI%t&vKr?WO?4(sYC%|o5rye^zP9hJyF*|i20UVPw;3J z`|TWKnHVK?CK6C5=eLc^!Ipht@(9M4*<_<;I&7V-0Y7|$*)?UlnE@ImV0TcG3HcIM zwxkBDC!(5HMQ-7ur3VZ#_#|blpq9L*$SjNB+G^n`p^?vfoYN9><*1_oHF_87{@U9G}l#8_KC<M*E6Rpp zY$f$=`tC=$O=29V2ndA*vD35x+?TV@!JA6Q8GCjG zL!@+cP-ypLGq$G8Z`*F8^j2cvu>GTZAvE^~dau3j7(OAn#QK#)+|mQQ1^bwrOqo3c z5@JQ)+E)Jf@;v@=zT`MEn6|GE-uQ8D^=ib!DptRfF#MTyXk1}M`H@?M3fpkj%sCbt z^b+$Vb4u9yq8w<&H7IhFdC$}P&MT$Yl`NuDGR2`^Y9+SCsgtMVu0jU&;kc;{HnFHU zLcj()r4!o$MNZun_lrO2<~I<{)~x(EwCHRs6bM#*<8~k3HnP0u{h8@u%fGb%cMRWJ zX-r8;O6QagZ4)I~2BRDiP~CmlkaElC8&5#*w)AG!@(4a!Q&Q!1$^HgS(-?i`Hv6_s+elYQYgf5PC0O87>|3wqR7I#Wr=F}IVpbbu4% z62_MoWSW(um>VoIhbc1Srvw^6)-YlMK`r5Wrfk@@goeo@zMce}j_%R0*kiW2Clyg8 z+YltFc7vMcAqDR!GNcGo%Hv+~0JnaT6n0d)@+b$M(rD=S-S!@RpTuyiR>BzJD^!#0 z3%HkW>a>w)&u}s-ZI&jx-PdBv`7I_)_qP1_m2ZwhJ8E`HWILb9t$uK(b7IgkpZ^!b zx{VY1Cgo0mKUV69=#iC9>cWjx^*X8f~^(*_f=iAYhOC@+yI7A=+gv!>p!5%h-wQ;^mhgv!kch8?v ziyr54kBaCuIRre=38l8WG+QDB_Fg~aC4_pFMW9)fTG0W@E&5g`q5?@21 z>qlpLIm!V7d*cDw&;kKx{Ig}|J&l<6%(Jd(xHQ(PBX2i-tGfb@Z*|GD4crAfQ-=qAmpKZB7It10cV76HiOHpZq}<-h-G(7jP{FS&3&RHAxie<=7z!s ze@HvpB`}oDQCfMJgYGmkEs_Y~^Ft_?2&dQ+5`xe)hHbpCK9S;3UAoBW<3Q>GJ%GQ4 zeAAPVCUY)vfuD>55?hwCy_YF$g-Kg}JdH)XrOJ++doo4kMI{Y`LKP7+zN}(} zRMIFCPPE$}FO`=zeQq|whGG#I_{1vs2VK>K6 zJg6M8udEm#u}l*#78Wk3qYXZlbK;WLTCLVoxe7Y(^^q0eAr_J1$X7_Co^~Aw)utwzngoHfvLHp@8hZT5^dJhtd}ydcr`U@-q{2K^h=HVDkYq zT>gi~d;up}Kruf*n3H9j6_*NP20SrQcxUfPe*YG$>eK<6QoIONm_|)Nhz)+XWl5f{ z=6V@nesnML73)M$erNsRX!Y+reDfYI$aS`nABT9hn^5L!fogC!IlVtLS}ZILlfcz# zROl@dELYMCZATXV!gx~ts^t9%;l4p7A8xM90!#HzNduL^emfLD` zWy?ag1r2QntWG$;n{QDOwrwPX?h{2w$eT0i_uHZ+m9c8X=L3cbR!O%Hmgu&&9QP{sAouT*LZnQ4Zino%ekl8 zGUi@o`gvdRDAV()H9{!VjIWfvJh{s4$^1pe7yji{Js|MtRyGB$Xa`0;OCRAtk1CCB zrI*ze7dL2bV+VYERwI-y+iWU|%X7fdDMsdqLUZAe_0DE|5Ys|AFb81fu}m$;{Bv|_ zizrmVGZ$Xb#su(|*SKJuQkYQT_Nw6BTp6a%(Lix`dKn)2Dkuc+a0G>mGUjkGqKq>CFYWIE+I& z1BnK1UB!o^P?F^P-Z;nSlPn6K|C!0RnWfpKWnT-iG@>Rb0x{2(J(Y&chq&PqYPAdH z(9gI>+stqDw2Ig!l9g<^ibJ6{z0owmt=UsGuCpl7zFJzj_W24}gO{nNpy;OS@z{>Tdz{DB72~8>Kff#`Rewt^ zi&TzhP8=%Y_6F5ilag&Np;QuAL~n@m^Ft!5qfjvgI4DGNV1H!RJb$lmLaQ2b*I7;3 z)d$>TFh=&C&VFeW!d}Fxc77rp%0lvH(aHz_RQ1ju^f2(1^@aE>nbd_Np<{lgCUpA8o&HYP5u;R6e=KH^pyj0-K=L)AEh{<;)bmhKtGRT zTQ2nZRr;z42naRi6JRi0o93&uHwW{EtUGx0g(oyDAHk9mrf<{CUX||O339O9tCxC` z{N)^A_y>8n%>8on{$*Bx2bmTe*&kjuBT=eR>W@kw$ic`h2dJY_!( z5D;eh1YH*0C(8{dh@3eyN&wMkL>J}EjPuvLtk^0n=^hVJ*uSKSJw9Bv?D)6>?=(?S zVd*I6nf}>FTULGZ&F7WRv0YOkt(wI7=AZBA&q$Jx#6aO23yjK+j^9O5dbY;PCyl-; zatFZ=uiLBL@Hb8w&g5vaPdY{3JR(UiDdsLi46f`rZt*x~^8fX!!C-hZlyfiF_TDqu zv&I9-4e59#&YZuII<=oRutF3(?FMLgm+ymn65#Z8kesF%w}iJXE|T!V1Ibkn-dTr^ z2FB5+Cl7FJ1opRD{4I4S!jBj^2SZNh4ZRPoD+Y2u8yYz@IkGri{RlP#h`d68ZeRmI+^oT!TJSz0Yg9OmelfJlf@T#7&w;T{SoXYJLc&_TETHQvvq?ozA=nX95Jy4kl#Q3FU& zFA*iEv&sp`9!0;KyKJ?uqLBl}&o+u4XE2vRY7c*15G+_|&uD)x|R=2?sH67Sho5;tn?b;|}S`M3*Sn z$hd>Ns#WA60PrgKnbYb@Mg7V8uvzMh&N-uwb~sbKMJebJx}PPu&>p9p)4n@VN9Juk z(o~tYoX6Sbny5>sN(s{WK@jHVk8b`TsWwBr6084Y8~D4kR=+rU{?iR<_`3Dlu+fr=>>;b2 ze@93hE~LQ^(Wg}PaI+L<+tx5^7TB$uRi8ZFZCNzQh11|^!(uWzjWdVi9V#ei2E=35 z-<$VfcoDkCO~b}7KjHH?pDc!{v&Y|yll_aD8JE7;{n!#s1+Hz@c$OSVZQ~(5yTFL( zV@V%xnw|@< zdJ~G?5tRFlr^|jUP`>FSPN`}?+FA45{pQg|DDuu?&tCfJ+q#O}Jcyv6uvM_TPS0x< z57XOb^n%&nxl2Hg%x(Tot?^j1C3B2M{w@aXdyMp|2GJlJP;Jr(5$4djFLAJ@>g-Br zyZ|1vxgc%p0<@6I)jkju{JMXmlo+p=wOtnvZ#Ar*?@x-*^p?SDJ>M6qf790gMdk8# zv1P_EN7Vk#Ki4ms*PKmBy4wv3OL)t&1d(Ns9qj@z5ZmyC%`ZuT9--e5A!_(mA z(iVT3{+Ei$|F~lHKluS5K^h-)C0D)J&die7T%UcPQZ<Pg*jSm`j%-DoZ-i0Gnj08PuD`!HELkspW3E)^COV&SJt`twf##!gTV`xz z2P}%!j!G-d3VZ}{E_+>0se!lq!Bh075{weQn^}tLC%gnKDY-?K*v!(r4=c=C9%=ZFra+!L`Np*Q#qWIOeNO#b8z~tgv zk*!9fr9vY$bQ|C51A%;WX&$U!_+3FCtIKIXaOvEFqR*@#PgiID`(lm5q74Qya#uKq7MqlUuev4L}Er=_w<5yz4ZYsnF; zD>DgIe9hH6Y*%?nnX~o}V|F{*qNLo7QEt~cYBCu)pD!o}n_b>dB9BFCBrC-Htwu0S zuzBqDl)&^xPSW*uC1-ZKeM}?rhOQ846a{IN6e$6L@MYcil_$7NOkh%mc+r*@b{Jm9wuViKUNbKr1cZZFiV`wyieX0sC|wwwHfK1K zjKwc5G`n0IGG-e;RfZ~MY7&`^2?SGSrvARj#({mncP~nuG?G9<_O+ zR|i;2EdDBzKPg$ufFSGgOSVb+bYazwb9I=tj^M~lzk;QhwB3$1;>;wU6!8o?wRHQr zvo;3{m?MUL<9c?^UeL(pk~fmxx#o1H7VFi^DHt zE9A33z8G8`5CZ@fqK(C@3=!@JtDbN{YaIVIshD;D>U+HDchKFkL}`I?$X4YcH1a(9 z~nti#-_Eob$0V>9(cDaGo8d3P$|;oq^$40tAC^QWxzcZi+Ykb z4|OhrNSb#y>$@YZGEZYbY%V)P{ufyD{8C)V`{Jh6EB*a?zL%7PM%uzo$cm*wsjZ-2UyQV?-4mCA9UT=knG{QEv9i!Z^+|_W(U^m0tINecJH6wc#9!>FFseDD zXtGjS)=|=j6hdEqY9;>lBZvm+t(8wBq<{?jSii!df+@CKulNGFP0v^ZN|sv5e)jhlMj_;&VnJn zW(I?Y=%x4Uxto(Esf`U!OGk^c)R`UnVqUr0J|R6axu>w)dJu(E<;y$1Tf1mytEUkL zd1gPAmYeLz@^jZ~eh3MT@kN^_(`S-OOeSc4Z@SM3d- z%uXZvv`6J=zG>~psEk0!P13TFH}>2|`v-E^?=JC3Vbt|j^(R68tKASOD=P5tJfVTg^Vx|H=*obg85!nlF*C?6 z6EA}Vpa>f-G+yY$$kBlAZin=CN;x3Eys+pP#t9XjvNb35$LLRWW6;90A?f3pNGUau z?GQt+qp5@QNGf=Sp5p1nw*ihjgPCyd)Xh%mhv>)>Sd+)q(SwDOB}5JEswFQo;!-uz z>}iUNGC!k1EQlj4q6XKBe_R{~y}l^w8Zt)iEk9GUPgfgydb!>sKeX+=QZE9Ql<;Fj zGx}n!Bqmg@ir8QA-rf85brQ|u!UG_M$#Qr%-2{-tBqN@Z-0xDURQA~}&i|QX9>%ue zf@yR`6|M0FK~7|^PnCisuAj&+qR!IFf18%YNQyU+QWWN<(Yu z)4M@oWz)(cwG%;)V?akMOC!Eul`4FM>m<1A-PG4$A;4g5*GBZSKjn;_8xt0Fu>p}0L1VMpjUl-xh zq$&m{T^F;Q^yBeNI%28)elncG*BLb0|BRDKclBSrE60Zvc9jg)Ke0#PfJvZ? z!^o40t*M66#d+RfS4qm1K`R$#M^!P6-_qAk&y!2z!#(lias83&-@s;?I{cO2vOY7X z0G2>@T_4C#b7W%>sT{Yguo15L(I}~EjydV&TM`GI&z%EmjkDL*>|Z(2;Xy@R9-L@p zG^u@xSzXckq*Jv%n?Hw75l4B|));y&I73(O;JyF!Z(B=*v$f&R6QrJmr9 zX5RjV5I!%BC#a1-;5Z-3glEs5dKygTTJ2nydY{e)s&7v6dRK^}bJXS`$b=-K%=`Uz zW#WA~YI8YhGFQDU3~Ko8J84o%1M$dWij}6mBr|WD^^w4$*eYl5+ntr>MOP!Iu4{9N4id3xYwNW? zlA^FLT3_C|tvJ9#KwA940_H#JJ?`VY7&`;$Ak#zGH1XneyA3Jb_w|mpz`#3v%I(nWzKaHff92%3_O3mz4z&!3Jc*~ z4yZiL|G;}3;-38$c&URQ%CSI+Pi=G0wfE}5TG=K{$_*i8HBu|@U#-ELDJ64U1S>c$ zp)~R9LqwaoPsnH1$wO8W*5>FCrCLlnbIqa~BP@`{>7VAt2(KYB37jJ3S6eMwD9Q*M z&E>wOOHY8rN;Y#GhJ{wP<RGn4Hi%kbROcC{^v%)&{+ch{Z@Xn4Nc;!UR-+Su_#%ox&yIOSTSn3+vKp)kg zHrUhPP-N)2Veg5qU1n<+MN#~)70!RrTKusEFhY9Uyyzv)awfe+%1QvNq9l(KCwH3O zZ|xSvI1q0?&2ttLQ2E644O!< z9w8YZ7Z4-KL+n`WwX~1*{Im_S7>@XM+%up*Pf_lYjPb)-v~hJAOC4Qk?#KZl$TJ6@ z6s%;m*98Y*+eE&JBm5vi$A7Q4WwP{pZ^MxRm~P$kFj$OVO2qNbm|mu5eb31(R~f=> zxdvKTYyeri9Fjg?-%{KZ!Aup(_}HL{>*49uLZAUQP?k>L)RcGK;f*AX6(8C@O)y`2ChQUGyY%#AcSEb6mL4;BJC5DL}$E5XozYP0bk z2Q-~y;PdBm$B7PL+}WEy#{=<9-Jkp)3&;O&igy2Xo&Vp;fhU+vE-~;O-v3c{oXjVu zdg_B4=ih$&=o6~7e`^8$?g8h+UtjJdxq|!s{!9$-e^>7Q|8dmZZPKlNAh|n+tC#u6 z^l7rb*`r%4RxjgH*}daBj16lL%#+dfpkA{aKM+`mDw1eyMlJ~bz@sakDf^#D3Fo#R zKtRj9jEM1Ua`gu45a+1BpA_%C<3gP7$lIG@D5Gv(mXL#g>~tFRn<2x$1W>;=^HpJ;Ba z$cOe%vIFiBD9i^c*8?uB4*8cZeCH{YvB5lMzv#89qsB@WXV*!Gm0K>nwAA4Tt*|Z~ z96Qu(NFLMU@k@u0LVh|+c95`Q*iYLvl>eak5&g!j!JMG2jCZB6P32Cyt!wCTtWG1M zRuF@U*{H701|oC)x77(%SR(Fn{N$ak=(FyN)+8c6YH()_fWhb2(u9pGA2F7=P~lGH zFKFoKC}~=hRODAY?%@nbG&9%psA1@xjJl-1S2sgzP)Ph`-kPWPb@`<|iH=g+hn!&& zgv`p86`g}ptBPxTi*Nej zTg}+MHXK}?;7o0aPPNj8qogaG9+Hq`_VK(j?_R8V5?Gpw{Sa2@t$JI?oqmJQ9 z!*pDnAmmpl@lF@FrTmceOqiaquUaGr+a9OdY0p!`OtiKg%3*M|^@^OPzeJOd+6&I*qa|mdw^qo3=G+x`Y@I~q&9g)uy z5R#lx2#xN$>g~|yv65XNX&P-(AeVEmb3=Jtep=CP0?JiP9lZ(r0RKta^+}9$Gqf*K z;o4r{6?oDS3`2~}Q17K#gQpD}YS_|+IJ=HADB<`7xleIKX^q!K7wE2oP-cD)e3=@<*?L0 z6OaCWEU`Lx7ZCWa{8>@W;W?deST)7+;-#L$(7jAYvf&#ldz3`P`AB!8`y0TKH)MPD zLL?;>9nZ?af7Di+P2m=f17Gyb3K}Le_ufve6wja06_E~rL`}?`R_s?^k>zsRL&fL@Pncrz}xL^1|Nff#|l0#C--2mDPmCQ$U z8W|nhgbA5oJ`@@n053U7h$t?fiBWTOT1}eXLebuR^sxowQ_F9EjR7%*DO)RZ_9eV{ zT_V3n-y@cahE_x<`34FV)+=Kc z*Xog-GgUth^7G8Z`lLFgTc?@}>d6wl)v>8ejrCuE&i8L{b2Mt$;?u-OGEzb4Ifsug z`tl44r3NpZsC0Xtxw`guui7=UF8&&|aw^8RL(aM~r|%8~z=>Fqswv5eVmHhewuE~@ z>aA1#0pfr}9cPm?==!A4sd_}&mO|r}vbtM?ab0o44Pq9`14D9ZG(5hrqSw=ucFHGm ztZC}$=dTreU7)VQ&Nl7WVe|-6@lHU8bWv1Po@va3oLuD(m-ni|@O^`s=O?tc)H?kq zb2fBU%$QO|GIcl-UZ!(1)8VGjCf$=+oy_sTlIW4_K~IVWQK`7@wdDev!IdInq)QQ<7YVw6U{5Y2?|7AC`XRR3S#Sh!^{Y65Io?27>Y5hk zmS_!}Bk5!I;2Ua;sn<9jKZS~Zn`UFIJGvLstxp+2`X+-ZBH6a_I6Cd;bQNTii#KML=i%1 zPr(CWfJvxM!2R^Q;T94J)%UR(QK$~{d7kbluV+q|2b8sou8@&l2MU(cxA@Qp3jOwv z49ZgqFDFA`)pk!b;>)N6NACJ~ewA71N8MagcMXaUIH2A7{FL+6pZbs)+3QZtRQr#o^|V|H+O^eMKOS0-u(h-o8CzN1CCuoFWDI58n&&C8 zmEv;ntwpXhRE*)+U3z_AP52Xkirt#^Ba-ZvWw<6A1cmfCthW@*t7a@9CG>RY50z4- zD4-mai}(HUljd9?I{`f8u^GB`^+dZBbFovdoyNLK!stG~UIK2bE!7b-)Ib`Qx z>-u#F>vMBEVnr@KU`0=@YP>?i0!%0r70CbN{OUu$b*R%{wOse;u zC0yOPIy;#wMt+y5bNAl97BFlN{V4^n59nOwae41-J zV1EbE^Zu+4EO3o;ir5d$$V_pqHE1k>cZ$>`<6|{bgb^!OFg-Whh0ohU(6R`n)F?!u zP^8UhsOt*qK;J&Eu37U$;$A@woQGz0ZFx?u5^wXzZpMkQMz*>ooxm)OEFSSFO5F3d zeB$9fI0fu|xf|mg)S#(!#Khs)W&<6HN-W~3Gy zD{JG5*p-6nZhV$QXWy`ai7r+lQCl#@12yp_;S9OPYxhsv(vf%%VQpXXR(SlN##CTD z>^^Qz<8)BfG9&3c&3rob)aP-R5=`jhPab~8T7Fbwk4fn54A+cnqGn|#U^%NwGtU~^ zTj|}wrseC6oR=dN`|Ycubav3gotZOy4)Bx(kBP-B#U;yoE~qQ;IkocYQs)V=sJ<|n zcY}H&rb}P?;=m}86wZD*5Ya|_{ea#otZ4|g7zcX->Oi)Xqm7T}g>S2&6s;@9I@>Xx zGj!-h`}B@eHD4$?J&S7mJ;_EO;E6%GfnX2AQU>;tA?qJ&4L29p*&vu8PZ5F00bTNo zPn;kl1X_VIG^QH(#2s}%V?0P`y*0d>L-o|W^NFjowt|QiuUvR?lNNhtPLEwSVSYi& zQh6dHT{L6y8_G&rE)frQHhRd6?oEvDtoh>Da|-ClQCU;kIH9j@8}Cfj301mFmVcL} zUuJ?{s=N64y6LrrvT{$ozKL%3>Jr~7H@SqWDP34_n=#VEq*P5hI-#lceRzf8Z?YMz zh?ORX@(#cyzLx#{?KU#|s9KE>rwJ>;LU^WuOtw_Yxyj{@GHI(I#irxQDMUy~^1@fz zWsMxmWeoOx??9^XkLsD2e&>{w_uL3G6B1{!@Rfi!>YUkVeN<4j zb^nK_I>(wWx2- zPpyB(ZaP-BeOh|U-o3#6>v$e=CO^ET{@SB_O)*)xVp)GAiGe&5LD1elKNEu>mB&Qk zzc0AaE-5HpN3EL)DY*F$^u>qMA;{6i@Q$Ty+VAE0k3SDL@9krvw|B_P24C~^MH}?as}=SnmqD2@J=rb7=Xat=X+{G~%+BRGpTC5B z@t`w>4t_KfBQ}3=ZC~ytGDNJIP}PoRtBDT;2$YS*?Y56Ot$gbGz&pxcF)%wXx7oP- zXx*JeI{aj#sQ9^VEsCyYE7czVcqu(%jcuh7VIO5+1n@$PmeS6}-O%?iv3e5%gVlv$o~wvh$^Q-&%3@#ZBA|5mT>k zORumX-Ln;`CrxQfYZKW;m}fiZpq5(=q|`78Nz1b4ouKHWyswmxOH2 zN4KAth#KU}<(p?@bo$jZ1Ho_LR+tH*?-6Hg}DAfAbZ&GZxN>lrLDV{q^Dw@9r$;x$rKzKipk5f zmnGJp+(Y9IX^XXgQ~1fS$W0`sI>X&c0I}@XU=uQ%asoJzbh`KF!=D1Jnu6CmRgczL zV7slQrKJR`Rq0oG@?PX?m)?wzk2Bv=+C5^dR_bP+`kpiEuQ0T*mF;!@R9L_A@xkW7 zb(T|J@+%&v?KqS+y7&f_cdUfiEoK`eQ*FmO;5>g$PgXuQ`kPF5vNXi@?+dBO|CaxM zAte95i}t^MwFn==>KaeT+=2`v|SZlXsP~dp7I5_G-uxLJq;gq`feG~iN zorCY2m_4ek0*_B!~{>V2*ql* zP3GudZ#_gX_;l)wU16?mV(5zWC}ziH9SvyweimB9S-23r#JOVmbE%!WnAuAO7YC{C zXAWuf(x|9a)BuDL+Ns4+NyE|Pi^Q6>b~BaaeZ7q)V(FomMh zRWCX4vEqn6jqrWyT34G#dKIv3wLHK(fQR1SGG+A$=!aKFOUelV092pPSX32^8vi(t zvZgK#*_8QH&ix5ypBj$FKfNLQd13ep|u$q#`UMX4kG|RaD`P`KMpIO3)dxnK`U$o=Bufd)q z&PmnFXm`NK@^)nYQD|%Ns`mTMxn{F_Mej?FwmU*wX@Nw!3jomM)4y0GbM%EPcQc+V z?X2!B9n%k1CUtC@yNiw3)OUDodnXe)EAke)&vp)7R53%*)0_Fa>A3g1K1c3jmzUzK z?$>OX8KtQ_pLr$eZPM$NeD6Iyp&R@v;q3&|@H2#hW z#Z2mL-TSvpr8=FF8WDUskIUR8%5d(()Yz?^AbFvx=9;8fSt6B5<4iRPkGIfsQn3HCv;_#ATRoXD7>_<}Tj->jj?30o^ifHaeI@Q@uRC+2RPNeiS*`@C=M1H^r*^{WioHOJZ_=H z0dc$}2s;F)fEeAms-tPx2CO(hnh>JEb+}bAC>%?rDdTz8T=(#<#3QVtZ86>Y=0vIS z^%oyNL9P|!0+rbJ{c*AE>iMsV@FSG@5s!=&+C2HUbs`(vv=n%F%eC%yBKI?sMOnfa zzAVa7Ju#1G`#jRPOx-BS#)*|MOCRm`kdQRI2JeW?sbj9x8nU8iXHP8hkf+m!%k8U@ zy}-XXi;%Am+&_3>5is>K(O0ae?|P*UR~nnl11Tvg^EoWWx#q#!hnCp3ixqcLxJas- zXe}U61YRWYgh*iy7<>OFY=YecSy55#<^}sC-_&U?EzI&CmHPv=&`gh4_3$u;!%M3| zu&a$H_y=^f%&N97MY}5|GSC~^XBLFC2qx)RXN-m)9JCyq%`^=s=qUkkZ*eecQ6M_1 zjP9}Vb{5hDFf4?gZJD>5E|*bV^(qnWV3CbXlreX5M+NdnmXwyV_r2^2_G&6#7#JHa zYA+%2r(Ngr#n2yv?+CIjY!x6|mV^q_rKMA~so4=r83pczfljw;g-X>5%-FjNPr@sw zTya;TmbO>bRsKxO|HyEAZFjcutmI_x%ydjA^T_0O&kD}aBPv zrB&6RVlG#t=Dp@tQjsk=EVr-7*WC>+FZJiV@zi2 zu7x)!ph`n+IpZKfufn+ZaOD7#YuytarHwmmygYcY)DS?yP5K<@U6c%~QXm~njn4LF zqR=oqDD?%wd@74k=Siy7*tj@hAr_76VoqZ*TCKogyj7*+vXENg&&!Upd0_{HwS{A= zW2CGy{IlQSd~e&hZJP3<$|t}!!&#~8inLU6Z+NS@+f~+B%+u^QY-a~&6~*@hLCH7{Sc*&$@BQQH{3R{a zk8isx=$OVJ&j*{uS&S!MI}11u5_+$Yq&i^^-R2#L>e}i=hwBiby9vZ*#(b)+wbJy# zK#L$%(kv{}+s89ZoIcQEtGC-+nZtebJ&Oj*4pOe2T0lTL2XjeOh1vZ^#ZSLDS$btv zJBhKa@|a0=`{2bi3oZPTeNaWh4UVC5y%t>pHF*z8PYMl88eaE8<=ls?!6{lq>FDNQ zb);~Dq4iUUo2EocHdCh7Y)^4*G4hwOZ#vnP?_!p{;Ygi?3B&Hmjp;J6B9j{7nN_%G zooF9WZ%}`Ge{NU2&blx;#!bLs%50?c#8JG&3WoBOjnzE6HFWVmy6Y3Px9FjCXt&G4 z5^=otUGLjqPP3SvJKn{`P@Sb|D~nbL2UFNMjJS?Ne$;L!$lozYXFXnHQOebdN=I0tRyv@ zbztG$vmsnazj?~ack+6!mt%vxk^7oAFW8fwKA|3OQ_k6;5t&zY7nDi-EzzbnqOb^` zzu!tbES)Tt(}2EwVt-EA1aMMKgNu`s^n+ww0eOnz%7Pj+@za(Mm{7y_LfK*UTA)lT z%_5xpckDFNKyjIwcp{I^otqIhm=ciF{^QEI+6d;x^c&piZZU)kMQWObeG(T)R~!4W z8S}tT$UdClo@QK&bZHykE}hLdP2sOY`mJ9rv=>Qy_L^}yr9}W8AkCpiE!TxU2>FQw zTlU*=6P-8CGvpz#AkV%vf2p>i?Z?@kRE^3j;)iSBh6}&AG{zr14;%x%!I=+)0~&Mn z!}X$9Z$iK|``e0Xh@5YWB0%Td2WgD@`%d8}9-wKUNQEzL@(38l|4n~MPUZR6eFV01 z;AWUrIdqYy*#)=Pr~J++=NX$ueEh0b_~!8P?qH%~SauXWdVX^cG5EZWvKe2v{bRg# zIn&(fTs-g1`Gr_D%msni6PBUlp_Yr4+Fw3hi=CL@xe^I@(b1~rlw>oBC)rzd>w{%a z4nAChNNT`kFNB{oa>*h@95w&qsbz5uV$aD4#lxhS)leJy1oCi{Y!iN)JS(3v<>T31 zG|UE7hSCKt+{V#j$P7YtTkDl}ny$SlfQPlTPsVRwDenM*F~U>X%37cO(wLDG8sa2v$1A=?>8X=gIxW*wLKT#jat4vbpNrG%1S z^5-f8pg73wV+51;1h41Vf#I8|uHp8bT3hXa707Z@{9A{#y2Hu~&quCiI z*ij4HlH~K@bl4ioMU)ubwe{mguJfQLgg?i+{J~3v>hp!Jix#hzx{{DJDnmg(D7|S3 zJY%O$ok{%j9|4n4YVzT8)&w`mq+Si8VA}A}A?~6aPhrsR>}9UHEU{Z;I$U1&c`nzu$-sXO1mqQ&d(2;byZa9OXL?pD#t#Hv721S zitA&Oo*@S>OeRMZ3m%8l8YkcBAx;hl(#TUr)E85|JUNrWcAE~Jbt*pK-9s58fZt@q zr3bJnKY%K>&tj#xQ4mokdyceKrE=UG5CEt7qJAS|w~T-N>l>e{Ffa)D!GPf^r#G5& z3v>6!Y$uMWNiKn={R0>u+>{k=%9~}l;a2lDw=c(HRY$+14s7e&`c^6o=O0p&8ti)* z%9_wav)WHC;4~$j%5bDH@Z6+0~tp4B~bO0pb4ZRV_{Yi^NUs^l-m50 z;!|X|ov%B8=kb4J0{`zy3jcN9`~Orch{qS5B&4*z?BYWHWZ;>aF0kUU-OK+4+6qHM diff --git a/docs/Images/Snappy_intro.1.png b/docs/Images/Snappy_intro.1.png new file mode 100644 index 0000000000000000000000000000000000000000..03a290f745a8f0b43ebee3e95113eb16b945ca81 GIT binary patch literal 314872 zcmeFYg;$-)(l-hT5(rLkcXxM!6WrZhHty~OcXtTx?jGFT3GS|e%_f)3oHJ*>`<^@B zA8=n*vsUkZT7T75U0qcjCNC=v4~q>81_lN%DIuZ=1_rqb2KMgm1Jqjyj$`lx7#M7x zg|M)^q_8ldyo0T&g|!J7m_%4oGPL|B3-qDxi;*x^#&|3T6jBzyJx;&)f-r*+#vp@K zzi?z@dr0Y$uYPUBABjIyfyc5D=BJ{Ai;nXQ6X#Y$fTL0`^)RjLY%DHxmMvW_RqJ+N zdT)SKK;VR~H9uci)e-!hvL%pCPCv&)hZXLU`hl4X;;X->LDw2~MMOY=@Ur9a5>C?UcFpfO2fRUOki&i-Qe+)x zX_w?2Uuc^r=7T~#h`kWOc+&<@NWlf;3Ee%=d0>u(vvrpOhPH2zngo}Fv&MG_$HmZNxE!O; zVkHd%3;>JV2H$r|qcB33HC(cL&y0thJY2k%(e*8HQitlP*yxOUyHpb&zvs=Sp+fS{ z@)dnNG&8C{SVAES1xh|9ZSB)#g<=p@Zb$Hb^Zsekd&^S%^yRj%g7LVXc&|8p_fxX( zl~<1bF%NjQAQ*}t(%0BjaJT#*w)|soK9-+Wkn7)!kvH(Th+v$^V*p*THYNrZ&^>qV z8KBHdbC$yo`5T#2CYb zB7`45`NK{_wDo*Q1*g%6&h*2%f_f$V5EO`6gHrRESwh@Y7k)7iHx?1Lm%auu(Z7F- zk`qeSPbn8G6FQ}r&F-U9ApaGnL=Tc1fPbtSr%8l9|M)e)nkgC?- zE$L^_&u9_i<5*k?B2sz_meNG0c&98@!Dnez!i-qHk+BiO8w3aV)~J`*muR6pO!;-n z+E2(P(1pnk!8egNk|ESa^!{e~o)zxbPEfJavAeLnqTXYxu@16Yq}!xZ zrF*h5SVmhaG-@}rXnHo%TXcS-nb|nDxKs3i_SV+YS(DSM@D^#8@>Cp>rx2o0D^@rx zrB_JF%P47AYu0hqmXop#+w|z8#FE4wBWnlYc9nMn2RF!T!YU=}BwI)Dl$Pfw6RkX^8_bAI^ZJE0`$mI7 zrp7F;^|bZe_27=_?z!$&?)6Wnq8<^380Wh4tw-&_m7?jQNdrbwkW#*5+GDC?&8p<8 z*5wZ6o{OKX2u@`d=@xCRr2#0`2rIlbg{$A5-$i z5G>)COL&Y3_y`%elRRE74UV6=LPv>SCas*v%Z5C|$F9t8DF77X?`Dl3|@Xh%6J-9t=uB>l`+@Czq+??HM z+{E2d+y!oDt|f0uZ?PU5?{yB>4nnU#Uag%kP0j1qr8SP*`+EL@f5ije^Q-Zz^E>n> z_YdeROhIpRY$=lovlg(OX z`ML&i8-JUppFkW_^ef3Ki3v%H_@bCkd_$HD!jTl=9&KycaL{wyDbe8%- zr}dAzy#1!J+ms)A1}%mY;W#uZYDJYDHD6rNtQL+rrL~pUuUo1s2vy0fh_kWukOd|m zj4#c)cUtTp9K=vFQNvL~P_s}YP{rdj6COyT$lJ+NNMr2zT20nDsC=kGmH3OG3Kzf0 z7K#?K746RL&Q4u_ax3kKhMS0F!{YN&^5NHuKTd6BaMVU3EI}9QL+eA2BPO=CiHQ0x z@D;jFyKc0ezcO0e+au})doz8_N@meUizo9u0d&iK$NkXuaKC!$xQJ+N_ruPZq=CuD z2uQvmbT3$(ZWGK1 zIQX8ens%P|G+n+iaYitDFj=r(#Ih(yvz86R27e=oz4(%Bdse86f$O!)#*gzhpn9#B zz037hHkoJo!|WsanRmDCLKhAhJ6Y%>>SM&C)R*^CS_nsY%g$R?8V(wIt>W%R!5#xm z;W5x$(4L5`c$S-H!g%ok@cRN_>!1Xz5kDpf1#D_&Lm z%j0lA@+~S@>1latgMLnOx5#R9Xm)pW0fUyplA-EDNTM1B`x3Uk{oEVV+wp1jrQT8o zS_~eCusoYlPbyBKvwoJb${5T@$RML*qT@S!nQCn2Zp;QPyey^Pt{#ptQ0gqZ$GI_G z?#vgSNudH>nrg=+Q=r$>8U?ygy!Um*CB&B&{K}WApS0U&k2l?q+ive~IdE+=+ecbf zyv8n6=KyPKX1?{HbL$ixwK+T>58VdBMkQZ_C;c07bh}QSA6NMfI<>r|fF2j(>%P<7 z9eOu@xB7bqtNk1?b@(#;n=cTz-?mEE=i<#yrl@^6UON>GDOwq~*zADr%eO;;XT55^ zAAbGhPJ9fw$v0NFYH|5)x%W~sW&Sv3!Vs!^ttS(`|BLFE*??8SlG&>R@btm$?&ma4 zZ$@de=%w=JLPxXv4}N6c>oei!(2*F(9A$ny;P}t=C$f{VtG$!zmg>SEk=>0xh(BMA zsl6@3r%-B4zk$Kpb21_OHM4=q8-nG!ycbMNTj*yAp^O3V=6dGIUKoGNrxXCdf^4A( zk;0&3&HY*`aKPqfDc9k9hR-v9%>{)ms@8W7e=Ym&m7eZV3=UR>=4TsFS~?P_r1YaK z)&LnWm&{e(UmT_VQSfNYqAh*y-Rs;dI2+@8KMJDy=Qo`d#!f=R5ey8S?Ds#oq$26n zn>q_=p{(wtE+frtWDB4-Ft#-`p?3q=z10Q-<8|YH%K}WC3<%u-);5mZZhXXl)Zl*0 z|1M@ACj6s{lNBGax{N%bu&sj$Asam_JtHwcEFmEwuY<8Ex1xyHznZ`O;v+V9ahm)E2I_CX{YjNSO5e!B56jE&7m4{{U2Q{( zz`z8-Btz2bH~c9FmB(v51BQa)d(ihofmHb# zX(rP6Q$?-G2%Xje3(*LgM@Ryh;}v)0UxslQSfgo*Go*=iJ3HDF}UUEiKFaAnlTu#q)9e?2MKO@rn=}<)4#8L=W`r=zuh- z;JbfL$nTsQ=8z=(ApR-ozk@h3&jI5#CHudO%e`vx{-4JqWdwg4Q4S2hJ{}Av_MZ~o zdBgu@G?Y{@|4f)Y6V-o%;a@7D|C2S{_l9^=;1G8U^k_dRE)Z2`5{xi!{HZt^1ow*x zOkzawj!WIc2rK*+7fA-0lo9ZuCPzA>$XFDYw8v+gP&%CW-@~SozN-)7B`-b0(mFf3 zB7}kp1uP0cJ04C!nu%EN7&8Actp7KNHbhm!831tixGaMZpEe?NjvwcHzto|=6(xOc zQzOBe+IW=sRV0KdumAk>$Z_L7W+YV`#@@we-Wx@oUn&)l_D?8(F~(d{v>nw zx$qM+Bh?>7^c#Z^0@{iyC?KQLsv|vhIRCOwk#drKIVvSLKk(@ESyyH@F@>=XOdSOn zx5rC!l6(-!j44ak<<+QHqsDVP!3B?2nXGmCg0l9VDn9Yw0I;hiT;ACQ=NDpKCF zJ;gF2{t3A?ac&M{R)WkZ*-YVim^dp{8T*gOyNt0_6IF=)kA-vf^Ps=JGBlgO8f<4u zuEtnA6Pn2UAKcV~_O|8Q1K4V(J6eM&lYD0iKBOE!)YH9w->0O{bLC?a)%}yPB$5Pw zFkt7dWOZ7iCf~XwP>U5&_`{)4_GAHNw30Vi&BVp< z66F;B#eca-n1gbUm$SJatXt4C?Ux7*{u__a68HRAvM?A6kuJbgdD?Y!XprHW-tgDU z=8RTr-|So2Ht3+?_^B`u@wI*>j<5m=oUMadgV{&NWioX8hW*gpU2Dxcv%6 z%J%AhA?36x4T6sw7#@XB*zS4tPUV$G+PO~VfV=v$1w)64RP4jMiEaK&uRM4_w-;om zVhV5Ai4KhIgF8lV{SI|SEcZm?ELGLU^2&Vaqu`??u4fl~o>H~2s_F^BzQ)ZpP0d+4 zje=$>yTbdVV>G5n8;kC92IsAD+sz@a!%UK2{E>)LT3ZVh@CMljZFA4bL&A6YKjWtGlXol7vClM@R;ShhJ#}R4pF%BsX-!+Z*%v@8b zww%;oQd%OG^p*6cBo{6=heEg6@1#&nzglAKP2+GKXv=?%pQVxauqd_8@ybj{A7%;` zV_IH{$YwFK)AV`BIUWKsshD)9Qvj6}wui1}L`73GDwMc3#Sl)=%M$`~CZn(rs7=%o z*Vt{<5JTr$XXM0Gj3Qon^oZ}Ffo<28LsT8t2Ux>v%6puL{kJuC2FUy_GvaR@W*B>o z?zv@cc*8p+x>{!{YW7{%Qm=Lm&-(0(bq`;#+b`=5rZ1JogvG#%$rRmzhSjrRzH`0+ zdj|N=L-3@*aQ~3KJ$HnJZqm&DH6^#7C9JT2-8SD-EKq&AiYivSff5pqXekSqxWyEBFXP$BJ?Wr7`lt_^$-Ctep4Y`C&RX+VReovj#*fvz+P3X z*p{d^zf&WND<4kbW@`we1XNl+Pb-%^%Bz)^P(6}XHaGV%2z8P?Q>TaA->a@Uwos6s zfEM}=7gCPV?ioN|l#kDPG0a)$`kCvxMctg4PlOD{_Kx*FL?l0J3TudI-F*!|@iVu$ zl+;X|1LWC;9%07efhM(^Ys>)>lMQ`8C{bc!2btLP=b% ziSEsUF>&%I&U=EdhbnSot9_NV^6UB< zS4FO{e~W#SIubqLPGc!5RV>==QWJ%{B%}8P6M~1sBE~H?s{Y7cO`taK zl5({D4 zrYl~)ZK^kSniH+llAjZ3w9vxTGF?4rBDeKD$|r6DuoG^Bx>Y@Kvc8;~e7X!W*Di_p zu|GJ5)E9RBnM^jD`;Eqp{O8el<{FHD$UBx4FjJd_`oe=j29y&5a8=C{U#A6#43S)+ z%MR(Q_hU*V+~;l)m7gStHcPAuuHH7c%;C4vl9<=rGwYk;y~t9Fv${f)7QWQ)+qU`T z&0?fLK=%1P0^7WKrok$~Cb@I+`Hta)jY-fvctx1NQS=hguBE{sQG`k`kQnp13Za zQuLGuG2dn2*{pPQ{aWpQB+YJPDG$`h%m9h#ar7^LQ#CKqnLim6boP~Yy0B$IP=2n< z!mBO_Z(muudoDJ@s5&~_QZJNuD%UHOp~-S=eimhK4x?!+e6N=?v|UR?zPSQakS=?p zkfG?X9K~R0Drd1TLCa#XPF)}-w9lW%>W9zbz;9aqEYS}l0+|(6?qwOwik_epOd4;L z0nh#LA(5qtYIjko_#abzbTFB`_|;cr+7=pF(=eG6_cr%dYC~zaEUvjKQ8HfYSTS9j z`)rL(s^Bz}Epd}j!aPf?WP0%UGXusdrqnbpl_v$!a^?(@nK7!-$jz%lKM+bA>}e~t zK$9>Vyegj!E{3FRrBzlns#EeN`YPG84H-{4bd|(Cg*?-@eik%o?epTongdncJ1Z$1 zJ<=6(Vk1`M&RFsabgI?jeLf{%0q>empxmL?hI_2@+m3O_CTehu2IW1o7Ws{JP#ZH; z?{yW*wYUIBJVsdS4es8*WcD`;D&w%Mx+xTHrN!tBc&0rPlVa)m~f{?rsD{h}WvIRfvxB#ycRT}g2 zY9$+*8pDh9J4@I%@JU-qR$+Lv&pbpr2wGGe$rMH{f^`I`!|xIIWVdt%vp7lCteW}Ix=*!u_7 zu#n`~4)NfD*p6nz;3S~31ncOf)^jD)#B*8K+44`EH(^SKid9|U@y4Dgr?yk{>oZen zwWH@gNs4;%6fUiL#*P^+1sT{jCx)?$ zLjr_ny`v*R!%7>eRU_$pETd{CO1f*tBcG9Hf2=TfL3gQj!-qK`wymJ6o-?+B=))Ja zo6Ub*aNu?Ik$cNULl;F08fm3*#YkhX=(np*(vnr%3p~28#G%_?#0aDZ?2k&G0~z;P zOhz@btVAHnBV{RVWu_S|a!1f?Fc#XF$dP}=WutFDE=4p|=|)g|Vx1K&AAvCK93bSa zWsm34Z7s=YQ+=4)z+-8{zPqHV<vMlp2j9nDS9KPo6b4=CZJ>3rt$yNy=Gn5H*`n+sm&0N^W%r;y``(E|59KDdc;W8g`y4gM>oRgXS`U3Kq zwP_B{!?TV=ws~yFMC-K44)?3^Xf~mZc{!%W>t0n^@tZRCEV&z0F2wEB4I3O%o*g_z z;v4(^vw*&yro=hd!wRr$6JyWT&LJrfnViP&+c>#+2nqpwsklf(wIDgYLDi|P)*_}- zU0TOy3CgTklpPlQz8Pm&n84_sXH+>dhmuy~?(Hd#S4Oq&NlTNqB^2`fCYFng60ARh zfWjwAZX=RrQVB27CYk%sRv-X?ytz@;LPZG+hSujS*nHkQ}fj>a$e?{ zYUywp{G>deIBjI-@bQMQHWA=q>9MKah58qaL8zrm@r1VgjaTpfH`c3qu)sJYG|@7K z`YoPQi6Hn1yX|gLp&;J9Zm4TyrhFs6y-xGopE6&#UQKglEQ2-MnUAh=kNA?e@M-yl z0*x{&L%qMXP_3^(lDPzC;2lNwPn$E=o8#Lh0~v#8@=0F#fKl!zB~@o!pW{esV{h)K zdwj7P({k0%dSEzer2f>7pQ^hMZ$!h>3m!bh*$m44n#=GzcbFq!Z5?ruDkHzbvi^* z2^eo4*jvjfdv2SXcm(l(fpv-t(MRZ#ckZTV|Ekx;oEtIrs&6`!0}k=DF@LA~|MG>zQU;hp1k+P$vK%!Gii3~F|VUq+jW{y3$22;Gf6#C>h|1Cnex ziPK?m4iFr{e5cI*bo415lPp#vBL+R^rgJFQ`Q~fI2?Ys>n^N(O)+NHRVMQPr&Dfr6 z#omv=I-?HR?rM`hdcV8-dK;qMq>&>vF}g@5jRl6mg)!-jNQ{)(u(38Ses<$6pF|Xa zLzH4}eohduXMz8`1&?zmEB^O=RKE>b;x)>nZNV*l@$l^HjS+RL(RH6_%9vBHJ#5}$ z3gcJirvyPyxE$`FY9D6?sKfCY-I@=<{VDywsu2w(9nkcpmO4J#N8J@dAP#-x1?t{# zAB@w4bFMZbdT@yC$z4r=dW|RdymlA6%LNNh6T;y;sE4p?m2RqDV_~U*{Flul==K!>lh3l*euXr z=S;R5^u_!q=fh=qv2woU>zx!-hU(57kL>P%Sp&L?t}CK6tK?^|gEhk&U=|y`R@=xK z*T9mXO>>}5*?~F#19sDLDs3G--<%pV&ugR$he3IpryEFJCEhw0Uqq{5(fg#bb7=AB z1>A%J90FYKuL%}6;D+lou!#5a7u)8EH^ybxErPCv_gb3aY&^3@n~CXo%Rv=h&&9nx za~|GL$svNR3&nb#0m5vAv*_X^v#uUIaw8*C0DqA@_hYuYu4hidhMbe;K~89dvw{}K z#h|mmTCmA&!o9oBAVlwY)Y8Qr+u>Os9tb6ONhdwir%uKhO-SW`y2Zoq}~5bTH_i6^-sSoJJP?;LAhF}jFPsG4D^fPsVq3AVvIKz)%II8 zt2_l0rsy8FB7@uyT2){l&pc|_Mk&9LazW_{*B4oa)}C1LD0ph{Pm+(6Y5SNg(aV(S z-+shguNyDw*GyO--7@W*%9YvAO$2W2t`>fjabk?oY}m7OC%Q0C!DpmYB6ya>_O@vk z7-f4p(2JS4>Gkg~+j-!>f^ex?v$0!s{WWIPhL)t#a{DY4)dB+vHMLa|f>UGpln)aY zl*okzh|{*fIA?I*W>@^Q6XNUuMgQ%%hyBJxAs#M903a~SJzl({WO;uo+^(rK`us5 z0UMOn4@MxFB?J580m^iNr8`eO;+bO|DWi@_LA{Jv9b-X(wF-(-J+cLcIS_Hist6*5 zmXo80_3$)EB8CNZJ8=;D_dQ~xIIAg8n)SojZ%eR0)t8>}A#Txfp~iD0#+JN(^i~JD z9aNFxI5K}$42Jy!?5rJV7vZX1smmI})NIyA2dtCx-XV{cXDh@uY0w!iDYtWx^&;N#$c947qL;EYIAH{$a)=CAv5Jg38tpb1BVlTh+VlK z3<#w;pZ$wH5_&Z(RdN@LxuFT{N041_X1-_}j*VRS51j^AcW6K2Ba~xH^YUjnv8+aU z?z&z_;mB9@QZ!$1uUCfk9#Qc&sqlrp|9Pol@+_aB;96OdC=6rGZ=!)Z9$V^Ts5a3t z+dhLe->oo^%HyUl;9{*AqoU@{$lyzb&*`S^Xv=mW!7zB?mL=moQBkx~qR0MNE`1ka zZJm&MZd_m^8x1eE=6LDz$tXQ4NA?##qeGT{rOZW=>}oGWqmj!W$NVA$uFfM@rQ}4- zC>m|4pzRcQ(zC2NZ6{wv0->(fY@ zy@bO|D5_1zR)kx7AV_7|UdzjbY)=YL%cbhGhVjB4C4#S3@*zTkn1_n>Y)?+4|@PQ(${iEyE6kf_Oi!T%=QB+ z@pE>9DqSDQFZ9}9-60`|$0B@nj@2P)H{A&7hlA^_7bntA$T3-Nk`0WaFrbw$yO!HI zzUo*tMcVSPPCq~>lhiAhV)IA1k{M4PWD&6}6ed=*5^PUe)dEk~YIraSXev+9`l{BF zjnqW(o*y^(HU}4v&-H3qbj>{V##BM34&9@R>E`>7{S$Vp;DpMnuC9J4A~;B=?78c6Fj87_*bhGOtnqi5kx z7_mh6X*<*(^VPXn;=RyKv=@4-^p2ZcFoWuWH!&D&K8i1*CMNs33;1WD3Xgk|i7NEq z&zg4Klj3S>?t#0WQ|Q{yl0UzRtu|)tQWrThqpcBFhBGJl3l0wEbQS!x@rq5THhwdh zvaZo%DN7;6t*XZf#-7ZA`Or4kPZOxzS1x-W!p$m88KEol4dq{33p|R@`@qQIo$p_ z?~AW*q!@OB^%`~##kK4OK_Tv)=B@kV$y3_FLl@(F&3R|1R+_Xcb;_2DTM_}0{5L@bGt1{{26Hn4~1>5eAzNSj-!j{39FA2f|RHr8`YU=wm~0pFTt&~uAi?v{{t z!IM8$pIpwRQ}_A^eXqRj){hwHmBo|&0-3gkL#{3lZ8bqsw|kuo=b3iDks_3*kg7e_t;A{tIkjEj0x;3rp3t27cDf>y9<=f7Es zpkl6WMqMAWV&f}5A~LXVX7(VLsqy{d)vk)*xmJv!ALKSS>wPkx@;%N*xz4*;l&<>L zxF|s5%(5J{w)?34q&RO0|E-?~VJk6`YvCu!1StY_8H(+S$@B(%#!^XonjqA3*e=?L zLbfVx?#`z&DH!BIe?4H)BYi=4~dfBK(rC!qmO$zYM9w^CvTEuL1 zJEuJO`-Jfp2rtW2`LX{gDf87^e~TODaa8%NXFVQM>JRS}A^fR8rbaCoSi_Z)0%#8_0#)vmlC?<~hATs2)wuL_~d zBanN^Sh8hnTPVA=$rr9T;O^AcNO2>m;Dx4z{6_bE#^?cUk4YPU^J5S42DK}VMu?f7 zG%7-8Id4inzcJ4VMrbKR`Q(O8@!34xt&!Mu)TtAL=thiLddzkjfmL!niTtq)bJBj- zMEN+0V=>kd#f2(4VtIEfO&yzNM`keRl&L`EA(KG;o76MYi&&y)zaM+^WjOq6)9t7N z4ry0@>!}JZz9~wa+;7af_Uy#WQ)470{fDoRCE?9c$Wp&y=2cnUdQOiQ_+I@rY6!p` zKUjz0x;q0%a_cZEb33}dDg=N7I)!(eUEodRzM~d2#8iZzDi!u7FTx7p1p$tN)kxp5~?#ju<#z)bj zU}MVg)}u0TN~s=s46Yq=e|L6&HLT&i9G=W-Gf)=4Pp{}IU?Q%5)rY{^hsXk}_DkE_ zbz$d;fER^^`bI{u{-Pab>gDQLQ{jU@7c_;TYYRvcaL;A$@y)4_(Wy zMPH+)-_aB+wRI zq;?((YvcJs34b_K6}V6c)~c2@;qq)4B+L25mqmiR?6Cmvc9-fy=D`Z;DK%QIH=v@| zXk36zWAEA$PAv$7>21EgoCcUc1s2;MZGYKg&@^e!nzbo_J1Qz|o9Ho)#8RVS5S6i0 zK5Lm+-27Fw>h%t^-nHi;AX)m3#Er!XtZU%5#^sPdleaPpjx`*6;LT3fCB8q`IY5ra8Z%N zkA=aaa-QW^YHl)fi>?S7=@K6IO`kY?h?I*PpVYPTs41n}hTZs!W#PQd)?K`a34s=7 za@9vkO|$rxXuTfYC0CY!u|r#yk{wg6Z%YUlk35h8y3OiZ9$qSadP{`Z8JZ$iS{deX zyGG8fL-SCbjg?T@wO`}1#pb%ssjAVP;iK{k3niNr1eKFc&(;d)#wsQZO}=iD6R8>F zIF;WlMI<-HHoP|(fAdtu@hcMvwb8#_S?16x7LO9Y&bjH(%BEX3@EV@)mES}RCaca+ zbFt4$YCWbEM-*}pkc5f|2gl{CCw@_e#x@3uV>Pj|e_f@H=n@$GQTOMbFzw~lXvi=M zabE%TRpX2qp2PkWiRi4xN1m}+@shR$wHVsC*3M6Qx2?(Ldz_q_THc0S3LuT9V#gZk zO8BH4pCd*Px!Yb|o}*McSXa(^4=JuTcF-Gx6)^BDzD`CiT|B<|Qc72XfzP6T#eAwT z+b-SAm_ZXC9)G*8Btx9)SD3cZABXh1v zjUZNExDTVl>d7C3p{~MJQN``s^x)w1piFBDbS$nm7rB)}wKKcZNF;E~?hEKT*+(M_ z>MN^+b|a+iqMZ_FKX8zvR91b`9k3j5RS1m%e@HV-tY2{jn|hJTz|kv0$M`n*X}MuEgX-c%*vk9~4({_mJPZ?>}}{_}ji z@0?c&fAApT7eCL2F+I36eh0JDCN_x@zlZJTAj!Fo)JmM3BfS8lat{Yr1W^C%Dm+hv z;z4x_4gf?npM3hDG(#?hbh8!Un6-h+i+Zo?^2qPRPQLlLu5N+&j*idGPA2SHk=*b= zH);Z{Euh2aXELe#q)v(naW9!Ge^6Dj-+BY-z0o%0VY=yPQ`2w|C@r7{>$n_-xZ`q? zlwoJ0_hjR|L^dzas}bJ3B+iC(wcqnJ)g4~+Bf_e$3r)&o%QqH~5 ztwf-aTNUy|zH-A$r!e+`wb+ggfvu1op>R?5xH%5kW})#7W%Mpp!L1*!EhS&95q{kf z5TBBZZMEMpfg_a~gqZ>iJICJ7jtFi-wL*BvCb*-`K2U@}R3|o`U7M zP(YQ6+tj4nq4u{!$5@P`a6ySv8y9E2^Y@n3hQ&oCb$)*Ir}17D<&p3UN%_Mh04rIX=<|ZPl%>~NEW`JtP2^wQ3ZyAv16aj9PWOr z6u8urKU6CE?neAIkm4D+Vqt31Ji*;-LD7aGd$kK8hkB;16xK8&y2~CMa_nkaf&m+} zXp8if^g%n)kxqn*>$L%NB&f~duBy`Lu>&%yQXImcSEI-dl3y(@WQh!=9Y9n%P7NT0fXnl`-5sfqCk)X?Jw6iCn6B zV1qvEQvn&$vH912LEhHv4nIzrERJ|HfUS{|v=z?c)5pM9ZN{v>(jWHxM6!Z?6HA<% z#X3#@aixRRCkCHGe11oYM8U{whh5#}09)PpO725aAmBC0#VP*kZEk}M%g})&Dta4d zW~)m=&pI{*1(e-Z19X2MtIix;S*hQ*eXhUh=Axw4jigbd6}?{XkgWdvmLJhfcqi&AFWj*(TAC3q1V!8QboZL_|k zcm(*FDXmlCet^$&2s1S5KjPU4WHH<9exNOd(zjCr#D81rd+o7lv&2t=8Vb!6?(NFz02!ZLNG)sm5+2~}KRG0)Y@jB1AiH!YSJm#(^E_<)-B#W1FQ z+b*XrFMeM&^u8mSqE9c^t)0Qh0A=Ty;wNqQ8GZ(bGy#$k8Hu-Vs{xm>S*;b)~9NJ z`05iPkI=20+H(-1;7tcw-33O6UAgMN6rvrzT)rOyHbl6@qBEQRj=c1@5P9>|SkJ5gva49rERUj?mBvu6BUktyAtTWbcv}p zp|tT86|o8*@iyqUbe$#Yx#q^R9RGJf;qQw*qY{kWX=>W)O$7&xG|{{FLkO0cXE$WM z%MQ?4XG$e7rDf$1Au~>d7shJgt1d&0Rml?D{|c%5KVxY%-z?)+?!V(~Vx1wNkk>m^ z%2ftTwJ-0q)*8s(ij_hIe<<+g`}q17Lva+f`aaB;vaA*skBiGf1`5_czhhOhF@}PG#;eUqV z_823*NlfOFdg(V?ohQVzW{pwx3495&+QR$h!1lrA{vli8#$SS>sMiq8!?!VdT*vh=p1 zN*$7={J-NY$$G9m(%EUtMgO+QEm(gIA^a+dBKhC8!+%Jr;1EBV=dq@TD~SI#nZIWB zSH{otEu`5shK}sNEdH-bU@+rgZ&aT06Ej7V{BPlte}!YNlL(5pdsV}cj$ba2!XKt*F7UO!kpJJ1 zdkgXgj~b``FVy@SBrW28(Jqh6ip>9rZx0-ypg0iD+*md4A0g@Cdjq(AC^`P0@ihf| zL&Qvwq(1zw)&JMb#hdAAgRd!3`XBLSeE)_B|9k8-71Do%#O`+h@wQm+WNy1V!2|zB)QeXJ*4;=* zh~4(#Wq8{Rf62Hnp$eK)rP0V{`Dp)u&_ z1$RngwH3x)U3)>>UD2j5`g(X>&5vM0ZHf4LkQa0eLwHtdhFt)?QOdTM=B*X8Y_Z*K zQwT(yZ~6Jpy6Zke+=+Bp<<(W|Fp#0Z>6bl3`NxgRZ9mbKeb){Dq{34B^jKLI-rKDn zu$#k4nG}B#%AxaSAM^ppCZ6IZxR6L?Rg zwdc)aGp6#RqHAWW43C=5@!(po4v6f(DR#-o^0?=`f2lsR6w?QfR0)fy&$rKO#IQkY zFHl^yRz@&E3>$S`Y{c`n1=-s!+tpW?cZ}5qY7pt%39xu4)+lKBz)0Tx#{5cG(o*eq z7;YR;pHV_aGc_T1=-C(3wuTRI_h0n^tqn}o=J3U2MD|5?TACozyp2LCmRVctht48w z=kD9DgZKb~BgBP|)iOM)qm@;*sG2Zgwby0uA!ZvAf|?HfGIZppVRq=Aq{8gFvgw5bvPQLIG7Dh4hjraQ!%v)t`=~a=Q`k@YX_Kq*QyGB&dj; z^r4d(h6lPxilg2sg=2lsgC4aTknF`4&SY^I!wQs~zewKIAIeoIW;GFmSUK`6*nqCJOey=JvL1s2yGt`0%7Z_| zJLQ4PZfS~D?+J3KSaV&!;bv$NB$VNiH1~`yNek;z^ULVgEuVb=yZ9joR07}r^WoN; zUf07)@usP<+Ul#u{2`Beyx+Y&xFXjcw(2KAO*WA0(OvI4)le$_Y5qv0-MpL-o{C!V*}x`D80{acIJxGZVSie&#;%Ui$_&hLPw-_QB} zBr|zLPhnU0XCouj^mJukI=T1DSy+F6)nZ`z)`+uRS|Y4f0$*7P zFt<|T*P5Xov$d=cPg*w<@(s#%2k(5AfZOua@-8ax@?KE%F=Oc4X7p$iDye2#ltON{ z(#o1-^2YWB?Bm|4k}R)AZ@$(fuEkWzqgsyfK~*MxtyJSnwL|N&Ajjfzd+}_sXtoT# zauXtW7X_H919~D^mW5hG3GpD=ZLV=w4Mg%Xxf_xwDM?d47DygrjOLuV_J(*sys%xR zrEeH$|Jov&gQ0S6=bC%`_I*yEALO|ZUtVMC?$!NsJKC(KUw?EqI&e0+UpN+V+mLcS zVlhNBRpl^_LbfFMmUj+9yQVX-JhrqeMJfCS`#y@|tF3@Nvx#$>|9=jnB2xe3ZXN`y ze_v7(g!R95PS#Ah6OM#5Zp-(?4skU3dVWI}1_*Hnvsrlg%Ll+deATRG=i6dAivXxr zy%>^MX0%D0JT_t9E6+j;UBfrG2dH54C7q7yiRKPC{5X0D!M^3aA9?g&qAvSE;}+>Z z#n2}G2zP*xaU5tnR|@R6m3zdk%?$JaHg0<)pzoD)#^d{XbH($Q}*=@l2Tp@)o2V9U8ry^efDP?Mnee2?bq@zK1$P;SLlgUr{pb@_Q>Y zz9VnCCRHD>-p+KgHe{~Nbo8@w?XaJdbDGzMrx)TSvYpq0ch9e@Lvf=v=Wju4wM->R z@U1p{cl~sf(U(JcUl<8om)OhtW|R9T^kZqai~N+e$C;(b5H z=CprGLgbBBdJknE_rZiGt4>2oK~n`_-GCbdHIKr=;OT=g*aD7 zaTiRV-0e0!;4Xz>kKV{x`{`N^6?3|iJu zZeMfnV5&dh>t^NF2l(FrXe!O^Sq-rxlv`OC}#S;{I{2P926KPIQTtrHr@yTADQ90oKLydW3KYH zZWTm9mjWAA$Oa$ON0k*+cpZQHFzYtsxp4uK25N|&%Rj~2rDX?m- z9ft7K$XE<9t*N&;QOJ(0zaK)(S6po{)Qcc)ss+2fL0b$5sMq74@rQWxaOkVrbTzD* z06VC4GHts40w~r_nn_{c^VLLMoo!)qyHbm-a4+Tu4w=jL;;bK?rs6;4oLwerMtVk z1?d#&?(XjH2I=nZI>do?<%&BkBDq|sM*7?JA>d0bXi zQNMqd#1lqeRAbY*Ii4jsHQjuXt-ou^Hi-RiqQs|ATE}c0P1gPQE^R;yx=+y)1yt8_ zBsqHd(LvNbT)bX>?r5&eir@hK2Thjxq`AgT85Q630(Fu3V^(GEr|W%XkwA6-apEbu zx0oAdqR9TQivduPCZEe0W_REDapQ#*A9#Y4gt;wyzL{Ll!FhXTn;adZDuw2`tBojM z4WIdO2M}&>xRQ31gtrFdBh^$D9pQQJ%_PLWFh z)27Lm^Xl{MGQH1x6O2kmJ!XWgIt?mjqB2RC$m0pr%YexB2M_t;pXYFaVQG%3hrzMf zU1eg?7}rYdy9hz6Dt!a_!J`9hr$OO))crM2bhsaJoL&?o*2%BTqm_C9>XiTV0!U0u z{WuNC*>>Q{U%nM<^6RuJT~z1;s@Kxv`oAOH7AXmc%5P$F3kLt2Q0ZX(rR!KQe{?-z z2uADZPA27$-#;QE;-Wp8N@i1~x{A9NuGztzN7GW zdDjoBbSZb79-J39D@WE}ttP8UzAi{H8so|Pqx=vE%8;rau2*UMqSX8_O%1z<@k5{a zIByEW6i$qD$s_pm^oH;=|Elh{C^Hc)Iht#wz;*a5D%?IlnVK^iy(X;*K8cN1q)-PL zO>_;faNF3X?_3RdZN-@S5gNK0b(wG60R%mjNcYo1T7;LEb|05M3a~#jk)Z#lY%NB%I6MZyuN3y9a9=#B+!#%|bzN9^yn5ZlG&J8z zh}+{#(;LL9OqP{^_>704vT3eOR@Lt>?Q+RvyfrjhIuvL{i}mY4ar=M7zp#x#3si`u7~vTNpeG#Uz%`8u-Y z+35urKSwlq^)llYo_21j8K^(_Q6N{9cyV598s<=Lubll;eeEVs6V_4^Wi>&f`{}*B zcBE@-j=WZ3ym+0AgH<*%SjDPqfl51rPFqh++q|lF;-S4qVr!MF?k1Fcq#4`rWkLDI z3Jed+r<$`tWx%FMwv*&g^FwrB8x>VLh_qlOOjx_I8y2A7pSNT)vne_dzm#UNbj8X2 zAa*nOJz%)_xrxQAthz1(vE6e87YR04`^#W!JgR8}9yZpx4AHi*#>MSV)uVw+Xl(|w zG*opH6+z?wBw>%6JnII&%US%x({_QRkt0Q1NLHeP@%a*0t ztCy2MXi!WT)N~<2X}8VJNQH?pzikb3(=Ncp@i13dfEcF6+PS~ z=YdZeBB{E?1hFd#10Sd7uIEM`(iSi|6cLpeK?JDud5o5ymciz}l$UQ0-%MCMXC}uj zs#)@;b_|Ks6(&_tV8{{XS=DMf@%CHJ89HY+8l6=%8X>5XGt~GvGdo(>@_nhlz_L7l zvcWR!fnvhQGRfnZc?+eQkBW16C1ylRq~yq5IQ6~U=}et#OKbK@OZyBb`_Yt$e3Av! zZu&lPN?&V`>OXs`55&&<-?pF?s#pg>^m;DpP{k*Hq=8^^Z&P^MW=sdO| zPV{YN@LS|sbRHjsubj&nYiiwJ1ID~NhJ<3vTq)*3-R#}$M42m&@X8u&3<^>Sp{7{fI?%9})$7?zZi z>(-XPgBh6~HKH-WIyTe$%3$UwBTurO>bE05i9ke{l{U&-`5 zZm1cVra~=@LzzF6j``9b8dA(K^BL57yDAdtFhHPWNq~mSv*x*^r!h}UKuy<Πhq zz125}dp`($k1)nUo*3;rqO3i^qwez=|Hc?GS7rj0{gF1wVM%nrlk(iP)|pqyXq9=H zD*l3S-j)-SOaPYTNqWL1R8bjgWbTjyS|5?2P4?^yn>lB!Y_)>*GmgZ44VM${MuB^) zZgyfL(&ccxUUP!-gAmTJJ}RWmQTtVK3(#%~^7Cj##^>`zd444?vC^ZY2( zY=XlpnvE9UM(Xc)T@C?BlXqRAE*p1ptWv3A^agS2SbGp|P*B0HBHCGBw5lsgzCevx z>Mx;=cCSEA>xgvR`odU&)LS%dmBu~96_v&1nxL)%M$c+xjxSGPNQ*Th>=%mminQ|JxbZY!`381 zqqaz(tF?VbTs7cRolF0rjHW@NmOBYiZ~X9R-B&a5dXGZgap#JUwWJ!!RJgbVy!6^$ zs$cX{xA3_3Hla0Np0QE7n?~kx-SR#yUJ`S=1<()OZVZX4iMw~^DvK%g7JIrG(lN}M zeiJ*d@Q}+>WoKD8I?#y3>nhh)P}i&JSi!TF(yAhKy~JNc@7BEYU!>!7Vw-g669KN8 zc$atL@KMdJ&RzL-$;Xr`fM_q)1qNoeQW>}Q+XcRgv!D1bUS`kCho;xgWsr9c9fcx< zN9(Hz?q+r>h*#W>xA|+rhUF&heRy*O;SWu|P9pNmJZ*mJX8&o^o~(ism0Nne^sP$` zB>R;%vq(k98h`TAVm@svh29abCrbQwQyp*gcz`+jLTC(HV>fz|RtN1Qt4|2~uWIjt z=oj^V17&wvyx*q&Lu!Yrxw4M*d!|!RAW`S}!-q!hi`dE&6uVveFP;AG>uZy-+dJRC z_7ipbLUs+&ki8BMQD1dG_q%s-RJ$$8%Dd-fPw@3J!TBeB$n1qJEc#gdMb2LREc;`2 zvY#y+4jwmM;u%u;IlqXP(RF3Lfb3i>DG0qDRUJ|YgD7zcI@O!DhZa8;ckc@66d&(i zNON|#V&N*$i@&O7C$ZPm2xF6;oPYavK1exw-a<%%>}=zcC$u#V!_=6MiNeQD0f^?q6(^I?i>!7p>~s&9ONv{FMzdfYDH)kh zTuPOb#j&bZtHk1EG4PK$gSg@~tWOM-0V5JOHbY`laSCCMP`$z>+(?huC*9pH7wI#n z&aNxXICP;WpTY`qaAczA)96yd)dmtEm!g~q%4F6JWb!3Gh3b+2TLpxi#f79^y2T7* ze|rN#Vd8u^<)cTjp&Q$F%hmfR@kch-3L->tV2xbT1T~TXptUjq1#q?+l3b=1rhZEl z*|USr&^MZNExG=z?aAQsX+uV!S*ALPl{OE0m_hFBhb2l`FlW2ULXDvNaXFu#AKRf- z*KB3)bmqF3g{C(eCe}*GRunvZHWjzSCHgVY(UV%!K-;@?7b#$suufw#Z6duyxsx-J zMB#etX6_n370w;yoAkAgY#F5zn2kNF()MjxCKO^g^73Fdf3wApi*PY1hqt38D^;#W0 zgWd75NT65Ihh*}FIJU)U$pU1sOY?=q^T_g7H{qrGF+lfFw0@N7GgmA0oN$RLv05Pm z6KFeL?I!U#hWETT+tDsY?R7ClMKfsCkxaJ1mg5v-i$x+rp5lPby4D9ON#26Xxoo!S zU=WE4Mz>wx3hS& zE&6jsHRsjNILa4+PV45t2-O(AAR=(#q-jvdXoJRu;GSlP;=>{S9YY$+7R#cKr}|9? z!X_FXCawXw8neq~&R7&30wL#xJr|;$=g6%4Mz?V|60nmc$r)NjO3ESkSsF9neyHtE zcyXoCDDLRmP;Y3({tuaM_bsdnqZRHF^;YshC2{zn6CPgyt@5j^q3vYrHs<$~c=6wF z_@c4IaC0y(1P z@IcDh)vf*oM8BVrlb{fK!G6u|#>U4F10%l9C_iK6jv(j>aGk@uwE&pZ_fR+Pv<;-K zFJ>XF7riiaxgIENP;hiW=|?{8#2**4l&9Ff8OE~gi$`Z%I(HmM{ejq@cN8jne@>D@ z`1u5rf0^p`#cD8Ds5Q$W?VQZ)s(bBw>8_v@d5dB?4sx!F1KJo%cl7+1n}*0~{1JTS zdpfPb`Hg@yMg6fQ=(pO%@y^&BE1+lGtX@a_yr9t~M39eMl#6ATn^nQqIY1Jx5gCNA^|pD03{WN{x8R zAnV84)13y-@y!6y(8{|vjgE3&3`V0#6iYW5%XeK5aP7q>RL`G#a;~iS?DpU49QYm~ z7B679gD#n$UVl{r`2n>+jVc8grf+C_YLBkdoOF&9IdsM|qZhe{y{22N@yZs>Sa*jR zq_ntZ9(Dglksy-)hD?#V^Wc2RMYaE=CDLG!I6=@->z|quu@5{d z%D+K@1_=t%3%?c#bI_yZ6Z|mq@YYp*@73#C_=C&of$4Z%s$+x#g9wSUS4vdhj!^m2 zIXWHTs+(%(l*vc7E*`!Q%}fhOclD{pe=B1izU1RiKX)rF@5Vw&nmx2ewcG3vNfZAJ zJQ5X7)gBP7$eY7+-#o-!nx|Uqx;TnriFRT?XOqY_#K3gqXYq{TEI;R94e1N`(m0l; zHd!oBgHFv+7%wx;Ru>|x9(}GHwh$n$oW?;st;jN~nRlxgoMc9`rcOtiAhzi5(bgrK zmfyvw#?vg~ZvK^eE}m>k`327wZgNHC6yvlh^h4X>mye=tb!Ujd4|bNCo<6cIg!;y0 zeD6s#PhL{Wc^a<0El1hefq%%)ryOFZT*zT~3H)w4x@gX;*)ZDUUD!uuz zpHnq4|2DGE_3#TCYyqZb-cDo?W_zOW%upv*Mf} zbGabH>krmaNoZ*NlX{!tM`j;caZvX%%Y;Qeo5ck5@nizBS2X4lR%_|aIGQbbrCX;) z$lHUj@SN~Zwve_P$QUkiA|)0+oPLOy867ij92EJ?++R%4XynQGIseFh=cZ~78(Fx? z*2;*t#um4~lyuPmAICYPID~z%$W4g;LL`nYl6I2o4IO#ZR}FR8R^HNZZVIYAN$u>y$5~MPAezlIw3Gb$llMX5t=}b94SVv$pBv}I?-Pqd`qD_n=e7dLvV?1Vd=_psyxO_ zSE}24ZOr3Y?sCIHlI|;UKaa<&w{J}_PBc4kQPzyRC?{&4_}59J1k9iPG%-g9B>zKU z12UiG<%O5EMthWA*~X`h51NrX`~&vel&xFhW#N3O{cqfJAnly}=Z=I$6%h~dgyGU% z;OKU#W;1gv5JVc#ZrFlT-F9Z6YHs`>n+=LLQZOOIw6=N$u;D&tQ=Uux#Z;QLilCPc z0Wa#xB~CLU_M~j$lzr(VQ#(427`Gy=!3iXp;&uvYs2a-Z`6w1FXZc;z`%la-jb5#z|1sun4;)ULN0QO z>R6SXCx%8-W^%v~y4TsBgQ1O$eHSJXX0Pc~l!8G3V%m=cON5{2YV}NV(27>Z$VAx* zPUNu&1e5{S4CUf7GQ~BMknYNf$JsZQe3Q&E@Nxt)#BMYXJ3Y z76A`v^J!!$hHJoLz{>>P7h2tggobW945wa2+5r2j$~6TodMc7Y;&e{A&bWNor1K&p zr%dE_&H~4pwKhQXBJXR+3C%yM4(W0!r5Z1Qc+^Y#&r{C#f^Co24$jD9Ex@$vCJ7(1 zM$X%oE4j~~gP>cjr<`0NhnndP(_9hGBr;Jena^DJwEaYTullw4&D0Z^HVQP5QuH?; z1n5%ZtFO+J>(_elsvd8*Lu`5_+zxW%H@;?{=_{r6QQ*h&#PzELg~^re>}H+aFtiYO zArBQ@Utwtkt=)fsGD;=m+q-;`8qh7d3VuXBZwKQ=8S_MbpODld2jTNp9-yhzm$Q&o z)zqK|rdxM+A$vZnCJ*dvv${@!7rzFjhVn;TT?t?ZbHtQ0iZ6=P*90Svf!@7hY=8{= zs!EB56xT$f%_lx4rnS7}_z)JuV0rj>Ras!e&u=0}L_(gdYs&Gpy%Q2;1~1#^B4SG6 zeNI{+mB}Ggnk(77aSVsU0g6}B?R?NtpM+`hE*+9zIJgL$)4QKtcM)q`k-elM2{7cj zcGwtEms|@2WG^q<+k*TNDs@21Inml%{#?Tw|&BnHEGmo~|g7Ls+QR zVal3q{KQ{u>?aaAL(#EMoFf_Sj4}%7dwhKUS^mP@}EH<-j+Y z^on3|-pnBwmOd>k?3f{5PQ!eD*3_7r zh5a=bk|t}~@U~YNLM^2$@fs~FR6SVnaH#W^BWS!RvH1gvM3RkoEkchAMhAEx?Hde| z2zGeelQTHm%>;HYhtENl&Ttp7{3NU1+U})KekgMouO=UB*tOtiycB{Mwds4-n-|UV z!RkSZ6he>ts29ogt0?grcER7Ioi2{F!D_s>;bs>C>0mz;B@o0jUMeA+Dii9d8M>WV(}e zTEFzaYc=E!*qt@) z-YfhfSkN||oNI`NXqz4Nj-rd@A3#(}juSeW#t}8smLy0h?A5 zTISpC%yyn^bG=xu9XWfGjrH97g6%cGM)+QuJ5nGrQJ?oJSiIr#puH136kFIZjL<^E zMI(MkP&l=>T{L6=<-z+IV3jCb8T2$5-7z=!JpJ3!Ahid#c7&EY`0aTh7?$(urOolX zimTNEhM}3dJ)iPb^wJ~c_Gc|F!ABIOcz8qk79*l)`&#{H9U(-`asd;BFYKi?R5VJQpTn# zP1QkAAvDQi8rJ5qDoLB0y@O$Mm7O}|X)NP_OQRX^ZqkI?5c2H1<{&tlD*F<&T`YG$ zG-Pi`(JWt9#r@b0Vrd$>SNA6Fxe~zWvM)o4^yDepsW zm@%Tv0=bpACj`N7zM3y-FSyC1acIa1GLzk>!sIG$U_|U>>`8eGdTegVzOvF_Jui(I zKh&h%+RKY&<=NmhEoeEXPqA=xbX8!JR1^*^OwHV~5`*^Qk3pHYelCTdf@H?S%Pf?Y zs^}c~J`>GsFxI`Yy%8b-ZMis50ONTtX`VG6f^X)sih;4QCqu8C{0~5jCTV9+r6}o- z6)62_p=AcjN+sZ18j^>#H+rK~*>p>>Gh1^S)II5ISff{rkj8AqB|qn91P7tKLNR{VFSR@@n$x{CNOMdNXTN)su3m+{$Bf zmMbB+--L(go`WPN_E<8NdnsE6zeF8XbD^we!I+=2hsc_h_XP^l%wQs17}(XEeAv{R zv-Zsj4p*k&J*T&$dRV$L$E%gA*Hdl&@}=vt&9f1FGoK1Wblm>hURA-oGi1x({P%Fa zjwtc21-J)xQyQS6*1G^|uHJ(Dd<9NXVl%=Cq^s^JjoaDoUvR&nf>*L^jv8sa+P}BEvmT zCJpd7=!spjyQZ6iyzy(oXe|3W?KC{+s{S&X-;YAQpK1%LH5&n`e3kS2`I{EXoBGm# z`I!XBGfewm0|9G%R1g??`ynu-B|t!=-?f1B8$zdYKbEblmHYeYBK7WXV1Ew8B(krA zEDo8Dt#_epAOwQw#Snl$q*`+@n{(l2>6PdYe^F1@MFk>nKQgjI1Bn3hBCVhh&X z5FRpPYV;55CgmuZnK3a*KEdK5vehdQ%B(kIg0D;dxpmPIM)6Ln@ZLeIa=huENNq$G zdI*yt5}ds(W)yhQn?lb-{T{(H8 z+g1&G*DBdKGa_Cy7jvSs=!I0$ERI+1hvb2$LSgtcCc0>D{p1Z8d)ZQSICAUF201In zsw4QEP3PJ3q_3227YMAGP8q9A5&qo%oc>O@4XgYf4zx6_b01;d4JYbQ(hIT9D>*6N zDOLy{JJ`5M$;+XN$U+OjkOn7Ei@X_-d-I{A8DyS*o8l=wHi!jJepj})VC-EwzQ%8S zQVlvPkn}xCN3bXj@h~s$943RGg>Uheno&_Mm{;)#ipPPSH z0*aN;U*1cI|MJqmUYmu%BP=w|J**hhc|Fus`@EzbHnu#Y0l7V0X)Ruv6* zKVmmIpi^fnt&bo=2?~5uWvj_D=Q+IqdP3-ESn`RPUblBRs|GBbuZah%Lb6I+tmkhA zmu~_q?22r??FV1(@jP6W=D&-H?>#~UMbtH~Xhugmk4yHr3$?csZZnK^cim4#y}6t* zb-rn%MKuqxUC`Nb8pH|{a4$(rq-<_df|eA7r-q`)fQjj#kP-9;NP*s%Kac(LM)sc{ z+z_CwKOwg83q09=jYh$mQ8)84ovv=2G`|Lkv0`(8CM%gfd2#+mdP-<+g*5loDr3v|4u zlj3^WCq?(R{0Q0 zcB~jT0+|aLfZSXR{SKx75cn4XxTV&aul6p7$0%nV{$*|MdqEIVkRjFW3PPlP=DXh7 zo_1EX@=^$s*X7BHJ!f{}wNKB+a~Px9mgt{wB2EA0{x4WkDKg6RxuvA1Xnx`FLzrJHmA`|+ zSZwdY(oQb>jX!bBmmulU(NQmD1oT{T8+Cvo{oa3wvo8TNFCIKU=H}nj*0dQy=O*u1 z%kU3ve$u)%nW=q*4{S&S2>GO4&~1H6-*2>aRn+z0c5-|(&bqM>A_3ZY}YM9nn-`zlK!eEx;1 z?-!LSDfxr7p4c$_x!V7hFis+9+M0A^zE8!!C&`FMLzztJLaQe$?Olw7nD#)DfdyZ3T zbiXBIUmhAgftovb!t3GMl&&n4SX4LF=OQU7$!2>1b)b@=G{`9LCo=oc(AV8*F)@Do z`zGx&P8XwNpm1NewKmU_q$ffkH^xHqvlgSw-)R4o`~S;vK{#1KV+c)`cqtKXaj47E zx$Ixo*4AdIP`rOZOOk005f}z`D1f@>!*%D3vbJTfeWislxUVf_uQASZHol+r4;}s^ z?LJo|8A8DOiK&`Dvf;)Bh5Cbrvk~h-Z;z{9#wjhI&0SNIK+EgiFooCkhg#^5H=i{a zukG;FJRxhjF+m-4bK+nV5k)4l80$@By#MgwLxm^$I7`s{SX(FpkZU3CiapEge^y08 zkARk2N$ugmkGVxeZ{wXpVaL~uq)N%Hs zZNZ*z(mQNrFeUl014-lss4o7GH4uWB^oWTc4S!H#K!wf}idOL*O4>xP@0Cwo`bf4H zoT=UJ)d1N`mF(<)fqGst;269-_aAPk52d zK+#i$lDXf;>d*9xQjqlk-+#FLdB7kCP0CQ|YDJPtigw$pwn7kTOY>PyCqFcxp@9=) z5nlACG-3Ha15zatqrXq>&sozGpaha;RD0XQrp4QzZZ-XneOna%xq|Eu&VN?Wg|^ZS zUaG1XB>gxDXtb{}n~Q&`)ujzmTO}+_|4R#E1ji{d zFi-ey7fnY0*fx!K(4_2xiu~2*XFZLrh~Pdc1T-R^jwMm z($bh`(^;DBF#m#B3?ovaM}F&69DJSgbIlA`6q{Ss(-T5jwo>1}f#{$R`{}=rea-Cu z0O7z0r502a2{<(@Y5b$0f0=ecYqAW;NyTQ!`M$r6>u*i|Yam2j<$l^Txp-l^?0wucaZq|(a{E&`>s-eBTG#JceIhVEY=p~7q@z8@66cHf??m2pAX2868jWEm36K>lHzdv{B zOEw4nztnD+hv{+STW+E|%8NVIlu)>mhi(lM)Thu=t;JusimO+}z?G>~t`u_V%NU30r!o`;Sm>_kpjjxp|mr(9@@Bp{J#mWOeNo*zMiU6&6N| zy_QXJgaLvCTApSMdD|EPTv(jjgESqX<71*`8tW=L#W>CEkFTopc6_vp*Vj9*YTIGN{!;7g@#v0=p2l0N^jes}qvXdh zhwOZmbG?;gAIp&A5=+w!-mqZpFMO+Uj^0m&g#&PORZB4{%;uzR_aaP5>Xb8NlIvLz z!F zTMh{+kx6v?*3*&F;OgnE$9~jOmvROKP__wY!PhG0sx))`)P>n$zzg0?8`qtA;caA?7*D-a`_22o3_w8pqU0m(mc z{|n;vx_8c4dzd|j&1B}Uj$VK4wp4A7l=3>Yb5#H7d0+a>#Xmlidi*rnmnVFqQf=*x4~A{V!Icc9q=5 z8dmnYCd2hj+gOlbt*~L7m9%auD-~EZ|M5l`f|*ph;3p=pgtI$EV|^@ZFU)9#$10mp z0F~hroM@wYRO=AaWZKmo(fLaF)Y6&rVjHy`BQ1YfH_q;=@iVco#@?WZ4Za-rBeMw; z4@=1B{osJ?gJRP&E~PvX4KRt=aAi*VniNj$%|zk?925YFj(sK$GPZ?+wLP?tJVUoc z9i2tloZS|%qv@bxw`$2VQ>$Jj+4hBHJwKHS!+uFN#Ag5l}8<`m;~txaDX zCKSRy4+!g^-}>?Ayd#kDsh&w^)_+h%&!@bjYO2Y5GIV-_=7zv+_Kr1nh&6=$fGK&EP+8d#;+3e}T)MVVAtU(OCji z-;Hux>mKN#eP=3XOVeUM^*ovpRcabdG6M+T04zRb{3P{X9_2rlmKYknonoL4ZX3Oa z#&>E5;d%W|%JCB9Rz<^9sNlGVVe)( z$~V~YqTB&B-sjFMv4rGr#ibWbw9xNsT;&YBGI*5@WX9x6Kr@lw()Pc5;yRPK>jYp> zMKwa-^j50}MX&M|znfqg-;GhNKSFaw)&PQ~Yswy20;I?;OwD&lPn(`Upkuve8CCf!0zMNFxTxxo!MZ6 zUMGX^e>O&#CmQAuRzj?~Nl8yrFL(*A+rLgn2H^1)xU(4tZ4$3I96A>Yy?LBB#Nm&DGj_F#y{#U$JiWQJqj)#7RCDbC)FBhJN>TP0>Y^9rB$X ze1v>ptU_Sd;e_sYO!`3t`%Nh*QEVyTb=MRlF-| z5erU_es*6CTOtLGV<}8i`j-cjGp|0i|BB^)B>;lI!0lBTu{F>mytueh%~8#~A8ae3 zNGpT*qy|}-u!{T94X>!w+&ElRj~@dT=*oC&G%M)R&8tRkqZK|Y@{&8nQobU9WsI>< z();f-`alk{c(qEz+yr(bcv{1`)-va_npVyGp{e(zz#m!itQCxwoB^L+j7e$of$Wwb zU%`r^OjjQI$$+ytMnZ zZh}hpbC&plH1!I2#T&%2r=FIaY6k_M*n`w0j4Xrt@zzi-+54f*^T#B!O2ZouDhZe` zt9*1{MF1(jqzYEi^!06e$(3-<-|c;M-8kS;KJfBZsR9 zJ7v;cUgCw+Xa8uRI{aW#M%PV;$G)COWkj~f8(S&Dpx^}M^`OIJtivsz(WeNVN)8E) zsX9%W|2HyrV}}Y^eog9RPoKrD*=J*FWp#S}V|v=)D!PT-TJ?&vK3AhqW;`#iRslNx zCDd73yst(WjGCfvx;08HgH26w9W-E_ko;a#*Fo`gB>xRvKFo5-h;};>oiPt%2$}?9y#F)k@NxRK?|tEK}px8&ih<%VWVrse1hGF4!OV0+&p|3B1mTPGtk{O~otS zB6acwK2he$NPlezk7QvCO$ClHx7%noL2``v!nX&uAF#Qx^%98$?r68n`8&_1S{`O% z%djRHxR#Nf5>X%8@5j(^{DyMf^w_y!RwWJl3H`yzI!F7~P*C0wve$ zX~wleeH){r(DztWIrz_}D93y0M15L9cPLUv4PBVs8;RUd+6!PlCN#bdRLQS+c65HJ z0HuDLg27X1box9`qu-2=!Id}dZJCfy&|*x!s&gI|oK4In%Ppx_mE_)p zd7qjHx;?vq3XClwW?(7Vryw7PWSG<5bk@>Qbx;*?Pa)EI@UUoJ`0Q6D9XWlx~ zN^fu^xOS~=dD=T88xN7nR^^9FoSqVP`qIIg=7hXTENj&4q_SQRScBi|afFV#+$p#R zUe%?$<*uf!Q^nfa;y`AVlJ8`^!z$+Ne5y0+lYod>g48$e8>Y7aykRhEGo3pB#<%KC z_Ya7uHd}qty;v5UImc7v= z-^@x?m)sIWgj@cnk9x>e)tat}=A7mq-=@))y&hC?j_v3rL-}E;(Gp@Z_}6~uKaSm} zj9Ab%nUwp*MVlwk*K2A#g-c)5w*1zN0yM#F4_0m`sPEVk_`Tb(h1H=^x#-o^UI`ZoA-=K~=>;%i>^Dl|88f!oTN=kZ6+P%9e z3vxJL&*mDa=O34nvt15q8SHd8ZL@b7-qid4jXy1vYvx$g(G@4+v+~ydWTb4JH%Fb7 z3euKbzNn3MbqbAhlSfx`Hl_a1kB?W-%7yLrWsS2RSwGbL@Yv>?#L7lxZq_-KAS$o5 z_H)!@XO{@6Ft>JWbmT(=7!<>1GZ`4Xdb3zej%)sOJ;-u$d2R1l!0Ynt%*qoHY(1q` z@(Nnf_n2v6-mU~&cbC{ATt1J1a+ zx&8=WbGv5?qA*30Tqk`4SIe2T6U8GUD!Kp_)zTvQQsk8Lztb%f?r;-X8kn<5U~q%1 zMn1MQBoiHSp?9(xHz9qI0;Z#>;b3Pc<>p@8r6J`WmLo|;XQlQDl_6$0A=#4FN7{XH z6qlz*_%`K`!}#$@7A%mo68kU;p6NtO#8V) z=kyM;4orXq>6Jlf>CAnD!@+%Q4o9nnHcf@OvCX3cK`o4L?{SRowJMg%MzWtH`M(OU zy-!*&)s5D4%x%8}T_L6`b%p1>Yd?t}i)(2ja75@APf)qgyNQWPz%ml*&^DpNld7N5 z+qheo2v*7h%8263anqP?oiIBBgwV0+z@?0Q;S%;exzg}n4V#X~ULL*zpK`(r8qh~H z?y;tc2!dRl^zvcGAH5=N5zy1jb3Cpa-s~emTi$G}$mQ)8!Z2V%;m8uOL?+w5bATjD zo6E|}>%WpJ`H=!=l#zujEMB((Y7Ka7a9K?S~0 zD$5O&aKi)wo|nZV#^L~;6^A};4jBt?uv0PKMQIu$Hx5C~oE_Cw5kEch zBAMJi5#EBPjGxFCTl*^)TQrc>947kHg($DJhfSbahyOi`tjZ-$QOKTH&ARg?2U;P> zml<_sCM3R58ey|vFh|#0&WMbK1-4#*^F_8yo1f}^+gUP?uQd;_PFFYL0qMo{9h@!i zKx+Dnt*f_Pu7v#l3SR^2xhvlYbcrDNFH1(8Dll0JmM&erN&|g@kHF^-qme1;yskNC zAz*@+5X06ED3W{5ZJpuWrV})w>SrB4Cw^$%u7laqyVMU4g^<`y zP}!MhMO&Z#J$%jAXRxgJBNIZwagvZE^elMSINj6e$yVMZ{(Yo=uhr?9QO9(-wl`Tr zzglG~reuxMCh-SYp_4CVa)0HqCWPSzz#L^XF2hMnk7we-2%xzwuqbE^?Y-8IN}OdR zL|(DVRP$Gd=x2$rw~cX@dC}mc?}5M+{HdRngO!iOlsMJ#mOaP$$&)6pWAyc!w4rfg zxq4ZLP}QWuOj`CT;Xfd3Q7x}rCY2tA4~)r%bnRH{bbcQYa#v7j6*$P4^;-yFJRTOd z+-&q{F%YBd@@6v!2mvJ3u0!u=p zu+me%DtOk^TarnpjAFWzHQ@sCLv zZS4$(;z9X}Q&`oC@Ikm4FL=9V*i;WnYJLMp+5z<% zmJNNr(PFKQfJ!>0BaP2JtYg*WrSRUdV1YMSrMhhyDFWMjoN%viuHD=F`fLrEW9>$e z_S2tU04Wj0WlcjR{fPql5$A+f6&RlzPX8-kag7ZPsZ3np$X{1)a^K#!GjcaYVhnC= zzMKd9q7PG?gVR*90)c1ZN4G^qMa1tlVVs-^KhlK)8!UcSXnYVpb`)e_3F_B}he#z` zXqz?moMPy$PF2f3HdG2lU>8-o`3$!M9&yfVNJsiTPbdwNS_%Z>x1U}b3gE77=BBiA=0kagOiDI@fl9;J#-cGrrB$MLVNh!TQ z+C9|c1?;m>vq!8{&k}W271Hc_puhPpA7!X@nZ$O}+~D2mV!&7rXG;hi{ul7V7t*asnGgfS{U2o}>&{TAwRYn|s&X*&fM^K-Bg1;I9;FB?>wMUXKl$ z`|P_TiIglA%-$7x57@;wpjkK1j%Tlqo=(<7zmoI;AFmk0UOd`1PtUnOdGT-s9h$x; z^f78ZFpl{uz+fQ|KP~`Xo=uMwovH#4GnyWp@wP2bEoR-w{hQYoR7PC~bHCaxRjpQV z$!Ft9H`_)F_oy^gN7x$T@l;X11q;`&Gb~jhg)b?dOnX4=$9^!(_ez~BJ?~!0ODbIe zOK$bZsS2|m2% zXI|Wq98ei2t;G{bQc_Az&m}J1bQnrj7GARnZLg6Cl!pud1n{Yr)6l}I zdu?sV7t{mgiR#2GRC`~bQ#L%`{ZdGCSqs%t+| z*5RSF4x~Vsz-gO>vCV~j1n+Z_J21vBZvfuUjy!&dmf^2FF|~3cG>r&li5yORzdeyD zY}s7i3Q%hn(CK*Q4TOAyxd!R42z=iX=FU5;)3WjOk;*Z`eyR1k4)Ci0v(M|(Q_^3- z&p|Zhp>|v38eL(gMTv>=$k*m2_3B{@;n@2~bYuu2XyYxtu_H|$Wj)0d`T9~jbw|qU zehx>uRdRzEES;OKvyO)5zRnYHfU3go3k@4%Or1CtExbvjj|(4pjxo${X2)eq6ecuZ(K|^IG+V_FaE9;fmX5w$22m$HnEq zNJ4N{Vjr;kPi$zXFaOR$wLDwXA8N~>pMN@FhVSJtpvT znfXJg=zYB}<ob=| zjzh7epT>4t;#E8MSe3`X;x{`dY^?Fo|Eo?X&?nj}yhCS+l6zae>lc;<|M~n!J$DC= z9SZ!2$J_$Fd64%?3D|S3TutlqLW*{T^d!yS6J@9+b%B65{RA!3HixHzW7C3LX3L|V z#_}HYb*H*C#lZ=D4$o}VPIPOmJ?4|rQ+7FwZ=XnY396<}3ij4=$=iy?b7TQ4AB?(Q z{8_z}%CksyGl!R8{D>v#&lgHY6yWuKZP-WeO-^JGVb7A{T&n zSRgfm+1iGoj;%TFjn|^0A1SapN15Dv0X5dUI%;Rq=I7sKR$ns;NbLU~zwr8PJBI8jBW_NI(l)kVSNkNX&bsr5F`>xt^w@1^~N|QC%l`c$(1lP`NO0u#J^pExjU#v#s zk+bzeXro6)`Z#!kNer7^1`J(0BS~Z~A6Q#k`<;%9)mo2IIj!B;LhdRTaxE7dxKU>_ znw6L1FTK6ZaVa8-}~HQrX4pV?S^2MzWP%(Nd~tVg`QrTlk>t z9T}aNOFuf)a|CBZ!EyGNo56ZWT?6;r-j4tOCLzd3En^Q&nyW>hsG`fDr>aaZ@apG- zzQ#B!>ZL#~&Zi68GFnw@x3ze5CUU2Gt<}81A$OPX+fVVW)3#OpwbRU7QR7PySJ<$3 zzo({UedMp5J~$i3(&M*1d*+G8sQ(;ZwBONqc;^QxsS8=_IN{~_+$}jTG|fSUe_$T{ zRHKolZbXWihxnx9X9!AXwCVxt@d26|K7T}MPe~{<((E2Z??KKsT;zNGDx6+M6@D_Z zmED?2mS%)Ce>5B4hrhBdgkT&HZI6F>r;-c@6^y4dhF=v@X1#M9F-|%dowLIZgkdN? zFPT)BseMl|MpET(GhyLW_OpA*sIQy7j;`o7Kk>Rd`KOpQUrhH`aFH|Rx%n*!$J#60 z=Y6Tn*QM8nS~VQ=U1K6awHXr^4IwOS?4#T<{c*jYBk9I3oJD%Y79$kPiH=q^nMH&- z>;H0Hxd~l1Lu4Vk&|A7YZ5}Zr+9s(oW%OIrZM{Qc(M{MFswb9$M&-Jz`Gb?R?Ml~; zbPZ$s(Cgc?(m0cWU&BFfgJ*vx&DBZ17|AI8@v%*ENbc+0Mgt+ZGqBm4{q!1Etq=1C zKy9X0@r|qbd4hv~7vAhI+O&hw-B1=P@mWM=-lSs{ywz@3w%Z`plx$nCuYGj?uPDrK zZa`J9ojZAwGB8umHt+tpuu^ z`%c{Y#dTVXjiE9Rgfcd^!^sf+A( zPXqowlQbXxstWr3575#=gSAj9f%4bLubHYPy*cy+I_5gi{|&WBx}}GRh~_rA(>>8x zyaNP3Md1uH2kW<>l)SVrk5G3}k=TkEN>hN5T^uEJ<-)YX8nSxz{Mm)! zD0rOL=<4wFUT2XzVxC&jvo$F(ihVRgy-3Os`z#`w`5Esg+=cL+KYu=K{-n$(B4eO) zB^-Xd3_sDm&a0BIR2h6RU?M~@{Plp0USA-&Y3WC$?=*;nF5$;#HwRmsC+3CIwus+L z0ntetLU!w<&zmw`g*rgKO}Fz;C>`~p$|5#Q^u1@d@JVv(j%0;Y`Q3<|Gdxx`4D*G7)r>d(ZADse4LAl}OWCdc%1sOt z1`OhF(XTg%>LrC!D%yb<#9Xq^FG9}#md-JZA`X`K!(#Y`zCCeFhmO%;FdVizuf!&pb&71KgQZ z-sO&N^f6`*<5t(YfNd2nUbgTn~GUOS&Ff z`$m;XXRz4-sksba5xz%7sbl0ms9}85-YkwD?9|YcExh)rx}1X_*qmwQ)m=;Y2MtT#Jkk^Fhx%ca&8a%K z$O|FvxZ*If-Nl4tcKh9s)=Jq?QjLlJa0npxfSmE{T~xgF?6}Nd{gVGa)`Py|9s{!^ zk7cO=!ziJ<(ygZbcF0~6`XQog+%q?fh#w*4$0V|yI6$Lg()W+!bEozd(GihuS+#5y zvP9%Tpur{mMcynCy0!=qzDE+zwttT_DurQ^;P+Cr%HABmcKn^d$M+f)g2^9nMuZSB z?|P8J?5>P~!}pDsg{3cM`{auNBT>m8r1!kBEw*#a5}{YI+^hD&%^Gj>`p{5XnX4`G zAf3|;%3Gm&!Ad(#(_;p*Wr_U42O45TK_C2)hyzj7!a_*X(~A{P041EWM#!|bCJiyo zLkJsVK1g-^;WO$w^(SI~tFNY@Z$EQ8$x(TU z*?aoGIZ$3!j2q1t1j_@?QJzrxf^3woe;i6+d-zSRFTu`As#hG#l?r{;wrpj8vgerU zf9T9JD-8Eb#5asFc;{K8-EiD7AHs@Ukl^@qw@kWKz(5TBpbXHsV<4B?*@sD1rU;Ul z9yA*9HVczH21t>*yHi|`NlH65#V0vhI3PZ{8T%olSuY_rmdc<>+pbznX7DdgF0x{h zx>QE2=qKMeJlV$0tF^K`FzgSNdFr@F1~m^t7Ii~VAG~BxTkq1a0?qvN@|4KAdjC`k z$ChG=7U0CODP($51j~^LF07=PJlWghXpq#=R<5fM+y zmH$C>t+D+0%o0VA){pd;8E)@g_(IY|R58!|gzMbR$3_&%6<9r_ZzUBKRsggdR?el_ zl7W0Vos&yTja(cCbzV$Vhmzg!?0bA>w&( zyKU*BgFNe$Htx@I>%?=iQt_L!u75O|5KXaR@0DHh(~nCWR9ax z3mw6wXNk$CLDWq~Dmf`3Ls_hPKI8qn5MC|*uoUIp-=am)LBc;|FfrPxJ}1SyJWVp% z_2c(Ha;c_VNV$>c#!(hf@h;6`y(v36II#D8BY_WSyTh`q#+Y7apbAU^@xRIpz2 zG_P2|mJM#j6F2)Y6cN9Ey;u%ut0_3Y{6fZ3aXBff^h>*?Ltk?)={yzyd@KeIilscx z%AC+MVNp(KNXFDP@&8^Ok^swGT6$?nvVrQsu|@>FeqH3P?78gRtY4c${-|lqZ`|j><=u(SE^s!aAS#67mel;Ec=ffjMd&9^;-ys!}TD_q|&i-~p;x zpe52vVn^ms#+CeY&~b-a_i*TotIpW<32+nlvte)D|2AXMP7@lYWA2?VG-U4%e`MV6 zR#skKk0BZSRokePRb*g8yYxx0j(mbQNhCHIwZZ7B^s%`Vv5oeE0Vi2(-zxjB8W+6Gf0f_zU41qbBAU^jD%h5; z{GFA>07H~lYcyPG^F#9!yse}RnzfH@gPzI;oRJ&KUjMKSks9n_Vt@D_5?oFdV}FMB z2g(*Tarbd9$TsQ>X>>}DG|Pn=V2Lg6bS4bS&u?z3ZMu(lcV!g_T@Yny$)d$KT-7k3 zz*Ww0CfVxpAs8r`+!O-%hBG6?8+t0mHn{4)`=5L4U-h?pfE|sQ4*(w-MDSxNnB5)s zkyH+u76t z-;v_V9t}H_Bt!<_QvLoQs*Dt`T-LWicUF|eoNDz6aZq9>G1yQG#hfg%0(2b!RluAi zV#u`WKPs8HTW8H1}@7yFU9BHk{ZNHa@Y>}yi<()FR=? zP`3y*{=q)WBG7Z3{nNt5XeNdvKR~`>-jl#>*KY&>xV;jz1%mPRj{2mToj}zMb=Sg!s!s=GTgs@hSP!GO zt^I$Wf%_hvu7JV6mlNTTP*DAi$N`Lpp8ZxalH=p+9`6Oo*&EH+p;cB#cICTt?2qi9Jc=6AMiPB=3N+H*_T{c586CBPW z=_^Ni^eL3mU91pxAzEuzTKp3I?!hQ#t|o{MPfhy$_PJVHc%~Pl>wwf7M%UVFgDvA; zn+{RI{6|7{clX)^(_l!})$uN?27;JQbgarBWd-KD@-#dFrc{MDDdKstKfK%7@*PAe zqHUiVAlH?v?`FTsGR|?C?A_Y-He9Wb2`Hg|+B_~}HvK~d!AT+G&++LcM2G}rE%h`F zRdRw-J25x9|IyC0FyciP8qY;K;1eN10fjvu1gUCujlZVAT;;EGnwdm63!-(5$|>LG zmjsa{WD8g6B8?*i%2c%8@uVaka)nomK0F}rTAD`{Hko{OC4Oqxs*x1t7=zQ;IFZ>a z4Eg;j)1oK;Rm6>pCI^!OzTk1KUgk0H?caM1W`EgU)LCHrJ9EXY`TsHce+S}l3g~j2hpFjWR<2}ks zf9-g9I3&jn6;hl7p6KGxTJo(xxZ1*K+K`Ng>9ujfI=509 z;4^o3c&fTxpmjbdd?lk~2H~X!)}u??9yUv0*K3}@+Ib}qm8#S6=yf!4eKMwqh=@=A zkP2byfC||^WpjA-;_oO)2!@;mKhzo zwV`PqPFDBzDHHd@tDIx8Q7{p;Mmk)UW)j{!F87+iMd&H|)5z&Jle>0!j@S*+N%P1s zKgiPT<)gc%RvzS7?5pbs`l?-sANS!YH#nYf>P8umA)=$OsrzE+UL|Nte5KOxQs^8%xB|Q zVVKyoer`M%n5#u@Q`5juS;;qK=ASlQ98>oTyf?cc`-`5S!YRi}e0vEEfpknNVkPmP z=KNVUHS*wN(De{5xhV}ne^*HbUz}O5AZ>Mh1ockv)%FLNL``AijD(~5k>1?=67~8! zLUW`9Q_%pjI~KZhAP*-YN<4 zd32D?3G(_;iq-388K-3lAx`(hY|CL*%r4Qo-!u2OVz`NB*ewG!{;H8(k48Zb)fUC7 zeep<${P78YEb;2Xgylws#9(#XN^I$y^P_oF;tB#|Z+zOH-aw%}e ziKKyPd^Hv?HpWR_j3Zf3>yB#-(Ws%rr&wxWrUvsGcR`p`x)yVlg zpoV#{#a(Rq6)84O42vtUb2Wz4)g^caf9<|X_HI3&Ti%y}&Ezs+vKX+vo6WaofxY{9 zUhB+f{738Z@@%HaN6x2@sBCV;{pJ!rH=QU?h@;UPd*#lgWMdRfF268`Zz-Yj`$vTn za9f4IPg%;Hn?-S*R>UD?JXS|Lhc>^K*ds@;IXzuQlu_o{RH>$^p(w=dKLXL&5~&N*z86+v_5Du?~J4CT{{`19mKApmC-hXiAO^h zD_)j*DwncveCe(EYc$fe6CF2c`z!}l_BMXsVdGKuczrz=6FY{)$TO^`5qFTDV@U2( z1d&07XC8(m4rjwz(a&*9Db4T<`w4b_z8C_+AgA0yA!_0{t99nyQ)<&7m%YP|iUbSH z8K37ht5)60mz+)7dZkkuViL^VUnnI{p$Zt43&Y{)3C`*?t)r}TzNB5;u~+Y{Y%|VY zg(95l&1nxHe+jRL^4wd88r`7tznx83Y)``H+1%76GXeL4@jqF!{nxEEj60o?7PLrz z-hIGgbp7N`aOB04S~KWpL8SWHa+~0g^!r0{k0T3xDG-tiP|GU?3M4eos1H5I!6cZWf&1)R*^)9SLywK4>03!p6g0&-H|?^{ z=5EEObo-xr@9Y6O`ctd$!BabDAgjElMugz}r`6c4G3OfK-0XX02pN_1#mg9Kjbky8 zp>BkdkS%p`;umqD=Vv;HJz0`6-B26u<xM65&FCe^Qk`K``q;#Ae6PPtovZ+%#Pn*8iR@JPDPK3Am?#cwVy#VQrpPWj#y zjmbDKkx{c|>GI0J_LH zZaYivw9Kuvnh377gicV~+x-wxtg}BU-*O$jd4)o^&^kyKW z%DgckX}D+p=VBbVb1%yuOEFo}TDWmgbyBT3fH{=BnfvdFYp zAktKko@{&Iy%Vs1ZcLIYW;TSoBFIaG7q0sFXB0}3%65q|BD3zm57zy=WWUD4_4-26=-*bC*BWDAlC!FQ zviewBr2#$tE->ywbZye5>&k}9fdpL^uO0cxT7Y@wFC*9cx{SG<_1V|Ns#xWjY{Ys?rBbx=OdIC{P z7cjMeeFDp{sz*CwY_LgIHXiKO{n^C93p}x40EIgb0#$=9Z29ZTXA= zu6exuaY3>lLr=0lIqM0DVOGlu5s*VEuK}Y<6Vu~wN4im09Jryr_3$dee0s zPBRnFqFGq6M${!k(IIrv@9flBp(p~YVmFNLSZF;&U~r(~dFbOuOVx6sK69=LZLYcb zmRE0YGix1rDyDlbELN5}SIjM;r+sMP&iDLohU?E}wGU2h)$i6~a&v)I5a^XLY;R4pdlFm73> zylZ)xk;-cj3(d_Yo^PJd(?L6Uy5xUb0v7*>Ox^I(V%$JNuf2-M78pjIuV0t4icu4< z+w(w8G_rpR$@}JzZENZ=?ged`a92vS!zBsgjm4z8 z9`&jKTq6!FCnNx?Lekjeyn+k%l~3a)Jvcht3d84M?3%g!y}_N72N^huad)>kG^4$K zmIEQ*^&io01SV2iUJl79#v{J$SKy3V>NjLt`5O%-A}N&gT*>r)q{)*h zUY-(Jy+Q4d0aU{-u68PR|A}Qv$38gX?8vLBwhSFn`N+Ehl|Z!b!|Y#aaQSE;>OyVv zf}Sb*6tG=ruTkrX%|*v{AU_ZKfz59vUQHl{m-~G0EdAWnO?p;}c%K z$*+A33%UNnFbU5UKkUKg;PqXz{nL}ZMB7Q$zJ`z66`MBh(A<(H?Eb{#wwQYm@ftL; zLfy~XZnyU1m{}6?op`m1$OU_Vc_*CInZx;sjVZ=EV=QOQMe@quXdAE;ev@}<4IpHG; zt|J08GS}5FFpu%idl~y9xgb1-(SHk?l}Q&fY&lO#t(h!)9tp*-6L)0Z$WZDht=^6o zuFLwQOStx*C|f8ip}(ffc~UpX1S_+^3f3M|jb@|^3RFnS;fzfk>t0`^o4QLC{Qih_ zN&+o5_$FkP;6Ll`5yy_cSX+)4Tk|7F>)VRv8&B>@0jFg|DrLAt7*sA#9@A9gZD><9 zunQY?Zd$$BjJh_g;NfnNd2w+BcE!1Trs!p-+wUENJvdgRR}H*tGfrTv7Qr~>gi%Fp zo_^PTzyH(@n+%^Mi1tMqnznq5RxFyRzBU{d*-dB4^&+%?f} zZqs@{l{a9?T?s({V^@C1xr^hAP!whm3KW~Ki*_L`(^1s-fI|j5F>$8l zM;*0YwCrOQqdw8-TI0Ohei5t<=xoxcAee+P)e2dO#NQWMNnvVk9);hiHqbc5LaS%# zuI|NRNYLf+Mhm*%yC*fw`oECvlvO$&1Xp*G|6Ek`f9DZJM?NJw!cg-KlaIC_)ChK3p#b(uB@0{8EuaCwNevB1ocRjwLBus=0E>;|k zpUPO3pC!uc5>n1umwxibcl#ql4KLtH0utk`q>LoxBGZdzGx8S>@mbT?mJ3ZG?)h>M z+Ah2HNs(BXU9_k@t?5U=>OlMN523i&65+G;PfZfO)HRajd!|(>{l3Ov$+6k|9qk%{ zo=3DI!`>}hnveVOJRxxL8odlHh0~wf939uPtK5m5a}lton{soKO|?;N>khcv;}N%+ zn9n-p^1yt5_^tGgl==G6^k$Wo( zI!hfg6z3)nH2dAu- z7OXCvOvo-zQhD>DZgk~KgD`|BirEhK3YjbPQivvRjy4jo_)IQ)4se6CJ>GpVus*%3 zD|@;;$9s}yXu+nLyGnUk^1;=Z#J-H(7yIqF@4ScD?x&XO04hVro5jCTkN@6}I`E}i zs@4GQ_luL39?Nn0@OlB|&2u(|E4TlWr-szizx$~UOWEwJm*fF-%@Dh%X+a4(3uuF=U$rWVV zOFq#P9fgQcIjV=?h`07ca&t_ywE8v>qT9!N7?l%iONPwhYFs8IJec>$lcKiO#&k^a zdbeq*r|TO5(vx?UlHcxNJr*!C9rt5yXP9mFLom-5^J7uviyou>AWK>hGAK)VpxEp%^947hag#9PBz1J! zlIQ&4;7FtpynD8F;nG^5giUOHyL9JLXi~p+rp4d?fr?D%qWdreZsksiuVBra5h23+ z@K4^9(^jUatVHl96}pJ@{tw(iF7>J{kI=h`$M8VZ&F;#M6O(rXz{>$gUzti5u{2A6 zpYHLIX`aEjtZ%}D2Wo1}Y?H>85@}Iadwv(+u@pA<8#(yrqrDrSu|2d;hHic@yt>VxmN*?2jLJoEH-pt|Vv zU(Odh^zxtTDzDCUUowQN>DS@1gvl$8K6XdM+kI0;JIKlYce327bil2ouEh=}i5Vi9 zt~PKH({ABZ(B?||e2b=8f(tHZ@C_2e_JSBvD*N>pkLvBs4o}FiEa&O&TFxEh0;;B* zd5K;FF?4H6HJa9pg#*CC3J7IJPaH5-%g3B4H#VxX@l_#Bd?>-Q&QU{6?b?KKg{E06 zrxKIg%k@cWn1yf<6gFkM@731VZ<95DRdC!QmXGwzn_5~Hh!GO`r8Lq$`+AbcM;zSv zbQI(jU|-ME!tJMw(f6pV6M6t*XJ0A={zJcZ<7`(*hX&m08J)wQFG+{LzP%l-)>dAr zHPIGLK%t;u2JS>k8xlv-l;jC>j%x76(YOaE{$-wvKcF;|AGHq9ct3R{b-Ox)!#~=w z_X}$@76TvVefC#?tB~Z0@YEN%+X`=HNRsqCVNNBcKOuuCMMTw8J1pePob=ROrE8|3 zY^}BDd+XlM?c-MR**PB)MWdS(Fa9l5avmJ1bbxEYTkXmm^7l#}BBb{&tv4Vbpp9~Q z@uD6W>otk%%!RZ>%#Y=5w;Y+~9OS(o#ck%(Om}~reuT4e`|Uz-ZBk;RBk;l{Fz&z% zNuzlP**KZ8yw`v3|0)4b!WTl&fgw(r&5ToNcYJn$>QlupPRYFP@^id~^{b2_HX0?L z719BWGBg2%G=Y=P$tP=ZWb&G}D~#7K2%d6?<4LGeKXZ*?kw02Na!&Og1rdi_YJEA{ z2lXv*u#Myn=)IJbPAm@P!UNpQSd4MG2hF$!LHKt{qBS(OHltC}KgMl7g`boavD!kC zfcVm(f-A;@o*4C4OouJV|)UT1s$kTnf11OK&k4pa|d7! zhS-@*Io?e^QF)72rhG82MPNStX)X_4Ho|rO8gb~IMyj8ws}_7uU7=WN9e?;z5flGF z!=TMx!a367nhV-mq?`lY-`jJD*K*Hb*cYnOQyk44qAfO zSa=4WI9D2_4oCJ9QpX4zE$Hx4D7Dm>IAN^LD=5MhgA%+$>^df9qwUj4U_GMK*6MjO z0wqt0qF#4eE0>3)jG~FFsk|vW4t7>!oTuy?tD&vDqo^bv;6+hY~bmF?;1bL2MyH);g99K@0eada5 z+OPSiLvSpMu{RU{)9q2kq8uQ>!B_{a$l$LVcXqIQXXr98DJQ1K{6lt03F*h#=XO<# z5%Hlb5kkQ?WyW-v?%-y2mdnnvr)uqvKe#l^gzRr9W-?h3)+N!h(l)_XI3HP%E`%gx zCRvVvp2svir16LaRNgR74$!+G3fnpem4B7TcfrX<5J#Xn zqS1J+l=bHaOE9OdsL6jqrC>=<-SJV-FaHt+O|~7)V4BeL_&_&ZrC;EY&V#mD^SsbG zW*qIKw=s#&@knGcBZ6SOKPe~&<4X}%c`Zk(?iV?IucuD}k5DCM!zofm^ru#=$7V*Ba)y2BUYhfJiiVm`_b zGoi&Qaur8^!C9&Mu3c9plz35qH%$?6WY--L3UW%HqMc}+*du{+8l(`sP$Q$Gps7ah z#8YS*?43+0(7%~|Xh;LV48dRIj{SsX+NJXFle}o<@jd;<6*u$E%~7z`JZxBT*VJU%QVTWY1vlU6#QL0wfA( zWE0o8siC3KbX7xSOsnS`BY0-3$#w(;PR1-|)8~hZxIdJ8m^~jecSF%TEdBB&{=Bho zPOFTr0jfvsbt(hV7Tt{4qu_YOU|ub=`6}Z%TVyn2h#io`E9rH@785u; zA+;3#&+&_%q=?kgFbFvrW^dvvE*3+sJ65%V)}Vf&=8S_OX~Fh^Nby#{Nib^zQyO~} zhgW^|k5FHYB|+D@{gWtgyCj*?Yotp#P`g8y44GEBRs)0oKY{R*CZ1Ed%bo#9Ov&G@-gOkF8rk$OTN`vZJa5bCa2`YL0E^Kdy-Whg3ulOY8264W>ErZLopfe!9*2G9g&7OqP^Gn2vL5lbel;mT#f%h^!92 zI@j&auNP#9#{K&C*ITI_i;tRO!!P&|&>;Lc=s;Z#v^cMs?0){xKQ*o#fHm<;D@s4S zD|W|x;)J<0HDd=mVe%Fud0B@_a`BgV-1v~Nd_|GV;etCUXs)LZ->W{pRATWw}D!8!5_`+iEaPoGzzuO zf6R$q3SP1V>`PTw^n({eYhQRV|M)>FOJS;808F?=i3go=wLp8C4IU zDbzjkIELEryRIBzik8WOwziQfu)l3IC7TPH8sM4(`$LU&I7*qiFDZChN~+JEy{>u{ zny1At%sCvxcGOWngHKmpleS3%jNYo60$NlFR@Oe~FAR?)c)R1hh-x?bai53YbDv3y4aDGn$%$=o6b z4GG>Fv?a5!KTWq)#$4n#@>!&01Lz&AJlOGH?|#VjghAq@w$TZAcTK@H=EnSb+N**4 z-!7_U3Tq)Wrn}2B`jnVtt-56;fD8QUu`Tv4V&!PX8+seDDXoYW9h#R#MmODe28s&{ zdlBRN&_&;vNXce@)7?E4bV(4jJ@B((*7o+t>Jbpt<;=G{@$&fTkK0z{um6K<-(Uy~ zr*}HB*vLAYAe;&9Et#*14tqzyL8u5)q0L^2*jlq|P>oFK>)4gXfUjV0JVA6I=QNuz zKlEri$G7js9+&gi@MQw9N2V(iojr}&pfgNM+kJwfN6&Pc4D|<1lIf3& za+9M?`LfpB$vVH^wBLM4oX-rapf^gOMt-WO&Z!T)J7?!jk|xJiR80&X)FTw(s}K<- z7ZpB)H}Nz?kC|tOcHKvg1w)R;PcEz9=?3H8Peb67k;EzQpYF7JdiOpxcDL+VSLmlb zi?xp*Sf6i_*+2OtpphedXz?6N;Cn;?OfT*UU3#{`?Wn1x&D+7^k)NNftkm-y(@fC` zK>uUon?*JLF#&SI$0s_SQLW@9QRfAG?r42*B2CUuX!UgbH8ayAnf3;5@*F9T{dNiTQgZ`NUdQty_bDU8)*ZT*~XpNKb2ep z2%5IKY7N=m5qj`hg>;nGIw7D9Y|!48eO=CJ-@#;g?lkZ9-arXz1gwL)jkd-Kfrikt z1&sXiPIF^~nGW$9XJ7t%taU50n;lwH%Ujt>LZ(YIspa3s#wNlc-anI9aUG9xy@%4N zYkYELWhFZR$GE$FPet$03$gnB4Gav1z9pPBr10m;zOB8}jPA8*ME#Ll^*C%TGnTiW z?e58^W}w42?KLE0nsqY$THy{!JYSFS+xsr4;@&gwUgVWD5qMrUr7`p%F<3#6~*I&#p5>}~VfNY)!tVA*2*l2%-%^F34xA2%wKY->9 z#&#}G+^P!`)17Wmm%Xm0Yn)lnU{5Ntb0XdK_5F^MO10*OX2$)3w=t}&0nQAQU{F&>?SF{%m>#u8z zjcu3tcj^dw+VTFfe`o=#Sfml6F*~gDYA5R;Svxz$T6R9;F7B53)Hb&x?Y-26pHv80 z-bE@ph1Q)q$v(K?%ErdFB$KL1?B1JlD+XgpE;0TMNKRWJPnR9txJ4h(631&J>`mgo z`7Qc!4XY;_P(f9yT1!HVn|XIg?v9=aPER-^@Jff~HxtJIYpv%m2kMw65r`r9+O^Uk z#rSdjAEkerY}WASV|+&8L6h{Q2y{qv{WsuU+qLo=Row4pnePyL)%>822h)sb%zg&v z8!N7o>;KLIxNWEQC~})ay*j*gj5EEGE2D5`Z02j;5;*Oh4OD)l(bQyZLVtZg4^b>l zOp4)$v%|E29s$RlZLRd0YchlIZI?9yp%>7!%I$?t=q9LjuN`03UlfN9s^Rv1F>Vvp zE`B^4j*6>A__S8Uq`(*SRWMk61@fq9-(&_Xz93H z&j%^hrjAlu!3rgb*H@(Te(2&MT~_M=(mX!yq)Vhcn~mH!EzcWrR2kPTzSXoQ1VWcf zf|T?5hDQh5I9<&G3~Jt(-i;)uLxlSH2E=bXM9Npfg0tfId}5m@_<)W1k&ZN{zS5*| zxkqb6*;72MQ{GrFOQL*?TQjx!SmfS?5%3&yJkX-gx6Xs4bqo{Van@+x9PFFrMR=E# z4ymWHQyg+~e`%Ds*!_myC*CsYEsvr2u&d<^A6E7G(*qt-&Xh0P4A@xszUEu+{LUt+RP+r`>)@T6c!&dOS2h_2i2-khp;ymd*WX@Uid*b>1dI|IrlZ=b! zy!(^-yhyED4k(ro^%}V4ICo4jmwpewi(C6}(1{N=5GV=bOg%T&H-W-d=>5EIm+TZm z?kY}M&ni-EYmdI0T8x}w>%p(duuK-`>K)1&|C1%M{F5t6C#g&GU0?p)Qz^_M0p!>3 z93jEfi2bF=O(H%sA04lnAs44#%^l)m3KZYf+CmQmh%MNNLr7sJd_qx1A`99bkHIuA zhxvE|sdxs&BKrT#?NLNW-_0+{VF_)J=ACO(9F;Zrr#XYe7HL_Gd38PFDSh5D-C=`c z&^c{_*=D;n1uGuEhk?HwwX{2N__!}0v0YU*u73A448-(rl?8FN-^m8rV&r{vKSbr# zDKFwZ#K3#L|CU0_?G+~zl>>oJ4|Xm7ZM{F)j}K(!MzPEWs@SlP-G)C|m*!>-*Y}Dk zv48*kNl7uvT=$uEzfN&62^9qGIPG#3eIt+YY$8Y|x_xnA2m5@vTG+LkXC?~qZ^1Yc z{P@Sxtflygr?YYLA)|rgwDlqoEW0;hLUs^%(eVN)uPLq^x=zU5$5CNoxpHs+bJb~0NncB&xj7D*G`YE1%-9NFQ{c@)$p; zn!;?qCno0rT~yc2L8*vFm= zUJV0SvF*m2vA=juC`(j`5bQ)5z=QSl#G#Dou3@^}T*uP7`KyT;21aEGRWzgzKhF%( zf)}p#1od2P)rM`Uy8)_k8P^HoG-t8u{KJuGGl zd@LBhziLwOxXyYtbk8FGFU`4uLQ}hc&IcVJwXTD+4cCP~raJwUy8m3u1%>n)Uma8O z0_>T(2x*Ajo9?fe0XsRo)*U4`9qJ~LNJo=!3-o`jfx0}~nq7zu751oNyT$}_;UsB1 zN7C0}FoA)p4wEUYRc$C^MTir6>4oY406*J15A%t?`c|vbV*h}y#Gysa0;n)k)a-oF z@v5Ux(^!>R%XF_tDI?{zzf*RaSAI}~vQWjXKV4Y!gF^752Nj+qIbjm4M2o*4gu-We zrl4bbmuU@iGNILBy6eb-2f;L283Wd0A!du?o4l49!dA8wR-UQbE}BoBu!+vTZ>gez zY`uDIHZXC4X(_5CuatXl3uJ3+u3_vy6Bjre4u^Vy^7D^~W!m=kUSFkzi|$X(e)4!F z!i|KFxmFU$R|*FXGLM9`2NQH+m^`fRZv7?a@7;l(RE+w%pQ5FnHwZK1f;pe^ohr7fi`R@~j)T~Z2^;>F#HTW}AdxVt;S9fIpi z@4esOTkE|)31sDj&pBuI%$%ZsctJ_w zju$KI#7dk5$=IWsl{bs&fg{qC2M`TU$(g{0lrn|IiTx}=yoUrmOy~Q8SLZY;-kJLv zoRVaY4feGaYde0^$Dwf2-+d{iBrG@^8Wl*KJ#Bv3^R=Or%GH<=BiM*Y|JZZk$9}uJ z&5`x*v?`s-Hn;`N6U{o4U;Aa*1n#7aT#IBg4-6GOrLLGSmE)%O)MlnnYmV+=KJEH; z>+Z3xs%5?%C+IkAGeTleI8k1ZOL(P~N`$!Ukg;>e|wx`Qja&yOO#K`@MIJGz-vLtu~ zlJj#gp(MWflD@bk;2Y;ZX_D&JamLR(?-+G^kN=$|AHJq&^?Km?B+Ga8F9Ky58)*jB z*>KTD+tc>t&^mZA-qO@Zgf;Eh41aS)m|f`r>LG1T!BebFC%>d;vm^-xt+^~FnDEBB zdxc3!8dKi$59*@tQ)eM(0KBZs5@D=V!Tc z^G2P2kY*ieFYea>UMk(V#!fKn}`hclM;IVFY8qK(00(hK-E zpps|i?Ivj!YdKmO$N>O5{=b>KXd+ z=X155xto|f4&CS?X%Bmy+q+?HjAKkZEm9X~i+!J6^-J?YB4el46xoHgzo)(#n=A7M&>mBFP zSdO_&6scA3w)lZHCWCIYGY8YP#c{lSl=B{Xss!GjpRn%tM@5bEMnf0f(2q9zEmWcd ztJ~SD={~(!+8=j4iO~tESYb|Mdy6-59&G0F=H;rNu+{0)Q!{0$1{cfWu=#6gQ3Yuy zBxPQ0L41`%Fh}!(jSnZ!r;lU9L~BaLpS+*1k`q52%nT#1d6^*Kx+^r!GiX-I9vRLE zs9dZt@(a37as6yJxnR@J$%9Sgww{K1RstBT2%zcgJAfwpkV^LRE>9R)4n5!$QOy4; zZHKQ9i7!2RWHXm%$vcQjCkGD}s=k+|;8twMWf+~mQJ#-_(kLts|5Q>xl}J02g`Y6% zbkq2A{oLfam&c~yI9D)D`(qiZ$%&BA853_4q5{Q2E!Y_iI>Y>uaz-m?>Ma4@qL>Jd zK?}Y3sVs%Oewc8YQufSCJX;M|6p`?oEeL?2+cJO&=(W8mqZRRNlfjiq#__&TDfsTM zwFJ2_1jGbkwy2%I2L-N-_;o*w@&bU^zL#@w5X=SzS2Q}T8C*(Iz8&XKO80JgL0>OY z8RSX*YMCY~)ej6WyQJl(xae>q`OyYWr{X&-26DzQ(dj*Q(t9(& zDdK+e<5p1IMiXz?dHcJ_%VKf)D%4iv-&@^R!Ph%Pghmy+&$uX^5Tp*Op2BH!(4`(4 zD67~vPPRh^P)baV$~vSJ)3u!L>~=@_<6S9gwnfk`p@(jv!X#b!f=KSu7t~&*j5V*2 zC55aWbEWbmMhdj)qBFK3G+0(D;xpP*b{q?-dE$dC+-j%en;9P5W(Dm>{N?-Er0a9J z6WrypTon^d*^<~o^j|mdtopHByfz`FJI))UW15R_!;+4$#NSo?M0N|+&DySxx@YBK zhTDV9`?>jBZZFv;>%)f%4Q!N?^B15YyyG@Q6_GzIMja1Q%+`g}So@MPCN4|J-u=3G zj*glL#-%5Gv$%@NmVWOlQuf^AkNfAe)vgV3>}xq{ZEo#|4Dg#*LF48wF~BzdH&>?w zTS>QH^+#~RWgH4yPnxi-N1e+ zGqDFOQU93^B!sK8jPvgAu!L|hxbR9h8EvO98d@(|90dI8+^EkkPWpCJTVWHfcEAHU z?vP``SyLJnDotE>2@}yMs;yQDD3RRbNtD0GFi@w>&XSfOR!GBk8#g&05WChCNaRPA z6Do1jQZtfKYkYSMDS#R}5(^|Q8clz@49)RL@`!U(TuUtb#HGQPZqO9aVI_Tfn0+Yz zv9rMVDZj>4`6GpkA7L>TZ`k>XJAdcLtDx;HCwWi~*-F(#yFtTaA_k}=ZjB|I(XrCI z_j`(>v-l3>`uWjgBNr@1zitU2qrD>4K7M5#dwPnWvz>G(z$m9aWkC;$fy^|Y{f4`r zD(N3_U5B(^O!sI%1CyeqxeT6OH@cyAUwHQ?>ek06CtEU1pYBXT)TS_%B@nLkC#tC0 z@a#3kz(y$07pI6WAp~_aVtVn3>aDz(FJri8Z(OM^b6yg!jd8C!+_O%LYO3BTsZ3iy ze_Or%+c%d5HN&E_irvBk@#(s!KwAtK#>RS|_9rSc&ia>_o-HG3JW7Si^BD1nU-`eb z+a{_Z#ON94nYWQ`Z5=#>bF~159(w@7atbzlH*YypIlMhrpi~G8xJilTd8iPZ7HE@% z-Z*luSb%XRVu7^H6O?K^Ckfxsp4bfyYMEgP&Cs0PTc}e235`zbX%YPB_>+7WUQ&##yWNjo|HNr6^pK{Xxxv|sFONY7rwsi0bC>$Be?4@dQKYpQjz8?@fw@w#ZpNYz=nhce~I=yo#BMwRE1_c-vc?*SuHQtf(-6_Ghn<< zVu4;0=gBocQM5k(c)02CLDo}`a`)4wuQ3EBudJ-yBf4))oIZzsP-koO>rp8&Fn12gYZ2Eu$;)}UfXQqZN6PaSJSl5FB&Skd5(`)r;tZe1ctOH&&}3MM*4V}5Ahk? z7nyts_g-qvYq_7 z%*=P!^Wu7rcqgy z2tU8w@R7O=s3YzO@9z-nMDK^RUWvI`qVb74pL=z?BaAG%!_P+&J~O+EvA#=nPi)R&{GGpkD*~Q{ z?lyBI9=Tl1j?u}+neTm4IAHnZrXk1LltNvNhBe1OA_Igr^{<;$(p8CZZn;T6Yl}oV zd<2S#>AHC#FtC`WpxCNiWqUJKqsa%=wq#<6Hj0l3w1?^dm=BP!)5br&N{s`Fu=QSm zj$8V=*Vqvt_)-urXz6@UpHg;W_J~*t>>)edn(k}ME@n-bPxA*$v>M8*=FI?H88QQ* z7>V*KmCvY;x=5+f95;aStGJ!t^EKf$z_V!wu2+RXe1NEF#|I6!*xJy#rEieE#Ycmg zT;lQNS4^G_=vW0ERiGndC7AUNfromh^q=aTIE6q$z^dUDc6hsC3$@rVlRn+&I#(Y$ z9%J0<>0P}O<1xTTQ27$rx==#?ShG)tibfysY>PLU5w4Ls7iFiGcy{n@i}7zhoX;6m zfPep*#Q#EhG0W%vCrMv?(Ss)86>(8+*MJ;uA{!B`%gG`{K1my-PHx-XYe0Q=3MR7*}(}Ab4`(BDO zl`<5=ovKqwEdw_)WwvC6YdZ1ek_*q6FBa+WZOOV8Ha3jqiilSOorHm1BIibCa}L-S zVZZK#xNG<6u8k%?759_LG}V}3pZo3_7b#D^0!Ux?QO-Eq`CbWj4Fq>84D<5j!)J#TK|bA02@ZecK@@=e2z1#16r6xug5ksrI3>1)D6%y~zN; zX~a(r-3<(B$Y}WR3FKp1ykiw0=InSw2nyjX<#k`6+U&BCrn_zyXFvN$%-Lh305&_F#HUI*{)4Ou9ITDJUOYC2F0>PqCStf)qlRtB z){WSh5a($rgtf@Lhu>WH-4h6gFV)z=g`LR5K$}ZH?C(AXdi++eOj4XvNQsS{*ZMwL z#k`3bC;r^A&Aodok@MW=eEyv2<3hbrMcKom-z3cH>WrdHw>Wz<)irML+cVG6YOkve z(Zxz@%fsm~K}seFbrBA(V9ov7bot&~UF)4Rv-(MxSI$P7Piom+ku|gUFg6(`-3s}| z61ptriuI`w}`|kCcO^{9J+vp3e~ae786Bf$OZIVn;t1x zmCD4^$tH)t*HLBh+evR*ivTSm{H3tLn>|TF5*0JZY1buXdb$wr5~rK#0|m=pV+u0{{Ak99)|$A- z`|h(O*1@PREaIDYOJC-loZgP@71Upq=xnt}3!lDl%gr{s)KX#0)_n1)ocwvHuo7Q% zVN|Af_{n5Td*FDZ5~Wi3PmQNky|u_-fwRE>3$m}GF&eExU)DkAI&F6H2ral&8e)RE zJ^m!spHA%mvYXXgzNk?H?#g^5SV`hf@!#@YySslX%&VPvTmYQFrgGtUNakwkoo0F) z2lb#ZakHN5pScO3MDSle0!gMt!i1XL!~lBGJ=IsB=`w=O<3?pm!0dqslka2=J~zPo zE#?X+XjYS@pL5NCeSS9+@s{c$#3Odn_@mevg9)}1hKRxeE#>EJ@+geptzLomhOI_; zBuIoM(@|1o1WD7QG9g}TL#CTfT#?h}Z|}iOgZ(xiLU$=w=li2?%j-6n28SbWLUz;| zOaX?S$5hk_lvyhG%h0f>`!{-S-o?=7+kPj6AcO*7_gP~Lo8Q;? z4{53MX#ux{IX7md;Mk4YiI#7-KY;7BtiODB2ac&gA9b$>ea!B{?nZ@(qBihrK20tX zn*uCzndllSggRdm!o1o_Lyg0+A)KIoeMHJR_S)eGOdkeXX1rASEc(a$VZIkk7O*$di6jP^WV@Lu~XW3$1ZL2Oa_OD+5JIvT+ifY-5; zVz%_#s_Q^}-^?$034EZLGE$q(VaVM(GQ~OY>L+TbA@y=C|ioZQh_IH;pCYYVY1c&nkz3Bg^8;*nM}ka z-z)f-Y<*1i#qs4}qy4;p02kEL6WMDuPjwzOu4HB$B2U{j{d_$~3Amx=XZ(B@peko~ z#piXqfX<-~aic$AZI-Gz=Fn!z<{o4cIs|uz$cH4T{;1IXEVTb9{Eh%;K()K}k8p=Z zhxj+jgRV4oU%qIUf~Ai4;2HAHJW8WF1bcAZ7-+|NMNV|&tH>G~fdcJz@4BzB8P0Eh zHCy-ElU1n{QwF34TP(I)3R4=VPid69wl%#T|63z^pA6WbC)sFx=N4ij1a^OVC($5D z*AQ%4a&j{*2AVPXUK&6Bn?sqIFipo!o|&3WotAy3L5?LS*}U^*(rA^jMnP${@#~j$ z<$a3N0af{g`PYFdTqhknPTkqe<9s4AQj_>2n4BuYYOg}g(~97Css8|7^ta}~ieIFh z(l)yYtv`!8KRF$9+xALRfoja3Wz-kvxo?mRMg!Fh*Y6b%+76LTH_cGtQ910*G!_kr>{@kFXZ z=UgO5V$R-3*zMUG;IUwrOoKi4+%Vr14gPeHq##-{Uw$dqMRKkoy*IiNP>p`$NP1T9 zLAgbFgNxgVn?B|eIdzH$sb@_toCsklR_?dkM$HlH1ES8FZ%BVWQx<{H!2IoEZExK; zN?&Geh8-GI>mn{WSE0)u{-hTD$MBvT96N!#A;h)mIg5;B(JA_UmGxtYy&4Rhg#s(g z@~B0>s6Ns4OQcL^y-Qt8;a9)!LyLWcA*CK!O@6y`NNJou-y>VTVML_JH5va-4jC7B zrxVNwbmPIC8#!rU@ZC+W7?SW%{caaQz2k{K6))4p3}cls60Vs($6##WaekrKokWd+s}y}LjT6{0-$4DXj( zZ1&oyJ_}-akG$r6{d)QvfPPmDALCSDylwGAACh(ii?VbbTw(yg^R?6MLeEf{0>6W( zRdr1W#itPu#V|x{)n1@;%wVfUB^50zt#wWN?w8JyfJ3oaN^Wz*Vmv5in=EhX?}Mo_ zty1`vibk4%lRmR9a6v(I#+3zobZiJEK#Z}cqoH*HV2a3f*r1(iCqG%T0jE2ZRG(^k ziogfZH@n#KlXOCp$C=CnqMBOxQ%elZ7g!)kq2?==|%~ue(z#p0Lr$NK#$u zuKt?0WELvL>PFW-$9hbEwXbg(Hhy|NLX+9)hb@J#B{X(%O=LCE4{kG1Bi>T z$;^mt>SXx5yHv6C__}f7f*mz}nvSbp_MzR0csqr$eH?TA!Js8pI(WHEuphI8E<^KM znfgKYWw`$~j7Lj>Av;jP`9M%{(xpjxPouLJbP8j-`&)SV8|i;)0O^h#DxB^DN{NHt zyE+0M-=F#+7zB`6Xo3u*g`D-=IrTl^yaiHI=2Pafco_ zaNbMT1I1G}$~P4TAu1}(0>Ol>?9ehe-VeYb#J+8{#AbTOeXQu;`g6zL3-j&eN;yky&Wl5#=2{7ws%|E(izj-+gO-gAL8Auc+^;v)NsvILH_h|V# z9RQe>t}GKV{xh(e(#=m~%(OhE!lpBAT#oU6oBSB_!Ef?G=9Om`HcuCG6PK z^9Ow3?Ey)F9hykc*5HVH&pns*g~)vi&n~zBJWUXte)*`Pr|)*oSM*iW()|bgeIEIt z)Ni@0>R&2i^=G970nC-*XNtw@op;Q@Z*z;pfLz!h%(Ut_|DVJ}Z1_p~f$`CJ=#!JZ z5Ym(B6N3|{6C+DaB#i+8`M_g||Ca82{u#?rt?ZHj5@7J`oSg7=Z>WUzpm2{qA)iY+ zwF~pAtiZ87@_FZe@^M#Z=uWZ%YtJ$-z<_Qoiq&JluT1n}>GkEPvWkw!!>i61iPYbE zveeyv{o|qn(q2Uq6XUHjH`L^*tZRlle-dR7FWZ~N#S=te5 zjv(@`cO9mE>@+zi(#$2ck{mIgZRQpv*aPQn){()T*0R*~81Tn%VPzn8ne6=?_PWGr z`us+_&!+5IozYqCu*s$HXI&5Pr_Edl=jz6{-exe%^9W(q_M0kwMTFm(K5<2*aXTj2 zX92)owm^Q7X&&o2U2(LGYojsY7{4&vzyal3(-wig{>LESvTyxo!iOevdX{>9bh0K112r7LqS+h1H<31hs`u+ z9E@mZUvYF7I_bfdb>i#(u2sK5-?W$<7(&78K-xyzV z5&)nkJEK2sDX8srP@1Y1XexmT8JH4Ol~gGhT%9=)8pK1nR>7_Z<~Y=oXKLX`w``YI zFsNWtQF?=8kq%+OkGp1p503>rUE`pDi-1L|^432$>F8++!?n|lC`<$}9TX4vzC7;g zEBvrkjcEOgjE5^{x@O(+*%P01%&Usda?vhU{q}fHFTDXX(6S}e`CJpYgP>=cp4WJ| zZ}U1Y<9_mAe5Z{CXbVva5ey+exji`=hLod^b?Av*n*q}jMdxpVbN%p(b^9 zoB9A@Gi`BRy-Rx3>`Tm6tJYwaszoa%MK51b|AP0m z70Bb9?4f$E`sV4@XeSM*Y5sRTAFfN3y)On`hFq~Od+D||mzSgMM8o)!HPW>=a2k;h_!8!=V!a9=-(=6>tymsR!#iXN>7{A=Wef2#d+)#=R^fC9sWk-)vC-+=|5W$I+6qM5(&-+ zSn+hQ0;(vV=d*f=AYVdCLcjwnRK2#koRIe4NE)bJ$PrfB)-EgjLSF;hH?a~e8((>t zTp6AASAi=JiAp{K0tSS^3+UIWw1+U~(hAbnM3N|F%?M#M5SN)B{(>H!14M^wGVt#1 zZfDu1d`746yA)pn%}!ymOBAJ_*g|EB$A)=cyki|Ug|SF08>-6e+`@X2e|xet4$3rs12%xWiwBi}ojk1$6f*k#1;c`Tsqe(654|+lOOWX7Z;O$ zmtcZJkAcQX-=JK3tX`f#dCeDaY#WObblHck>Fdc%kFf$4px9y<1JfWr*elF;OWpy_ zfT?L*b=H}8fUggI$=V;~(NgH|eH3Ht!En6(-ogy0kS za!vsM@NdLBQjOAbdfWDWW4;^`b(m*)mUPU=0H>5qZm(w?Jw+u~0Q;yLT?|Yd=Z^*H zS)lfO&WnRfynu~?@`dqPwOWHZ*}AdGP`#3p-QmEK%9imbUQ?QlH`|KtrIW+>tzh$x zJb@|oCM+j`n~z>G6uu`XQddW%XPf>W?uGB0$c(Hs zvMX~B@Fkp`eBMoVB$-fnWY=2+tS1$XFyIdbKS>>y71Sw8G$vcDSmwN4n`UVB*PJWx z-sJnhEa1%h;7dUj=gEl0(e?%@!U1 zJfq^nzrhx0sP(lxHV`K@ZL*#hfcz#8cdaONL#^}3lEB>zw)@elHAsAeO2)XG?a5Lb zlo1}tRgi7JXc_{kAh+pGb7zZfYj0O$bmZMfXGh(sqb%ElnH;AG44TbUv@hMx@^Qxe zsvo^I5F@l!XR7|me7@F)#$K(y%Kk9r{=)1W ze2SbHK0Ase8T2}1*Qgoh+{jn?Rbv(YV?F%8*r>cTMN_O^tu5Ja@ljGq?9uf<_*eZoyk!!i$*bHY6F)Iw)V^ukRef+bssy?wwYT%AUwXAx^Wc{BkWB|=b**+Nb z4D}3Z&jAqWr1K_$z+c9u<1OP>-^F+HlANyLtMZPs!GvLF*BxT}2YpUG*xG=n$){9A z9+V+={O4A;GNV(e4@B3{c(@+`B_jDDR|5jW*_{|omO36mNP6NRMa1)xhpO?2`k|Dl zfU9&3DGQeZQ9$a;*(2m4`NxVu7DGcLFN19t_;`XdKbvDgNO*ATLTEes2mT{|=QTTgMi&z)f@lWOv(>Hb zmFuKwuBA?oJQnWJZAdVwu|kevdP6YIqO0wK;d1GDN?m z5Th*)V&ABFN#c!b@w#+j)BSRv%gX4jHsAJCVFlUsX}E~|2b{#VCBMz?v?5nCu05rL&&YZP=a)3uwmd-jswuPxhCdH=lIz zvcB}_&H*A&@{U|fp&WGAvCJBONN%DfHFbqV=H^vbw>Rk;M)O}ez4@AG|9}E+8X2$= z$(O%j%FLnR{gggGmxOscVdw1Z$>%F2^fzH zq_(*|zgd&js?45wQvNh?4Cc0(Q0m)JBfsCLb~k$)NEvMe&vHQAG{Mfk!bh+Q)N()Q zUZz$AJV82Lw{kk7SP3>^`5JZhF^aG08-lZvrEco}tJNjzVZAYQ+2CKF(b=rOtr%lr z-KQ#0ucBW^d~-%ULg^P;$7+**{p^yQ96UP`n;oWDICiRDeNSH1ot4_jb0aayyPxlm zl{fom%fQipI@MiJg3wXAFXq|E1bzF|gl78%!>Cdd08*XU31b$pg& z__P!|(eckGj0xRN-7;PA2+*VW2vm;;GZHr-JedOuCPN$WOd>4LuK$!yfXaggQzyrR zi5@G^^33QwQ32#wV?Ln>bWF=1|7x=41_`n;eTjH&K674r{8t??FsuNpGn!1u71}wo`Lzd zOt+krlT&1~WD>O>0Ex&N>iGWJt|>8ih(`+_wP)7%+K|_(zbeWA(gIg*C27{SM%YNH zrHRn1a@F1rr|mW}sg(J*-wSOjd93^Fu1C?r*q?n2Q&OpUK4k^q%N>B%w{e~XecKg) z%lP>42rD)PE0(HjAI`l9%GblAT#?Yw6p0Y7k6AyL3Ki%`k&U2d5$&Rh`JH)rjpT19 z>q>QD&U|()@jrG!FYfPx+uFJ}A&eC(d55Xt#>RfTO`QW{xfUE%_jDmk>CT2s3Ns3R zSw9ntqCK?b51+vDdIRA@KC8aFJf$Jj0*FSo}Kkl-gaYyz%W&Vd>8m|y8YTu66K}f6uD+^ z5Jckdgg2xB-IFq^dMlBR;RG)4?C4mpd$_pQ86_%w-Q+y z$gbdE$&AyT243@(&m8RUeQs>-GaB-n)h z8Htk_Db*=TZ}4`k+g4#LP1zBhTx5{EsbV-k$o&4RN`nvCe~$H+0gB#`Vh`T1)z#47 zy-)~uw37Sg|IWng%fam$;WEU}gk`j{VfOoA|FyzTY0E4T!Y>NzM%_OTc<5{61Y{Yt zcXOTf;`TbZTr7RwhpZ@Yp+;*QT^dYP_|}vv#6O=>9DkRY8?Rl?OWxSzTm655?TAsVxwrO%kL-Q# z#S&AW9U=0{{mQgtbVJa>`K&E>aGeH;b!i(K`d+<+eSyuo!sRf9H%7C0Ftxb$8)I61 zEwin(^IUacIhCSIXNl!VZ1Kaa0Z|`0*&MLWOKDym25H&;LjT*-DTK4z{Nd#PVd;m; zbnR6Bt`$u@1BhKpndP_i$$X`t>&S5*MuQ6W{T1(a$oOw1jVh_41j${rqQFAT!mKC5 z3yN}+@P+gpHt9W%4-&;X@rqu@<0$h<`a@a=%RS3e=drmLN%c%xE`j?-gD)MGD^gMu zcx53g9sx5V^0Iw*bvdU}1Dy5Ef1iOXR!<@ZO=Vma1I2hMYOs?RtrZwTJJ=f7SGf3P zKU}q!i~j0M@@I|+G^#4q)|zi{snLb1Y(7A{zto87-CjBIgrS%Te!Q~&@o@gb21Q59 zcVuUoY8=)gzz4zil;Ml*;rFwL$3NFaE>v3LAj_RxivfA;sA3xunRbrn|$KlU4P<_1;%}!{(_Md5A zkK3VYI_;LJ>OJ=>QW{4>^C@Q6V2^huhS#c|Yp=vLJ*rv};*%B?B{y_=Vm>61uW-$1 zWE$aog|gX_0?nY*32dcq{%Zz4!8+av>iu*L-`L|DtkruUve5nRZmKNH_-&|Yb(Zr; z-)ik|kP+Ka$4fjS2l_3{n>Nmj4bX1{&Ups+N~~g2NPqe;fI1{*AqWs5$RR zA#GeCjMp73JOhyKm!*r5ONLQ#c!<1Eng2#?0_<+e#LkV%4|u>trt3wf`r#|@|8o3v zUECp)5n4_X+rCu0D7)U%s?}8hrxzc|2Ab}68`todT8x24E-^gxE4KAq*Qi>D!xwl9 zpI6^En;r&R!mj>~i|$#ukc3=L=h`cIh8`m(>}q6BfPsnBpfJYTSCbAip6w3dnn!`| z^`R&S;kLG_iM{o#oEk!r^rF^`nzlYk$-=Afu+*LKiOEHDDeXe!jjgGlHu*hlKb}l% zF?!*AD?ti5&dtvTdt)Y5PN7x0p^$5C-!qh+x7Fd_SL|>1G%C9}bo)14UKwo@)yU)& zGLs80s*Ks%9v!JFEMPg9d7o&|KgPX2Q?GulnD_zzB165So-Z=_u?OBTk?^yPmSLGD zZV#D@m9eLyOotfaNlDmw!N9%h6Ms~t``|;Tli$Kc*`oiP$CAJ2ao)tDc>VF?$NiX1 zKmVq@koc6+N#p*+>27-|UfNz^9?zb~R6>YJezq*L(WNIGz7kS0BS2h90@|{>m9iN@ zW6h$6<`Zp3(*3qgyX_b^ertTX7`>uPR~`pVCd*gLtzhW}cm><)4XTiW>@nbhub@?_ z5@RpMW6#51G`>K#7J)K0;GQoP!gKBb;NC@hfn0iUp0eQIJ==NuPtv@|i$&%`R=61q zoW{H|Mc2T3-6rp1TRhmRMk%pQe0h`HI@mxqEI z>C^i9zEFmHl0ksSWB$B|DF=cjW7+vyX946(15CNJ0}sGNS-J^c22EvTD9_R-e8eJ5 zPeOy03H3UyI#NV{WzQJ`7%T4k{&U?g%bzl}_sgi4#J>S;T241S=Qhu->*c4ROd|*S z!rA1(Q|I-h?U^OKrHEUwVpadce#+}ccNsiQH1p5DFTm$ZANDj@)=MLN(WzKJl<+jA zaACZ)#(C5+X^-QS`)-I+hRkO(BQS1cL+_NPPL3c(mUdKS)UJIYySYreS#z&sKd*PT zQdys7Y2G-pN{SXUc_I01a;?|_bR{QW=Tc_nn6-QZvAW{FGQFh%%?O(fGDkPB{EdLz zSU(55P1F=}@G1klS)#5Wtc{2-J!J#e1L_f+WhUS7g!mMIY={oSWHj)^I5jZ%s`E9< zA>9`Q@(n|$w%i+HJ|aKWrvV{7?^!6Rfvl#ul}ltPz!#0yI#uS}kkQwHXZEz|WoQYl z20Iy=vC>y9EhMQgm;4&ba=`n^GVu5L!LrDI!oYwM*5+sOK(utql@hP_5KCAciB&s6 zCH6N^|9pT2c?b8E7tf?9=N)WVC5rSYi!>pnoeu(AZAlFkBIDG$T~9qv!{w)68m>orAw+PA3S#TsO#JZZMyb zWOUNu>v$)9-5GGGc8RTZ8pn+q6ZCxP2xC(s>I_v}v{uQpn>QCI-ATFXr@hZ$=4Uh% z>UuZi9!q}hVC-0q>M@q_EWiGbPVg!THyQeJuKe{}-OwBQcGX{;YGI+p26~6r2R^!@ z0u;2gHsWmOpKxvCutTZhadiJ^|K&oFyiv^8*9P)FY!vsqI0UOg6c!QgrO801C%XY_ z0`5p3YBBr0s!8=9-<$-O^mit#l<(gMFO>O$KD+9AqJ?~{E~S%rs{rVyzw0k|fc;d+~oqj{$2_7TS*~#&b9gwC;H9lH&ihgO~{pMV>5* zIb3Oi6N6;kuS`74h|^Z1MaVC1Fb62E@^vV;?uBS-w*`FmYgA*mUm9|6+Ih$WNC$NsZKMTT2vJ;J|Ju zzix_{nSA=EFMFPa_8sK5Yq9S3N5s<2`i643TVAIp`I=q3nuYBpSg(;zK54Q0Oy#_n z3!*lS^ygG2rA0H#7Me zH}X`bZbO0*-c4|PZQOA9X83 z=h#7l7tvVwrVvUi5W2B_w%=~f+6xE(VdHA*LixlfT;)#Mv2RAOCoTZkU{Q?TjyKq; zx@9n%iHRz^OsZ16Bo7E&wP4*7pqnoWOS`F5%-Q%Fc-;)!2iwo zV=zL1zAL(qsv5C6PSPxgZyAmNODCrTBeQBS&WR5;!^&F(?-U3T{05KR8jdE{dR^N- zkcYeuc@*+Qb(qr|RGsqjGJocsdI#l-0*Tq{S*wsIA>;&^g1Q3-SUdyJOY3CEwMbUV z;I36!ckr|sIO%^0C3@NxAD2q#&LF4jvc8n9Y%4p6aX8)Dj!r~j5c~9bD3y9iL@5xw19@u%oTI@k}z6r zD^Sb{evP9$etT*?u5qvz@*;FKCX*s;LDKsAf3q818NLT^9B{tI$|US!V8jDl_GYBf z0P33U|Fi)1$9AWEc)fdEUNbZItABOb8fllwL$0jswx!m7H-*`fQ}chji0 zG&}8h`w%8ys_kVV9FOKIl8Pm->5Et8FwuDwDi>RNi#F9YjvSk@_k9z-i0hW%;+vNHz#Tv&8{z&*;_n5nq4OoL1v=L*+M;q4;2_Z zodyD$mXVDF&senI_!m#3eyYIjrC-);nbwF%&kP@)fL zspSU3LEZ^;AiQ`@&U}$+OnBv|4=Gyn&DZ<2{y+eB(Yfn5Edr{D>51gy3Jq$F_ ziLPGPDsSHYH(yD`IEl18NkAB;b9r&!)j33LV7r~grkbRTv8 z4Y!E$@oh5XvZFp$c+zP&{bsp@8`JomJWy{ZbLLv1R?Sn$YAJ{^2j zp~l?q0+}0R(x_GRz%wmXt4nY{i~Fh|#V(m|Q##J=^(!lC!^6|Eb;JGnF18PBhIgRh z#*_1q_aAJ8;4eTE95*)o7{62{@F68gEUFAT!zhB!N>BG24$OLEi`_5R;`YAR%cRVZ zRHyAI4~7Nu0esu3J8RW|FO~$GocCtxPewj7+e#+=!A{2kfXK?7_tZg0&B`A-+#x&# ziuso5=PuZ)iul0WL=tn$MR>n@GMbMyKfxdZ&tQ%{4*m*OF-eDPm8rS_j1Crr)f-*Q zfSi&cSV#eoi+LWpZ&WS9#1V~6wz7S2{!hIuo$HlYbJqo!ZC#pSiOc2z|B{U84uI?Y zK?$occ{DwNS*`Y*YpD#+0LFf*f z+8JrMFtKlTIejL4zRK6HlcbZM!nLyXw7yQ_ZR#TeLzdy>_D)AsKegh84wjqo7VG~? zE9k&zpBW&;sUeZd6TE3mU0r?~u{m9=+3);31d=el&BM-CMeAU-FmWmw{()+4Y@l~_ zo*S8NN4ODop7h?yq0xI8Pa2Pz#n9=t$M?*cE)QY zx$d4w(*f7R+Y(Dw{{3$xhJXR}0LgJ+trSzvr%ApikjQI&SYI~@Z- zE@vmu0(`K90mGnvO^6fz@|hf()$a6k!?ceZ5d>LnajpmANFbg?x+@l`wv72>L0i}P zV2O@Kq>^sA8aiqJrKUXS3g{Of865&xj!&+vkpo?VmmbC?2MJ%H%|@cX!|@ut;_(L6 z#eixbH=4H^${tb{e4M4nXZGW>z;?eXNEiH_lcw$(;p7de_4GYk{p|7jKoKcwK)S}3 zBTvwYITA(aBuuGy{Y!KwR$f6?-myNlNmPqbvNnE zR^`#ZobL3H*NEg|TM5@7+##nMlst>!+xIQAt&ZviGn@KO>~V-ANN3+rfsUhMuh0z4 zS-YT&m+)=R6H>^S7FR`50%pVTbd8_ z7h_Ft{q0FUv~^&|&h;nNz+!#zDS0Ymq1MY85&lz~2pj;G?K0Zws#@&%(~L-ej$l%s z7nO6>R>90^;U$?H`|}=5(TGw3F${Dj-_bj7k~d>BIgq^*W)Q<5`;)ybFRHm^ZNwQK*=Q(+{*7JAtGCuaZvRbxGG z-kU+_f;J9Q@a5YmD=TxJ)0K7KkEAsKKTPr0JZ`n8hK0QpJnbk*ktejb6m}VSu7>`gO#uH0oL)m>1#nx-BLgQ0&3KjOw zm@2C#k{hq`oB{kV^Bb}hzm=JFCj0y6cXk4+jG@q79C;`8zv{9!Xs;eG4ya_U3^*7)T{^Jlj%cvpG4-p^~frgF%FHtKi1da#2B`;F{G09JkOIQ24WA!mp!s&q~!e2}tmER|^QmfxU6>AINJ zV!zG%Q>F5@q|*7%g->ccoh6T$lKJRvfGSD4wu$>aomTkg1_e{9ceMZ3#f^y%)0_H) zdcJXEcJ*R57)#SHC^5xj)pDwVyH>2fcXLKvI5583rUXO+J{*1$B=Ow^GFW0kadXWZ zDsDiUAQ4}@P2=PTWMoGvT8(rBotmBX5vZ9ei)jH}H<_kdKCzBQeyp4Ls@RQ0QVcV3lKu@xU9|}Wa2_erzOe|RldDn@#(pExT zbZ05D134^b?;l80I6CpD(nLjP&Poo-1!FglyEiba!}?&UR; zWji<4{s9L6C>qp9kfU)?h(nD-uWzvq(k|3lVWhDEu4 zZ=gdrsB~MDbjQ#Nh@>FhBHi6Bpwb}C5Ym#;Ih1rWz|ajt4;@3D*}C`n|IT$jy&t%) z_hp_~aj$jXi!yl^v70MlBRLxP_{9~6I^Rm#O!mx4f78Q3Az$NsiYk9Wi5_I<-Hw>H z$Ju^q($_7IzB>HYhYMRJ<=U4LBo38y&8&gvIe72(xs_ddz&S{vmBc`^1i0#YzlzoV zckbQU;wTX!`jW1BnB^y9NqGG1T?W$)b3g?*<8a_JYqA7(`MLzOd7`4rJlGV|EPK2~ z^MT4=?4nq`eU#}Cu7N@*idyQD1`qhUb z%lS37hXpz=Rho{V$AL1Hj_>C``GBX0w^bXRSCSZ6_o(sSj8R!ScM%9`GsAt?P(O^d za8%9fuiwoh$##fA_A^UTF%#7E zc!(_D72K${8U~%i5O)c4GH!pzKg6t-= zB*!dz5p)dt6$-D4#UevQ5{w6eJTYiJD709hI{oT&*^7x?SF&D{r;6bQdthAh+1mNd z*{42NVZxTs8$t%^v1o&viEGwGHy;}fvH1Le_*t18EI4q{_m2$=w$o zsNHluW}BemyRo1t_+AKS-Hv7GuC@DdeiL)Ca$d^|pW3zR(OPFnx#}-&>q$nLgY;gx zwKrPKuD5g=Vc=1nI9BvXBZD6Pdey8)cg zx#4+5+9GMUsR$Ld(oR-@0xn+h~YyaH^dne&mDemUAur zBkeNIbaAzV;UwT0bPX!Y{CW1^0MmeeY@nvqdP=ZT7PeL;3^_;wifMV<81|6V_RI9J zmOzPj($qJV&*@;62p%OGukh!>=|pw$0nw2@4=c*8w+p#Vn`s+I!56DVLU`(h7WIKK zO)x@E`pZ(_na^MQf{e1WTPG!c#206$KQi5P=fGmypQ#+M7!B_5f`J@z>OzPaX2k~p z{s8W=CxL`BfEuhZiKA)-8cE<~5V1B{{Ipp3T9GLqpOal7;kcgD_x01iSb>=a3PL$s zklIh=j)`oYt*A*bgoasymSDSrCE{1d`QrE0o+lT!3od>_>*>|T7AWmHmFjKA`*zS;_s(-`t?)};{taT4-K(cMwMh0$giS6k~>&{{}X604d5LvsA$a4-- z=^u3N4o8>U?!H(UP7NQdJ=ghvg%FryQAuBE67`novRox;UP+IjA(MukR!W4r8gSj- zwZ4(MFoi~bZc$7LIENI{k-M07HT8{t-#(Pe>xLYi!u%pP?J)i1%OUjw)5ZXU4+{H@ zXwIUI_wHzbw?^5VLE(7YeJ$lgRm?M}Q9v7A)-CYzqhgk{5n>>XUqr6qFbjM`+EdH<|7$Y<>KB*f4E-2XVF3q_55}ka1hv zOG^R1eL{%Tm91)HWmHUg1nxoMIPw-8u2-8~L;PFRrTWKt@rhy>g+FI$-^jvwKnj|i z9rtEwXKeu8ZF!AzzWHBu(UIAr)hIZh5E%oaCrh+O5#yVtC4hkPU$EXM#-C##C>B>a z0MXTkhP_a_?_W2c*%JY{nUs%yH4&3-0bKt>rh;xsvsFCs#(W3SANH?Adv!&-g;esh z%*M}LZ`4nmOaZ;jb~U(!%tjPE~c0f*9=( zC+??0!;v%)JZ43GNoNFz=X>@8N!0yl4aFmG7C@?~C?4cJUU$$aR7m@&OtAi($+!MT znn#1~c$O4WHYOThoJi7}*$|0*qih{W(+{#%h;X!`Jn7#OIyxKa1~)zDmi?;-(0=_t zdH_;j1$F*Xk1&4;i(obBo~;R#VQP;EMqvf-gc|iru(kerWm4fV0hGya_YQF?G@wc?YQoXxqIRc(1CX-D!UyN@3R9O{2akl^4EX4hE-{ zV1|V|Bz_$Bv>|)I(I5s|_Ys??8!vgzH_88l7qulKg%Ez~I=N{iaE=@<^Xiw33J!b-T>BosVsxF} zkU8aRQs0)HANRKJ!Dmn7)|Y>r3xE65Fqi+d7RtAmD#wTTYWPi}MS4_D>LN&GXd<|u z$mS}0m2nw*{#!EHsp(M3Z`n7EAG?=VVk}`NCt>40?HYlq6JWpS zUVAnO#HSE&6cTAU37%{%{(gwq977V#k<@u92_BmoOzCjxD3PcSEO%Ky$8tkowcI<~ z_aQ};)gDHkAYpvpI_=$AU6@LGk!)02k=Y<}3X>*Htlxs_&emXFWW^74beR6bsL7S( zu)o7D@vF;qfp0pkitM^Is?Hxn09EQ7n=#uZWCuRGr(HKS8+Q!Ip?OgyQ*G3EmwQ8! z7Z(frUgP3(!bjmx4x$OSgT(_zjxotZq7N;|iGJkXnEvszj&(>sduJL?kE z6A(ppmB&AAma11{zvx|NP_IlKS)n2MI~9B6$>WB{cOAlpdtFVHI@RXy-o2~3to*?C z6@cGAFi=Gn-t2x*r5R3&A6WcRvM-d}(D+<9T{7?G-2Um`2G{N~z}?F`S4e@`9mQ{Z|!`5r7!F?pZ7YsdyErI=f z4BThUtdYuQv^Lt47{R;(naW=<_W}1vy)xW~eeJV#qovmV?kf+0*Qc4e{Ci&?Pm758 zh)rAj&*6(*JX&dO#=PH^VW0PZ=|ke8`E2*yy-IB|U28>hykax)kj-Ulxw0yfpWS(= z(B{cYC~Quh8p(2aIl-Ay_IV{A`advRs1~)qEU&o&DNKTtBA|cE9*N}w@a40`s**TYIQ=pC zkZ{<@jXF>JsaI{DBE~G_z+pJjEu*0HOwE^)3l!X5arG`Za2jx`&ThS2(%L5qCzIEG&)4U#KISXFVW_%n+7 zU+5UwrrCbC#39Ci!jF`ylSWzjeGN;l79U4z+iuzNAJRqYa72?nuUu6O3bfRg<{kOa zII=sYBQBzamdI*k9Qb}kGq7o+)aY5>#rwaV443avjQLHmHCH7qN*j0Nc8q3$cxOMp z#aByUE}_k`B>1ppOcbIu&bq{Ss|u`BlkHE&bsLbO%}~IV$t#;LRLRn*wPqmhl0Y+& zWPaiK)rqWrzs@N^YMY}h%bO+*-IH;-cMsdI^Evt7h`cIQQkiPqq2Gz<(2BadYCn9}=YIO-9}2Ntg3#}W+dKYpjQ%(aMLOTzjnLRNG>v~?)g@dmbWVGyJ5~bmi4D>fb5?F z@zWu_RQ+zB)-B4a^7Nlqec+OA~HoEu+`pN*Kw%abL{uteGv-C|c z9Vb|^N7h%|2zutL@SwyP=Nr6vL2&0j#D@EAN-W7mbY0~f!e zr9$00@5X6LiQTw`AI>vweEZIfyFXu16UG@8(R-Nq*H;;+H2lo+wgN8<-t1VZC)ziB zA?ShLj?o)d*Z^y_d<7zIY;~^w2Foj?eJVjlir9zv{Ek<0J>JxP_6Yy()X)hXJT%MJ zm4N)>0&>hM;=q3N_Ulb3X6)jXdA&)&ul=4V{Px>^3Cvg!b-Tq1KNs+q4qz4(dRZO7 z>vhiWI2ZZ8&GUKSejZA6^DRfwE0o!wb@Xfzlo`MIv6Y2e%=e}V^(B`<4$osj4X^3f zma}QvMPnCl#VeDxBm<0rhpt6L=M0r^C9li^Sn^-TW}$7daj1%G!rT0x{TQ5D%!gT| zpD0qc-|4_9#HeR%ZzcYP@mW-TtE(m^Ul}FC6HygUer)cIa}?|TW>6>$vg=M zh}+HrgCIU1v?V7D#p}L3=(ExP3Pz6yyy0lqs+biLH`p75loYER{B2lZ$H>?enw}p? z%Mcq&y(eKF(N_D=etb zXaeshey`W?`9K!el^gfk<3k56iDSTTiWQ7!3Jzj|YDlfLuhRy3NB1uk#b}u3rRdy- z-IqbXUs|2D=vdn=%;gQIq|=^pf+~t_84oQ@t!7Ky2$Bcj{}!^dvvC3QPUMg-Nyn$n zc!YK9Y5D6!3sc5KwiUEYD2cS6V1$pTgrXTdlIbl}&{{GtnkG-;_w7FfE=oGEE&5Qc z(9s4S1n}iPr?+s61>H!x8T*8fbfBWZID}@v0V;rAB*nVjT{1ur?VJUQku6bqH)s}z zf#E?kW7uk5S+s&gsx1X8@BGx;4ts*dM6w41MtN7-8pHR;o38$eSh7E`tVsD3d((h< zWZnn%izoO$ZRdyT+qBKx_fR`f%z))wwub67ztRE?O`iFNU=px>f6B4?`z!pL+J=rw z*K^cBS}n}>)I?S--K%QnOtpKHa3XL13j?3;)J?8A!REy2>x7P^{rB;Zq6{*r3J)OQ4^W9r*H@?h+C9z4ruU*ggSGQPjZ}=?I3X~ zkM*j}v^w^o(<{`6?*ugu zcj9CM<-4}Fe@cTXh%~JWt+c_Iy$Jw8oKtFYjg?6c&WWEqxbL_Yp>OXU1%(t6qtRtSJ75S zc7W7Pu00C9$dyTa#f>@ifkV@#*F}ld$mlOV^Z|oHFK{8y93W^+jptAfd8oqrLX3-- zDL?4>#{1IziO;|aUDA4>8^WbsfZCLT`VuLqEN<7G2I|Hj15l{r)GW8s%IZZxzJV?6 zSm=*MGr_7Q7lwu36DSPn#T|u4KX$f@dPaKn^!xpa-{n^ppf5vv%Q@9yx^=y=`jNqA zu{W*WHR0n=C`2aNAm=kx`1y`Bth8G@qV1~5n}5{)YXw@kE0;%5x9?_y0EbTLC|Ez| zfZTPm-H_#*f~8&kh?oPkwOP{=f!0Mru~;!3jiv^YZULwcC7#dz>(l?Q5HLcC8QEC(q}dPOm~sP8N$*+CdfM(*logQr5? ze4gZ>3)D}#wbo`RH)hEO=0R`u^)o^x7%faqW~0>!i%v1goUTSNuhq!0R`EVu zOUt5`SDWal+<2&&vpw_cCgxM~>=t^+$^_*NBYUC`;Qz+;2w?i9*|3~tCsYU$YSzt)z z*ft?8Pw!<0Mmd;0wW837Ss8KiyFEE;e<$HH`B8ju;wG1_s@x>+V0Yn#%y6zDR*`mL zX#8IK^A}+U-TL6skv|^&UpmnrX1~BcaHFj_6mC`mctq$tiEQ)^4n&qC=EcBsDA>J2 zduuR|hp25%xGF4L3Rqh=desb9{>)B~@~oy=Hj2sw?=6;v?Wc#B{}M%Al`8c-8; z(Fi}=9FBGaaiuR)gHvB-#s+;4+er=_^T#RK6SB>XkSP(>bZ21>`O0BEgYl$&Nq2## z!0Cg@2my7KUD?*z5r5`F!g>O&`I`4V1sSmgCasXk{)Uf@;4Yb;M!8jl2a7*9>D!8n zt#;yeFA&9PVP6|`B)2UM>je+hYTUyNz1a;7(in=(-c7Is`G5!|)Qs{mpNjd#``?y` zd3X3amjC~Wny;&3Snw}~XCGSc!B4!6Ov<{e-OYA$+f*Q?#Zor5p?GFy^j2%dG1f)v z+lOf){WbGK#gu1zLSnodg-t?BR|8^|a#`%R5t**>3`V80OKEqL@d%1P7Un;#kALup zTx99@el%N=1+Ze=RYhwf>Dx~y?yH?HU{~c62wVY>v$ls$px}MGzy|OV_Dz;iTd>pb zO-10Lgcq7KJe1u^xy#XLzHNPRxI6 zppiVH|D~qWAaY6Wu&G8Kx|pYoS)-Sknfdd*`5>$4)k;jY-Y$%^-{|x&5cZ$C=0AXT zDY8xXM&dJ1F@P(OKo#%?P*;t@F+gZ3G1}4@5M*azYfEc-uH7%0x$?q`aO``3_#XA& zI@L%eDxt6Gb=0V9u*)54#+Harpm5UiJ}sAoh6ZH`_ID}AyLw&B<5iZc!nZ_FH`Vaad7bQ{`W263GU%{iaWk2=PcRz<_vB%ZwArJOyA{`&S=g4 zs)USl^c9cgdP#TVz5Ma?a-QpQ7Mcow2$*iZ*ryCjZ|7>SK3m-LgZkZFF2T+H_q?*s zy;2d!5udb!9xhIZC0?~%DN7%=UPOcRe@BY?#Ot3n#ta5-o^A}xf4QYk&=T;|3?m4U z_sV-|S)28{nF#BlZijp$=}nWx8gF7B%w2cprE-!Ux%kUAgR#>YxlF%Q#dPn)77y*E zTZ!$C=j!bwTo{T$Vyt5?OAK{c2xhQfs(F=0((aH!JhUA4ekOk^D|yNkL(6g~+_*R; z1n}xFJDDi4VF-bhVaABahkb|NQab~@e=#beAYUc)lZ3gGJ|uiuIaY2;?^D7eJ;rTs zx+4ho8CTpcVD13VP3-eA;EU<5dQzC5)Jy6R7GvCfHIx9(v-ZTpT5Rpc4j#xY-J>N9 z(yUk$ksuyH<4j(QXbyctq?D2Mj#3Hi#T^TkwXC0KseYrHUmY^;k#@^1;R)X1Hti{5 z5URi#786?Dq#hrnyswXq82!BKyA}awfpziS7dSp%-AGW;RJcG+H2Qgy1{R3PG#ivx z7xOKv&qFPNSAN(0uAp(9>6evvoF|Q&^`Oek_zQjE{A{6vL*1q9yvi!vh3PkzENdme zkZ#d{)v9>8MMKx}WAv!Jl~?VDoaNV#op%^#f1#cBwj~mtYhPI0-`bwi^@w?sv1w;3 z8MsYp>}rk%(0gUMEU$l0va_+7x}k#G-R|FXEf0Dwit+LB_2tHh|GaF&A!ghb$7RZJ z4E9&K$oGp5!RiGD-1{E*418zl{N>#C%pWGiLB1gLT>I_nOv9s|fKw-wNN6*)!{zaO zEpk22f>D)52R?t}HREpn*($edn8D#VQky>Z@)BGkZO&#IcYLC_p5QFYS?VId-m3LV zIX!2+=`>ZO@i=lZ##xZKlI`C!A&H__RTI+EQ5_zx@& zmM4Stq?`+HUS?S{{rQJeP;Y()23`1R3=O3J{Rt_k1?}%k06Dy6a&nOrIgV+X`JeY3 zT8_hX!#E9^KCIpx_nN*|YS%lYl%{|Sz8Cw@dCf#-DW=eOKdtA$Vr$J>%uPap_ALqu%Y5ZQZ}_XQ2-F%d z?5c$nI53{weO?^*n4((%#mn7njD@}QgLzddk?nHv!$c!l33F2MB`$rFrfzQN|P>3 zTF%kg%WSC-k}60%sInEg4A(M2Xkr{TqyPPAE{r8G>TE&T31UF-Q&-aPLb+yJ| zPE4Rzrs42ViA%rI`5M!bm%j|&wlv8Kfyzpf1{7GKjg6W_@d?Ua$@mqAiFO}7Si;Sa z`x?kn$%KJl1DAg`7@&uHMeFXdB)x0L?=h*xBNl^*3B)nRO+ocjMG1CAAYBBIy@L`` zFZ@h&6&NH~HcQ!J1|0OU@g$nliGkV1V4*=MZfgxmTCS9y%!bz+TBJ!}Ps|ER0ed^| zDP3F~laB38UcGunQt5K5kG(QOR9EZQ!M8v8-(m=@k*7Er{b<$VFCbgH7eB=W z_%}0UoJY@aCTL$fcwQ8qH1bx&h61P3UV1DwsehD`R!h1+(c`(U(iw5{l_FHJVOnG% zTJ&p;EZM?&v!O!gScj_JNUola7Yii{s}>Fh^aK4KW@B9Qphh*_-XKG6~;VB zws*>BJY+3i?C@haAC#A@zL>!0@^@4G7ob|+4m6{WHudyO{JN_k0%dgVV_54DZ~cjr)kkZOi1 zC@}rdyd|``{-uS;DNZxayjLX=(b6o|{8KJiV>pa%!(qNoscaURl$PIdgMyxd)@ejN zD=+p;+qfkAM*Q)s=+9yz-Tv7cCYx;JU9>nKt8T1zkGlY|R-8rT13Nts67}!x>Uq&H~a> zIvLy>NgDJ9EP{usFa-Op@aU4{k~rfO#uugJ+ zFqAH-=Q@kTXep-u;~%St2u8)6yXkCp{L(8Kn?BOHh1-0A!_m?~-Z#h?DLtQqe9R%Jbz64^8AFy$-T%V6H!gQ^>N5SJbvo?i&>v-T- zFc7+jD+MB&K*8+3O(}`;lqS^hD52Y#Q?l7*w@TJLOA|25w8Sz9mN4i+lB@QN3c zI7sTx#_YHn^iO~9bHw@$+(m5cX@OXazBL6TRp|SJ8#t2t&6tLhrfBO2kFrzmiqq-k z*+;nYQ;Ec0R9A?jX(DuV&Nuv=JFj{5<1n=P7W!fR?8IRkB{N@XsYYjNK*fPI;v#Ls2^WH9Ok-q6z=aIqtDd@5mr2pb>g`TlG_~i zViKZgyOQ^+wrcv&HR&^K>S9(0cen*rZ9)aN7Rx3h!IE20u@@8YAv)t5DzBAJU!~OR zKIc@I9n0nT$;iy7M54|5PZ*X4{Od%Iet&4ZL;Z1q+!wg?vzxr|r2dQ+H-Bwe7GCH17eN+V^OQ5AdnyF*2HrHh0nH$9lCO;(Nh}7nEiZgH0$aFrKe8o0j? z^gdomK#L_qNJDBMTBnL*9Nx2`|L#5tQ8P{@&b$@3GOM$09kDZUAS6n`%BmP4dzo}p z+AHa({6$&8jqa>=>29TEivFS(?5F5BD(=_jCnHE5{^zC0 zO>oR6WtKl%HcKFF9@_F$(> z#W+qCs(FS`^4E;FB^y2l7lJx_E$as#MxOi)r@!N-$lT<7Xd^lUq4nFLGd~R5L{*VG zQ5V^3h`!8s#86Mb?jIXS@qF0*)QrA+-tX?U4_B(fF!t}B{{CGyBF~8JC7ME4WeuSM zW zKDae|);9?bjp?%Z931r77fVWZzr#C%OKOvv@(Gx5?4D zNkNec?sv#Jp&~R~%cNS2ggrdwLI@L9VnX{Gja-7FC3>k^@xAKUW!2z`l+1E@@_5XA z%;0Em^64*kd(MAPKN4s%6AltYGXa_i`Tc@SAScqBcwO?e6zq{pyL7t2-NQ6@MUfY( zqd7i@UxcV`knf^v=0=}Mm1&=8?6TZMI}?Uax4#$qpZiKdmCHu8E1UNaF6lAYqFITY z1Dc&zL-Og##yv=%$B8tObJ*MKwzzyv8(;r~N*Gp9M*XR~K&i^)PQ=S*7}v|GAHw4y zr6O*Tw&4(Sb0+UR-t5u2KezPdGApV9^3m`{`^I@m0`Ys)phh9$_!-RLC9zqOy&I?>jL`*b~ z51Qgolyx35HuzWt8l97AEAxW5#0WbN1AleZ+)`&9D$YrQ_5hpp>qRObdF$M-GBCcL z$EF_vagh|(O)urJJt@v0_IZcb5~56foBWH}kLG$-eKR7{pZc) zvup5Q5S_^yL5tcI4R+SY#%{vznW}BEAx5CK9MSACP~F~43Ofk;gMqpnN2^9O-#u#g zr~bxJ+0Ce*=kX(P23up?g6%UQRT*3$)rA{P7(XOcaSxw(@lDM&Dw{8T~MF zHnZdW5FTytZItKmV-D!C5$ZF*K-|`2hR` zT5)cWp~>qAd=9)p;472ISXJ-7C}`hqMc3E{Gu#Z&vdp4DxzCynaanDMTu3GNhGt%= z$@g?nG$beQqB_l_nicKSRLOoRgnVF-)o+_D5=XXmi<;u!NbT!$W4g^V7vPinu*t2GW<3~ zTZXYch-^vOxS{&g$qgQ(BP4Pp2Fc~|^fw*}v}qN<$r|-} zGJj-_QTxS@+q#09q&=fuf#mt|q;V%%08l7~!qv1j0W-NM%*wLEsdWmld|9*{JMe=h z>lJp1_-B}Rr3($4g3axer-ZJmhmx2+!Act0NyZ=-NbXk)8r+1!FGqgrxD2!K#`~Vf(%WzlSu+5_6WQz3{K)(K*^a%ANz{Ony$C7RX0^xSycNKSYbqQ(NTZ%f8c1$78H;+3%IcoXVCXthU+2lkFme&D^mxwP57sK=`eK!SZyOZwO%5dnCnX(_LLBEFx`*rdJ}f&!6CNehZh0&R+#Bm` zYPY5`v;3;`MfqXBpXRo`bGH%gkx;nvc}-e)#c^5S85<RXzud(n=?mTs-V74gx9!`y^3BUoc_EI1EfoYD^ z(t|{J3`}rW?O@OM&X_jT%AF(vbr>*|K??i!W6qql6h157JO3y$*`-HSBL<%@qVrp< z+Z-1rsxP99iFij+ICzh7C>1)smNAk`(CR1^RH;V}hz;Lz6Ro*m5-sHRbtBBCePDPW zYj4L>Srj8luL~xwgQ$qZTG5)eq=4<-44H5HKKk~*3D-q)91q50a0@y-`=#Q+`+l1a zuLB02=1ltb1WI%R6RHtK6ez#E?UvC7NC8=BiOP7drP@vY7+4f_;|h|a_H@aN4m4Nz z^RS)J*hSb)%DEqPq#uy3$Op>h!%xBb>2C&tOnP>3e}@lF3b!Q{FaMAt$z%@em^LAf zSJ;fei1oW2x-q+$#M{_-{MzIh95(*4_$%O`LnJs$uTZ90WmedJu7(HB*aI`?0kI<0 zp6A^xZGxIpp7M2iMit(uL+e95eeMYDMn@}oSy>&xGgi|Q*%OVhy`g&EKY0dWD=Kig zneZ;f7Cl@N1#uFM@;0U{jDZGW6r=Q>8YLXcp9j!?J<;~^hva3&-uz^|-y5$t6P0yK z3-NJCflJqH@yBjd8?`7amu*l!v?{cX=M@EfFYk~PhMAkbZ;Woj)dUB`l0+r=tg43d zDbstHSLG$2AL38Ll36J3=MkErt#+0X+1C_T#;oCPJ}ld=PXGbJzQ0}GuN1qnp=)N648Yu}X?TE*lg`;Q^mLuH>5dnz_`YEvM+LM*EJ(B%E@wh6!j1j$BeVo< z9xCkRSw4uyF$X^926~)Ue?#>w}>BTy7Rrp$Ts)u?@~%S=ywyXQRD7+?K2U7Zt6`@VE7hY z3f$C<)&9Q&lY;7#3m~gHwe@xwxD4V^Voi3_{n-V*8HFh>dZ|MS$Twh%Pl1`q2@odZ z@DKU$2(zVi){P96=p`#z)-1y+b=~VL^tLcm;t-s7DWX zW+Y_hJK1Q>Z+pdY9@?Z$p)|*(Z+`j%rh#Qnom!VgP06ujBWHWcMsU&XW{~V!O>hC; z6NFpgxf}OT_bRRIGv3ADj;}SEIB>?L>7uNFfQi!A3eG)eZOZk`P5?`UYM~8PQ$>!3 z;41%BT1FRe)rQu{JySec1hmRX3o5ErnPh6RpTxB13SoaEt*r2bjZ6o1e#Z zaZvfCqASRJ(zzMkY@*Zbwtq!`>f*)T zr($An^*x_oAOG>no zFk6*gz4~83;{W^jCp)No)NAB>RpIk?z_hx2w*M6hU-8{R0+eCzEc2WoHH#^*sD!P*6=!w@g`oR9a%fFtiz3{N4Q` zbVb^1jP==v1ncsqpjTJxtR&AX?5qQ+Db=>igH)5q4rjcTzKxCOoy#{^_xI~wy|K8O zZ2oDoL#Ju9_PK%571O_TdLDg-8~+CZSOq)6wB}okY+lpLm&wt^st*UkV`kD9X6XsI zIioabfr;*KN9YW3xAjZ?rgGbs+k-Pz?3o)**)2oK>(Hs??Nw@U2O_Q#L~%cD^|W|sh-OmYif)`B zCG1tmUM9TcuIOSCBY(U!O9YQ7?# z@4Q~6rZe-Xd2$GuXw|4SBBH-Uzold=+QuyRbogj)7{PgK`0Jg!j+@fsmHvf!Nz=>)t zmorYT;ER5NsBcaN&@mW%UC;#pNCM>FNPQ5k*``bN+%?v@yv^lVzqJxW^%mkxj9GMx zQ3TqB)m0xAoNm@=T=u;d^p}=BC)T;%e@N&jA9>`U#jy^L>xWIEw;rl2X5+}#)>b+8 zq@Khn5e?Ep^|2*R_BO^H#z*hhvQ^7O{kyQ-k+p6q1$XOzjF)5ut8sQKlmXsMI{^_v z5jIHr(8-fVHeF*}$lyjB$c~>r#mQDh;$BjFk;%+@(h)^E_hjLRd~Nih7Ai$lNNjq$ z0D;k}ZH(WO;OY#4C~|R4Bg`3PR>V&V{a$0h8TUwb(njFGW?u#Rg1Ju(T`$QaW0~mI zEz2?kXs|3w(bv%aAG4SS^JCUovGi=Zy&X2F2l(Y3bNQr|S(@uqvC4uR#>nCBp#G8L zZ`CsvVzhi8bs>5D%aF#gs?{sH&@4{8rm#yGhjjLW+p1Xg>CvYr(m znlw&kunI2evqo3|OT`>|-lW2=JYwwHwIXkKrf88$Ag&uvk`>^(jOtrco#GiLzZAvW zG)FJa)WqzYZVz&_Hfs9Iw|HE=bz^8%;w|n?y~#ZUZq&7UyPt8moEiw-;OYqZz{W@P zuPotq=FKjbNgw~&#&&yBfD+PRsiMefgMi7;y*Pc{vb+_GOT+Uo1}mu^4)%tRnwOrx z_}R(hu8N-oH=#sd-uqnkxegc#nBhUqCD8xz1@+V#aO$65sTv;Rf%O}wk#+;dSR{(= z)#${s-fIh)TY z^<~M~7b*!OP*Q;86qwIIcxVpxD*Y$Jb|;K%lY$@FxCWA60bCG2=s*8@;_Q>6!j9AL zBBYBl(HN+Tc_~IJwC1^fMFMd9!0r{DOY0~aQdKinVKdl=rx!1hoW<_&JqtX?+~!}V znI1?p)7@KZLjQOH?ONL`76CzZS&Dg_SfNrZr8Pf zgS>-TkL!E~SPJ`6L!N-n9R&_|u|5q}g`Ska1LS)mDIp9*jIfg_`disu-cy-W|9buD z#b4H66)BhbbE~SJtlxiKI9hy&ow8fpCDR(9Og19yix#Qn-Msf%$jBWpT5y9JOdzRG zamVT#acyPE!sIdgbCm{vYJ+3pBwP^};%DY(p*dqeK^&hfgUTM!Y@~>{?u2J>ydGHR zE=DCNsIAkLfp2ExM0#;O8k~qt$0{r8@0lfo00DOWc3vZrtWK!5_AduQ>@0aB?}P`& zlZDI1YdKz(gbK`^*^S2=l+AJl5I> zYH=7mqk_eO-I&_>@GZi`D_STBWxQV0uLYu(X%>fn<9b_-?UWKoYnC!3sU?Jdh*OH* zu1zO?U1?=~+*Y1tGVcsf8xhbWx-Qu9;n-+lBt$JI7P=EOfNpNPEpXCxs`Rrr30< zy{lgyU(WyPB@7ADjMFDSHJmS8bEqi>B|@yuNCR(9;!gVKVS~L6H$fyFfIY-sMRL8s zWIs?AUo#7CaTFts8C{`QMCtR(cLPKpA0gpaQfk;LJ3(QQmxo#j?y+Mlaq~_gMpTI0 zvw5xb6E>ReQO&lr5(G%}ewsP*c#9tpMb_ct^Y2rh{(aeIbg(*x(9r3+6#2Wgw}4 z&ET-ePEsD}Z`$OvLOUQ~sg)uVIin|M$duq>&)ARj^!$5TakaYI&#nufXQEQ^1tf+$ zKGJCJ97%~Yqu@C=I9bVoF4o+&Q{_HAE!|Zq{b~hNfHKzJMpSaZo_B7iJyEmNUh1+_ zD%b95w-8yq-Kjh)0454NGMBb(a>$9fHXS4DxDdFC3eRcnwDr_>ionXi?u}+fsWd`g zKByUk^;8CU^5&O&xf|!IHGQKbFgkV(AV%BX@&!^mp68}em#tsZIo?xKO{j&}n6j#m zxZ$5zqNvye?9O6Bc@MNzixl6i?15j)t}H*@hq+#5FpUv4v><$#TniQGdt9MiM~70~ z)>0DYlKOtF618)dD>iD^+;bH}^MX!6Ay6Evc;d=ZhTG5fRs9@AckK!$$Y7B)pwN`xZyBb>lp}qF!o( zQ4hq~y6ZclhK1>OOS`Ha0^~O)R3h*ta%`IZnkYwX(rlAb40Xw8T*53w{>(`Dq)1_T zdEx1I@!@m82lh}2R|}kjSAo8;*? zdIf9;8e7X~+r_X1ne2Tu>|R+m-UzRX1_|}Pq2FW=JU9omEm*n&^})7sKEN@-w%bQbzieQ#hO2Nv6*Lm+O-P zAyp!wD>O2b6QJ}j&Ja!ABN!JQLP9HR@9p~YV93aM7h;a?qia4MasDK+$VaM);BFl0 zo3gQl>@1N(%fVg7$_)DD67+#;LUN25%z0M)N#2Lw03L`XFHV9FqCEUD@RiW=c`3Bj z-UR6WKiTYmN9LbCi3dQsYi8fpxiTuzth#($R!8^XVjx8P_8KSd;fC{hZ>#!-I&me0 zIh!XWd_Ta5@uV%_@%7r2?`OSgk}9;6K^&$C#m2XvkLJRW>E4Y)0ArP3M9AWPg2_Sp zbFEBRpwRsDmzS#@4iUbv63-UOa?^4Vqdf4)=A)|9w4EWQ$jhMf*^#l;@^r{|Az{Pl^mW)r6f{_OU6S|PA_K_#~$wi{g$Sm~{9WI~FHXJ{RX}$2ZcCCG+H;2#db}UWRWjtmX&q3VV4Yapx_G z3LKWJ;cTMd-2_Y-(nF%qZsGYK>Ev9NHS(TtNdYM*7Jw)BNly%O5d4_twzq!ZC!(fh zJa+2-eEC@sIqK7LOy&oyv-I}t-O5HAbyH2UGJCa%+b4vNf-2Jv=%5lm$TnZx^dfcX zbgY^b=O6aM$U;Xd>{M7w3+5^LSbIB?4H?MjF*{>~rNr_c_l5P&3Aq>IaPV^#1b9&d zbajdcS2>!XLD^DrNKD#IQXXR7_}p!$$+>O*evgB-OplareDXcFEdjG9k5UDNExg+w z?acRd%G&@t8%T|M8gx{xZ2UlXHCGYFdG!k|x2f_78bvva-MSPQ;EoB_Vhgav5K!-) z>1mJ85-FXHn?4DsP}qA8cxdg*Z7rgWIV2raZONLVV7gg(jywPBfBfM89k;S}C6BMR zi`&LGXb&tWi4SNh8I}5o;{?D`2~A6ZBwSc-BTXSTAf~TZ`~}aAH&!A&y5c=u=@QOt zKo!N9*t)TR$yuI_2|H2Ap*-}H6{Hkz4?{{|~+;><95+i1?;X?1Uo*vJ{B=e8y&_lu`+->_!j@AKZ_LrzPn^L(q2gg7^XBDG)^OUfj>W!Yo!`tD9Ao^^EqB15LQr=Yg6g)hzeMmFmz5C5cv=boHCx zj{R{ai4osjOZMZ1)0J9u);zfG@-)Z3V>N3urAtxZaNoClKgiU*>XWN{#IrSA*$f#L zF8%(Q@X!|sp+Q#cRE``)WP9R%Jf!lyv=6D#)KsBCU0H^$SWbpIx;vzOG1$|MZ7Xe@ zq3+Tc%*N-!Q=TNfvuw&vUk7fN>q!KY&_=~+_WfbtUIuG^XB%%k4*Pm2`5hiWw0ij& zVxtNjqz`I6w0<)DELZnW(Dy%FwF2z`))&pzBcmpnX7QbTw#CJ(c5>(uQxG{#`%1Tw zcKGW?2Dw4&q?jwB>(!b*!M0HXo##0&h_9%5ti`H)_1o? zfi#X|)wUJSjiU^FaTiunID;?D{l5c1Z{M5nxK9d@OE(dV4{(#61=lDp8n6*?1m+WEfS&G@tukh`1EJ#{ALB+w)Hoc7&6 zRERXF%4)c>qDap(Nj44*8JXc4f*#ypZR^aX@EZInaNWLa^z7H zFzhmTAqlWbSVB;bq6gtB0(f@pdX*bc`(XzrWbEf#cT?+$CkFAUSJ)xHp~U)7xfCns z+Ytqw5JRthbW;3hOiZ@xMgXtCSKo8lNe^%kNN^=SI`z=myEC;xT7%;^U8HP{_q#VN zTQ}D5VlxSyiN_Vqd&2mOjeV_zP!nsH$84w2pbjLDIHTP^q6+_4z#vosi0+HR7+s$1 zjB^M}KIc_F_ekU^x+ki7@{7U%^&RMK;u?-nW^MQtzgODj)`_H>%HWghZQqAx^0yB$ z9rFk)Brk4vPOmk!`Ij#6EBAwI0L>1sp$S28m!2CBw=R{Jh|}eTi>fSJOW|wfoWo@* zYmr-bb4$gpQ>e8*w1(N|@xhV@HQcxO$qCZR){{vS<7y4DS4-Z?ZXwq5K?>lJ$F*Ey zk$7MWz{NSV{+Zy9 z`_`&+xfY`ps()>>#x8)hMZg5?u}0RQNMrrl*?3o4QQatWmVUVLP;YRQeOQ5%=WT?b zJ#zMoq{o$5D8!baEO`Mr?kT=6Y>l+F66oX5Q=()2r=vuo+$HKo;z5Bj#|kfZVC8LL zsusGz!)N+%3R#}+fgGgh0NbMt+i|&Cg89ULcZ+w0CrWY*4<+k`?%f&Ojyy5*{;eet z=`wuC4HudF#Ys}{0-uD`SpD9n>t<8x*{DYMOh%qTPm0_2&X$eEay0N}x3bCJ?~CFxA^X0hN8mn)n*aFDNAv{gKgj~h2vvezU=Im+U~AGy>17_3 zokVZSRmN*M!jkAbW+QevV_Ab@v)>fm?)k>mum5K*%fI2W{{qhnH2%nnJ@W#Z?h@Cz z%C1HrY4d=wT$4^mXTD~T?I^B-6e@`Y&l0DZx>1A9{pG9Lk*9iS?f&YSAEJ~vChp2dRSkwN$CbBaKI5m7YX>_i#;x}9;cHVaI ze4`P^cVLzx+l$OEXxD>ZiU@q#0yKFYuj4-^qukzsD}8iW7e$`ST!CtDGXT8}tkwR0 zhHDQP!TE8hEyKPOF<1(if&TvQ4$P9HQZo~fxxGZE7zfN)1Bl}hEpteswaG{7K(Sl$ zlNcKg=>DMFG+gIJrq0?n5;3JNuI=R=58J-;GpTn5 z_4(`b8yk%|eo8z|$I#o%WPVvJ`l%BRFi?o; z4%-8~4*4!@VgmwEKo-&;0KL88%X^s6+-_zh6uiLMfah(IWV&Ox6fv&-y@{#(=qUrZ z-ufXdcb&HcYCoMCEvmmi?r1`oTWlOm)A$f-vGC!s1ovSwvNh#G2li0spDm?W#%TtQfv4+PQwnL`VoBz|25+?(?O8hLh3$ONXV0*}ZU`+IhaEd-rSqR0vH! z1lnE_)W(G4de9!)Xx{Xf{iBgLUitU*mIY*)PAG^CaTqAgaUsjfu=qx0a8>$syunQp z;!RV|@O`|2w_!W@93tO*7fmHcabz<=aXIjGLDK!7k0_Pc4svQ5yjsMSE6+od1 z5z0qYwu_9=EC9heOGIirnMb)n)@*0wJF!Or#f6Q=0lYAQG?os(dWQCIrUi`k{T!2n z1!Q(*a>eh*utD$+CQ9)Q;1^nX#M%tt6Q1ld2KnNZ;8V~bLD^=TI_p}*oaBomdSr$N zy37wYL%P;YJ~19*R+S*k#C@om@ZP1fkS3On9p#H3@pYi9!<8;H%w!u>=ypWhnT~Jl zlh&aefxe0qn5sG)cG{`7o1a}W7FLjMn3FJ6cKx>TVHUqtZGv=&4iS-F-?h1dq`0=T zLab6xeF0e?`}*ienIxxZsmV+}b$npOK|FO8;)1UqO_pyaZPL-31P}Aewi@{az`(+8 zlATN!^M|TkG(`1T?T z*&wE;8G42-OkEbtmTQ1B(B>>AP9Omk>tNQbVm`_v9USShhHbFgJpbG!`ndVHzBKgd zKM(9*qYe1aqGYGcyRaWh0O^2MghGypCKy1xeKY<>#zHxuN2`!c2o7yAbA(pS_FIty zELs)5DF^LB>abr$A7!a2??pkNMo2CoF*q2| z8yxzX^jmOw9VvANQ#YR&EmJ(#hXUxlYSw^Xa3w?1LJgSW^cixb2@Wo_WoR|ubFmSO zl6(F7_hY))3r$*~aDD@_@JI*yJ*`(ja3J^bYtkHJ15&c~khBt7_`oZrqL%WZ{9j3)}N{7bNmntsw1lxHFckyG#JkZ=^sf&=LCJH2sVo zS6G*?_-i}L7bj*3Qrwo}3%(-l^ly|vMHJF(Q1etTo4o_fs~jwe?En++HUIY&C%!BR zj4;MY8nQdSPh?p+K&%Ht0`HnYMSG@~R0dJaAvC-@e#7sDDE?Zz{!rx9=TO#? zf&Bd49_j&K#5^}@GtlB}W_r>|O=|)4bqq+9NpS>O8L|!h4**yKE?JMDm5FPk_9^jO~|`aL-JkG@;E)8;kPMP zi>)vMC3_?wKS0uopWt073!?z=A=TbdnB8L(Qt0S?M>AoMVmqqw2viubmbjW?Rxlzv z`4W;KLrVI&iWuOSY|baHNR-lP=99@W-p>f~=bKC&SZvmGFm|8{bY|+i>?k9UHo`Z# zS?U7EpW#6PTU@=F(>A)fCoejE%{0H|@aYHnS;hXXY)_If`*ZABt+ zBd$YaYEA5M?51LkN!%mj1g9#>xz3MoJ)7vmp5YDlOj2CUj!6rl~3-x>5>8V<}xw$NOFz|<25z)s>(xn)*YEgaNXlil11f& zMU$J?=WOkf7~6>v`DV*avuwk9liV*wZzJ_jnrXp5Z&$BP5veFfIPEOV&Jad% z7Z5{yArRvonP$gb`l?q7KMj`{g@!_9Aov@;+WM^pd|?cAKOzF+V@y zOA$KNb~^9Tob{MMC48ZYG;c9r?XpB$mGtrIAnc4_l*9cHsajVh{VvdpS(9%;`c9ZU z;G{Tq*qK=+?^oTX-00;>AdO^GZvT_|D%9Px?FTa7R~{=bJc60HG9^e1i|8f{1I^vfJ^9(d?)RD#SFPG1X977mKSin9Q6*P>auF5w?h|FO)!%D&G6Jr< zcdp`Ivq6~0J2r1ImS>w`v!uT>YByxm{Hr1Yx+2fZiC80KCm?aAq84O^>fkbLW-gQO zId5QycNi*Xhs+6xs~RDHpr zW=Rmq#lkzBv%TE>%l>|hTFKaVsw|r$%MwREfZ`K`Zv{fu0sD7w5Ohb%)3KW~D>kXF zfycNSxZNr2dd+u1LGeBU$^2E8={CBPu zAv&TzpC#$jE*}{d84eFD$mPAl3|H0y?{E)fRuyBERp3t>z@g#s478HeemxXvmV*%E zM<4!fF9wv`JOHxGEMk}8@JE^vWWZNwm0%F!h*A1SLu|NJa(foIlbKIw5VgPSp&;;7)dLYRS63?sR~@S29X|w?RDBz_ zx~a}&+enJsE5%n~MJp7$HeR}DOthP0L!0ndGm|faL2a*CMlrAhj1oMG$O94e z!QU4%9QFHj4`%W{K;9IfG?>^0Ndy|nM~B%#Kuczw8HEH$bq+(z8}6cTZU*lyUQHF{ zG40mA)0oi1%;#{oQbrcLy3e<9u>S+t-1x30dIpnT z&D+r&$u|0pj2WuS+^5lf|2A+VaFaB(bI>8^Jr0sAWCWyy5xPO3JL`;`y&_>C4HR#Q z@3~vAS30V6I!s%D>+6K}-#Y6)`Vit|js?iMr~!&Ni{IFI-WRW{um_P6} z(!r?EHze1Nd4s=&r!7Mi7ExRa)eq|&G!{^TmvhDHXk!2vKVkSQu8x` z^fl*^M`{QmCQwH65Op9sMS`PFB;Tp)5MH9aC{GTJE}{j+AK@#M z^v;Up_r$4sH(|Hl)wP1h_5aQv$VH`}viy0fOJj}9IIYo*?}_w%I%BwwGia+!e(Qae zImHmFURzy{N`l%CjN~MTH=NyH5Fs`b;?S6vvLqRBy3s>RnRaBB!5~t4@wk2ef%by?~z(=BOi>eOlIr%4wG}AayAtsz-t4HINa!+OY!sW?aUUg@Hvbs zjogcUNl0;Anz`~DW}bq?Fk7mkG#dzcDytKM~731nPI>1e{kRpfmYiDnOg>fgG(Yrri zAT(NkFAO_5UHEH=#valgMOTNDQ9it!9t;YmtxEMyS zw+ovPgyAuf`e(VTrv;<3cHN@>}5#t-$(loEgDS^18*3=Xc4nf1cP&M9hD9HQY-*ztFVlW zS4(toU9cX>$?6)y<}8E;VL}Jl78cs`^F0>4Re^Bxv)TVW6$8UyBhOGJY;KN6( z2iVG`5;7g5Kd~s!PY|mV9vM45h1sNtj|oqtt{rh0O*C)aQL^!j?(Fw4Weg8Jfp!5f zf_~`ai%k4c4NX8;VwD;8e%3HMe}*d8)$ztIDNO>~V{CgLRh7(y%d!=U0%#u3K#v!k zsy&~Mv%SG~KN{F&u%bw*&EKJxkGTLrn0>QCIf9w0uhuF3H#Rj;$w!r(#d9;&e#pt; z0Fd#=p2Jdb@3Y*TV>RqGl$}3M2=Vb#BA@HW0OP+164c`;N+xZGQicV2*-c$$>7DZw@dO6kt}87){GFVM z!*tDU-om`N8bvkP%Vk*0@9ltHYz|>}-S5;{LSxpdP0{aEQHSgDk`QMQ<_<1HGS=dv z&|=XNryR4ai&J;j|B;y1h5i$e^MU;P-t+6CyF3$6nliJr8ui*q7^pfV2UT3DX!g73 z`KGwe`s40S_qYw>p`vWCDQR|K%NP5f_Q2@83$)d_a~^1H)Gw@4zG(gR#sB*a2q^d* zzzGmogRl;iSSMorR5rR()lA3J!JsmF18?C;sMJovh{E8292Pn^$j`GxVYp|Obg}h$(R4DT;TN3StH2kwC@X)N<7qAX()$?b`-y4?Mb4jcJVQN8uaYIq2 z&lv)2yl9dkyr#izS_LPg(9zMNob$cK+fe*lpSa_+KgP|bZx!Rpn{0NI@Mom|Z$s9e z7R!IdW$F9i-2zlyV?f{(j~7DO0>cITtD#`eVs|in$@n&e{>Y;@U>{gu5gHZ{#k+3^`ykZsRKQa1^6NauF z?DxM0FEAtsVb%69)h#va_s&~?!@IGM+iUD5hX8#=T7OyQ`e=-3w*2qye&Z#ohpe-( zgB!C@FfT;M33jn3Na*Ky{tP&N2CN)}uKg>Mb4mwssz~1Qhvv5}*YhDdP9EmthULAGjx<$U#+r~!7yDoHRxf{fecY#o(EI4E5YaBfj6ZUE4 zPl*ot1t5Dwc|tkqB0qa8%dcj0Q_r>=Gt6t$n}{+ZL1%o}txA#ca?tdvGDY@^ZdhZ(aA#d~l>Z$0mf?M9qsFmR4n;khD7Qc(mg zlScDQ&W(HfH=$sb_za;=ynI*wshj`P6aQm$90K`3D{RS+0vY=LuB+Wurl=2ME@yq3 z{*$}4pH71W2I&#WsR8+OoCr>bBj$OIPPEh2b_XuG4*EX?PH$*@iwkDx1w6pn<=HB@ zkeK+!H7ci58<>OL{DYlrZ~z9WmAv2ousWZ+{)-z@&r>yj)tkE&FgL6HH^GkRuP|*$ z_~B@QbB3SlG(Q5CEt}U7N#h}oErLS4C~{@sC4GI^;ilHfl4OvAy!sscV`LJOPpzSp zvIj!@Ib%f5b)loQ|FLcR5s2pvF*(RL&Cxlinibn&bcID5&RGjgRZ4q&IjLX!s7uDR z;<(crw$XpuD_5&+fI}Yx6R9FD3bAv3If{_C(0Zd$TjxHLe*gOEZ?Hk>Qb;{9fwE;| zbbKtN(UNU+*`P@4kDx*r(VhC_x-vV3;UJy#D$B2NY7$lKT{j%Rwp0+dgfUz`8pZQ6 zqAP1~yj7o_|CQ{5uBb>3+uq7p{3%8I$0^M%(U9HL$A z(VC{2?*^8;8(+zCd!gr!e9riE=P2A~B6LgDu z+5NEg`EM2cuUi#(2~j0@OoENCi=lk}!s1WmOj8S!d-2mzKu=6BQ4Y zgwSr9ZDZXc>h_m^LxZB-R~)4fwJg$!-+?HL)T5=o!o2xqXd_&`F{0>0azc#j2)mXS z#86tp5QCu(3V!E^+EWO|J-Dv;2#gzP16JCi`Mi4fUn42s$|3*MpO3!)r+M!1!}ltK zd@qASYg#|{>??~y_$v{{nk~XrS65cJ%9&5NIm)5zOF}%Mo=O|Z2j+xxdh(gelss8w zKg9{2|JME)-cR{CC%(3LofPw%vCb;#&Q;N;t+R(j`A+w`Nq$lX*It92)3cXG83f@v z8a>LKR_9D%eFxF^BDql}prxF3noff`|Fw5trX2X4kAEW=h%L34g{q-eA`AfAy*1W= zr&IA(*@<>S=ZSs4JbgRYZCl|w^i{IGH*_w4C*uiw(8<0Nl(S9c7emArIQP+x_;W+9 z#b9RahB}{TO|*LQqB5CZ?>D52+E7tuzx}%qx*QSh8)_h`8)P~efCidEpQ6%rV`NbYZEptPCo9Cq zYr@J6q10Pelk8bcoNFfG)xK%;_)|8{U9m}J?0Tt-nxy3P=u5!|<+(d_g*H;;5D&1L z#k1l*RIWX?604~YvV%Ut(T}XSAEh^a^#Yry3FGNkOGP-VBzdNY{k`=>i~9K=HpQ9z zSn{iM_r&Se!f%SEWIZ2eenbydbB^TLrc+%7Hm`Y~QUbQfx<4lfI-HrLwO2W9VTSri zQp*z&P*Q@LLt{I;^yb$+HzD9oY%;P_GR!6s`F|ubKrA<-e*z-eT=t-v4VCrVnz-pd zqzPIm+WN@u`V^^~jkT&C^4lU00Lz~wlYy^kLg0iWllfy)509@PMvseIg!?05rik!ks}!7nUl~!Lk|@&}TO#}_1Qx6~ zp>~^eI6HucD1Nj%uo_<2(}-pXWd=h5qAeG>oVeTNHxWcR$;w7bEuIf3m_R6G3-I=A@HWDB#l!o4o)B00=!rmSWe1Cr250AZHgrYOf44d9Z7+PO7x1H7AIZ0LzwKv;t0R+SVCH8ShXH|YP}bA-POD3R>dP) zL<=ZtM;+{c_`%gmVEDC_g!YTDQ%|feJqd|>`j1FV20iWb_oe%7!2x|Qp_KkFWeU#Y zuqordGRYI^q}vavKkveU!fJpemqN$00v2q>t>aF@`M=D%&@4!%(6#S2UH{azUCQNv zY5^npcm7CA$lXW~sDj|H$iCa$lTgiMv35f}VaO91MzX`H%n0zy3x*}1wKeBKDlI?P zbpR5E;xhq`Qv0OPu@Z83n44ET5RSrltrZwgyrXW`HE?orEi^y+!U5&943?H*m)PGC z9u9hT?EzxaH=)DSc5jZV9@d69HL5N1$B64<#?zE0yZP-fvUQ##qdWzrb$>5YM4rV@ zbd2ug&rF(SH0TT6>_nf7pupnma8^A=jpneYIjaH%?2-%2{p>A_#o%$&4FvLaky6n~ zkI6{LUsuG6snI6qGJD2rK*i^lqfJLUuBm9AS~qdsw?41Xaez7&{in9$CJH5UB_MLyg&^_Cw{r`K%11O{d+WG}PjiG;R5h{R1DO8USvyKbQKRMs5z&ZC9 zi(KJ%JU>A}awkM|SVX2rpGe@W54xSAU(@R8#T9%plgI6a6YN4IQ}U8lXWE$M^LI(K z3D|>b;k3xjaVNcnN723fZ2ZzZ%nTocSE~q%8k@R5!=+VzSPwnxVXqsh*U=Rv!(}6w(QK0R#FXJ@`h6gh1y9H_;6lFRRzJH;}BSAa8p&<}Nw*7%Tr}|7VHYZR;E7SGi?NCPc!`!pohB zx#x>@fMENjFup*8ZbOA<#VmdEEhN5P*OFV=L&VtYrrz0`FSKw zjfaz8^ND#Q!xON2OZNUW)5~;Tu%2vTrhc1dq$nqyc|ks%bguQzq}*dm|9+&mGPhkF z-=lv4G=TKYYK@5HS*;j&*+dDu>9~XKI2nb@E3)-T4kVY8L9xQ-Ki?PPc$O)*1ETRI zM^dko&99dfzAjRB)V~F%uBz~gJa|N6>qTdH7i6wKHF<9)XP7v! z&ZmJV<8(HGYcUoEqmK?+e`S6esFLb+YR%7a$uwQ#;dB4cZD_An%l3UatBUXa_A@S+ z(5nZJ5420|m!whwmKzZvWz2{anFx~P= z-{&6NgJ^q-%6_g?eck6$DNA~DfA;2xEgA;_97K%$>GF5Jxv8qv2YR@ec((;1enWhI|JTcXV#k`3a;}-7kjt6(K9wygRe#|D@&N@p5RjIE1lrjgQy5vtCG8Z& zfDWRabi=Uuba^m<{++!f4+2x}N5s@B4}J($Bqjlb)bo`D7P1*MnjO%TST?)o@_BU!6?J4Y|m=iwh7N6nl58ko>+f=C6 zfAclSf?vnQZq3h3i$126>!`tN`PLvePCaoLvxxmlK4uy-$URO}uV+39+us z2+8iJAlK8cPeKdO2d&{Xm%oF+AdW%c!}q70hR2=oC54iVj#ioC90$u3Fzu{=^{xno znDjkP^6;wV-stM!Sqf8U_OG07AEq3;5rsdIleaR8GJQ@RVW+bzygAd$d&`^4lgHPo zUU-(|Gp0;B8jWU=RWI5qRTcAA^U=alZIf<_;{q$*$17k0OQ#4O@NVD5wqTxl$$YfY z_u{k1S#Cw0&(+M}&hAl1)Vp~v_!5QqGoO=Ly#WvVfw%;X*SlQr+lJFl-0sv5`qaQQ zA*ruO109~zjgVPV3Pg05-kZ=0FI7XOuORxIXQ}&M9_Ceq-*U4Ij@wBKEgm^ONF&~c zd+b6oj)j!d^mIW|?*~roz3dC2U#hXAKCF@GJu!^^W#7PwLf8|(3trKa09&P$;-nE( z8N%g~yN7Qonn1eRwxvs2S4xg`p$3w~?KTs8%nby%24a9#3X zG|)X?d&b9Y38vmHYwa_^^ib@OP(#$g`6V!WSVqVbRf}b75+di`s(Vu>XtM6(Q)7fQ{^BnA!gu&t3_(BcnRqHFHV9x1_%LCNS z%C$Ma<1s?)ggxJTD!xWQjbhs#ewv62;RvPabWm24`^-3a6nhUPh0D)Zv-o~Z8&{W1 z5tRQGyj$Dg%3 z_H^xDU(9^nTycW)kb=tJ;~6sa1478JYfv#xV31~KQ_JZ}d%OheGF!KQ+@1TQt7^BY)8(fcu5cz_y_0QFph(}Nv zpusJ=fWSnWA;sqjgKt;kmFmr8-f_Ehd=6n}#_K#Yd7cs3!No1sd&w?9sVSMY^w^H# zp-lQH>*RE0gt{WM6W0Ibjt6LAbR5v?H_7y;$(!4Vm8*j%WY|1B)!D1n;{_u-3%i z`B$b$1AIAV&}5S-k@v%4A6RMo@w=0U&?wQ?HMRpu%LwnG!61NsC>V7pinGKf~4J0P0!1RVpi3UJEz4Shb;V!A%yFNoW`ft zOx2*qs|uD;8%)J`EwL=i#-~u-o1@Zt*Uiza2=&pZJ@p~E(W4zo`#bI&S#cq|noJ$T zYvGFtsmsv=b7(5|8Y0_I-ZE-8DS0$mWVpKL5FTKw!gSQcNOr zMqYHML9!qpmp{hZh@z*=`#phN;rlXQ_H%Ez`l7hkt|f_~iuikuOJ`l%Y;IznbjRj> zCl1qrt!8bHB}VITe$0&g(=RsLd8RIoH;Z2rpA!??tb)f-cC!LO&=TX#DVuAV*n&5# z8ZS*x>iA(H@$VFwj<2^xUYr~!&e!~0G@eh*vp^uTWp z=(Upt$S8iJCebznlElV7PryfUl&^&=0pDDXA002u=x}7 zFBfLb^!0~?RYwpbTy+X#oJ%?GNSK%0^0Rf_nkp#y%uh<#o;=T2`{iNVFM)tqcS~+) zXtaYCnCX~@++{G{Y%^+dD_x0QU+VXwrM2Q&ojE3~W03TSNcvb}m&1}U@xJ(#HhEH) zO7Zy)Nl+Q zCw9MqMz;}wzNxw}_1t20`f7gm5%B9*!)?hJDLia+7&Tw<0*#r%j{u2F&L6)SDdiX_ z{hWpj#`Q0sY*>%d%!D|uKwh#pnIEmY2ua5Y>8tJK`|)Yyl@syr7x=H5p@PWskIO@a z%G+<(fqZ-GULEZd%b66}k=&Sx2=%2qD&NW_E3t+v2y4Cven~;$3ZpS5&pM!^p1=H= zT|O8F4#XT@APv<0Qc9cuim^1&qLrK`NQ~oQm-`NZu9#D2EJ46C3@s%GZHYX4h#$i| z%Y!SNl6-tTY;?&ndq9*;=Z6ZH?)Sw~*^hd%%~+V#lb{Mp`7bKzI=wKx01O*-TwrMt3{f1%3 zF+*snFYKyb){ip9ms0v^u99hLnxBO;N;~dzE~M~!AY9aF#H)VUHZ^eBJhKN4EVX3%C*4TEw`=PpSs<^Ly zztoqx`>YsT_k!AkmbmV>Md2&YRW@QXjdE*x;j2aip zt&mm7Dfp>5s=L~vLiQ7s&phrEn~@a3U)Pljw~WQL)Lr}|4vAH!w-rTuRieD?!qqlp zA3qN|#@jcg;vtH0t@X>OIY&M7K+KulB~;~Si!kJ}OqSULmFYX?9|;*!XXL-d+_ZO6 zOBwSMo9O77)FWeMT<14JX$B&x^M%w8$YPu?(l~?Jhf+XKc%0#dqN1dvXg|E6zY|2K zUc4v&ZlV$7X2=prOd!sz$gMhUMktaKgA-sap&s4kS@KCm#?^K&9PW+H6-6?9wBZOn z^#wBSc%;&d#t$i<`kUdJ^{J}gGpmnW{;q!U*=WQCXWX&p`3G>x>$eoe;O%l3wH9x_ zTfsFVob6CMn5gZkhI;KrbPi-nfs~d93h%MjA^EUA$V1c&bK~tzVR$<;&8*zZj@e=g za!q2t9Ims?7 zsxSSBbSTKkiDjKuGqZCiALVNaGBIfsWF!SJk6&J1;!Oo~>aW@)(cs~sp&=UyAp11B z6SNrUk_}!>XQJ-d!s@qE-P!5%WavU36AqV_laA(s$~Wh_=%{x!aYaI_VXOnb(6Kbv zlR<<{6|@k3+0tZ=5th~(gvkemHwg+DxN?bAP&NtKxeod0lh0?%b~ApIGbWADExrur z>!mZh{9f-&ogd9~c<&X)w0o2Q!#Ft=5hfU(wz%Doz!JZ?nDpg^66{P)psCVbqkAgP z?2u~hMq#SbOui?+ZR*i50CGGHdUP%7NUKR1(SrM|Qhj|3F?jqwbfwyM0tRWk};DB&IOixe`=ZS{5nJ*9Rn!%H_1Z_;3?(s}AqZ)3}dwWC9~ZrG;IUTM`mxVD5ZSp6*@py44k~w=t8xGlD$tzKq=bcv#6qYq# zr%$5?&8fa9dpDVdRdUObUS^_BAvb2}L{3;6+0nc;Dv3Gpvh*I7WvJSUY)Lc}`w_zU z>R$_ZS_p{qg3&VwNg zyRK(v;-{+iQ}{mh`qY19vWe%+q{&uelhtO&s-<{@lq8pa7LHT}5+cA~G&h_KE%01{ zf4Bl=qP^k8!;g+9CVW)3jsn_8KDm)&X4Ml(UnCm-srblt}UNgBrXQr9-CtTF|L!B0fKvf&2^Z`TwBP05`h z%p^f_|LU!k-eWkMgB#+sLa;~maXNYE*kT&*7#Z9-;oe7&Gz&A7skBAO2Iol%{-~pf z?{~&EacjTN{megd1Sr6sVQf-A$CVat(hKG%7VnO!VUE%!B(8F%puD*5(Y!!@yfPPX zA;Y#GGk9JNe%wy=o8VbtvOJkc=Ft#)n3lM+ySy6~Odv5h->f?^-&`iW zY!Zi9*uh)eeJ;|ll``Z^Z|qT@b!;2D%&-^WlE^31qDcS~4G$NsiEW(!V@Lbvsy^fA z%=H{F?FMI?Q;-`r=B1Cqs#*O&Ph3VM6#=$Py9QK22b1XmImZ+MS0%*>9_k*{oza1)BxoDrH{kd z9Td*Q77rxHaoU$+qP||W81dbv8^2^9ox2@ zFXvpo-?)GG*gxv2RSR?0thMgzJ=E6p&4L{3Vl9qq`Q(Rgl#j>eFonmkw>ghTK?Qh3k#rGe?q|O-Ock| z(wB)*rB2O#-+7*p1pcoCQ;rOLV5{VAcAs%QOIY76F~S?Y?484qT*{0bvb8(FYPB{N zYbg|x$KR;|dwLfx@aE1TJ=XPrOk0-1yY|F(w5@#3QpkS7spfcfJ+CV$x$KM?pn=}a z)^X$~pllGCr3Z;8DL0kyV=wmV>#xO>lNLfQ%cx?!LJV zfuddGc%}gS?d^@J!X>Grdi0OV@Cbbfl?Hj6$9?%8PcWoQHlF5%qlPw@+-^A+Ej`Ou zDB;7mt4Gi?n9%;6=t`UHTZT-jJA_C80LhI}`EW9>iw(F_mjn|QBEA`>IgLGeh6799 zE%Y!uSNbkrW+-V;Z3v~`1f3iz_fsWIdYL&`$SIxy5TCj??*ew0Hc632XI#gW6Dqvm z7)5G07vg&D?RU;a@AnZw{n@6UJd9<>D7DB>Ri2>jo ze)#0s^z`d5kaqKjY*{Sg!H}lO6LFIuyMn9FUz43tz)#}h>rgqs*W@_Rlm@Nm?duSK z#DlRVjnP)i`>UIy7K(9(%wW!%7Ke4aR8MuCcvJc%)+*tZwoM|NC6 zFATz`*lO}By{}9Fd6fPnkz+sTP@;J^hTd~|OR=1DsY|SuML?5B5$@TzHA2BT$zq_g zb^1{dvI`2>(8^Fc+5%^`bbci5-78s5w@{_tXboeB#2Y}{rNT77D2VBi8O6#I974jgWvP$8SV{AmTLhXIw>re(NUYpFFSS=n zti?MloqQ6z6l_MvQX0S^CUx`q{6g7texOnfSJAF zD4qLnl8sslMoS|&fJKiDBhC1F(4aj)qmhIOQ#;+$!d=D)Z!Oque}(^z>g~z~`9*VA zG9$Zs4Mq54&luVF(2gz52W46grKWJ!m11x1IZi@xmn9$#?B##5?e}`3+b%!PR6a&Bq}_?q>%+KefyjG zWHooHqVoO{ICDaeRoFNvNBf&zUnX>Dkwd;snn@DXRx8eJ^`cm?G5e(y>g!@E#Nj0Q zG;ZBoj1->(XBqvlMC>TJHIC->8G{AYfE4+l5g%2fyrFOwGW|2)9K+LvyH8pP$}7FA zTwoU9j9u6&CpCZ5>;3b~S=gBmHd^trv4$;LB^n&C4CNrL$|6^;*b1BGuzJagTBHAP z(Sz||eI*Hi;O=d2UB@F$iuk@#9^$#x{KR3@aR0j+c&n}PhEmNad^a^%Bjm0nG#3}F zMP*SE@P+JP0HiDF(brg0JGoF7#4TMUs;#||A(Qs$$kaKPG6LHdz&%w> z8&!O$m&LY#C&pJ7#YC}}qr%%(JCX>;pR(*_I7nA(WH^f`Zis|yzaiA*YhBN{D&__* zAR!B`Q^OuU;K~NQ$mz*zclpQX0QiG@UIj zRqUVEs-aP(Am_r|E5iX4tL!4o%uFd6WU=4(qhVqI&0h76k{s!;6I*iDd9{%r(xE_g ztp$A)tf*J?K}bT#P-O=Xks;C%%5<0Jhq%_a7Jv2rykWr!1^!1uXeZoKM*|Us?d^QF ziJubMWxBaJ$$OA~aW1;u!nvBPTbxctx4G{X#|cPCz#1tC#m=;e-gkA0&gU)6){f7z zoF}kb>Hm|u8;|I(NOVV(^E5^R_2BmA;)0H8BdvhxRO{TU=wHs6Cqa^%W~BOui}uSc zz>zMRk!B_h58Z3;IoiAAZ><26;EUHGMQF_3rfFNEPSGszg@5 zCj8a|?Y7iM7&_>ry6fFO63JDl&RP}9$!>B2B(^MXjnyd@?PK@%x*asj@@|IPwB#v| zV_y`I%N#O|_v14Y<&l4y8^#e*K$h-nRA$OrV$AnxFff%!@T@vuktQGGG49VLKbe&Y zj4CJq5Fu$K8G9#w&i83{S(Nth$8ONkE&K$Te@(gnIVeHvfWfLrqllZQGz`VeHJI*| zckOH36}{rv_*-V7?|hHn+*!qaH&$2!KbKH0nhdWoEYubmsN0j{lxL2_IPPd=!A#*S z@&Tl8q)^*5gL>C9eJ%3|EM_PeqheaKQHNBO@^?5`0|(F89H`U}%Tkm3r96^GLWi_2 zGxB3>CUmS~jcLzfu4sFDJp7wSb2m$SCPd^NcIRz2r@Ve_s!;FB#?}!Iv}vFMLBA!V zM#Aqn8TA!O!ESbR%CWZY>}qs{o1Nvc$g4O!>OvUQOS`7=jU7cmI=%_-v+HUtsW;5- ziHrS?0V}5%f}@D+i0R_IGwYIFInfx@Fu;ja^+7qhWRb;#$wvLm@EoV~4HVbWouM^o zM;XP09!wnS31?i`m(l3#8~%ZSLov^|XGF~#BIFnhs96-w2YBDgKUQLOuxdUf6p6+(N zw&PDPsucGHce?E|`Z_-r-oM2f<$bJucLy^-Ow35__@!)hFjUvPq zrrb^OJ}$o?hfMO?3K8Z>)nmnN9y3yA$QrhRK6z`2{Bt7sTVA9P?5_j+h0`1mYDv+p zJygM%(q~2b%K0qyly%tENdwtyp)h|+3vmudhq@d;XMsL>hV7Hb8<0A%mwx@|&`DK7 zjcf<<<3^Xei6-IXI7aA9p%Op|$y}3Pg4h%9Kw|_H-sr(zftoSe8ufxxF+p&f`c-;K@Xotixppr_pC z?m(WIoDP2csp3yihWcBTo|@?~_(jESiI{NF^dYe-N6{SQPa-|~D|D-T8J@qnqCwRQ zi4CX6cOExyE)TXP2<6iNt2s|@QCNp~6x};|Z-VPH;^`(`Uu#CWK%R)|bdv71m6n=} zm#^UU zla46)(5BcD;cI`_kP4 zMxJa`F)N_~3X0?J_h~6ED+Tl@b;c2Dd94l=i!dKJ$-@}o*md!*`azKr@o&TO;meVN zW=>ant2(p~32}N$Pky3qb|C0$7>USpd@vfs@3{mS7khVIkiws5AE9IN2YyoY1|*DRm9K_GMFx`TL58t-X%`evP|)BOga%XqqXhBC1v{O!G(@j7F1 z@fLXZtA?xcR*~jSK@}wXiZ`7Km(4-0Nq0X(*j^&P+|j1UNZc+w#tV2Ng8=is;?F5PNp3wL=XJN z4sHlBJp)IXbicPiC9h>5p(y-y8-t@}s#KlCCD5~7UQ(=PcLS~jcZOL#bc@#BQ5o?b zw#l>FiNs@+(+yeQDqCrRrV}@=lip>8(an6kolSHxDI$aw_)UV2h^nR|k1`JVZ^97R zua02rvqu-;>)s(%Xqz47=i43%Cn}n6>@91QwH&j5*-;vIcEkz8Z97|>7v;ed-qnCY zrFd*Urps@&AtdG71c&L)xjT-RLH8N7SgW3zeoA@;g(Dz{Dh%0cTbOn&TG7s-} z@tCypB7;)({kH{RGLQ*xVPRVt1(+k-@$aR%!U8V{ z(RmkbsI^=?EUsS-4N+=9eI-Yf=wJ9Nl-RSH1=Fjv-!@cKu9N8FHRo#zTG`uQ^Yr^Q zzXcWmB{g#qf!(6oid-BMOQlt0l!(fP{+el(bDvMv{BXcs0;|E(RyFpo@3`5=jWxsn z8{OK#19?2EI#W-kS9)hutc;!QNLtGcEs503yIIX270j=)^o@`(#@;AlFUKn`y+P{a z_OryoJub<$FKo8a6wTj9_BUb-dA~6%aDOHj5?kL}aS4Y=Lxc~GAL(Tn(9~jtBq?>m zzVl_TF!yx{TIqsvs6#66R>oBxG@M*^lneF=wM#TrM1cCs)ts10N6LP2+W%m6{}Ej) zAs|%6jfEvfAVDdB{3?+$=xs*XkN`5Py#_Z(RTO!yGeLj*!q*A5a})(}_Ryi(6P$%V zq9$jb^J~{e0`(5Cm338=3_#=r#NW+|#5yOE;+A7E1^h6TtywvyY! z2pl=WuhP?#z2+|Uv(F#{{q2*H!$W@Svr%!&^XtUA!RiXk(iByx{!&IrBIo8}fk_0V9t<#r^OHG_?<-Mx2ebFrG)yZ%ZQ z@=$f3Kk0=PitJ(J@?#*x4}D+&CK&()PV)QZ zIIV|_wrBS7(0QJrvXPkA(u^5dxCx%3?&`ijR;dQ@^-lKE!Z+$|QRip8Yolxa`<`XI zAV2M4`s~F;5k={D;0}8^7lc;wTp^&>mm)9flJr2NEvNoTvI2#7|(fSZ2+nYy~A z7%DDnGE2?a?i2MSFH6qYU&9{^RZCyL;j zOm5#UmtQ`2xwR+~S<1(u{jY6Qh6UnpP;FHt{7(G$##$lhL7h_(M%@C0UZwa6Xz2Vh zHAA&VITL~&n8Rz9St^=HB85$S5GMcF6V^c3FxPCm$q-bd0&JMI!pt8@`?9Flh5x3o zz5}}-x@=c1-%14>zd9gmxTS{+iTuZU|JogR2qke)&>0;P*$DK+zqjw3(&kQ(P`)k! z)On-t!IqK@f=Vn2C(^m;ZB1`hQ^UonW+)$=>E0fDf58%Db#*nO*ZhQm=3qr`?l*QY zVX-Md!pQw44Ju(s>SH6uveASNQo!lFfFLUw7xEU6tp~_2aaY)R7+Uk$ofy7c3W6;w zleyQSt^mOHMENYlpxWQ4;Qt)7ucZopE%5?AeYGqf{wL+Z?y7gIont+tjjQY2y?JPE z^!ju`mV5m~pCa$hI-Jk9ORqH5M}8^Hd61W`4D}~3CE{ReDUmM`+4R67SXUwMR439< z%sHIrg_>(sY6z_vcGpON=P3~Sdd0WPNp;F&(On<6^5$8 z<7#(iN6~$e=Sf@#0!G;F5Z$CeKUf}yjSt7e^EuIhNTY5?BD1*qw*++Z_9zb6;?7FQ zvt*Lb)V26&6fyy%s|7>MEhuQ_3(IRwZSg}C%Q^vD_9RkkkXR}Cgn*mRw?@nnqP}-E z8=y`MyQo?xVe6Y+oe-+ktT3xX;fRy)RU9wMu(EX#h^yjy@ly|ij12*MhugrCtSD{U zg5%J}u1;fR7`*T%$&Em>g(~kk<0KNO1c6H2&@)mTa>8 zb=idO`7rFaCb3~_mAzZUnQTXc_Go}KuA|Km4VBltzde-cw!pO!7*>TnO!BaE#SG%R zDlYJuAy-W0k|OQl4i$ooAXdH?gWbC-G#7*|nK3bL!-aTsa%X0NqJ(<#D{wLW7O*t=@pSfmj4c%gcJkK`m#rgiz6e1xkyZT zrd4O-N<&4uOvr{d|Lc^w&ZcHa$dl*Un3h>^klM<3?<83noPwwO_GeBD8rWqoMw zDB%1Yqq_Uy5aPP6xYSGpIxA+u$^tm;Q>NUvE-R$xY&Ww-g?Ap^@J!r_Dp9!<9Lc$P zwp<$ut)n2?;e-Kt0ClA4hWAAj@7SADz8TaBrA~pe3!(4~5*?bfWPyMmgy-87grMr6j}U;xh+6k{xM z_qW@F0Rn#YZ@&ipM`!}*UHZ@i8x;WiT9xje*gTR%#>?604=+-8wuGK>v|LGn;)J6;9S>k>i{C>@J~cvpCn6W`LcUZP6{Kq;qt8p2k)HRA z=8=z4snKFJc=R+Tf&Q%e+#l*4sH~XhZ*z{N(xSvRK){IfQbu(4Cr=4pp*duyPA*sp z($Mq?N6ewh*sl3|Qj|AC6D9_&xNdoDL_C{juf2l4!k+G6U={r{>K$|?9 zMd72^g0N)V z8qN3TXO9nal9dS^&Yn`ZmQUoe>C};3#=1>g7Y~g|$?Q1tS1FURw*^x7r1xLzJu({; zC@TYO5&4+>cwYa2kfzNl7XC)`O{qE7NRV@etIhCoHcf7AcQ;!;p@AJw^AYGDL$gk;ENQzJT{^ zK{nZ840*+^ucg_p9+&g{8d_E|< zu!T@+WTYG-`d5wm|GBJRdFU@kW$Z-sKT!m+Tt#GB#cZ?mNm^ZOq-pait-}gE9ke0U zz0yVK7gh9D`B~J9zT$66$=ls={WIF@BV01)xdS?WVH z8i`~&IqL&y4tB#?%GavLyBbsY`e~QivUKKFjqQV-$i%@??vuQ8IBjf=R<1=mkSK@0;93z`34@iF|SJ}ybxCv+4eQ>%0P959(vO*XwsqpprWC2Dv7 zAiO+}Oc1ehT@2W_bmcZgw*Ft4i0@v25B_}q)a#EG_m)A(uf|{k??YM%d1wxnB<)7J zq;>DB`W~r477phhEV!PK0U3oEYeP~|Fw`>!U0Vvt8|&KHEfpKwM{CT$@1E~qS*TW` z{SVeNmBJN>>+MHB#>@J&WTAa+QCBXPgU;GRwsQUu5_1y?ob;6#R#KfAZrsgR_WD#S z&xw-zyye^#e|xraA$4V`aTamQJV6hJo-wuVsXyncrF~}~PoKeUcQHz!O!JMW#ON9n z$-O|p8=%nQv~4?Czra8~jHSeaGiIEd23yKxlIWfI6{FAAO`Bzj|7ciyyv92dur|Gg z=i6@?SA~#51vMg*B*7$DgY%7j=7q5I?R@=*(kg zTWVo0rbR~>wR1urFfr`De!Mr9bG&KEoV6uQY_D2U(NZBoz%f5gMCn>h zK$9)Km82(CVeINa)gBpVT3YNl!r)yJ0scY!#)8~JRMA38QxC=&HebRDYVjw3P*6s7 zOV-QOlML-?TE2NHYlA`$aU+RpTU`Ov@0TIyY?lw%)Kq(PR5z!T(9RkJ+m+Zsi(oZC zwx1Ft2mEnG|7ytiPWw!FMuZN0$C8a7lbYlf=m^c zO3#5waL!5462>o*L@juRJqhEdTSD1WcY`s5wlZM z|7~17^fbR?$Zijm#_sp;<`FP-;f-X26-G=wYfS(dTJbo#RR z92lrnHrpfwZCwcnxf9XGha!ut;=V;`s_yB3QkI+>DP{2lRQ@NDRQ#)03?us(Ia#Fd ztJN=EYXn=xCCy(S2;W%BCES!8q#(_JA+VlUZDG~ACu2lARW!vXU*azbI^Mg~G|?~c zY))8fHY?8zxEiRG8&^I+=(b?@$l=UYCNd%B;E=2M(c0{Rjb%b5U^@@0StTaThn?nK z4}Nn5wpweXWZGD)Wj}g|G;N>ShAdZIT=XMH!UHQ=zg!Uqv3b02CMMrW;nm;=Dc2H$ zqb+dQxpfb-r~`RE++w>lt;Wtb;sW;@FkeDT%$1{gd?X#gE50nxCHaXjA!r_292F<9 zRB%0>9h%2_prNj=u3KF9gXJR4G!{6RnGlqy>PXP5j0aH=b-iz;JeF#x*ayvK!rIWc z60VNv>7wu150t^Q2!k>*GKjr8@1ZX4_IaVGq%?7i56USPJIL0^8BR$o(VZeK@Kx#& z5D&5GX|oyMa3UPHq(MEsb2RLT&Q4#RB@TPm*R}g%r-Yr=OUYm7DTu2O%%(}cUov$W zkRPz-)Wu49hF7Xc6qIxZL_22|c`GXMn{JZ6xQ1wu8~j&6Q;p?!c3T24&fQr1(+$K` z0kkXcL;`|VgHAdL&*s^`*_{k4f}z;dmbtx)_id5F-%#F$l*Z&>q`q~MJsBaKg4U!j ze_&?TR?(wsW4;rWYrSG5jLN>Fxr^cT4HA3(6I=$5J^FUfB`zQ$gVkaYN@zEk+WwI{ zs$$S|tcn+@`4D12TP5u9oCeC86mvbZ>uL^du-=}B6QXf!YqZF#-)L3Z5#oe5k!q&I zEcwv=#-+@JfJ8O53p|{gQ-Fh|G0q(C8atN;op?Y$@WdyR*fi>lLfvJRi2;R;$Vvl| zMyLp1e;?6{!?-$X(ADcqDl$wXX!5&5nlgG;S4*l_*%;1V@%yS!ohIS1q0?Re(ZU(< zkTbTypBQtsBBQzo-)ND}`N1Ek(Y?_V2{2e;2RpaFMdGu(8)Am~z1y}&!|g*e%QOpD zo%RhC4V2|VE1fz^{ZgJ6Cb-*VNta3DXs#Zojja_YTS_|dSFG(@?@3OIGbZmIK*4EQ zRpEW3EvuQgg$L~4^DP_XO=0FpR;^g;40$Qln-=-_r3tS>^;}8yRqIAoR#jQ!)tu4# zGx*x!b-?ae_;?Ub7njgD2dz3U1dZB_dS#67&ExtabO=g?ZGl_b+Qf)C4TqsnPFiDS z0-uQB$$@|_MXay*h%EiQGGTyJY!7a_j&NSSrO)I?L;Ul$!S^)BqstY-Lj*)Lh0^6k|O`rDcvQ-wtWx$~aDmI;yEAkAq8_qt|9xA^C8 zwb{>um|mj}DreQuhCh@GNZh*^Y+%}wMGGk94+<% zc_K2!mv7p%5$e&KyqHw`P>-Ll!D0`6isGSG+)|6u54%{ipOAkffr3P@B|B(tmh*5z zB6QvW@=F36K) z7gH?$dCl-SY5_hH=QMQHs_7bSB899sso%~`teul`e0%m)W2&*D7jhh^T70PLX0{c_ z0RlIRK5**fa|8@s(XP}_Eo|cFY9-~TsL~Y*?aGIv_BuMa7RbnX(l)RsGd6L!iw;{( zRS??zsAob=;B099Q4r3)IWKm3sn^C1>;3(Lp7ng*W4Q`s1RU)F@N^nU`?;~aj`Pee zqV-gxA|5LQ*c{o>w<&ev;i$e`mb(|EkGaZYd?8rb9&Yiz&2RNj=6_r=TZMElXtGN; z8%U@gnG-ga?B=oxZpL>#uTQfDTN0^1<*x3!!IQTh%m!+Fm!O{0*lIoDW@WO0ShwjH zZRen=HJAfxdX=*~i}R)aol;PO_+H?2zw+xs`|EB^9J?KM`FCM0dB(>1-7}4pA5~!w zjbh7>KQJ3C3C2GsGCW?hqbdov>gg ze6S!!joODLHe$Ux1c_~SAb-BCVz;S+(CdFgI8Rz=en&ViYJGB-f#%U^RHdaLai+89 zF(LKwnMmTvtzh83@lk;?SW?L=L#x(+^$3oL4-M87w&nytwS)C6;bBK7*DA5p*Nw5dP_ zSy>!3bxj#kCt6vR!N$f05#ek%S#;sl=u6J9)HsqPCAd;sAM2yNA)I{HMZ*l}=6tp+ z5p>4s=O|ayi_e0~?q<~Q@45i|fyJDrK9q7}#;XW!aD%9Z7l`YaLnuLS1Jw_sUdaz+c8;R&W>9#ZR9_!%?`py z&jIbiO8JDAKi(sEM97#R%c@1IwPaJZcu}h>~W`V1ds-wHf*?{aU-4I9>tItF_&|ccXN7M$S!*!_;da?85R8rE{`hHM zGc7d+`gCu)uT8s~pca2)Z{PEdDCJIxUV)BSL@ zL((YpLzRmf(XUc6EShDe+A!bXoU0k;0$T|1r#M*j=U+O}vDVVYKg{4GaGIA4P)FCv~23@nfOB@xMZJ?0mtOB&5S zg?eFSKY@CoTCJ5twRTB(>f#XVuYtSw63C*e?ho{@UH~%pgqAKY-hJSw3irzA>Z)r; zO8HszH^lkeOYfLBS>bwlTp6poEWpkl`S}Qpy9!Kqt)1EMrPPz>RZwoU<0Tnb2dYt zBgT`@bk?&4$4mLmmYI6)mgN&O{J2kfhsiFf;&^oT=yeBn?D6uX`?5YC#;G|)o7IPz zZR@YYw^xLJL98P3qEOeW2_Rr_Jg-zVHo6g;n1sMJy;1nv_q&xd&1SIzqa<5+?xG%F*CS|jK$ zio=7a2Ks-x`lACM;maugtOOgV2vE3mVB}frSl;N@>=Xb_AqzG=JCzE{fq~1N+bpan zlFy~u+T$)=;9OZ9#HNnpm!nFe2ja09fd^Z?~yWCtAcb8G^VQNwG~> zB})a7o$w5(IW@1KNW+o6SXkY?kw>uMN|-v&Zj|TBat>^@Cq!9BaAAY-w((aK$C!eK zPbPp22>J9&tt&zw4ocoGob_U!Nbv3b7^Euz7Si?G?c67s_>)~ei-lM7z8WAaLgU|s z{~A8qbEam-!u8wJZa2>O6Bi`fuyTRuUS?k@V zK7+x$r9&bi+UD|V`|zbf-y8T#KG&nIhcM)_cT z)H}r}tKS1~Kk4~DJ95ogaXB&~$qZ+P!i`LYGS@)#2wasMQzk`zArKy)r!j0;4!IC| zixSEeH%@Rm9JtOEt|8pZ)8p*tBzm@+ju<>(2=;4L%F2>qsoy+nseb0OK#+^`MrtTW z$~U1DBR;Cg7xIoO4PhMqazVd8K#UB}h>3(!pTh$`sn`D{f-EUKi)VY6(*X)*c(}b5 zgpGL1xBiXy*g3Md8?E1CMcN}$K!G2bVe+>QA;BqVk7iwTOrIL%b}32VDq<&@%olUY zNc|P^kLe9p;}YzS40r-QiZ+>oP>b zYA5YoVx`w6j^(_1|+ zA8ztmxT-7?s}GjkB)) zW2z3sgEJjboyn)ihDlK!E$ZjHI4CA`bO%WL?qvj7{w0Jvzn=Rn-qD8<){2+<;^D5b z6W?VZ%XIqMp`v|dk?h+WqoK&*ZJrU~t z*B=V^xInOzzIfn-Xt_ZH8HxEj92M)CFe2G|Oewh56?`Lu;tVcx{qJrDMzq|g zAEt=bY;VyPZop@24xw7i?}PGC@;XuQrHCmm%zdvcspXu&m@JhY_H9TA>(hE&+5%IP z7d~c7wZu1@(@U*q9xzoDRLAF{dp_zR8rkj`B5K(@&=W#8EJA!FlWFiK`Pw9F>=S-r zF3IvSt9Ici97ZU#&Oidn)re=21w9hN!UN67K#P2l%Pk1Bdxr%h02It_?ZlYM%CQkP zXLQ~Rwn}JwoY~2aK-7uILY8R)7HmIEK=tGhn*J1QufJ-tVy_G~dy=qF`}^feKBH)^ z!+YqFAIFG@iAh&^D36{Q97xRzL_XrdxvR})2o>oHH#agyU3$z(L@dGYqs1z24QAVL zkBtesgD6%foPWMw`Lc~)Mj+oo&89&!zw2W5V{fy&q|*TQncOJcq;6QfP(#WSBAbIf zmoeug+Oz}%j^CrJKW|*LI+<_()Mm}#g?5N}ft^R=_#Y(a(`J#=X`Z9GYaTZZdWvP@ z?$09u_!wgMn@KY|qIe0Hz=ly(lJ!i@dO&bb4`&Lgp zQZB;ofcJ3aF2z*dDSeh3z*-y2up4JOZgr(>oaZNNnq?ckkL_tblsa%m>R`-yYi2cn z3l+rrT@mJMYtOaCl413-TY2|C{?MJ`Oth@~*7fKjQvVVSp5g5P=Z%_`5@pDRxyutV zlUu^I*qqcTLl9*+O#0Sj%m|CaDZc2(wYw6vvP0Vmo~tOP&eLtPx8W3OmsD2U239`a zGQjsQ2A#i04BIiz!mmoBt>WIo-5E?~kiSvMG(Rh#4L@%|)ityH^j<#c@H}wriQ65@ zc`{>BF;6;o9JvisqtT1TO1B+eR}ka7177PbqQO+2rd5BIW&xB z<^eQWm}1Y_{i}MzCFO%(4UJ+MPz7+gZ9I2fwO-q7oz_ z#0xPk0nh05n8Qy-G&rVtKs6)aaz`+r`>@*9D=LoQCKVclnHcJNmABx^K3-$)dqUUe z-FK>p48lDCDjS=3Tz>5OaEHGLi4}yV(+wKPwnVk-_vS$Z!WBuV;RH0juW>#4s&=%n zHtJDHwoe3JFTB@kW>D=48v5JjR!_soZ>;j%rW!CZZ~SgG8iBvAAl*jxY6Me$%NU=( z!r~1LZRa`%C8`dihK4q8_V;IdzRxdmJ$yD`&bNI{(R^gzPgrolQ!uwW8Re z+*L663WB?o>J+n}B z`x7mcY~vH6llmo$#59o_1-pe(4)*7pVaG>WMz(lyt9`FCS5{&L7)@Wh`?RJ z8_S5 zq8D&ka@_LD6XJ?wa6cG%qqVux`g6d~_9N_>^z+WbHuu$u)`g6^U}})X&0v!14z}sD z)wTbzc*8hJdcLxb2nht~pK7Oe2s(YM%UVjUD5$StEb-^_=&1Df6v2EC+xO_puJEKC zELhhSi|oY9I54l_I5%0wP$(r<(JSWrB(YcNk@WU$d>ck%^%$gxfUBDk2*AoZxqsGa z?N*g|C?Sx%av zh$4KrYhP32`+ghsQ+PJ>m?=(uXxgs7`jam0IDT#4jRJ#6kLsFUYZ$23w-bL zdeLtuZvQPTvV%R~s}T(L%hqIzB!swKe%HP-`;B}N^RSKLzAqzut9X{CfQ z-H9n-J`Cw)E1?K0PJIPA69k^?3V{oL7RuWH(_iF1iW2Mun(|8LcN?jDO;fyoy`^?% z8*XP_`QZpz)pI)a^@~bz5{EDGhCN>$mh$cQ_Rp*+lv<>jst1uf8@q z%AFXY2!H2%Ka1dB5~vmS%o#puB^5gbL&i4bpkTmBLEoN5KkWuA<0Vnc#_;PveG9=g zHdepgUzA6@+}9RLDo`n=)PVrBml~B4yf& z5=`;?apK-cvNGTsD%vWMfx?_LYVC8%@miv!ng>R71&Tak2gf)mI ztZzex2r2veJAU7#k9PCiMk{=GmOC=soND*fC{OZ1Or%JTPH$QMZ&!Erx=m&*Mu&|_#>-L)Fadj zQI$tydpt+X?#gMvX?BbgQN^OIg=JkE-Yx7Q=Px(QwCb-~-n5;`%HPN>5uET}dmV%xOS3bM2;>tl@Gx#kb(b9M$p$6dXq4su; zFt2!~zBH4h2P28E#tM5Fb6#m2F+AB5t|+Il499gm(^0v2>+1H_=%S11Zw}l0Lel6o zz)FfEBdV{Q)CBw*5DqUie54l^Z8r2SF2C||vbl`NCvvy&v(9V%_0<_XgYjt@4*nCoUmvu-;je;ddM8HOE~j^J*K#qeGb2EP zqzSRy*L}V_@N0=G(o3@UP1bfctC1WG7l^sh`ySOejn>A4jqr1=FnryId$nlhAvL*C z!uleNidXNb{c2uBbl%39kVLv1i&3Ob0#kk!S3k1wlBPP_D5H|`Jj6X=+*mJzZHzc{ zou<@5W_)WA9>Uzj4>?h0RZulin*u4W?~t&HGixd%e9-=?{&@HrmNf}74Y z3bMQX6Qn7Na`P`yMBkP8+yRS^kW{QK=sOpcmz27nJm{@&dM_x|`6fY@$UpwS%j(gPU$)#>@i>RRLvQXGl5+_U0J5j&jwCRS#@9Wh z`9vDbtSS~cW(ubpTmNbcpl37PMpnrQbNiPeHz^NG+=B@psG3S1z8Y22?Gnfuy$64{ zfhnD+1}|wPiI$gUAvqAov@Z?(zi<{I!idZ}swT`2EV!83<+BEzW>NhHc^>|cwHP!^ z%%phih7qZ{ik%OF0{Hc)%7S@22Pl~@&fG}O=kr^FWc0djWs$86=XS^Pz4UD<5X$X$ z^of~9Y4jD$wzaJuZ>g1FTIVu42XN8kK@xtar&C#JBiG$+ymWCFZ7g8$A^lm$M=Xhv zs*yvH15~lkZAuM>8NpR?yf0Bp=UV+UQ7Xsb&(0nh3ddIlt65Z-ksGcv>PW`;vtX8w zEJJ!nkH9=5>aNcjHbg-bk~F?(H9A4k<<`+w)byY`{@j3drJ65!zDvE#bqEEH+{`cm}N)RpMStnSymyj87R_n;xLOe^FS`w4{^@ACei)DKP2{q0aX z%-%XnO|78;;*wXJ$Fy7X{BQO++BqZ9K%dZP*JeHk(IP0pD z5IzKxcgwCo{JeP_gN3IFxPsxdfY-C>_U|3k9~Y^ zrtdmN2ZuzwoJeJ-ydp|#4pkrf+v_9Fk+dALXBCN6{kNZw^a{~AqCWBnH}#Xdhx2$l z)Xv`crPP=iIonLyU(a^_rM*8D{KW$QZc&t1hWEl{RG5@hu7NX6HqICNCpZeL*%*B# zDfZ6$hjzN<2UR8L`pgq8e=$RXe)X&Oks&dTZ1q5OLet&-HO!_tYR@k`+%sVxE(!{^ zWL(UXY>@8hfv9iB#i=zHj zBs~V6|6}T%!YkXluF(prl8SAvsN#x^ik*sWYsI#0+qP}nwr%@l@BMz~`7h?} zeCC+lz4hLPL_60@pKn1MIlm>u9sUM#c&i}6Tk}thYcY&9ogq1XM8vhe>XS7&kr$iq2{dZ{7Q!EW8jc`$ z=o&3Bf_Jr%xx!J`5KzVBoc>Is9$nuTj#TlteS41zyYBmXF(8T_bOeK2ykTMqR$Q{f6 zztQ(S*9CQ={k3(O#X>?Kb_f1Q2iSASA;Cfi62gzze&tw!)G-7rPuc2xch4{kHc*}O zGfzx|^A{FADqG_YjVircVaSC5#UMTM)99=7N0F6l4jQe3TVANk>N?@4D6%Wo=VFFk z3EQW%gD68IIL40^?hoS)c8qlP#+vFlsxgu7H?~c{A{>yD;c>-&x=IwG#r_fm`)IE? zc9ppW_<=2r{k{Cp31SNO~mV2npX!5)D2 zZJoX7y*x^>z*1Xi{_as4J`lVHd9JChgtvqKOxO?Z>G2Ip=Q+{6Rm7f@NN3 zZz6S#Dwhg39`XM4y=uIk5_h4ybiF-alfg6I~Nl_LCPsB z^V~ROh?wiu?S~ZpE12h*NROiLr+vZ#({h8oyba|GW2FD4JO%()T5DL~WqY&wG4Whv zy(7YH8M~5}zcjX#&X$*^z~hxPrTCFwaFa6N0w^?deeeF=(_BddT4qvU2!e(Bc}z9K z9o?`nK?!)$?#yu2B$G};WS6@l_&UPxC34jG$`l_Ykt!97jKJdvMMamn9T=HY7Noj0 z_uxs?G753 zgA@kGNktRK?w`3&rV zw?d5lO-`i~cynU>OF7E z>*>5@dS@VxBmy;LUqp`=1gJfJVTOhAwjEtZ_$RvIYFK(g?=c~X!`!SB!Qb}|dI}Vi zuY7$=kswMO{1pILG6J&s$|*tpuB*SNol~Qum^Ly1Sfxz7uC7k(l$RS_*(|;^Qf~v# zf#7x1$A`C(yt<<0AnoS&-=7C@-fV1sxh#_$+z(s;0~#mpcIXS$Qvcl5F%3-78 z8PUUrEpk{UJ=_C{oAom_q%#TQ-F$*i?K1F~K~|KobA=yGz@ypz47L^pwUGho<8V9G zTIrNE{!+W!=@=4H+5dk#MnU{6^yw(NB}npDHD*llx^GA7Rcp5NN^P=(S7ajgfL&l4 zL6Wy~-Cz2Ngxbav0=($*-RtF!(IDBN%bIaQylqL7hr^gDekS=}L_mRCJ&{vZ9Ev1g zf_>y&-4WrURk{UJ^Mz%7m};1w0_eHIOa>PrE(m7@A`k}jLhbn_?b9Cs+JCd83j}aZ z#E%mp_L1MXi!PA(w?ABX(u2cBmGXZ~05z{}ALsFy$$)D?!9ofZ<5v{%CMWnJd{4JV zRthdw_5sr0o?RPj{>G%JH8Py!WsZBp3v7jlgGD(j&I>Mx+>z{FURpc~*T-5i2uv+O z^yzQ!^qcQpUzklyT-`kQRu;B{B(bAle(8Ny3_=y@5@oU4v?Qv~7L2=2n3kt3V66~g zoBO1p#%i+5;l>#VhhA-ROm8dmv-wi#GfXqrzNZ;{8oS8Ut77u5@1>c7uo0#c4=4O% z+yG$v>8OQaw;2i1y5To2nUYBBYnoofB=7$O8!;!$8on|$NoCKr{*b%tE^0Qw@k2Pj0b4c%DNz? zujzx+=sMQu!_(X&ziR?!Xb-cRn!cgKS#&w|x?*34GM}W%Mv**!szjgB%{?6XkNz5J z4&z@^5@BSMom z0(VWeaBLuQ;IeWpcPR_uj;@_ugokEEvCWipnR+)m$tCV?f9s~1UzvN#_y*4UZd*i6a-r%HI5tENF) zBcszrjV@pR)Y?J3WZR}9;qDJ%LDgET%zg)7)2L>C0s*OqT^UWiBi5QaK$zP0U<57~ zj0tzGKWZknBHLCr%3Qo@nlqW>iF^FfcG*TueqtwW(~ntaL9pn>8$`Ao>&*h9(mjrx zGuUprxV=|?SEfHiZF}_0%)ZOVGl;l=oqD|-z2a;Zoaq%z<1Az~>l}Uz**H9pD)D$M zXh?G6NW-HOf`Hu1=)?0Ru?s~jAI=YHV?IMi2~av1>ss3gF^lJk+{U>c+ps&B9WqK2 zv#veB3>Ky<_e3s_&6@D7-n)Twda;|^+cESErsEP?ZsME9if6F4eia*2Sn->H4BT*K zLp9l%-oxjDED4NLJoiCN*Z|cD??Gi@DORompuV3g%LzA_29tuoGD#ZGfX9*v=xql% z6;s`VBOb@y43JZKtZ$W&+JMkdP1EVDq2a4WJh=w`YC(!}pbsr;F>rlb%jF<219!1M zcm$Q&z9kG`&`I{r?UbIavT}s)*%}tFx&(UG;T2@Xvwzph;HvUa+Oz!{+7^Ie0zt7tNrQWzY zWAF$QPV1=feQeO^TACOs9pqdv39Ww9=bR%axm*zr6Hc1e%KPBJc2+Lh14b9{nqlWC*L5q?a;Vo0#n!-#*vLA6@O|p>Q&%U_XjQm&)P% zcQ&GNVVgrFTN*BcE0TQ6Lsnq19im>7G+FhRSQ)pXy+h(%kKM5s$<``w+a1Qw!jXjB zo61hqL6RI|VJCrJ(Q(A z-9SYf9X;*&>1BfBc8p=tZPPh%{gU&Nq2uaq>M4lfutm>)Wx~c=b=uu9pG5l{eT~u` zV&cu7$~AU&$i zLDV;2_6d_yX&SEbWLJiEu{CA+Okwz$Z7!H~bSG7|lU{Y(lTTYj-1;}nK~K}mimdA3 zFjlvo&U4i5>g>|s?$k%gliq{d)Z*?iDs`~qxT+exT2d0`M?>|+ilrnmM{xw7NR%~9 zGQ>=s$8sCyq^Lvv9BgD;vN!cAJRo$VOvG&^2)ophhgB>-L|VDBT_ZbTBus!8K!DiP z3i)>F`3Z)5*XRW&Oj2<~yXikf0^&LD&`_n3L+SmQWGasUr zaxHem5R7b1W1%;lvs+Tb{hh{4ftz{h)z*Q~W9X?BBg>a&fyf>4K{yjwyJi5&1_Bh^CtL%( zX>jq@&G$48R}kmtA90_nD#7c`5nkJlWvZkb=-N+39ehPyZf_lbw0N%0FA;AK81GKk z!6ZT7?1?#|BEeSgZu0TZG>&Rp615HCJl4m-h<^4P470zekk`XLv$}uX9un75J=Fz! zpOTbDQG9nrw-JWsOZuBk3J+G}YtY?@j$jfkdCS82X*?~$P}@e&SWr;y*q8mXmd zJ&=OfJkCVl5*H$d@>skn5L;~F4sjM<``Ys^?dj9n%3-=t7Q7S^V z6mT`7j`G6+f!k?LlM`XB5Z)*C`E+MuI_3RiTs&NM_l5j=*;13uU=31{95*m%V&P84 zfI$G#+RJpc^-pADCxZ0mQvLoXHeFT+$4B45#e~CEsY)#~Uk+pC`|)N+=-=2z*KMY9 z`?;zO#lvvnUG2M{W)PmUsi2RBhss=Ci%d^qX)#;P0svCG!h3VYSzG(g#cS(kiWoN$pylQmo$zih3?@AC#cjK?D2)IxB>wUiC)|3a-Z{oFF!9sNl(<~ zxQ*>Q8z8;ZL-Z_nGm`w*5P$U7a2q>xVhrfvE?Bl_q0@DaNcO7r25Mg{5jq`jZv^4J zlCPHBYl=Kmbv)VOpaz~W;BoakYqLfo=dieTp}-uki6%@TupS$p7gqa zKQdpre^?!ADx<7%wl?nVw72;5RCjZb7g^bVlz zf9-%PZZ-FhW!{>KGz(9!8n>y1&onaJbe$|9UJ>V*RG05qo^oF5P``6TgB|ahSu(aW zlSLMvsSnHv_@%o`Xm19H%GFK1IY>5oqG19T1#@DL=x(f{`qkE`NctmTR!n?oULU+p zhWK(1PYxpmDMh)IhTUI_b&{Jvo^X5pH0Y9)WP4A+47PX-)`Hz^n0KJoqZH(xThI`fWfN8$)Ho&0Brw4&5EdH8e+!VM-;Fphst3k zG&(L8vWWsz494NXw{d>3Q*Xs{as=b7!X=lYTBAa+=}>R6#L~_N zj$Na0k{lgqakXWP405&`IyPrnrDYHH*bfE|MYAJC_pJy|U|Y{g7kkj7S`ij==59FK zxT{o*({=mfD44x?-ZpVdwi>SjJ?d=*G~KQ8kB-kGW0n^9HxVss&co=+k8CyLX>vpK zb37M5r8zX@33&bh0V(O%@2O5EnYHh)8ElO`FHXlmPj*R z8rL8?*aN1sb#*sc`bXb5vGG0ymvC8N)k6AJ<%##$$jZ?acr#742EvS9MegQd-N?9c z@WQpE_fthvw9vMBcf_BRofyC|LSED`2govLHWuG z@{aHp3`W!6Sc$JvO@wF1tZ#H~?6RHG1CQip2$_BD{q-L?t{r|?slkwju^Nxv#d_;Z z&LoM6;kJf?aLxK_cc8mQQ0)p0PPwY~pmMDrZdQ&Y23OJNu z{K(P=NR_0_pc8*e4f>5jCeD@gX5f!Z?rLbM5cIAE2zkHv%T7K^oJ$GZ8Ru^sk(b1y z80b`TtvQLHgK6z=MoCZ1zL=O1?PJD2)IEpKpgexiYyzR}T+Dk$7BxOBJb|TwEyjJ#{U4m&F9$ zkjUE{FE#zu3&R_V%T()4Ln3H2TCyC@S0G_vPauSe=}lvWTgMrBu}keB2)|)hlaoh4LFx-%XWCsI z1uelEFe&f(A4Z80i%K97xAm>yg=BQ4AaD%*=2FjD7^R>ZF?oT9RPlu&P2w5P`uRCzW?Qfh7>8SPL zJ-vwGeV`4Hzi}cR!{v6De2-HZ9?%ZnZb|Cywl~9Xs$V~KuddNK>@CJ=OIYxG?5L;g zbjWscGWwd}(X+IIrYZ@sO)@*Y_sMtOkGzBf>=>0)f-53-6E8jc3; zC993jBT6fVRjb)*oXq3%WcOmqtQcg-Wv|tBL_<%nUYEL*v-G|$K@1qr45Fr`L8frr zi7mwIO1YAG8twI+s?e}?a&@lSM#&~%fi|LV(Ahn-UYe|?t3P{K0bkFF&&@<_suvJp ztr8F4YyJM|r1#yhgBeS*ElhiJc)V{?t0SCPr`AA6+h9Zy8pBU!n@)M+)xa`i3>VlH zMp8SKo!!t9>uSf9Amt{g(U&^`$aL#!lJ4PpFkeY~F=&CVk*S7EQ-4Jx9b+czm%U`qS{AnVLjDVw# zJgyq$SXceFcTBYpw#-U+V`cnE&ItK4fR=KI`!00Cw>9eV^lL9Ok&OuX=Ef#UX)j>l zW9>EzC9EOY;=9t_C0($mTa)oz4@_Ay));O4?RDk`|LM?jL5)+{WK!TTE??XgnW&fv z5?uN#cy@UmJVtI6NU=EDwsE^A24@yT$}`sKka1dEeBfA=T<6v#7zSA$XnZfqu;9q32X^@UL zX=Hp!cB}j~$Y^W)86svgqU=M4GosMn^D={M*;n<*7DX+F7}CBX8lsG(3v0AxCHIVI zjXu$`aK>3&A>Ax1tyV|b5y?6uUQB(U@uA+XQ%t+OP$;H8PxMt@<|#3Uehj2M6F7{Z zC&5J{P8q9g*9H`oC*!tRgkC<;xe`q} z=v1D(p-auj16`%4o`}q_euOYARF9;M8W|d?w zN=Xp*ry%^_Y;)56p^q2V-W<(Ed+!Hxp|u8I3O8oJZ5IiFXY5aoxPBAeuvWWEH0I*9 z#P?qph${#SV4*r`cdZF7AuuW2((t^(>gIO*`THqjteDD1U8YhxZ_=(hOs7%FpwatX z;|!0iofTXZ#C`hl!!T@(HAthF!`?O+<-25ZI2`+=A?1liOHg|;0=_SGeqFWV=i>EC zn1`aObN24kg>q^ZHSkP2Dd?!)9g(dL{iMqAbP|8XkGeSFak;B;e zcB6b5m!lm?jGngc^!37c?;Z|fEK3O<-{p6OnrFnh*C$xxlWa2lXi7IOo~Zot)YXjKMRt||EI zT73|9JDOx~V2;HpdsEVfWdmaMGC=CnIS6VWe!l(QTvhM(EI!EYF*xk1}Da2Y(08IcBywnqcw@yUhHz8SM7zEODi1{v-#K;MZ)PZ zpz%z^T4K0>tI(NwnWxV0A1ps?9B74KfwRgQC<&HHA!UNwDhWb}sk)HgX4uGg5;Rv?Jl&B|&ruO@{?#Jcla8E&QOOe^yT6n%wjVMUD0 zmdHmDU$7g_MsYE2h49-4Zc~2~q#)m!`_5`8(J{?oyRsBu?dSaP`W&}5n?mYI?;3LS z5ZI2%MuYBhZtd(f@6hb2t20&?-ZY|9V&hnVPEDz4*EyZ9qZoFAY6vM)Xm)PwX);dOzIST*zK@; zu*=|x?owWs&;q=N8$gFG9NPo@A|ts?I?U# zgpTg*w8|nEkA~he%X;>O4Z9=X1!&f#*FZHs7jT*xK|KAq=pe~2Lj^P)?=DE3RLpwz zCsbPZID26#S^ zyjFo z%;}_08Sur3O6@dLn)QI?xfNiD&ns>MuqQ>2f6?a}ak%iKu>sLG z%D?nk3y6&;BNb=YIks%F_8f*wT*NWMelN?u9H&ly7E9r zn9L2N(%sH!E;-ZOnF*DLu#N^BL~blD1@sPb1`BHX=EByJG=m26J_ zN}QJ>ew_W-p&dUbd9CW4@D*2yK`Xq!k(mce6#9y&vp113r5{(3$#Bg4xe=hB?Y2iUk#IhV z963(eI~OG$JkN93`2!95M)3YX8k6t4I((i@KVr*KZ0UNQ*CC2uzyH##dFydDElQ8$ z34~u9yti$4TYEU#ueqT@v{Lw|33jIq;^LVY&f>Vo7N_qxPbc0=fexzjUC_y-SEWv3 zDX{5Iyh-J#$)5l*n8WDtCn1-LL9sWfUu z6lDC3aml+f-9Nl@uz2+c8RHNDLw=|XSDPqBF02G9s07TN8I-d zD=knsjL@O6o!z!_14O^N5{T!Yb7rf^ldjoPdBXG;xqJ*IabL@v2;Sl?f7=>f+qNn$ zUMJo5>cU+7xkxL}|0s(y^hlMLH^Yc9NnPB14rV^^m4<*xeH%~cuH9ZhrdPegH*4s7W_d^PpY$cPeO$PwMdv@ zTcxk8@gN*El2i5mg5?*jF2x=&e@^bCBdk4>x1!u4kH|=obVq&rf|reO;QX5K8{R3} zk9j$!`-Mf!?33a0q;#xxjK?S|o|`b|=Jy&!h7 zejlp@6lks7T`L6Nfkn6G!L_EH2lZ-H!uF&9`ejSRFyPm++H|LfmPQcX0@5yNHLbp| zTr_c?I9cjIIcrcK-MR9F4U8R8++Rkt2ir-P*`v_jaP}%6Tzm+HOHD?Y^@5*W)ip(; zv0LtT(!vZT#c4AxE?95D@q~PP_ zW@XH1UMgn~InC=$%Qv?)GWn{u+nU$%65MwUMvGrw<2|ESJeFr%`Jdu1mJ= zS>wABoldRRAb)p%xv5nhYqz3z2+#ADwd8#8)v^c5XvGy#kH>$N>)F8dmLjFfjG4&t zW`5$AX4^v)cb<=@+?{cn9nqzF%T3MdTF1+ck@uQGseS_V&&Ui>7e$i@%@(_#mhJZm zHe9v`5>s3ia)l8tmFjN4LY(g~`F69Ng>EluC%NS^SXR`$Hbfv{4Y2!btt1&AjVt09 z7yIa9HG4ugazyC;n_N$d3+-;hxDUhLvC+MO8&rR}7l&vz@YA2&>K1s~pRd;)H_0Br-#hmU)CO9VwXX#M0 zymjJz4OU8MsCPz3&a#AF%?c{fH@bZLZQ*Mk9JD&7fSw<`pre+U<^nEF318Nm zlZiAgZ+4vi*j>#9QX%;^MkeOtK|Goinv^V^znSl;5B1cyfudAj!rw_QlpEsC6|3QY zz%H-RKu2y*8RbJZ8%+{WhWbb9<7Er_UtF1>`SChPr)7*?NFusQxi7gH_hdM~`UWeU z)jEe$gI989j-vU6PTjzRy{&!v4T8G;4O}0>Vlpk-1jF^#P$>u?dWmPp3E|a~(0Hac ztX0rvGWacR?vPaIOb)FOlYa8jR*ki(i5n4pjm5F_s2%cuTmYuT%cvy_WzIhe#+Ch+ z&ls+Bg?N^B#vSxdet{eNMEbR#bd0m_3c5liHItj)2`Fr zPJWka9uKVSnCs+yno8ARlZ>~QQV-jgZ;DbPg%NY0ItLP7Z#4$#CY26ZNc70w`!wo> zqA`T)7}U@=qy~Ndx`UL30HWMo`p#o+>N*8wMT(LYSKl{t3)RrBc}b(htScLG|1cwF z6O~f!2=%X|i#bbSrc=TJikaEt*rBpBAr7N29~4)wc*TxeBE_awXbVw?6(6nXSXh&m zFr*Ag6TpT4+vIn|1d(!y#*AZ;)@^szP^lQrolJ3G3z#|Czj+(GpGp=H(gcQEp%{$D zkgb1>ph57UFIF(Jht3j?l7Kowd#{dRKAq0;?ktoJ*=PqPwKCx?U$zH+>$m#!EH=9S zc(_fPt44dNS!$=pL|p28VE_q{X^QO6t)L$shlcfGR}Jpe96)q=$`XC0smj6`$(sQ4 zpmGb=x}|4I2xkQy;kwwVa#c_FC%LOVxgF!sH-dO1H*TO*TOZ9FH*;7bu_%n<&qQT; z00a9LfdnFWInOV4HAsJcu`xx5vk zXQs-i&@4?35*3*k>X|ac;y?YGCR8o$iPL$AO!f2M1a{B!BpJ0w7&!zsj!R44ghles zH61Hg?_(rD?lGG@7$y~J@&xSL4ov5(Ky|0}lMm_Qt#%hEfigJ0X~+FMjzm+Gcz`v0 z0cP}se2*QoOHL7#6o$ff-I@E~`(?kXNAOBEt`bB&SfM*So6X+dV68VqH~AC<`C!=R zgUULvA-1*qo8o$4%2C$Zyw@fS`2gaQKz|J<4`+-o4b$*2;^F|1^daGgAS*g{r67*65(?${^8v* zb97IxQp^>aw0|{O933lvYI$lMumO2r5-)QPmoNqdePQ@MfQ z7@3M=(ZLSjewP9a^i#?HCgboj5{^RUPsjS!<1uycq*4^~oNZP+>$v=Ije-Jm9U=ZzJ%g1q6^ zyFk9=n`KI$6k+#-u zNP>+XG(FXVmFW7x8LtmH_}`&4V?EA~n3W-ye>IHY-zFL{=;Jrg`Hv=FVQh@w9TwDY zERAT~+*Wcf4Vz$(?G$j;+@Z4{^r5rdWrDEk6fgK5c^Kyk)>+I0@>Y8gZci)prAQyr z9YJskzf8mC|BUecr@Zh)1pI?*y)2z`{Ex727hS1DoR{`=qm{3Bk#-)YD>?baZ!*$w2?fW^TzJk264xexWy zBUge~N^DSAjP8$auWvY>+!(VLX5S)Rr(-qzD_T-!4cvruR%*HI1aJ7ha;SeqSb_$6 zld3k~k{NHy*ztb-*gvpzgKgv)vRjv4E1>}6J^Q5VGo8+*zwsjsLHA6l+IwXCU%CJH z9R!HKSU~ZBo;|haX4{E_Zy4Jb)IT3>h$v85p+TgoGSiU45__<+4!9(%uw@?1F~bTB zyyJI*`;g07u>efIFRJSjLIS!K_?jI+7Xamp{kG&cjRggOPm zEKJ=BtwD)~pdAa6i4F=2wgKBXhYwx&3GIA8Dk<)yp0Vs>^?E`|8%>QfVZ!b`lhkY&rbBlfW*(@IKSod z{nB$pM)CdNF}KXcd|A{zPTmqYnECl;G$o@>ar}Y0D3u@%P5})n&?J4b7hcFQrwIAj zw9eM#l0Xa1V5BGDeWiy&+7h}&X^*!I-J1u7onw~kR3KgBJg=Q7*B0-e`SV( zhAxje|C%>}`GJCh@(W^ErbdWQ4l{cF`T4oYdN&{^GPX-Sqhc6uhTlg1;^3eFVOZXX zIvVDF@_esFKxJn5d~9^o_lvkg)GPs{O54~IeHa32tX$|ufzTnO)8x|KdS8?=xbV1pJ86B#ZI)Dl&$3;kJhsFAJDAn z3&e5=&X}J|7%~H-21^uhI|1AUz_mKuFc(!s%ID_f{ohDHm(q9-ERM#8_ihg-DP4l? zNV(ReM&vTT-XL^;GQWL%T)YtM;*{~&kB-^&)tzH+PFGP;Ih%h}FVpJOtd4~=+ekR2 z=b3ey%-|~B5={zqu;HAYokit5ua14$-Q#MEeH48u%6K7C3pPH?V@Eu-KbeymdNUkP z$^Lr&sUc*g>FGtmDH#4S&H?E4x8KF3_zCCjA>UVN>3Y`b2>(zp(9Adt67hm_I9CY= ziorV!=29|={o~O=f_WAAd6CW)&Lc*y{~ue=BH#t5BAcnyXe~gP{zAy1rakUY6#T0W z=ml79XBKd{0U^OhB=l)F{Um^*;tV_4oQ~#1PO`q@w3A|5z)r9^ZSdW`izo}?BEVYu z#R zq7iqn?O zr;T_`<62RHJ(ajy^B9=SsQf>_r~VB@w63*FGK%#7IpUTRco1*L7j(&oRG6PnPRxjS zp7xE@Cjm}OUT8SYynr5H7FZ(RvjnD_Wl-B2nzufH9B3$I><3pi+{DMy@z&YV7d-uOe-oGk{2tOb5U1`L7+G{pM(5uvt z&Sqn(4jN&y5v9hKS{Dl8Mf zrfNR%M4|s@on7~cR_@&M_jFXg{>x7{^V1mEtCs7nHg*7;Dt_bev2@Ei#N8KeVFvL3laAFT!I#U0 z>Re<~rZ3~piM4QSqjMm}yIS^XuYFlUvlA%SVZp21gbfjuvB$}tK4yn|uTm#iE^dXP zdI93UNW(d}Y}SDN&xu*Ne2QE0wqvn{FQ^MrQEI2MojkVGTxEi$jOVvJV5-r`reFQP_;0oFY7>5dH z_PdX_%bQii>d`4L0xf~ifW@DD*sgGHzpn&h?$8b%0AliM|C z#o@aCX82mNtmPAP8+205;Z=>sWcGRT-of^08u?x9Wb)+G`I@;pOA;wjD6CAXj&LRe zNj@iY;PWkC4N&vr>YC#D$H}0srzgqsZrw``^imZgVxkJfMK}QA-WZ;BpnTW_Em6&S z*s*#|L*zREbh#M+SoFafy6Uw2a4Opyd*b{Z{@?*8m{`s?Y#Cqs)Yg4!a2a@h);p)h zRP)2M#93)PZLrZAhh%(U@2Ju$1@U~m4v-xCe_1aA$O^{o-f*l~5&~eETq5XbSc;g# znMuy=A^;Yf!ml)!tLpO4l7{Y3nmQRE{eq)E0UcfCh+B>|S#L0+dvOtcJe5W1qEp?< zI(U^V+zRj&;5&zyCEY7Z@maSo`Y!TJ(S^3YgcGk8-3exWW^p93--ABMb*;3H%ighr z+4k9wsi6{#&h33PXNK@1AV^r}{zZtD4(Ng`tDAS7GJN8{YF!E)PoupX(>d4lyca>u+R`)hVb-zRM?t2Kf7r?^g2>=R9kRyE|^NPL6r*Do6=Aqh7m z>hoj-!R4p>(#Y%mn))W4{+xpC-RmI+Jp?RyIvp^ns>9z6b(7hst=kNU(6PDO5?}gGkTYq?~9u)HtRI zO(GF&4eGD^*kwH84{pBLADen^m#ufnDu1HltR5+cLRIHP*Wn&HR<^zG|44b78gM|7}s{mFm!ne3X#$q9$UO-VDMk*WN%~N(aP|C z($`BZI)=Ump2;)vB!)QKZt|JO(Ai1#H8}@q{6^LZzb}nYFMnpU+VF;b~`qrHqJ`X{S5&$6I>Z;6wuqewFWYc75)@;osG+^8;o^YH616i{|qB zl~K^aX8)>Gy(0+R<9HV{X!3qK^}}bHD=ecU`?JuCH%1|IN~7JNWcYp+#dxW z-jxenp}fi0->QjgFil%NpYgC7K~gt8G)W9nrQGlzlcTkPU)#r=Vxo128QmXo1|u?* zZcv6D4=^e67>f#iwmg-6JX)c80vZH{oMS-`P^Ya=#k;*Ftkb|v+SeWXczL3eO0`_# z;}KQMu1f3JtKk$d2ly>xF;HE(j-D&{&i*Zg7NUjO^xD+x?xjTQIhT6bl;nh<$mMC1 z5qb%14G?Hd182DM<;|{{%r)o9CI}zED`;S2Q34Ko(s+==s({T#G~>OB3^v||S8YJF zF}W15KE~+AnpNYi>?A^pT4QVb@IZy@mFVPrG5TgsP3$~j@gtxLA1yb$G}XeM>H$?O zXPr9@hbZ0=(S%R=;k&Jn32}WfM5AIPfgZH02-qrNkcHStW_1r`VKpP$@GsP9u*(vG zS4kX=-XNRU4LrI_dQAAeJX*f>j9IF)iKQeMc3tMOuJJy)Uu_QF$EabTp@^MvOmZ*U z0##Kb1anvB;3OoJ^?b1;zp;f(h-?I*vDoE@-Xfs zT0+87CE)R^nxkAa{Is|w|5gGH+(Uy|kmsBpYk5|@ip{(Eq(KvGFH+hgWuz;@0c+&O>FLTz`Wg`z1Su{ zlU}b#IXiFU{8FNy+KhxVpy6nG*A^3!HF(HUap{*Cr55o&r!1@IeGR7xShPxaxxSZ zm-OT_Brc_v!7)!QFl?13_Qyh>YcB{j@uDnD3AzNQ_!_W-4G@6E0S9%TY>4IC} zOy@#a8x&J_*tPs4cZ$`*Q@~_eDkg~u#_U#txUzddHL%n%ZsjrxB`}yW^paQo^j1%Z zu?uW|!{$@-O}#Xy85(DLm8*?7>mmW1CNpw{wmxc7Afi> z60t>6hWhF!W*{4UEh!Fk0dtEH&JVN>U&XWFU*T@Cv*cLA6L&Nodc@Egwjh)T!oKj% zXL0jRRQD#{Z;kXcD#ZVDIqcv;2!(x<2Y$^`OJ_bJict9c7HrL?q&_K3W;u%Gsv$66T1(q3@$UG;O#JMdhn{ULjNk<8Z~&$_3kDs zA*@}>2j9<)9DO5#0cy>lh_k?Qkp%_4PUK6BC85(q8UkBSb`OkL?k+~# zqROuxN}_xj)(dERciT^ncam0T1)0!t*V7Vb8g0)QG`X^{TdLt4lG&T9k6HF6j*Prp z4f($cS2{JE?9@JPX@?KDIDhZs?BJArD0;`!4Zfqf%0L={h~y?{jh5|>;o5716Uq^s z_WzHmvkq#5UD|$ecQ5X=xVyX40>$0k-QB&odyBifyK8WFhv4w#dCocSe19gBOeUE- zyZ7#O?QauqvFC<1rm^43sVH^WB@aaArP(M!p6EkHU9`WbYU;RnIeb2+`O>dUh(vs- zIy?_TFBe0{3uTl5_P=m;SEV<@lcZgEcnkL;tosoA#g zKPG|6-HXzgaGbWNEQt#iXB4pg|}#uR#`L z6>i*!4iU&vIt{7>zCq%0lK#k8xD)Gyrbk_}qJe%q3nYq~pKWoXZm^zLXl5cSO9j;l z|8^d0Kvjw3Ef*E~{d%ht2b|O0l32=5srZTtIui^PA_0ltqTPtqDZhO{z#!8^F*L07t*wgqpB2T6-kXZOzdS7BUNr$mdH&ZZaEEl9 z?;h}!(8Zf4;o~Wy?5=SDijX(`Ez)BSHVitnnW3s6hUVkN!i$){*{RD8tg^d)9t&f( za+b`OGo=JCmfW6LG7eya$%y^SHk9(( zxuC?xf7|-DuR1ZW)X~`I?J$BsNEFjM@pk8~m%YWC&~^X`K&(5$+ie?nvz@DA_x=js zl&Y9L1L&P@(IOFfc{NPkt(2m)wO-e8v9WE>R5VDlTy^( za;K&NwD@#f%J|5xUUV7{~ zwX||hk}|$TxwfvN-lxUmMIn;0Y$8aqf!6-~^$lFBKr&Z}4Y;(V0qM{<-jM{;5E2mY zp{`H1DxJ46ldg`NA4$flMUs`g3oGXnUB9$x4iHJozMasM+uUHE?Wo@stX2g~m>9VP zCMc*4lst;*h!P=+yEuGOROZ)xD|7ujP9Kd|9e=KJjL$<5RoGxDnlIeWbRd)H_J|q3 z)`Cg?8k$%2ntNRuP}}mWn_1KKGGN9vhH}<6pn~o{9qJ`C5=A2(nsZ58+bU1(R!8yG zbvA11;lt31?zCbM2wSg(U2$VR8hojqO0f>J@$v5 z{uI?hFN2d|!X{z)za`x^BIJsIW)!;Wps;d|L>}-?jlMa9xeK&cMTYiX-_DhBD5{Xx zT%690ZAX-#_D$n&(zh#dB+i>kXT_#%E!~*9PAgLVp4DidKWj2Xo~uVTmu~Q~V>Tb{T08v)JiM3^ku%oxVI#ulz{7BcyC=Ij)wlv|O51t0s z{zT@7S}8EY5QW%9`WiorBz|Y|05K0ik6FMkI{RbJFI4n%(E29X|A}S++ug0Qvr+y> zOe1x@Z5<-Y>R!yXl&hp%?Uf(EJJ6=6?6W|n;H~1v3X>fk^gwuMEdx!c4;!y~$|j_g zaTF$~<@KM2rVV8H8%1E&uBF{djWFp%#6*!J&$+yqCwj!#^b$bYaN+BB)zaw8YWK9b zXjn(jHG|KO_wCKgLzbqhzeqNJUbll#zSn->bJ|u6c&FBW*yK~4{iicFNUCIi?NyX@ z@D63T?a%Snmwy!hjhN&%RVu$aTS{Y^&yV=q=3VplEgS)$6G77Ls`n&H%|L(4rw3ff zeW4toyQFfqN((oz-6eP%r*uWz9>_UFirHsxoXQlMy;4hw6v|UU;DU;|G-$aJT?hQ; zygy?-xMI{3aqoFnL5jPq?#D}*N@P{a8>iDG`9tX&`ha}SX1Hnpl9p#D}D%iD_#Tx>-a;qR^`4lU3YA5Gf zvfxBMnBaV>rH8{BJC#drH*|gkb+>*w5>XOERrz2<1>ra<-zK1OjLWg+uw2h!EBboO z!24T8MNM4f&lQyradEP*>c9i!Xb(~CXG2lJZa9d7MjbJn%eJw;bbG@U$e3eXJw#30 z7#Q-0xbl^EWfpWb?*R8Mt+MaA!Q6V2LS-Hmn8I!b0X2qrvojO4yeN2oB@(k$Ty2-* z-~GfFZQTx2HWH;?UDG%E>IG#CG7Dm7-wpDH=uZ)dj?c?Sx^k0Lve7!}x)GDG7wdiaq#D{l7n5|H$Tq}X!|06dDk@cXF82;{9ji8{8bLFAgMSsqUX&cS@!?h^b6&dal2y%Fv@5N%ipTG6l>OD zAn9z>Y$jVZ3!J#$pmA@CGI?|ECp`xg#xMqRd;Yp}lzja1C(5Orqi!0d>89fOX!v4c z%h`#o$jt=t$3e(Ueo##cJ@k(;-8-mQK4RVc+E4cT?9|u#I~CbUY*=^5y%4Agg!SJX zL-&K_<-poFF6A-a;KvfNTUysxk!5UC3f{$>jv*RD(9@AfU$<=NjA?bsu{*=m%sK)+ zqUxFWnX&vZH^r9zBGN6ih4uV(vYBXqMXi%lV?SeG9^+In;)S##uJ;J*lFXX1i^6tc z*D%{he67*G{)&<>uR$$8N4Q58%)};*Q1fkVF5W`6Yq_B8=)8yX>vSwnBi(YgOrl_J z7yQ+vA69h-A4)nY>+0?haf>fI0^T^m!317K_5;nPtlYd!BCQM;!&Rv1#y!4a z$pmh=qM9xA-jEG7r(duD3*zI$Wf<~AVViU<^VG)n$({+hfGbCr-(rgceyPZf-d1`K+dUtKAQ#vC(Y%Hr-}; zS&%WdlbjYUZdf=nWK(D`|E-#UV&6G#Qp$ePpM(y~e%@Eo$FlGS&R5|oPEpsQNfh{k$AZ`~1*+9|0f_1|JX4QpGz!JRjBE zo%!jzjDe{h9@qVS;@j49r|N6Xxj=%B6+yjw;V%OsTFbhp6ACHqi}5l2LD6iohStk8 zZk<`#wEiJ5Wl}L**4-KMPIYBnY_W#t&NP4==jruYD>}8IlRe=o#j8;?U`Ak$H^~GIOPFI=*nN!HlTF zMz*xGcu$+Ioc+A)?qjRT1SKcZ+}LP#xE07H+D-w;4k)9~-ZYTnd!jD@#4U(KTZ7C` z80nSl*|A0GJ`1vOu8&5@?YsA`juYwu6u$sPrewm z-sFSXVmu@_GubQ2gK^kB9Bs$XNhb>2e?ly#;(k#k@N>;+#bik|H7fFzhnVlqj@mX9 z7`7P`WGCKJr*p-aOCwyw(A*|AS|)X(B5A-~V9eT16OiI1KHS|Y zA6nL%PHrC@6vtm;;pJv~?cUv}VBfaYt&BP(n}I$i&ymdDcbQ?*KKJJM@E!@Gb0!k_ z!ZQ(32f^h_MocIoH(+~d`zCo^6E>|hp+gu%1&+VBw+t(9dM~skxwO3)VfVK-$L^dr z7W`gWM9Cc{gglG;c*DX$ngr0k_d$x1Uz{h4*Aeux;7?~(C30>+;nm0b&(I56eRXne;Y~=CCKjSh+U*@Cv`hMWPjo};<)#5stTGXu_`bKx(dq}!{I!6*X z?94(<`*A@qcH)}lUzm@v%oxaZ!JdA?wml$8mV$P2ZI|lF)tGC`^``rEMn30Wd~tN5 zqXZ}ntcx4DWF-{G@5wH8vA`XPyxWi4hE0LzeZcZfcYK!fu4kd=I4tEVt`Uh;xBwk$ zKKYcw%rXsLvC)br)7!y2v%#`^4`Umwj=nc1El9ao+l_w(wL+UUNzA}30~8*{83@Fa1&z1FK=olRYc^u69O(Qodz+OTjK z&WX@@yTJ$g$j^^#D7$l|;WSIfDr|x$>8%+TDpeX=rHKcqo!u#ri649R7Wrb=F2~f_ zhT{rCg~xHQ$4I`^E~~Krj>up|LGlgn{fU)?N++cep%J8Yvi~F=af{e~pn-Qx_()Vx zxjybeE6{h(3t=7v8Jexa!d zqj;pN?Mjt37r5r621#enrg3UKlhl^7bzNOuW2S!^NWXhDCJC5PE8Na&5rMRAnOwH+ zXZ5U&Zt{B!U+4IZd+T`-+?X)*7)7do|2|5-kd6*-oB`$!x31tPrWVDLE{Zw^`Pd}b zslK`6Q$poH2`^my4Z)U&fU4os<(3_xuJqF|;4`W>TSDIFyF$vQR@lwy*nOzS0%74}IO3N}0sT=$KaE-ok z(_AN?A2pMgAE8zu2k~uasf;zz5c;@oaI^CIo;Ly)ryf=#wJPY{(x_xfY-6ZvFn;8` z*OnbO!V966h8KY|z1@AX?P2FkxY-`^Bdue=`j0Iv5g(q_NE z;;1Y6rtr>W`be8+5*#z{8$0j%*DyoN^{;X9-(4Q>wc0?a_z{S6ZxsKW;QjOr1cAtC`Vrm~~@eHk4dAwFJe9fql2b?aDh1Yl4ZS z?yl}Z+si<%^8~j$6jL4yW^O9s!LWdl%VNLU$Dik-hySyZi28@S0<>9ZXrxG&KsUT< zRHtG%C{g@Bwp(2~q_?ref>r1bnv5)N)nr0NY=mO*7^4E_GbD$I+m-G`YqIx=1PK>K zVM*t|-CPTNziG>Z))NZ69FvEnm9ThSxHkC|&w?BR{PXt}bNFDLUXQ*I^cg}0qV9%;->@6orx`!S#Y9q|2 zH{xiYg&qWKQgkl?qN zeyti$2Hq5;!7NH&9D!d ziU-Wgzu7?Ax~R~mc>if^a3MXiv+0^%ZI!qL zT^_GUSktPgCR+E$D@@GMQ&~CL1HA?CY4v{np$Mum0D&lM{FTvEZHc z4*#Fa8Z~(jocL*IxVVhi#A+r@FigUO*lu!3cIQj6_JO4TA#TDRW?6W+YX)O6>9~dP zp8^cf#O%G8l3us34!4tGMlLO&WDZCz5NFIfoIBip^zwb&y`>X0{1Xq`!T|1)`DBqV zlx0zZw|C0WF{raq{Vl3=A~v3=CCv&9 z&LLuXs=>^!?~cb#@6_$MZdb~+jD35Kn^{c8rv z{r9rJ1Kwc)omh1TRumnbh@34ju9uniPMVqSL7^ zvYTs6_wbGmq3+#ML`SVc;LbwL!P6sk=pM~v9zgR+F4ofp5?s2 zaP8046lChht03qAb#pSwbUF2P-XpEo;pHp#+PD@L03A_(UBd73-z&W>oMkos+rFhC z1TQ8PY&G4Kpm_Kf4kHHnM3B$z#5v+Li*4mWmNhC-H$6NI0}|s^Ibo?h>e7wBV=&9_ z^XTe{4q`$zY`q=byLznO6`=pmeG`s)%m&)bS)@2<#$ zOgJHQm|)c6d<$MzIXH;jkj&}D`O;wGB)(E_@Ib|PKh|T2qcOnL+@s*l#W9yMn&M)8 zw;HMEXy)VjOu|%@>xk&_GtlKjj6IA@JG*45`=J!% z;r^62nP)x_7P{TJqrBL%!_|)XDNP(ZYtXbcQUh6TO)0A97dH5)dx8^-45W=qlv;QE0!<+%PiC~s_AaY6fpxqE%*Dv5 zZO(j~(tr&*fMo`)#Z;h2b>I_pdIo zke~nf=>F>o206&MQAy^~VJ*nkB7zSi!Y#_maymYxAut~=3lb(x?gwsb68*Dp0ZDq$sWO3gA=cQ%{ z_Ln-0^uS^HK|0&o%uihuQB)!-r*c$ggnF?b1-b5&eQNH_cgTN0JFY(9Fxmln>vkLL z(`(dPIM3+RI804U*osmctPo)~xh+FiE@nhy(}+&4kTd*6gaU+T;dR(wb$a)jl*!Zf z*;0DhZ0opb5HWngs1ogpkoxl)0*wSJAUFp2o`_-49ODDEz(kEFM<66tjAR1EmgYiz=ebd{m`n*Rsh7y^Hx9K3KQlqA9x3E_X9 zoPXWp@FXPYv4&mni1Nt20kHOSsVb2kFs?gHs%&u)!6k(()OSZeD3(0BSmG1`4F5!0 zB5)%1dts3*gPn-q)xq+}8Yu5xqWiN772bn^#&!Tdz854|$+N@BowFkZMU;KquP**iiXWXgPyOzJG7;v6Q zE8Ha4HcyBqXLPS;hIHW5ja-N^QSk6eg^K59=FALda%h~BWjy}XeQYf{@bo{RXDK3J&;=x2A4>zglq6QGL?hWvwFidce+}jljB4i$vLe%;t3v>8Ww*Mt*_bO;$r~s|$&dOM2@i z{5!H)8d7|^v?ihTBQEM_lDM%?WL#g1K>#xIP}Q~>H&Zkz`M(v&KY?#O3KrF1I4SMO z1eC!^AO~7W4|UK8>lFFI-H(VaRd>!$pW1h`;yM1gI}DO+tD?|*H?&Palx9mI9@eL= z%s1A2U8tOPzqe*YEe^foNkjruW-3xo%*Na&d8No6PxvPUPPjDjHzPD|c6#dHuIB^T z$A7@4lRq0Kpk!OwSeyA>43Ot3^xH>`cCQfpsx_dcEJgs#-EscD8K)Qb;^%>#1k>!8rfU_cQ0f~> z*=^icDuEPbi4w=WvRM=&F>BgJ+FO^7P1^=V$4{KkSwDa@d+l(g@l89Z}@- zVG}-;NPC1F`}#dj)84_Y8Xi-!CfJ;^;HMt7VIjprtlboM+S@y8)X#HH?{zlo+DV^C zK(q(KuTt=rhK4=~=|R9e$u1?LqHfe;*%45o<1;`YSgt)R=gTh!C7Vr-N4w)y7iOMg zyPUsWu-y#47^`J$sBB!(;+}2_x7NfFT>@}@d@B(sVUl~T)1R@g83<8a*c&(?#ulCT z1n0cUY1poS!A_||v#s^ICT`LLv3E?^FEOpp_X)mMwF&i-vtp;_!!|23-In&X8nEyS*li z)Zp$(P*4vVeJzN0w8VnublPNN=%^!RzNrm2XAVTWzz*eV*dF^}508;%_;Iv*p^--L zt1WpVOVo-v-v?8HVI{V+zVEjHeD%4ooHZTl9O^W4LAJn)4!6##5wQSldQi~GRbF=e z2SYoH)Eq~I5SEBPhZluoXH5;NSW{}KdAG+MRX7*DP5XDqTml(5*qQY_Zr4|!&3c=Q zY&s(_u~W=--ZUqx;^+vy%~{Gtl@dS*9;-t3@AU^0l5ggvt1EgRU^qbT#yiD)Gz#qt z>~kXM_WmSis8?Uut?SJ6a>3jf=^bQ+aX800Jn;I3aMZ*8it^e+Ck9uZkGl+}e@pdu zUCg_!j#w9WL!8)?;uD<1j|E?vC-FCciD&t6dILiJ?!=RiwTv~%#jLgikN4Cc;ENhxa&j%Ab#?KbZOB-&C+LKz_+k-x^RmJL-hazjlfu#Q)TWzd>P#wHk zWQ^HZ)~9!O4w3hB7nQNzlqcNA#!jP(#ZQ+G0T2Srfzg=&Kxl>GTITYKK}F_iqbG)Z z5yCUl|16qWL#cYl@(U_?LP5L5TqyH!&Aaqxc_P$#;m!n0TOSA^Ro9 zc2y~;WP)MnPN#?TH6?}<2eQ?$Zj*F2`io%yxn60~h{9{A4e%wVs-h+8KS-pN%NVAY ztNL1#g2PEaqiL3jIL*+!kHQ(AJZP5r&qY&NPi1?2#>&ufX;pOY4U%}&= z#zIRsa_cvi!4gQ3k5t`07jz6d48e9n2<{rtx=Os${BD>Y;+y8{75TnkKax>dLXFy5 zKILDzt}amXm*FR63OYr0QnFX0gl3-$rBs7OPT$mu;4hBxCk zxNKIcZ#XDkug+Mle`cq*PndhL5BWF?a86wdnW=Y<3gKYsq2@0&+9Q7GA<;%L(2=1q zQlco`&N&@mF7l$kzpC%Dx*yXZrVGMC7_L|m{KY#Ttmv(IJZqqhQv*<;wJf8OBxZN4 zfP?_yoD;QbUDc|Hm8R77l4%x8*_^RhIkB0|76;Lr-=zZ9l(vk`=@>;oPx-Yu8(qnP z>eSR!5zISO@b3p%;c1q#KVd0Tiv0t|1qkCR34wbLu(~P{}u-IVAf1RDNG*Fm~n~A zQ%p&I$j-vf|IYQozRdTBe@$>?C{eIk!j!ZE377~lbkw)96PROnc)ofED)Ko^(wtiap>jKP4*1$T^!nS( zkvU^S=}^FwMP1EYauH$A-OZqg^g3`E88t86#uJnIv76=E@C#wCDHwYY9zS>;1-=ci zvl90oPn0Q$yLLQ7mhiWs+d!{Jv?dvY$1AlfkYnHE7F;#88(GEtUN|ka_*q@fBYsxb8K~e?4`zUEK;_HsR ziObFE|9lEl^Rd)>yWPi)SFolXelz|Vp!h-d*m9S1qu%_QoFiG1XYB&fh4KSl7(E=Vs@ z4f_L`u?n%n=7*{cd{x>tzP3}p3A<*USDe=(&}IBt6Ec_8#p)^<+55}VH=C!emS#8V z4f-?co-wmGtjVzZFsqMaMub%J`SR|Gue9Kxt@CQz_e^(cT!cI0<3BnCl`l_S{m$PmE9Y;;!c8G@|=uGOH80{Wf^@0(o=Lh6RMb933#AuZnX$ZR*Owd%Ewb z;YySJR_(G-i%!L6v}YR;O;SiPj)Rn^HA3V1i%Nng6Xlw?2ZKVyh9BWZDS5`CPWkI> zu7pD)HV7L?@#&(SFJ)u5rCDtfO-|};g^zIdH)@YWw_(h7v!gXvDYnQ*IhEAc#pg^o zF1v~R!->2#od6U(Eqv8`8t(2A=Uf(ayO!eJy7uDx2IN7)*Cg2s zO;2-sudca^qF-rkMiuZdN6;hFY`h2uUL^4^4VUg59(?vqvLn*8y_>XLtg7NtqI{xY z^NOW13cyK^>PD9PzE&7;89o9bbL3+bg&U|mAp97SVHl-N+I+49c?lL2VmS2x~ zv(Yo0?iFd~fgZi!*|}%sS*)ridThUIyyfj^8iPHfck=e#Qw_`XzA@%&HFaFzL8Re-rt=q$d0GXd;&({is_;PP+6M9VH#3juA)__e)hM-O3Rh}zH^udLNJsC z7~|B6;$LSBQ!3>8wKE*=&6dBKa+6T=UHzoB5f=x4+ivZhw_=|D(qhc(M#RnzU37|$AZJm!iz z)F!6zE*S)jj8gw%K53JC7LnWMIlJc~{fpwuDu~~jR9wIAf$JO5516|Zkj;VAUHo*+ zS}u$lj?Feg_pWx521b8#HyWh7$G`w?Tz%qk-s64y{m#et(HxQHZeAr@XJ%xv39da~ z4&i+OztKCp8an`mUm%V2>#eJ&*Jd-@` z?K+gZ)9^HNL(r=e!$`kj>VA%WzYt=3JFMCJr6nTfYK2C%(ewRy{C1S&Y*-GmQ4Vg5|dC47bte5}tcdO4MZr>hSO3$!RE7RTv&TwA`YVwSZ@MP&n53HPOU$Wj& z_0WAsMZLO3Y-0)3mOTZ|jszuO{3_+uK`Af>3Jg7$0?;}ONW-6}JNlKHcPkUUR9va{ z>mh!E>TjFLI5q}|1Gc3eH}4Xjcw#Ndl4B#~UC#HI4;t|wT9ne%%y5`hC1%h)HhL>g z)n&m3dVlG0_=A9VmDce(uQJC6-FB09>;SjvN0eCzcbb%aL?i8X`ZtN)OPmyl^E;!; zC@C=kmvEH^9syXW5=uarO>>mn@ud6R6JA`=nJzKubQ+zmZkC{3Mcg8wBYlL>GHXxJ z6|w?4%3nOM7gg#q&S1A~2c_yP^g*$DD8?raxcB>kOGXc&f%oj~JMkYr?2FplU#gja zKV_bmcj`k{fE??+n=2(gKzLLRht4~ci`@Be>7^vmJj2z22Sk3Xr6p>dO#;1th}W&8ojc19$d{%3PLAGnf_E2ovl(l644 zgOXmjz?O^Tc?ZAhOwa>9fGgro;1KKXo8Q~u=MsWFRu$3AsIu27MP@cn`y4E==!w@G zn{T3{d0lYFm?50)sUKj;y&<16lruSCy~e4aN_Aiq4XJQ`|kRQNNvYpPt$_? zgVC*Bnc7d#`$1!v$8&+v9#6jOEN{M=q97RMovPEnXNu|Y`)ch6kxiGo4)fzEJM7LGLk)s{4LI=RQMLN~`=CXE2>hw-zF#Ec zeH=~WZCTzIig&3~U%)%E;u+W6du^D!GWjm?=XlfT?XNH03YSRAH5Q{3_owj`6vF5) zF}^+gz}BZ@t6eRBPN#JkLX_@+R`Ouu(!eJ+#I9FqI>NK#jJKz;@4hDz^d87b(2wXE z#6OKIP&#*f?GNs*CbyC-DnUF>)8d>MACs{2Pk6)Ow20JYCm4ix4J)$>)Xv~S{Oz3)RHMhcL;6{&T7L^jzfyB!Q{naOGND~$q!YUXA_cn63)c!Lip&OF5q4@|%v6gmGI+*#K_}O7 z=i_kvf`@-~-R^hW6#JN}zF9qtyS}7l95<7&f;A1`qM z4AU=Sc~T3phhCI%;g|0!aZDzooxlj~k4w?7SjTInWiTvyj3P zLLS(4W_y<1-R>N|9CnNn_c9T3amjVC>W~UNkW!|JS*GjlZgzVZ@1TtU9AfPB1JY?r z66T42sCVQ9uTFnbCb~8T2s1Uxr^ROniKuFQpR;wdSzbk47{Pf~{mU!%DR& zD;DLC52}$9TGgt4wbprqzLhk|Gumc@|9e>e&?=FW9CVowwuycf`vYEjFJ3+fp7we) z(xYW>7E9<#`f2n5hnqMAA(4kjLr<>fR@xfRx(4*M=)fPMgkH-X#^YOk%Yof*7W6M$ zZYJ+QgGLQV$bde1#hvaPSDhnm4kW?JBGui$#2Kf)xi6AUACEQL?>%nR=92>Fr5k76 zNl9Wx`)-ibqyG30OuN&)BPV#fpPG1bwVgQv=y)!-e;uu1t@Yj^Ly-xtj+DwRtng?~ znD?O_P(`;coN8^i%n>(VQI-w8>8h9y&tt81oS_?*v%WJj^mKJXT^C0(Iy%@T%@xBy z+hg?k4f!$l94@D9hms~u`to($Lqw;|qRhZx*P*cItQ@S}40Iz6K8XkHDczZuEpZHd zczTVb)nU?JwxJ$~JC2WLx{@kv%f>I;^SFC2@Y1&#Z$w)=(A4wUQR!~*5rR0X{o10D zW_v>cC%TP0^Dv^dS40js_$v|BU-nJ%&4y8_t>k(%qH!J*>q1*irgE6@oeuH?Lt!?( z5p}9j3-tO%MiqC%%CDMijW7xDwIILZfm6&%qt9!mEdioYhy|=GM$Tj!=4}ujW-BG- zu_qIR20$vRjZ|5gNqwq5CEU1ga{M9;M0BeK{*Wg2D6YupjEJtw@sr3Xp$(o3x}-~c zpLYwY_OL?;0*|So*ykbpXy`nebZe}%kxf6}>!HJ5Cd@#>kq*V4KJVD=+y1Z;w{Me~ zhmMGI`zG!tj=MR%7X9E_?q=lKG^q;_!@2O~PtqazB3)7ztm2G2gTF4OeA=i(( z!8)TeKj4}we(5kbsuPDS&7c;ep>&h-H1p8^H*npc6wHtDMz;s{mlF?Z?fHg}1J>}4 ze)Dfrn-8OV#rDNOZ1#CsG;hbksd0p_rC~H!vAeFRJhn?HS)0H@x*9DOT0X&Rdwwzo zMe~cYSqz8+;hnTmbornKpNKe+Jdm#JHlh~pqZ~}S?=z%#V27Gt-*K+H^mZF?JKdE)n#GU}2r=Tjlr4KUkvt$%&+70AH*f3b%_*p$2=>-Q9}! zO;5ls!*t?nh%UpHrdhVaQR5r^&|P!$zzBi3|7OabT9NHos0I}`u;R0oN_tWVm&7eU%&s{ zZgF$!*ny+X`S@oelQzRQh`rMA@+=Pz^(HEm)P|4tJ5&=8&O%u)bmIW#+Jy+_n!jsP zi{ay?pL~O9!ES z9Tz7F+8=_{yH+TPA~=gbDiAM5Pe(Vvd(~k?H`Z*kcKG-}B>k$7cREks!l05<#KzfZ zZKef1Ss(lBhyrlNszl96SPsO8^B}M!uvScbzi8g#K;eA?Q$Iz)YEWEat`y_B%YQjy zrs|dVyG_;e^$3c;X#4!?M29F7en+r}53LqXTL&6w#yfM4RUN*PQu{1_XwWZZ@L9?C zEN_NYOAJS-mjtsisU_d3UG!)<%o^`c+$`4Xj2dL~GsAVJpOSY?Bz)b99emGr-uF8o zP59a)OYl)f)G9%1f%Z>;QqY7IuOc`44_0G>ZUISKQ4X$*fi_z($)H(z_u2^vu0e!1d%l*!8% z#EJ_j9H<5sSfr);K>TWVy`B$+Sr1##Db@2pif}J_97}mQ81Z*8d9N}3m`5ypMfcI@G%PmsWW0I1H=hW-dt1IEu^b>e zj8xw(>;r{1D%>cnLCnzY0nTLC(GvIUme*Jhac75cw@rFU6Q2E4Kf{SIBtCxYqQxY) zNp~HJlEQyv%o`c!W;f^{_lV1kAp2{ImDqYRlN(i0#|gF2Ytb0bodCwH{hxex8vO-M z5j`DIMhXU_Wasx!g;7uR!3V5AoO8_Px!Oc{PmC&zu#Vj%P@_O}uDC(qD@x z1Rq0&;~b+~Js?aphB%S#$qP1L_R(|EA|jLk@(%H0K9dffN3OOW-)@_Ig8Cg*1v<6N zFnybl?R=gH#|$-hD(a`jWF~aI>y}g>FVt4v7S+s2b6MN~I5KNjJ#HHrr7ptlMdp6p z3I*U^U4Yo2TVsBP2WMxwrGys4+l1w}O0AlMa_@6)q0+P6X@OcS1)R?u{;5lX8%ez& zo<egDUpL8NmYIN=S z@O_pclCVQ8BTYK{#FRG)t#=C2Cl12MY!YOxmcr7e$_)%2Iiqp1+*(ZeJi9+Wd*E`d zK1ucd`8fK@e}nhVEUpm=2A}S(=@K0H!LjfG?V?<2{qs=&4sDs+qg70<5A%&Hn)?-Jts9l`@-|?u?ZF?x zE*^%s^>*869}@B8hCrf4;HatE7g24(SqVS$;wh>!>Qrhv4B$d#aDSyYJkJd=$xOg% zNmR!%C4@6ygL_r|)!lg3hz|OVdm0L4I`jW(B|g7-iKhU-@{Gr{k}X>}mB90&6qpQ+7S z6FUmu@3du(-vUK?Uo$yz@41$T1;hfuUT2+xnO~P{mlnEohjG@BMD5OE zRHH`UD*7_wh6-QB!Ez5i98|my#%heYsuoTy_+=N zo}bSvU^Tr(Y2`#s41AL=UEpilV-SkX0<8PeoeUdyJ zJ)OB7R`%TM37kHXUsYdIzw9O)UGw)>O>4RE9w z+tCzN>i*{pLV@XVM*Cbc(&Q&d+*(N4{wN0io>-X7-~R?DytnmNR}i1JYXQ%ZX@5ec zbv9Obnv`{eFq1(*P+6{R+$0v035Fj7TK}~3(QuvpDF1@eNL$MV?bQ$`I1Uj~4`m%# z(-eeyD@u3^Nd8kkRdBUrxSz(1%WXv8ZGy*9q|FCrH$ClodUt3bmCKrtA>`M@Y@QCl zf*9r<;I2E`+d-=eX^eiTO0Jj7?C>C;<*@uR6w{9PL*$g#_NJZ**jAn(&Y|1avBS^2 zefYPDenlfkwVcv2)kF-rC<^?Zy&h60W}8dG!-jSCcJYI36=*xrgSY zsp3c`C(TtA^R_;o(Ob_*U##yKa*m zP(R?TH=uyyFwq?~i}@@BgWcaCoOJ(zx6*S%IqBj6p7LrJCgh2@ybFna(P=_akuUl4 zZKCmocZfF+%b&?Bfl9ZPWh}Y`q$DlY-25=1MqK`}p7zP>>aXtg2YhsFKjUQUE#7i957b@c==>D7(QT~ z)6U9PA3x<(0ia6AEr-wjWKH3Iy}S7rf^4a!%e8%T3_tSzi9qM94rOgKA2D7Y7tY1b zP)GK*OCS2pYv`$)Fv+-qVY5{JJT*SK8;EF`Vjli^1H52~<1mevLp;JmnmSxxtYV47 zkY0_FqNVLyp*j+*+J%2vp$Ez{{CVFQKi6a~AOAX93YX{IofSU~F@ix~ceXHR&vCn- z%pA`%=wnnO(x`)r%34RF7zH_kKNpa%kLK&CRhsojA-tm(x`4_CFj&rnFON5g7_} z51J)bDt{7~IFU{x%MEJl0U->LSBKkJWpBFuv>eB#P1+zysp?DR{`i^uP<5+~tN-Oq zk4FcL1?D^UwYUqtbsNS_hT?D*w~8f@wMD5gWCI zwpSnaO;4a*BG`2@!h~3QOJ!QTMbf=8h9f^E?qZamI6=b}@&4-2=rMV+j&m|W%aw|W znYq}VAP&kkNV!^PrR{}3FiF9ESfKEb70>;k(^?oXH-huC*>=2Gt5bAZWl#RmtXlvF zVz05>Mr}cap(S_<1>FB#)lG(jPvh@RFT9l@)Vz*o*e5;OMx}k2f)% z{0$u*BJX)C$_Kxim#pv9(Q|FOMReg;VXuYah|9*0+}Q7du5rJ9mg!?_` zRZmH|;O7Alvt0DK#1hgI`546WcbyP64e`}l#s=1_pmlxDhy=-$^wf)91M-AvxMGEq z%s)geqQ!5*Py*8)kQ>fk0+d@+YNC3UaICtmwz{l?jJJ2Ytw>Hx-aMvypWf5P$$`vzCDw2f<>RGdbTs*GufMN&;E z1f)0s{(3@e_4E6|Cl1+PONOtM&tD+%6|D3&!dH!kZ_tZ!pH#py;g&FwlDEfuZ{k2jtxJVwbZr^A3lhl zvW@#R2FnZDyu@Yg4H;UL2^mcW(sX|-KmbP3E8Y|G{fK?IJgcwps-q@$kTOpQQ9r@+ zS46WB#e{*}h|Z>182sOj|HlA0OfV3771OKV)xIl6^l{F0*z=_)OUHzHxD?w=}EX{cE4Bon5=AI%HEVFLT~&RGg5%GqFhC1d=Selg%|8f~XHv*^mg_LvD&;4-QzU73Yn927K6M$% zTb&V`Dg!M~E|b047sd;Y&DL+G+TFV)tOo0TzyMy-6eG(Ws2Y#&k-y^{IUu%J;crDB zv9c7=IA}{Lnj1-KRKXQ1mv$CWSRs|1Qj5#|_Cl-mIG5RKtsAzeMN)68B0rw~TK)bX zd-=aNnGXvyll>IO5guVgA>VWjouiVlUM=ZhRc9d!Zku%z;pV>H#816#acDml^sXf& z5O^q@-7+gJ1sFQCdj(H(BA&#La#z+p4TRKL!dYUlGlFy*JTK6EjGo`7u1ps`lt1|J z55wzfW&mH-?>$zpN+$#5fg|&yXiRBrz3krzY>V5Zr~;0-+3O2{;Rg{|Ly1gTK(89#_tk^P5!_LQ#7zEN*9(LDegJm zvX5x`+mpt53`?&@!AhA#fPVw}GFF+5*dMZTm=_nc-_ly5&s8enr$aam#c6C$WaRzN zIXVJR=zI1sg0ADqR5qjMH;E5k0*RwY(Z|)56#dWI3^2SS zc3Wb#qgt@xu~A9=#b?fE+U={kX%e**gdiwZMyS>F0};Hd=GvuPlvnIyV`YVV?CqwN ze%oH~DEx~jkrc`ME<+*P3Bm}=#h$<9z-yUYUCO#K9|WXK4wME329nCwXwVhqY~~Wt5fstZNa7-mKW^}`JV9=x5sETx=F#v99n&o794C#H&m?NA;|dR|0{g_$C(&6 zBCs&2mF*K1kT>l51$;63GA|9Izu}?F>Y1QW`!h+f7L5zpcpS6u4t}gKrHc2CJ;>ba zk-fd94u+Y?Zh_}kw6{g_`tAdjG+49$)X0{w^67>5y$079VBY0|$DnrddWRSW=pj~l z<3u-^0bv84!>7Kr=f}CtQ1;%T^Uhd4sRqoCd{q#;)~Vy%zkkHf2q7~NR*{trggxuE z@*28AESvu80aL|b#oUV?Fu*$wWagmcbHQ}rVHZYz$z+sAB?Fl>-s0FT`Wk0f<}PX! z1A98R+U_z9#D)f9Rr>MMrzA%n<<2yQm0HP3c;3-Z%On;!C88H0%M5J9zeZzxYof|} zJpl_&EoWe*%>|l~%Y2zep>E|{zXW|5V*j(*%dUB&^gzeb#cYoz{jGBMH|+St#}vzZ z$KL-&jQ{fhSApK!I|r!Z#RcEdWh*^;9c`ptjWWYHTs|f1KRjXiTffu@{x&;)|MUiY zT>OHx^u$dUU!ITTB2d}VFz2IfC*iH1FXSp?LqiI^Q@9||T{6oOsa&SWbt^}H2jpI? z#=>C(E)u7`o>_m{_x@|cbMH)mm}4Dr#%+%bS7dp>;mQ2mKTn6{HHe8fc)UE_^b9!f zj-&;mQ&16wenlmM50M$z+1UPAXBt{u#)W6gCg#Fj^GO!6;cOlGz39;rc%NYW*RP#iQGmyW< zM}L7J0<4%n7?%uDOGPEv<9ys`r$yf+*zlPIoC%aOnU)6g{`g4@)CVP>thTM9mr54L zGdbM4JBEBjYKkol2R5ZM^PB)@U5@bV^i&JrSr#Ihcw|U;S9sX#D@OhOVRQ5HK8rb@ z2nkHh`fJxol#6g}>1C0y*bU*f?G>!!D3g{w#fv7PU%aJoZ$f)yG0EQ9-sb%Uoo&6q zKnik1o(dUykifa)A1RV5g7zO5MLCS}SFBakC_>Hux{&`tz`t+UnwsS2DTz#Xj_FPR z|Na+&{#LDWGQQX69sF;@;m=siQu0;wK0s;*6M`iO^dLXO)3k58^t?kvts=u=M%8$w znbY<9$OPesT4Suc)xmd=Y0uK7%GYa4wepI(NLUv3(uT-l=ikNI+8r?CQCn z*Jhyp|DyA&o`?QcqB_GDe)56-F1L&5liiH4jY zz((@J*v|S)kj7KSM%4ZoXMr`%)KST9@$u#@GaankEaa5gG&@84j8n9EtHjZAsM^=O zCUCj;<+-1AYGx#!W?tJD*~=3LEdS}R@?Y}7vM-BRyvqH$2cr-YZ ztg!#Q5f)tXN@By+DSA|AQm8(RixT6_GG3y}ye#5q+GD6y8|#!n{1({)nlX2(uXh=p zPg`zKAk}k9*=Zc;Mz6f+Sq_x#;&A5GG?b<~)fe~9ez9_F8!MLelE`bgSpKGqT7KOf zf!grg0Z)u;n_*Crzar+FPVCw$yby@?GZ=Z4(Bmf-XmLRw>}Aavs0V-`^^bhr5Z#@b zgW|-l&E%7;v=!Zam%ze&3Q)YX$LD#wdC}V4sVL1~LQCW1$-EpW*-Na;wa#zEz-DO< zgZiL56T7*le&M!6135kGFH@qaEb*_?`McqJlj_baQYw22_~Phe3QtAHZ>)5KpSr7cr$moXq;MX?!ob z@?GtJ+@(*Th__P=*?Fdu**-=bgz$R?RJ=MAyR*fI>qE=EW#&l90-QkRo*QinC)OaT zEWgJcHV?;-l3d(_v6J< zDD?6~^LDnn>X$K(Sq1gYUS+4h8R;u?EeWo=_W@z%iLZ%>2ZDLXT#C0WrL5&B*ulTd z1;ZTS)J$T6uN~QO~#_TB{XcQRr@<#;uENEYipvI+5kHoNEz^hS50uXA1EO&ejFZMVBrQyX&8o11K?uvU9psNadtvU~qq~_X;h(8QgltzEj zOu0Ue?!)0}9PLV8^!w3U+Z7$s*4?gtqj%HMl{bMiOF(4-CKS*VKZ#>}AMtTH@KLQt zdsEDhtMV^Nomuf^Omast!@M+EaiESk@~2hd5Gyz;mXk#(@mht^qR!n-k2BdZ^&g*< zc?OWBmvCzZIf%_PK5Xe&sdi)h81RtD)=0qIhU8_HeTNuZx$lZ!VT7?p>*c5W1SRl= zBg^Y8D`)8i6i{|M5kC=F?#3SlD(0&=9U{j{j$o(WPT9}~t60B6fIM4LoN4t&wrK1e$n8D|Pvi~&9 zzM5#JrdbjQx-Fu?1(y%O#})v^@R}i`N3T8Xgqe!mrSto4+|0}hD}ABGGtY7FR+j-A z&_x>gzJZ7|9ApH=%lx5{~6z0K!-z?SX^(=6ld@ zI|&I1@bWtXlsHX^+w4Vv6Qp$jbSDJfj7y{fZS}=4iI(qpqk48MfCX$~cqPbBP6Vbwrp*6wsxe!ryJWw@wP5 zz3JxP7$VD$5dCviQ$En+Jld)IM1X*jPEg}aCl{&T(^B%|LN}YByM2|Zh|tw|6WPMy zyY5&KQMf=2X51Fvr~~f~T*f0&+%X#jTZ596wgn0yw`8NK9P3H{@##euAWXllxvN`E zzoDdRdRk*_L|8!@SnX^i>%y1%#WT)`+e#L%C;572HZG6n zVaw$^jpMzZvJ)ehR*$%=0wkr5IB%LdO}`Kk_dPe;N_|RXYFM@Tw6)X>!)%(rNrY06i2^2wm%a9Pl0pM}2`OUYKGNGz_ zGyM;SA>fATEDZBwY8XOjqNkse1Dbi=>#izl=qy19sj-!#bjQ;dN?4bOdk zS}npMmqcGCV&D`iV`{5xDac^nQ*PFk@>tG2bsbrATJ~W}*j2E389cEqEssi6<79r- z;BJGF`l$0jQ8t)3<*2q768mS9vC5mk23x^d?C~tCvF*zCDn+f!nfJ5B#J|8Q`2&pj zNV>~TQs9^4@f=as>}oa^vDPx}HloLGI}kc`lyzyQ#P5hRlQR6v=wZ7Khk<8OtJvXS(=X!nye~WTCE@*c_Bi(fVe>nJCF^!NK zV$_~_ua$(>OL@sCgU@xMnq>sdI%yOzV?L8&pU}CBU9t+M%;8GZE@J%`W0DAX!@6bN z22){xpDsFe^*WwMedP(_tkA}@d21_SJfIeiMH~Xn1r4``H^Mm%8p({+;}5Z7r*BC< zM)PX3}kK9L4U+PT8~4o*J;NKGoVES zj509F4?i=2R;nT-N&1Y-ek|RYiRkZiMKnTi5bb8DA8MgYno0ic>d=Tn|I3 z9uzAziD8@_{v=j1v|@C4zN+~6;BQ8R1`RTC+Z;(zEPKyF)K=U7$s-g5yzNSuM@6!P z!+8?p4o$f;ngQfOV(umS87k zSfE;PCG#AyBmFZO|&wFC3`Dq;_DIw2_DOFiO=ZVPtG zCXT36*`**(dL6Edlk1U;yk_9grBd9cTEeXQw086o#bf!K6_t! zsu^Evqw%>lZ8EJCQ{`7Bjj78QIWG;*GnYLuCUI`B;s35c)(Q4Nz_ZBF0xNWgOiltE zhka2!Lhd7p1FUw>cl!*kM*;dLy>jRofH+Ve^?vzNYN(Ajfr(*Bt;5+qS+1%GZ9RtI z28bYpjy=zWt_+rKo2qN`yL(9K!Ss9j3pMNy$_IJVJc!r4Zr6$I*eZEjd=3@bmb)2V zt#X214{=#1vb%oQ-L()NWfpVuf4om5(c84qtNUm$*mtMrX^}6Le%5>P!zqqHF6L@n z2>$PLV|S$(|IzNwr|FhqIciIN@$L9$Se;1*$xvYlfY1b%?zZ62@K{B|=&ijK)hw1o zE%DP?1>HYi{7p~58_K2G*+=$nb)Y8q_aj0r&H#R;K<@6R{**FX-g%e$k8%Qy87n0y zA)*80#h2dAa*968vb3{kHg}0%R$*g;T8|YJ&}ppH&sqbm@+T;ZYagO=ItDP0Mg0~) zi{xmNmGCoP8)R@s5f8g{dk(Oz72I@;*q93 zC}yW2|FA5f2!w_(4mO`_VWw<{GUy%Wusu@t>IrA3GqRuB=(I$}A{)<43wv2SExP`T z0D=kLjQzX05m7N(W0G=MY6#lRsP)26Zhr2 z+8BzR4Cw<5Vq%(^exVAJz;+w7b?Bq`?1kz2Kly zYfjEPkoenu$2!-u>LYvxF7||hyP^e~(SI{_pG`k+%qrK5jO1j~lFE4Cb);{lJz);< z;^j=*fyqWkgb?Dp(y4tdVa-Z(eDJYGE|XNk>o{ZYE%r&%x1&)J)bj$o)grq=l8OwA zRT*OM9@g^oxgYZAdr%07=@vNQYEq5TX51de8GjF@To24u|C`=Up}tL%xocP4HP&KC z+-3Eh#T*$OM#IUL+>o|Sqr=MHlHDhY8~D*G;a-OmX5i)tA_#>}JPrF|Qwp$Li!}=D zJcNqdFi<2!*s1l^h|JxLfI-pqEw_%Z!*SJ+i!o_zm^9z1RJK^ckBY1RrB!@3{og2` zbeA>#)~u|}kGe|xC6GRN^)m>4|2?KvTvQU1HucU?v%0aOY9I$-iW`DMqD3^J<(T^s zF@AebMb(!h>C{Ck*o2%OxCvF`<-(>KR`CB31$QbcR(5g!IyUyR#;b2jaEoqi*A2OD z%ygI-G}OEH8Wm>XExI6X$EtJcz9qoBJ&$p=y+X6O@%~xIxKt=tzrRZgb8NaaYWwkM zCBU~5)_(5`g@rqktNw?ti+dpE+!I0CyDLgbEnuW6HeEYB8mb__*^CxxcI|(o{sGS0 zmi~MttA)idHGMNxN)m{JI7hLDLPUrkctaA4AFexy!;PKJXvmgqOR;x+qrjtJYg{VQGy2l)MxjBc{H1WY1FGIN5$T@)x&ohIjLc75f}FGKm=gN6Oi6=DdHfD5 z-;FhZuw1)O9=5z|DblM6!6{-P$WX;CKuB2VfE>x;J*jTL^n1Rf8@(LrZ8xG{#`Ma8 zY`P8fr@ymX`k?)=8R&w7fRJreYAvG;AbCew*-+w)8^XvF>Zdrc1?7{{sf|cxbTJrM zH+f5sTc?l3wtET7+bD_8frN9U7V*}k!;kt~Qk|xC4TW~|olEr*r@Xj`j>0_i9)>*} z*{`e4+?SD#rZA>xKaG+XbK?7ebH&YZli2@JI|VYZ&AAQi!+iEO!h@!2GaeVi>^-8~ z%UzT%lAN`hFyWB-rFs=J@$dZ@n*O#g4f`;P#6P~48$coTj5D@9QysM>3cN+i4SGSufCf2GGC1(Vf?#wZFZ2*o2<*A~hv<oo2hU<#U$NoMZYky5okrgm{fbMA-jp2qwIyzdc2WO zH~6QJ{`KR(Z6R2Xko9YQvE%S4JjEKkl&8AY(#70%8{95c3UT__-~0QQyE@`FwH!b_ zu4Gibe+!?6OCBoED@|0J7mt%uI`q;-E#B?>gs?DpM6bZQK?D60RDphA4}jp(+yKTI zjqz4Hm-PM+`P_WI|CVrr+I;cOQN>TcKO4e%QczqUK`>^DVyVpYUEJLUpNh}#B;c->pwY<6VrD;`MY6`;SjG^u0j zkTxm`Tg3b~p8Q6DCC$t&02g#zv2M?cECzuuR^}!n5J5LxEjL$$;%-^8n3!Cl{nb%iz#Ag9wX0(?VnCCsEZwDr4 z(pBR3;S8TP$VCRix}jhw>LiIYzAM(`E;kikr0VlveQKf!X|F%52f?YKRML~)KXKyo zJD5h8S2+y!Ufc*Qq1W?iNMa*c(kXl}F~K%|(|zy~U{k9_Zg<%sKP#WM+6ZpA{$|qP z4*P(B6t=xy2>rF8(4xUr9LKUdV%RAe`z;Dfmsg8VHyKE;i(7{kcIBK@!rccX=(;!V z&7@Gmt{8cM!94iPy={}1KKpT}fzz&@Sz96j^)v1GO6H$*f`M4qh@Qi$^q%&l zhHoRdLwGKqhzO1x0CwoB?hWtK)CSPVhobc`xz`BWAg3Ycg^z zC&tFU*{$unPv`8VUCnZWYwjctv?IqP7XOyF|21Snj4wI^DK<4M<;aL?Dk{bZ%4Di7 z77cwHs86rp&JtT=)#D~@-w)dXr4cON78OzFQNXfj4H({ZS}@7S8+1HSSp8%Ew{mi_ zwgbsyS$c7UWXTQweR1K>M#W6b@ zm1+q$>7=Zjnzye2Ude5fr1VCw_rISM1IV-k?n=T9U);C5BwjR*^nSj$?`%k`D0}Cu zx&D};K~4;X*s3+h&QX-6U1*@cAJ4W#orI#VVR zo46g5qTy}_Y}B-~5xZkth6Me$c*L3n{p-v&(|aNaS~wV&A-VyA#&(m?r>43AV{;~q zeD$=^8BnI_h*YITdw0CC@Jn(bMq~Fe?Tejmb-g3#WAUX*)`oh&pDZ7N6t;vzM%)#T zyybw`$L(7+8H|LV_88<&D26NHIfpn&PIob8o@-oMwBsJO$6{OfqV!oK4{ulWzRoqSv{f;;S)FRKeC?`@2H88 zIW#BNRs4&FRA4{8T~UcOHj*6Fr(d#HNJ`cvvnv#|+G{zVlE6FAFM-&{ZTEEJT0J({ z0D4QQ;5KrQ75JLhHsVz(!f`9L_$`Zi!?xo9>%*?RaOPb+A(%5J=>{=yylmzYPJBR} zdT+yY4<_Rm9xE2&A~cBHOp|uj$fq12lsFy#Xx5Ah>vamT+`7`zL+oM&pjl`x86;B% zy;_a}AJL;;F1k&`COUcC{C;w#seN{Uv-AqjIS!*xe#RDEY*Ega`irlmu->MbaR!;L zr4E^ekIaTs^|~Zq@aPST3V%xBDaBV^(a^w@U1@*}ZVCa6_9EA!qj?|esX^(O0C!5# zDIW2FFQM3jH!8^u$ZuC^#b=Pf*qb20k#ItIm{N+cz#{?1&ZD&|%YbmcjK)W3Um`UW zW~G-8s{D16q#`>1a?c&0zu=35Ss@@yv_HcC$C9-MF!` zyET3;0~K1^s?vBnc+6itNPNi_WMB_a$MQEJyrA?)f5bE1KkfJz6?>umB+-7VUak0p zmJx*~0_2i*`qhY2xmR7U;l5wW$UpWBD8ejsx#L^spC$**OCi4B#;sXd%6XLr$Umx_Ydv%Fsum@IF5Zd65} zG5lPWt`bX`@3~P9bOAo(_2$2-ay`Tkuf0hqHZrS6;!xSuljWueuImmy*1b7ZX{P5` z8NM@q*)uR7H8J(h?JF~7G7|Q;l}RPUX?axkxCo-i%TPQsok|m*lB9>&m~Ey7Pu7ZS zLB_7&U~DA(pA7gb43~^)i5MQVjsIj2{Zpbm&zkd0>~q4YEbWu9-{b@sMt}oF{Y}gb&Y1Td$7y~o4^;mFz3c^=mL}= zT6WMBdjiY0T5vZ{|1Io#5H1Chjxh2a)bRY9>IO%@nEqe(Ys?=b(}9y1VFDF-b&2ZZ zg&pQr0I-jYkG4!5h0&6B(!hgWWn9gp&1PSos&!(!Bd5=UwETy>asxt21EeZvi2F>} z;be%L2@TZvy0z#*sLY^{$2E>auJ0U}bS51uMI`eQ_PIjn z0TH%sr7?08%4}=PuMpN*?HRiz%2F8m19)7-G8SGRjmNu*IKQK}dIIyY-!~8dD5R4I zn{t+JxCygp9S{7sEPjjyeGK1p1$S~AzBB?E+7g%$K8=5ojGo2Q@7QwpfozbEMK<3O zINcB|+*5~*iq+R-!`YVsA1I!6fMFof>R2*IH_PDrPv)`)AOU6>`r!r`>5 z)r4%+Gs$pl6JMHtkZSNEOm8~C1E}}&Woq=I#qUK;nIk91?<3%a}Qq}-ow zq#<|nc!jwKHXd7bhK7`L`9(haeOF!7Po}Ed?;ZQ{{zIw}qK_R0V3MuerI=`O<0?c} zXr-{un7vuF^PJ@n*>T>X@gLJ^3PlPk%WKwOmO`MYs07+L$hyPCe#3DCVTI}8ZTjXV zO;u>1aFAJ&>&NrFiFRiOqz}c5B0Py#gj1x+{+(>qK3?tAA--j@yh5;@+pW8qw6{&3 zjMM??^6Bk>ja?X3cPgM4mzy(W*?&jX=&!l0eQDR`e7B2^#T|2Z#a!4rc=wQn&~F20 zVRVp$@%G@0>PUMNRvw3PQept6$}msAeJvdr6l5VDz@eO-W_A8CBn)RnF9^DkYEv;J zV{Rv=b($sY8j~v_R0gdPD$hI%jI&8)TJCdBgd z)i?H;NJ$lGCsfmEE$-UCLN4NmY{uQcQpO@oqNL_08JX{VV>H(tx?`rOvN`=Gj#?H9DJ_#-XxJ)hT5Mj&EFcH&R6A!fbwk5lv2_UlJSi~RObeL zQ=FMgA$}GM)r?{4^wxMkA~|CH^9>r*a4;udr@kb0SI|d0ye;9Xvs^aqGu^2v7sJ@Z zU8u9NTH(l4Zsuvj$BztAFUZ`*7-hyh(sn-!z?eJ>Fx{lfNwusV*mdcuBWkqY z*M-)%VFc^}Q|-our-f)1wj`1St8WO$&~Oawi^=tn|FOFE!eH-_uFr>j2M(gURuL{w zKgZ01wGuAXwlv|L%URV%5-nIq&0x0Gkk6o%(i(1Jt<^tP++#flKn2L(9e_oBTZ3XH zRbF?4S!2sE7w4$o0E5}qvKj+pB&`KgMjN_WrPN!WNzy~f;2gsgdA2|F_JFkb4yauQ z5`xQ}AUI*asFeAB$%{S#CxasZ6Cl%&UzFv2$NSt`=b%amBj09#Q?{`C1#YiE`pBtR z4+R3U2nn6seO9E(UDvsD47*V47D%BMG#*yiQmzEVoUL7 z1DW?+YST~p+bA0#*fgN7t25IYc>we{R(s-B_w}lJO$mCl zw=C^Fq~IgU(G?G_*!n0bjJ+qoG1XWiBPahn5eGQ}fk5QtN}pKwIpYuGs=qu}a{!1m z(3yXSUjgSGY=&$h80rI+5o+abA(%u8)5B~Is8gdhviwn>9M0;`RBUw7uzZWf?D*25gnbKp+3 zaYVDX#9B#k6UcV|-D*!U({lEqm3yt)Z1u`q|1lFEoVrUDV}IHgchXBiY!1yUzp1yt z94}OcySG0zwtOQ8P^H=xU%x5nG>+Ntyv+EzG6VTYFb0wHk;gIAI$B2pUU^v3nJR0&ZH;E7O@hn|;- zKqw)I6QYdNA>4ta)-3T#9^oLmgEy-wEkrVUM9R@kD`jlxE3|~NHCf_rbssC^!kA&3 zZ(p->dC_o{0d6xtkZLJJeYEM8dV5GXXd7hB zj<&UBWTrdbS1+k$q@zKLy)&grLa&93bM`&`G4sw5#cQ(LK7b4fZIJk0hyffS85X+W|awXE!ru#4EPUJ zrGD2tR@PIPM|gg%&%q!7atr;BDPbXlwO{Y<#=|jfE}X{S60=^5u#`HVk}=b#zy4bv zBc`JUtO{>!GwGIwNw`9BRUibd=7Uji8pK0iTTf@A=Ye%0#Ek1At4H={g4*}uYQNL5 zk4?{po9Po>%%|wkD<;kxH$>71BwSq^D>1P^}*3*Yz`2F5Sfg#UtoTc+$F7;EDam`5-bUIwV{N;&td1) zR%CC*Hliv7cR3+z=?+|kz`)`C#`hj065q4qCbHA9>jL`{ znE22$4QQ#Vo%?_Dg1%$Axi`0;;|ks)-|ebROl0t0P5MH3g`4Q@=Mq?$iSrD29lp-M z!}#z##JB=3@$4kyu*3*jX!TbfYrH-qz4cuE_M9`3+6&BE<{32wArBv=p4ukMMZ&V_ zweoI^O88p&{JBw}N1TZcH{|sq-g*3#tpc9Vy6h9Vq1?G4?@emsJg$R2en@kLB=p^N z)QgSS)$)EZd4~Id1dwj&R1f3JX9#(0gWomZaM^P&NYaKFlf!cGCoW0Cs!BU4RAeXY znun_-UL=4vxW!AxHjP8*uNcdOT_mUyL_KWNxgfiU6M=0QF00+>TZrdEk*>PG(9)~i zgL^Wu_Z}V2;vl>3@}pj9VkM~f7s>vsY5&6<>ECM4(^ttOC^ZAs8-uAdAQ_5P{al2+k8Be7G!leJX%hceG} zg!SFk8@3xU8f(VfZg%_~IZFXIZ0BjzQ1-650#YTp?VKjZ``W*_DE&B>p>P6~v{Ueq zWd4D4xHH9hh^y)O19wAn%ODj9i-yc85_Z=mAcl{_3}&M@_OEAlQIK985@-)IZ4kf< zyAhWY+gRQMyr+QFHrvKj8%dhEwSaBJ0$)pRP2foMz@B(}^uE5I1THToM6)Adf7%BK zR#$;@9RmP;!sSQ&+u_LgF?873(fS-ulL|Q;e`38bk;g~e!&TOp(b*Mbc)>p-dq*g+ zZ{qav^Owd?(L;DVBZ+LuL zLN{cJ9EU8Ck452JtW!$mwmuw}l$VdFeTq_49i>7|F43fE$^$k()i2sX*YeV$RppUk zA31r+dM~1Pd0q(D-vL4RI8yS`LDEa`f_;Ny;&uKxc1~il?R4$?H$l4zfT8UI#>kje zw2&L;3Sd4>!W?PjAw&qoE!2p42u^qk#@a}0^9FjNBpvI=0WDI(v}St6la9b8yuW09 zA9IK9hMCffU0X$jD-<&&598jtA;3InP8J44pEN|VFRk8^^DAq@BG@~N*zmczmvdWCK4-rgUs@t5#9#$O8 zJ_)LIr99C-!XonkA1u)L{Nxgx*w};XpOi`-CI#!A7~vbR+lEvrpm*EB^pBY3WnW8u zvTlCfr3q$6y*a68MYAh=`L&tlxo>Mu=!VfaJ9N<$YOenerC)@_)?6lpd3m2%+2KMr zdkBBXI-EGm0@pKinCI#{SW`Mv+`6W6Md?$1)x+L=sX+cz5w4%5FGpV1i$d1V*w&kV@k(tN)cJdmWS05i?$Gr1>K*UV|rcjP&3wCZH2<(us#mVhgR% z3iB*{FhEPc0q}#2KE7FFt2=uR@VK9n>_LPPrbfze1n5VLzui=(w0!P-}X0m+jS zFvXIg@FY+RuaqcixbU`#{VocR{YqHxs*gKG_NkHTZ7=+nytJdX8Ai6QxJ2xUA%zxF zDnzVQ0+4<_1qGWkhi>MSfgGtiHbrJ^U^)HJ{?E8VR6Nn ztZa(fxk4U>IPkZqpkzv}g5aQ4B^O zAvpg-+vy>gS+(+%JJ^1AX`RTn6H)qhpEjvboQv!f}pF&7u%i<{ZR`F zquOdl`uN;%k3R%!f~0UYU!63{#L@`FlsF$4u68&i-u8~#=1k*-MqO&snPHsmsQ?gn*qG?$6vSUzjcivv}ar<#=yR4z9f-i)}Nh(w??ZutbSI z%45O6n_cz&E>K+#cDGw84MGt-8=?iwKLwE@K3TO9Z_XO&0i>3dpWYKo9YskRbo$wE zwbe}2c*DajSHadH^%Eh0>K|ub+Ih9nIs^&)RR+D>54ZdN+`P{nv>HA9&etdfCo+T^ zkX8lgN}An$Lox3UK=T=QnJBIZRxoFF!vHGvgndWnW( zDp$q?l0$BbRZ`&+dv1u%2gT^fD%1TO8eVY*1s{=h;EoeaYVCq39@a~ms)tmb#SKC< zXzq-&e$Xeyt#O-%4{B_rLAuDT1_Hj}NIZ}Ve4em@dm2_iqZ(lYTK!WWjU8*5AIT5jYbmD9B`UmI}Q$6mj@Cy3igvtWb zSP;9PU}~sH!gn0tuPMAJ3~MO#eMB|$OWNGjS=3^wVD11HG+Zh47oppR-@BgbdZS<@ z!D=i$@moYwq?B(Oz`k2NZZATf#2d?xcC{`i~B?(U(S?Hp%+3H(cPIzSMsWEovdFf?ALVO|%$#Z=dQpj~nWDA=@ ziLR-(WDBp2v@E(!&hv~8GD5VV66U^%F<8hxfzF61&Yy2df*o+@K6YUSjUqcd(B__0 z`05q)lhWNcbLm$o;jE<>$q*O40n@gH^xib41!=tjrUESrW!~~_Db`vcHwE5v$F~2& zYnUYM?}k;Sa~2ZptrU+}$?;x`p9kYAZunBQT69V1xx^Z5^c}z`Ahd`|Z^tw|2HOcYnFBS8~d!6}W~EuYh|Q#AGg`DPCE;@?{)u(6QB) zWJZ74X9BZ>kMZZVrka|!Uz3xb`bx^&&UGuB70qkYPPu?-Rng5YU{51rNV>=SwUe^b zZU2MeyVg0mFG6mV+Q+|cZ7+H)(b~!Q#e{Xf`>616LDEWsrNc!I9rwW`W^HdXywttJ zFP+CuDZ6G;-58BMw8PtWs8gLh(d>`sJ}j3t6SlQX6xMnDb6o9b-@Z7nppDG3!I4$_ zZFj~Wa`oW5O)f|e#yRJELKw?Ub{vo`^N(992ADoTW<7t19M&wU$X0m)Dyakqov@np zqLB->0_J`xn3bVI*J}WUQVp2Z0=M#Fz84ADV^gG3OAatp8ePAUItJ>Cnzxpp2~oDn zU%^=vV6Jd>yu@W7>C5KNHBXx*o{`3K0;`7KkLIfVHe%*S8&st992~8r0*{rcXs8_Y z7Z&0mW6o^dT&)%0R}|)(!Ff^4Yg#8)ocGT*vMF%ao9OrSJf579JA(f?Nd^NI*rTgMdgA z5s;1)QE4h7y>|&6LJ6TuFVZ`RD4=vgks3NmF9InP0qKNJfP}ywyjSmg@B7C%Bjb#~ z$T@rOwdR^@u3d_emNXbE)fg$f)zFu;-4C7nO&SmBYx!2Pq%Z&2NGTw{<|4Htm_9(T zoXI;9GQ0F7y+W^0QfBB=^!(fX9OpMbKV%V{0pCT0d)?+A)X98r>>>(+maP>%LgV`% z#jxXc_#e1)(*uVu2iz8q@)mB|B`u1KyD%t+_%{_rUE${ak7e8~&hq||67khn48DwW zE>k{H`zgN9+p5x5RSNxXDOC``Co~!z=`~&WS|&c#-fdXpDCE-TjBc%OPW-#rPv(pI zl&{AIghJx?Yesif1~nX~oHcqK=YM!GY>#JbvTed*E9@~LLoBzZG#6_Pc4M~Zb|t)E zCvH8qx4NeQSoK`S89x=$DGHe5z_>-fvzv8=9!rE4%?7Z`sKkv{| zW{g?&KU0Pu90AEark>65dQ$I6ykAJyHbX<2EV5RIy$xaI>gapp-i~S5R-FZrS6*Fu zT?mz*uq+zP9okH7>0;{#E*)u2v4`>i)Qd8gM~JTt`Gb0Zs|~LSAE|~n0EsaI)@r#f ztrens`bLsbl~L_06G*|_EU0{9ufWi#^ns$`=*zG{&|=bq0?wMfjV~5)rPT(iMQRX* zs$L0YXw2w>PZAV7AxC8|=>C#B;8@yq;E+>+n0o*z##d0^rXIRZS47Om-P8S;hCLq2 z7rRmY;@KM;I>SBWpoFXM6TFoGbBImKrO~GEcnf%8&eXH)U>WlrG~p#XL#Jw}YG1>kW`#xiDRQ7TH%q8Zy5kYKnl~tv=T!BgTnHhJM@~e9KH4G(RlkU`SEw)WHtd!c(FEnN(3tNes zo0Tkl*53c_p~^1z&Hq`wyQ-e@T&B65RPMPlX&HK0%7w2R>0QhbRWi4HK?-kNcFiD# z>7Ol4rAJ;~+g*BUxK{0;JsmDvkR5ai@#c92DwH_UQlSgfFFystX)^v^ZpckfI$LTz z+hfPjOfxb2({1X#bGd<2-6Fl~{FnRvHK-hI_1-4SbSlm=_oyb@)nTd8E9nym9?If= zBJ`}$KrD&r_eyjfOX72>WBdZu*O}@5%ObB|oPgzTXg)f@_oh9Yo8;NwC`9JbE`?L553nmv@xfIR<6&PJ0A?!Y?mB@Pf5mi=_JU zWhGUd%%ca~eUf$Pg0=$rY#Qq%!PrgW;B#zz7!NiFcwscy%>0+kH*^aSmU#OzH=P4C zZ=qF)LMr(=%lr>-SrW{>o@rb<%4?zV;#rrg!VJ~|NA)(No95p)#?0t%Bw43F#Bb3q znqu`AGG7J2bN3*4Y6rSZYMV=0pNkq_Q+UYOK%E%*1{=)^Z5Mh9c4@xA%vSJh#w6vWhOD1#{?z1pX zykg;V+~vMbVu;H|JV#9biRNd^ZNAfzPInfH8EUMS+>C6Yc=QgixlmuKZE?#(=7=T) zOudt5q!aaC@6pjtgYgfX>%GrU?7dv8;e1?t^emxR!_k21s>d__?{^e$acOaHT&^L| z@{0LHcf7)E2VE9;sQl9$gQMO;j%O{q&f9+ljZ!UHxU9uyIGl3I=+=>YNGYCfwR9VA z{>3-!P+snLMzZsR3Gy)J}kBd#Ys&e5x=p*hcaR0fV|2{WTFPfrCY!c7j>)&dpOl?7;b! z($9LQy!P>F79$Yi^kdRnBp*~#o@rez9#jVM9@a`N@W?>gNq0QWT6f5lzK0h2U%mE$ z<8ca2u)o{mGU0?3;nRJz$9P|znC8YU;e@nrwX+sEuwE48bf6BhaWqhmx9IJZaaKNx zV|GsDuVSgGTwv#4wTNv%QA+|3+>T1udHhU(Z-->c2U5IwN!eNdy-LVxGM=+mHW(j4 z;kfXrF}?SMEI&h+@Tr54%Lcsz*p^ZJg(joCS< z$_*yT2qif(ML9C>Xt|0uQ#V#AB32;2@REq{a8NZkGdh2uO?Y$)^Fk7CYwB35#K7Fq zFc#Z4ggeC(!_KCsl6SOIr>Axw6PC>(?r0TWkTA$%^zvG#bI0=*&sYp+ii`GSlaq&i$!yh9`Q+p(W&eTSQ!SGd<(#Lex;G(8YX*3f>! zsO#*qAfw%=1e=)GDeXE2?4|w~zD5d}Um8bsviQRKvy7~+^=4P`4xZW+eRWXwYo*H2 zGUL>ws?5X0SAFO=q{}pwWpd8Vay#_NxH|&ETs|#7ZCKVX;UAc>f3(&N{`-Z!L*+c) zv~MJDelnu-&Eb{W_ylfKSH?;{jr9(a7LYo9)7>O;V#7T*{aP8U^p2bSvCSq$L?zqr z;U3B(n|qXBl(Swlaohd9FXZScEeW|(bjLU{QWKqbjTx%H6jIhlB~CDJ2K6d^vJ)dL z!@vn}*{Ui3phH6q^k)eel3(S+k|2+33KSa!_L6g2=IHXDl0d^shQm!P9Z>^&L_Brjpg+oC>jx>?M|& z;RwAYQ~jmV?E^>R$l(u;o^Spcxm!a+LvIy_a5(SF zvy9KHQU&)I#2oIgzVW;>49#c^mSDO+abe=d#1l0&wL1)RUIo{r6YSnrwuex{;y9GA zyQ_gMg?B>rGs7RWdr%t3rIHs?Q( zMa@}SBiE69=Y=PqCRAswukXs5X^Hu^`MPN~A8E?$N=p(B--w$^Rq8c}3_r5mVD7PE zaY!-n_(a&IlZ9O_oqSm?`!f-SQP}hj>K`bcXk<+mVx0WyE}@J&ZO!AF-o;=L|pG)qX zDHAa9%jeH`C^pz?=v?Q6oFe=qqRo0DaW%^RYvIwM_c$I!J*KvdxQ5CVdHuccRt<4` zTdB^ePiT5fJX35f+EKO{#$MtzdY##EY1%ijM8A3Dvy<>abCf`i$$YP#!Gv{|eXXEG zL$wcz2RSU`hsR)#9#5TKmpnP&>#}bc%pz%-o>coXmc2ug`P8dnsurwp6a$`99;g;; z^vL4Z*B^U9rk^nN%8GC>-qYNSKjB8z;o+@!#!ITBtE*3R_cvl0?P$1k3 zr=MdHrL3e6S(%B4F0YYp(rQS>DY|xG&6sU&GL3C9;(@<8w|rK+2Jan z<0@TIUsyCHx(#PfNzKdP2=n2UxdPkS7U-fge#rSZaQ$P2?~G~R4o#P7#7c1g<6~$l z5=FP*XC2%7YZKg#^AQmd^4T$&X3&c)FcX^H9{->oTasYCT`K_nQ0M3#I`35QE#g3Kf*I8FK-AiE` z7gk-Q*(L)Ighj(KDB}e)pbcPbH{tc{tC09Z~TOmQ@AVyo?^H_}9dL{c_DiIeC1I zbGFFSUzkAxTs|NJ+yU|dQM$?Jh3+TEnC6tE0#y202RKGl&hdxnbzA=Us&^DNQog_# z*L76*`Sa7Y%mn78P^sk8*&DIWhu!syd=v7_mNnyExyDfe(VQl8v+CVsJFd$fUS3`_ zs87c0kuP^kmVdY8Hwu-XMODN=VHd|g<9|N!TpPDw(yeLNLFtnEdep08oned!3h0sJ zcj8s_=~Kl*Z%P4gn_G?gc#eM_5^-kdjPfA?bzFXV_MoDcfTso zjv#Jpi2K_6t2MUPzWO1-Ao4m9Q-gNkiE$ntEG3=bmY3Ww(8$UDvIPiq`|F>s_!}{Z zIGW-gjI}?QwEOwb2oG(Nv%Z=$HF)(ZSqi=F--S;rLHNJz9bwiVwW~)oW0zbqFZ=4A zQ)V^_e#ZRXi_uyfHh4W(I+sZwVafXlL8UKE-|ha0MTeA&dFyU|DNa#lA29KWjvcCj z)u{QWLw>vJ1!kfZo!`v-^xM4d+EB3ivzw@AONmo7jOMGUnD!ynWG3!^*P?|E-=0xJQU+JATIxw>n)YOFH}y|Tk1C3qhKzRSg9UbEZPyi zcb9B%t27w%GB@-jcasdIcx%SCAPaPSxcvka+h~>^agjU1+q= ze4?dRTPCqhjJ?PlHVI;T>1ZS<2EPt9&!greX&-0GZIZMUgQtAv`5k)yrd)T4Cym>% zJhoPWUi*dX#8HZp3#$jx2_N4^GdVZkXh6%;BiPt!IkE@SMR>#KXTzC6rXhUEYAoizT;$6CG94n<3O|>P=F@0bkpY{#=_UO0<`hI0Vrkk?Sqc7Z+MxZ!=k1JZrnfB%Ggdu}am28_11tpggNZoa>7XkvRc z*j5p&Xw%`|u|Q|+Bjn|w%t-2%>ok>RIz&DGPWj5_liVzsVOh{UnJ$GNU&^x{#BiFv zKi%HSb@E|UzB#!p*{vI5npZo&d=X=D0BJBp(*50!zdvtelHIDJ9@zc-Jp6AcY7tdo zbc=G zVQW1vucPab5M{r)hjQeLY*h!}!!I=e4-%)!Y3Et%D{Wk9nM0O>@Ku`7Y&+a!f3Xhn6uy)2a}PgQMeoXpb@chzD+=0 zY5`DMA#}zIaF4u3>S@b6uVa3g4xT?Jh0crKK&*WR?Ufb$wOx+Z!Z#(cFow^!KKAIk zVL2~_eI;wD{qNA9y^qNc01+FV29mt*{Em%eBpn)3t-u6e6;QsNq3*`H#xOSmZvU08 zZkpl>KhXVCWMNN8YSZq3WPzm#U}VKQ7qiT*(px?lFBDu3WZs$;GV1e0@zP>U=qfz= zQiYy;UrLp^hAon2==|^|D=hNUC`-@KjLEdojJQW;##oVdfu;Y0+C)LFXE7JKG8LHJ zTHsBh5fvZ*7Yu?ziBQ?F2-G!Drac)=;m4>on}GN zJ@QZFpDUQXmF2z2e~_=fk(U&G%?xv z6iMOCT4wZRe3_hHQxAVvCA>oeTeib`)bSkZ9+unRD z$}+y%p%80^Da`yYKp{qRcA|wlWobk|`Zt=B4FM{2aeq0QcRW*k-iT5#*m&)a2>WPV ze@7oABhOFDt55zVYoD8UedG~i=!6#|9okoUp*#Vjse!~4HnqN`;$RL2?A{Hv<7D=~ zDlFsaDqX3itz9cHrKxI|1pMvPe@C<%jc2Ys?OGJ~+qM1K{hgwh!@r#TOwd4uP%^|L zjFzuWkG(6WZIi<5v&$$OHYQQ9C3K)8>MAiPMZ zJ!2nf-Ir?G$XZLBH_9(dyOdjm>-k??@i}{Vv+HNzpNDh$`}c>C-!!w~I~dM-=&qe9 zI1Uc01k&kiEa2z@9{5#sXn1%l8XOJ zne84-zP9-NxOYURk(``-)2QnrliS;|u`yjsOUtz##!k;F%@Fr#Izgk%aSuNtV;`lt zkD=J7L0|m7FkCXgo>_YkIJzaZbW?$ENkZXnw?fv?n8-sJ(ZazlK|#;E9lqM`4I<~R zUH>1;a5@{DFPiyO|G;v~4^IEEaGe@$Tc)%-1%FBfYT{r!eSJWx1t6C4lltlYKjh#Ge z?8a`<@E=RO<-esyz75l@LSBIlc((X4)$rlAOcR`F3dO{ql>PWNoPf`El^DW#&pZ}4g5dDYO@sk9*$ai#J zxw`GvlgRT#QK@_|qByM7^MNdW9rnZqy7Dolo^Y?v4EOGye(brW)QdcPOu;|0p=<&Z zKvKzxi*WCj1jAl9sLAzX@*k`J6)x_cQR&oy{4Vz25%g{258dmxM5QK??7@fu7k0A7 zg!Ukrp4-d_G5`v_YZdF-ui{}|m%1Uq?>sbjy&d>=%kLsPBoxRA=^~g2J*8>#%KWC*Y~#&ca~5}%{$GjS zb=t5^{|+RyIm4N8Ms^;*b6xEAdG?`DvU-n+@1abZntxisUd~B&SQ*n2(^Bs=D_>Mn zQk~^7EgrtgAPRbHSCi1SBP7^-NIY1p2Ba%NH5({+NM0?u@x`3Ro?A8bE!pH39-A`1 z@0|kgx8^z{H6ay>YUC|a=j6zyEmI7y45#c3fZ{`!cP3p2IU9;zl{}^X>k>Y^TNJ#F zTIBke=P%uHvytF+V{30zqrf-hEenBB*4L-jbJZKlbs^se-6~Hwazd{+ zY&JtGdw2M=qNQRX?&dyEkn(-vDoVP2oAYsLHFs1e>aT@fZU1)j+T;Z>53Hs0Fbg4R|~z1IHXSfAy$2Jb5HBgWr=bEf@@2 z-@kPhZJ%iVo2%lONSdoS8rdJSw$X6t0d_l?H|5B#Gmn@gaxoh45kWkgs-FR zYYo?;<$lt2AuK2liOLQ-FIFa9F%CrU0FbZh)?}_ot7*8jN2>vL5CE9iaPw9U4I*LI z#HDw${X+KcR}0}*{pmm5W0s`n)I@{j`rFS*Z30UwkXnc@&UxFt=WyYKHh)T~BtYl3 z!Ro3Ng=zS1pvQ4MrwR(gO?|zR5X?=mZui~d&p&-b82hD#>|F>r)b9}VPn}`*JKny! z+snBqed54&XItgYV!7wtZ!^@3;Kka3#lz`k80d_3$Y}t=Hh=igSvJo@__xhtrIVZU z%8Z&Mpxs&bZC~I9q#UOk_nMr`ny^o27TV+FT}?DHDUr*zq=qy6`)-kyf6@t7Qm86x z$0VTO$Sj)>AA1g+W)tu=;0Lhn;=Ux#WiXnQIegZ={Ta*>Ne#RK2|L}3tr{SO7O*7t zUs^{>RE2W`zzPNfHCggdZ6S`x}`6L+?Yef|r!<+RD>@DFCT52yP)USQMaAH?`9J|?B&BTF8Ux9DyS+^p-1 z>^`p8UG*a3HiNWGJSkbsR~D1E7FV zG>py`(c*v@s8NAmIxohy_l6&2(fjrhGkApqEWUBuqW4-26E;X?FYZe@C>QFG$gb#T zs!HB~2+$~BFV6Wn9E(j2mG0^AG+U3OgFfdw3xWC>=irQUDe86;rPaj4p@b>c`a(gH z>*YcHnc|~Vvd3F)jD8ApBsQ~*UmLrn=#KOupuaQo2)33?CDs^dvm|V=@loHwqmfiz zAetyZ9qp}k={wYMLT88;JvCCs9{XS}Y|oCxcSXVQFW@pYky#vs!rsZ8LGQ0?kB6iq zoE%Ab+}q_$3uxnawl*g21`t-MD~Z9FbJvvOhP7m{c~fDMYusMXOPiW4N8iUj=g!dn z!eS}0=2)e|SaTWE;ame+S2i+bJOa`!uIZcs6fXT7^%GzeAaYB`nxfpZRhXI!hy)Jn z-VBNUXa+!fwR|6x0M;iNRd_9h^56`7(-a8 z;#!-`-TauYr2e%+)!Zht*b2J2=N0Y|JQMU8W~)WN*wEfpl0|Vc@c7L~I&dt?ZuI7Q znEdh1qW^iS$11E_>d^o+Fvg507=U^bSr?t^W%Mg@UpODowwd0SMyBY#C^v3nop!@) zhe~4k-L2x?0p?^hZada5b8)ntO0thjBO8R@XejHzwmO+!*%Bhs^dH>Mrv6Po0$ATR znTA*OM=Kz5eZhXNn0+PbuCiYMNi+~wV4%cWN$5pLacfLr%+SE)%--JK(aR@*MDo6B zw&KtsW;o5I0O`3B=|_Jn6Z%hCn`__f{^e>S8g0HPFFN7{&^0v7@+Aaq4RpmDNcCAb zSbU;SjtfswrcT@})9-n{kEbKzCSlE|Q0$LC{J9~@oo0YGB8@s4yj6L+qT%;65Py8M zn^4|7sxHY{LEcKfeS2E#A>;cO9puU&#?TglSsQ<&8>>W4*1|<5|B~BE!HN5kTx{z7 zWF)(lgJtQYU0GB71in#_>r?#iy#Ty+NABG_nDO&Vx88QMJ}M!q_(IZsQC?cRi(Jb| zLy>1@%WjbNA6bFJRqs1adG9jbWWVQ;JbT$87+LD3u10D!r}=oXZbAo(UTZInAUOG< z+%W|SQ$-db47MV)M5|)6iFD<(J9EAqIUnViROX7oOgh^>=sHIgY`chML_W%T<7d-# z#@$}Fom4+@t8er@Fye6=#zgKqt^MI&5l#-y2{2xiTxC5nO!_t+y~7r7v5Sb!c+<95 z8q@Gu`DAeqmItnQ)89{l+2_E~`9xHsr*=9~TH1_5*rLl@Q78Dr~6!ZY52Smr}4TrrRQV4}!wBM3o|Pup|SNE&y8 zV4foyNy+5QSImOFJnH@n#+sh^i&Mrm5lbCj6)^R?w#D5s7x7=5=i_e#FIjq)D)(hF zyIO$-HU=1x8RiN0zdFE=A3U*R=p9lI&>XikQ`6sBW>tXDiS zIm_p8_*S3ZOpU(A&f~}8W~8>4&J41gpZ4}Kb9d&Gu%o!6d(39{4tAR2Z#aT~!Ngw6 zZf+>^Q(J8VXC<7{Fj3SdD)#dYW~P$pd#=$fcmu3%S1yPSBaJhiX<}|xD~UB+9ssW zlOIQfZSC|57Npjt`7D|*nh+z88foXu?4FgNh5m2UCMGI0BptC#9+N^|&ELO&|3JsX zZ`>>eW#eY5@FBYJ6Dg|P53tf5CFb0;!?y}l6_gC3h%;73wgc#(6EH~auewQ#I*{Ivy`TKz$AwUtxEKtPmvap_rdDW%NK7&|ZMc`g&iEjatG$o~ zRxU3&G+>g=x<;8uWO2p6VqLuot0LAq+pCLs;-aO;w3oHFiPzq+GIv~7%9aqz&SjQG zp+(4vK*w=kbKtM2D>p$lCu!f|!+q;+>^(Z+ZuT#`LkZ&GU(8Haw=AoQc#f?Mp>3II z(^ENuhZqT(l!&Da$G9ZJ-}!F<$wPlZOcWM(u?!J|Q61Al+`(zLclN6&)v^4NMDUUT&^ua~m?m(rJ5nJ%8lLmKOFFIwvv^SzOb>w4_%@hc_l{&c}91MSLORnhQ^) z^!TFEl`dGE;Nb^z!R8+vEms)|d!x@oJ4dAynQ@8|U|X0Z;RDseS>bHGKUK&)5+AOH zI}d{%Oo<^E*v@0fG`gPgsT>C19TdELLrh5QRa6SLO#uJvm4`COQX#3Khg!hwb!}s) zyZ7=pK?i{Qp(W6jmHK5`L+WPgX(}znR2tsfm9dil%t6m5$@S%JOQ(w`*QKQ%rTH{- z>`1#2&%whM$P#$MF$a^b8aVXFHgPAqR!@{l-tv^=vTgf-{#P`-9?nr-~{Z7 z_R|i}3K-~$tCnYFw@-iL-&r90&@Nf!%#0VHXC$<5Y(2rF1E~Dr;IE2Tk~3;v+B8SU ztM@yOvNx0X9E!-iu?{Z!9}AuTM^Ygo(3RJuVWNC*-iX~2^yGN5F`ObPW*}Go*KNF!pEhhw@ z3%tUPC!)(JYMlrk1~j%Y#DKjg|Jo=p;$8a-UGjDE4L00f0$br(w#t6DkfNhqjX8q7 z5=JEnzjJAD|F6r}5k$ETGa$LC3YU>M`k63#xp+VF+e4`C-q8;tzY%v4`2+WV4CC$) zIcv>)=Rog1U>N8G7+2`~7P0n|_aE*q9gyk=L{oJ1O&sIGpv;pk=TRw5rSuZkUMXljrN$8f&# zc5mt#TwPW3uS@k$9guOoFT{)~(k`pH$l`OCuxis8am|+L;9CN>3P>=$^b%M}&aTQs z>G9E@PWy9%Ce8|Ald~x+DuNL~{8npL3Wm5thfzHF13q0q?{yID-F>HDHmOWDmut

AxqMulvsyA6Q!V0mwO3ZYER+M z-bV$JgLp^`mGqxJ+y5`+^4HTY5QWj3M(;xJw#yJ1c_oakcwXd5dU7OvRf_)wF5-~; zQ{0sm_7FDS<^3RJ-u2Rety06y_}2$U*Q%NjD6^V6JaGzoFY+WO^yk__`VNGCWV?P)-|u4 zbWurfj}>1`yaD`&+_To3*%z+ zKv~jP1)JE5g+W&WLPkT$La?U#ub=)qy#J0fSpWbXzHMb4joq>t6rkg9#F98mj6FG< zO_nL=S;bKpTBv?=8Uqqqa-2`Sf6RZ2H5SWHR=E1d2LIdyMV<@1%%F0KwH&_F!)xn@ zsDe*avC+`k)1W5G!qevgmh%6n`6fion%HO_f4O#*J;at3b*q(Aw6{}C zh-(_VSbjcI#`lpq!`34C4iGRoBUW|1z)nh6{tjUctH74`$cJvXOUZcE+e z(T4rh1|xW8?3L;Uy9Hp88fhuor6RCcuzafb&W* zAskz^sVF4bA<#4rx2K-CX`a~$mPFa34pX6rTL(3j=!5wb6!j@Ic6|56{&6%Iik_RNrm zg!E6JIzeV28Zc=tYRBqPo&C22IET!4iDc;D&Kf34TB6Z6FNet}bQ2#=m{Wkb>-Z&< z@|Z@FJ4tsmMo-{~>?xibe^%T2Iy^F6L~J3}8q@=t>N(|vJTT5%=UKIwCpmi5=H5nR9+idRR5MwSd~ zXu*`~aip)SYm)ZF@izf(n8ZG(`DbOYX-l%$#XFd>Mi~ZblRs@4@n$g+>kXBGc+l5@P+8=sl^vK|+4GOGV$Cy7om6T~b^bq1vk= z=Aeol>80eISdebH%V2jIiypL!F*K1>W42Aw>J=8@EuO|R68bGb;|`hB>-P6HtXRro z8MvMc+zg+uueZh=d-zU`D?KXM*27~P;MRjR=4GR=n^M0W{cIG*rl(8v8ZlA2>krVQ z%MA~Q2sjwjx@|dA(PAY#F z?%d8bG22EssMVdnjw&5hb{yYl!Q9?pCLzOm+VtErX}ppof}D4K+E*8~J0ZPxho4lO zR8QiO_TPEaZ_V;=e9}1Q{3a!by$a;kTJ!mHkLMF3qHIEA4zO&at>iX&lbSFW)tVn7 z8f&~mURRkW|3RdNV)N&L#mtWW>R}idD3p<2Dbu^rnKgo1txoQ}4l6{zDn&HCf><^n ztedy5=|6A3CqpS}K4Ca=6r|+Zc?Tk3?v!)jH`aWZKXFU_q_0D?`D7H^ne_@U(LdmF zcpv?-SdtXf<#e%OPpBqZceKAQl)L|=CBNzIYOznx#(>p^=LaaReY3pDN~3y~QVV;m zFoBwYyMDB{;opTccx{KS6Cmm~YRGg+p^aN|90pbrqf?=G(mt0oARJ$RhQSSq=NeV5 z57ZZxRw0zG^45-siGiq2X>m#^PJWmFx*`XaZ? z>w*E-=ab#>bb6(yja7XQaUhUR#W(lEq}ugN=J9A|>HCHpX?IW<1=8)7`(mM%vLB!b z3K5azvnd5`1uxGNCnNu{=D%;2sYs;gfk-Y zXvt~$M`HD{EU!xgh5;j{Vje6$MmzhU^+ruEqA81#5$^B?)P{a#-)F*?Dz|!!dmqEP zTIi4MVbtjt3eAZi{0KCE#1|tA*vE z4owrHSLj!=zjA<1hC|U&?va=XV?rylkE#95-m0vF+PoOaacrv4XRVel*CtUfhS9-H z52)dk9%QH450O_(=ol}UUOM>%*=K85=;AK6oiva%%V7fL2?x4rCQR7(5F*~?z7Z#c z^zOsq`L_kOSYN_%<%E+SJ51}+gg1*as?U(RvbNDZfYMzEE^>cdCn)FOkz=H{Zq41m zl^*MyzFn)Cr7&~*6z6vP`H<>Q#~ohXsLwpxBLj!%-EP#|t}*NZn#D_9_<=&sl>n2o>$R8g3T7XYt%p$1a~5T4~GT8l!L4HucMHkxRNA zq&KcLG;fxAS;gVC-A3E*@!v1r_P4X(dEP(L|>jbA31qQ^N$4c$0Y(t zlv%w{^@V{${W8;QVWq?jNnKE`Gmi;KM{*?>N-XA!g4YqRX1{P>$mHIzxFRZ6U2=`_ zJ+-rD?zx|Yf=%*AcBjrCp<4(CG2b43;NMpD6{SN;^Q`NF=nsYzrRNM3bk{;c+2O&<=P;kR`%c;gy$x1 zo=p_r+OX5@I;sXf>Rdvl&Ar?P4D?Yi94GbRmT*b;Lg+lqxKg|Q zmC7-KG6iYAo&rZ=>n&=Uyn9a(UVh|*3-s7%UOy}1rmajqJGkBTsDa0Dw%d18LHzn zB&|0^R^G%UaBI{ICv*=R|REy{Pxs4V)jVyugmWoy^ zxSmH(UQcpyE{R--j^@~9_EbeXKnAa|Z0V&%h?B&}cn?6B7b_!?dqg=P25Tvah^e;pI6Ak5L z*#PoZe3ZLCf~dKuu@{*=S-qm?s|RR=ARBcBNl8icq+IfNB9VL1=iNL@nYOREv9Tq; z*X!loP@XN+Qa|`%-kyGrnz?b`YwcZw!P)|Vk%2)Q5j(h|V?jNt?Td8XPE`WpTUZxM#(%7+K5lpym?h4Lc;mnRvLLuNCpY5*E?1?rABI=Vv;3eB!Lau@nxq zl(vJxwAEN#veDS5$ zsdU5g-SmW*ns4qWMFFZf0G#~wtJA3g9ssgdc7+$arqMXym->TwNhTR9Z#KWwRO;~2 z&<|;!#x98fc4wKCNqOWL(rV}!5o8;-R}K(pkAb=$nqppUONK>O9KBCR1-T9l5@x)XNf`IDIU7k zEemsmE?!lj?!Qd0vnR$7s5SZ)vo4+UfGuOJ#<;GHxdHEuM|w1yv3L)D0#7e^AXagV zv5bRix~M)S@PHQ`vM$HRnxuCPQ<;&Hzgzy(L)Hc|G+V;&GvSBS5UA_$J;Z3Be37(U zBQxv9{KpJ8cg9J3_J@r{A>W_lKN9k&rg)G8F_m8J^Vs=V-!5ibhPY_SIS_uOo4VF? zrO`sg`qGiZ)+flus|I#5_lWiSoPHjjYOl~{(XzI|vU*uV<2dph(x2cpkqHFxd_IsL zVKiqw!q^COx_>f=h!#|iHZTyqcQ6!&Ljnjqsw&;);K~E&AtA#k6whBddV;aXP9HEW zPnb4Mc}^ExQpBcV2Q++<1BA)68df{R;DER>93|?_GGd+2qF~O%igH{aa{rl*wkK<3Yq zlP}wJw_l5l^nz|iRXdd^6m>sCXw(?@^?YcjU(MRR>g&zBMc|$Q zm}bwz0$=23K7QN~R17(8`_S}O&?6RfO%ctDPVNdKmD(pT9(nH~ z-B;~bov{lVSr!QZQ4d`U76~ksETa^)Z z`S#^&EDrh?EL&TZ(Zgj~*#miE>vO)zFvt_hl(E?3v6IR1hijR=Goby9MIGIJ8oYiH zZ?m?5Z0iXcFEflcs+~8*rn!!zxtp^LUvQxv@`Yw71@%j-S1x8d7@s(WLr+7s(Vvga zyp&JB%qoLX9BY`J9QZ{3QT6msIL%jc+l%NrW7)A9*Mh~xPu=KRS$#)q=sb0aXzA&K z@p`esjJ;+IzBo&E%@$(yO|u4s7y$1KH|WD4p0TO*4NRUxj1BOl(+foi9ZRi^w@uA$ zbk2j9FJmq*`Hd{bFgp}9x#o?=HPfW0`9-CVB1n$#v4!RF^uB(CP?H8|6R|_yyouu<(a;X_@@G)yPV;;sRLbyIA zMZ@G{P~N|y zm2@Akv^7UNU2B;nMC-uGhPM5Q4zo#L#p&@kRebA3xXpnQCiDbddb$)>W0P-udZf~i zZu2{h$Z~BP7}%5+t+I$3lN|J3qO*$&rRXFYbCb$D0zAG6onQQaD-U6EWdON#dDTbj zbDZx%p5;Wv-wZ;`MRzKdI?Q!%bnJN!uMn5J?Iwpj)OmCmNjZF{18%g^$N=3A_d9|x zQN!W|ZpzJB6kEDt%bB33$lLJaCkkHpC8MjoR9e$*A;C1#;?Q6yNnvTzqm2Oux}qKtQWfvOEau8tm#qb zsoDi@mQRE2OW;rL(bmuH7>r4-iwM>|$oeFF5KI91sLX8p3#38cF^R6|HE`@o1!FHO z7-)A?NDc43=|*b%I(^}Xt_n4r&W2)4XI{O$KE5}v?Q1^MFnb%n>Ws#VyXB{`!!#Zs z@(UJTyQpL&uDg{pN$yi)Q?Bv9`FyNYKH!&W%(3f(!DrE}o1W?&Q#b6RPGQ~GWnWXv z7=0m+KA8Fodg@1eq8VdkA$jE>^oB1J<@Vs%vt%<{>*BNxvX8@*@GHUxG(7nHwK%nTqxd zKFQ1(Z&U{_+j+u0+?ZbEJ1nS_Xp1c7Gw+I`p;5AYw&DTN3f3sJj7#v;0RgS2uU}s} za6~(k@U$)`_!MS5URftc5E50$7@Hd$Gn?3EAYE_y#8IsI1{=}T**7s_7}hR zkFN~$JJ)1mi8Tn~D!UGMMQlN~_Z@zqEAg;nE>kG6viNPDShDdDX+%&bc}O3@u`l%+k6OJ7oYH$xDibv3F{@TeA^Zc2g9PMq2ly!t5Z zGLbWiCF}&POP=C6_3!$I@q7K)JhDtVjC_>xle?O!q>6IPef-%yDIq?qb*;bFzSSao z)}yD#xMDkq>I}-x#}KFb%2mythEklHJUNfMt}>N^@?imdiNFt0gu$GRaJK_w4RD&-pz`eDk9u;s(azi=W39 z=Cm?fz*$#x8P;86n>= zs%Vhqo%C=stB>zd<#-of;@e5@?e|t?e68CWV-7WBmEs*Ip%h%EYJ(lCb2_lsSpAX(QWP&kVs=BYwWh+Gb zM3dkB5*|VDJ)n`OK=5Y-c-dc_eOpA+6?fHJ$~4RAEwl_)65Vd&`CQYqU-C&h{Ug{C zLKdM+D@x?p|LY##5yy7U14zoB{J)_|>3M5KCw}MrKK%blj;QGCV42ntMs`HC_l=N%~2WCm-8sXYSAw|P_~D$h{1jm z$gpLj6zyd&B94a@Xoi5a9mL%7-RkepM*0+*mA0l$acs?)x;hb8jF`V-c{#;m{w757z|MC}Z{+0f# zBza7#TjPMm@o($XG2a1&Gv^u0idt^d(w@pk?rh`Rjc2||y0WaAUKZD~nJyJGn#i)o zF;Ku<22!swCkdUg*tyLQe1zQ^$1LqrSBcoOdNeBQOtWQ|mTS6;y-8>+occk?eAMhU zmEJ;B)}?gY$60j90J9gaKpkj&r=aQM*nVGYLc8gwAupyrGu>L0e(Au5zQOTELF*+rerX%qGUO%xF z_xTJX(93p@9pN)hf0rx-?QF{mHe~BAuQZf0U_P^)0Bz6y&BB-%fJ^3+5k_%{4oURr zmwZIBO_2!E5yC6Hk?O8ug+o#e8)a5*sA zJkk%U%5ae0EsPMl9DCxI5Zv{Hi(z$ly_H5?1!=Fm_}qS~4y$_j*MOI1Po9gdl~1t* z?&O}^O#N4@4rp5|84D+nUArkWu1zEiQXYX=3hYeM(8?`OR@0ED~jXC@W3Rq$hpuy#Rv6j)u>XNAKGcFJBcyx3SO)Z^ z$YGM@51HXHmSc_b&^q{wMZLh=v6hzkUTZk1!kH=R5P!3rj9P*p(7TOXm2lePXOR4y z{GNXITg~{A2((|0)@C_20FY*W7F62CaDVe7>*f8iLGBN)nYwdO%Wl{#o5K8D!_cX!PMtGRs|o z=@$NUVw9W>AJ7`o^PSW4LFGQ=Ds*CChPj+VeHij_s{wxu0XM>ziDTjRsjbf4L$~-< zAOWxT^uLzX{gz|NFe4YAxVyV=2zwQjG%CRj1gF;Y27A*SzYy%-B#%h+KMP2O4XOoAp-vPam&P#! zgg!z`)`oIRpG#nE!xhbQ%U2WMC08|pDNny~ADmuC=*Wex=Q4UdU|3w(H-(U$1U&!4 zKWF;3(?^Pb-9^Fq*0;Se$++y%`9Jv>m#Yccd8wRQ!VwqnQpWv$Qs8B)n=}6)^pIPt zCQ1P@VW}WG0<0TX0P6e~XJv7(g9C0V%<`g3ERaTjy!qbrZlR=5>qOEG*CNMpX8V0 zrstRD7QqB)#oTFZ!vxCRrp`Hy&7=gFWaYJ2&QC+W7N`@4JUzi$gzG)J zJgKbCxh`&$K5!-Ypay`dxnz!}xejbuU1Ndq%?BaC3e+6jtc^`rC>CIvqW#02OZmK? z*||vVWu~p$yqi6PeCx7--2K<3<9!YI7zJmMwk8k}It;C-i@C2U0`A4N1hg<5KrBJy zH6|C+n5}sSC|7uknzw1%m)fF$Z^6rDhn(oMxbb>BpPIRyVA}Oo9sYi4Xje#4jzh~R zvm2F`JNT9%+ikFQuKjvH0_n#h%>i{Erm$V}x`*XdppGbAthe^H8!p>j(dWj0$}^uI zksZw`*dSW2(q{=M7%HG}W*{O2;$=G1+LlJP1MpR_8e2*GHz7SNI_>#`!B5MX#lW2@ zTus(4{QWO`g2qXZ7tevVx%D>2Hy(J>Mu4`@pSbhcF7%x!uTE3w+g@mwjXT-z34XYa zcx9se{*bfgT(Re3N7;$)0J|fc6JeFbz;DKJ|Fb)EbYQVdj@!N7`%6bHW4*DPg2i)< z!tG@N{?Y6HanQ(^Tc5Z?oULx=Ey*P16WYtvYE~<-D~cgdnO4qp+RKA+du_42a=Yn$^eHilBLMxr+OSqtNGUMsQj~iUJ=(xMeN8Ds zc+0SupY30w$oiW=U-0L;qowpGe?ebufOG0ict08sx@)Hs!rLByz;iU^-SE!R{wfBc z8u)!GTtx8lw;RPxixeRNETxeChP4DB<;^M0Rid<3qWxeqaBKl>PGu2pt z-luGKj%nms3I*J)?aB+ahNA|*-DP(kG?#=ciVCe}LN4PC)}Xk>4A2g( z#Tn#W^`ZI<0=cLyXGg%)9wYCCmUl^qO5UVD+=9cXab6Bb-fi3*Ww+FlL8yxE7Vpv~ z@M!KggAcy9`#dF;_O!V1p^!*a%H|y)W}=XuC)XzK9ooPde`84-)cyxNQR$2#ho7{< zY65QSSLmIx>x$YuXq?W{4?~AZ(c`ruw{3LQ`E4f~SuQ6m*^c4?2f@KT%#DF$P9WcE z0jKu|LEJr``eavsBq3LbU2CCse-}^A+!mM=$2p!NnS*{nkCRftr%qMhTA`LV0f7A+ z-@0A>ad-{Vgi~-d$NS369tkBK^a4W>?Wd!cF#daLGBh^ z%U-lZUl}m?oR!!m59;-GKlCh}3Z790x^+5;q??10XymWAJ-Bm zYtCPOx_Jj}^6g^o$J@#_(&FOuJ|UAJJg{`SvuZDd`uI>Cyg!m2a=p77EvL?D-I4=b zirc#M!AKxt=#QNmNUxyq5&evfy4V)s>#NGvZ+UCR)e9`{tp=7ap@$9zTiCezuqU zDePwre%Y>qOY>h65c8Q$Th1gxmVbSeD*9p{<)^@Q|EH!$sa<#4k7CChH`3X^xHbrv zc^_Lwe*YScWgsNMvQr9=Ia$c($kU7jt+A{ZM~si${C{09r|hf>c43z=GLaWiZA{Ts zZIW8>qJ!!)Z3vi2zGWaBQcDJ0^Jh51<+46sY=8wbW^U`b+&5nuv}~mZB`i7Dw4p=g zll-MW4e-d`u;*0EpP881T|SLOepXxH)?UWiy`g@`A^ZUg_KMaEcA~QLIY9#i<~uAI zeeoo?!HOozF;>h$CQ}RZgQas?AddQrdU?8@?H%tsHF#dFf*ofH@&Yh{sN6nq!D-wXYh1D__5nJ~c2m z;HhZ|K3`C9CD7d3tSa>VHyE)!Qyh3hiU!KEwKTC{*fnOkD5NXDu^bEMAJ=}o8vd5s z?Pam9tTs~Pm*r#B*YgP7g>Q&PXx`fzlapTq}*SNQ{azs1? z7%%ULW^BBwx5=(UoVknMiAG)_{)s}vAYoR+U2*baXE zw_7p0Kkmg~Eb9RvlW%KxSt5lT?N~9tz&cld`gGtuomearO|PB{)VSy|aeEP`i)wPS z-|dI{re;7h+VjzV_|L2Fk+~4>oyez%DXEv+or-v1nc66Ye+kPd#>*ffubA41*?ETu z2|y$p(JwsbIWC)=XFo*4_k|lL3X;TLei`ph2d~_-B7oU1*Z5Z8IYL+@t*TlmI zpOthK&xX@rbxo_srxQ&|=@l|f=)Fa@i>-8pB zcEvNj?S5JkUN*T%4>K#}& zq=8SB9H?w0nB|phe*WFqcipYB3bA-+^mH&r&8?w@33&U~_cYgQsy@^I)aFQ2YH#{- zA(XqK8X*sBmPpvvg4inhr?jD0vZln4jv#S`8g-w`O=&7WG87cU=vjI;qy30*WshE{m*F3%-sCCP;Zxtz>MmV4R_Yvk?&%*sI%rUd_TU2 z<%LW)jfn!%jvdo`bPXLEbo%&TaR#W`ky<@60s_vK(f?m{4=f#TZ9I6fqiRb)lDqX| zeBI3~jpL?kVk6yirlU@5l4l`|#v%>$8gM$u&3*K$^---Pr>U3omlh}7O#OD8S#%t8 zo>%#Ow}u)<-%2nG{jxKz{b?-Uln}QYk`?uLc!%()viV;;`S{Bh{5Yb$@*`@^4syL= zpHe>jLS`d44=EG|1y-3X4zCFh*v-6)IvWogXJv>Nzf#IXvy%aIxSWfk1@7ONsCgul zFi_Z6+BJS``8W13{6es7u4M;0iO(W8J@wRzx_1jUL{T1z#zr-&@?@%oS}u@?8$t=2-LX zSHM{1L!q~uLV61O;=YF#YX9OCJ_4XrT#!@CP zBQcyBDFyX&ECCw{=aR)(>G+97mDMYgV5Uy_9v`Dn_l!_W#Y)yHg319hSBgvVoW@7A z+#0%N*h;KGy7|;YD@~ZyoSQJztduauY_8|$ zz9H=9UBzW!sSE94a1nQNWF{5Ls5JYgj?BKoaVsWQoP>>L$+$|l3o@D4;1Czk zYLoq`zS^w@kDRbA8+rD5&kNKzc73N5d1nj0 z>)n=6AcpmlLQ1sdZDzN%F_*0?uNCKuuugRY{iIm^bpp+z?X5@F2O0@8`z((TS>Iwc zY2-g<-~HR(x}P(5*FNJ16U)my!B_7TcZK9iy0;lqJnjbAJIG~6NU9?DJLx557V7Wr zj4JSh=}PVszi2ioGP;kZfsTnuZ^mdDBo$n9*2)o$>XE$!3u$EJE=iYQ^e zGfPl1$z}ZorZ8hV#-oZ~&(=Ilz!WlDDQ3CmviK%rV-9cq8?oK|8o)Bh!dgjYJI+Hy zYv_#cY2Pcx?U3q8?NeK6>YG>cV`KrWnR*d%`wwM@Q-!pBR-Vc^rV_X?+JaAmEW_9_ zF`n)=>^UBkudTqoy9Ar_Z2!g`6cvh&J7%BWXLbl)i9W)0nbKO_{#IRm5|F2Wv0D#P z%evXSSPl->5taxC`~JCmZP4&GppjC|xPlLNn|&Cp@GPGF*<#=Qs#%SPKoN^|ND3tt91CN+ie4*Qczz(sJQ+uFsxr& zJnoccn&e_Vk)tlPndd8^TzxfHz~`ftCiW#_ppLSS394|!cGiGpZMbBGJ)Xt?aPv1~ z3uHx-ix(BV_E{XHMfwsig>Yj^XswGOeYW`*!*a2qW~}+^9v$xZC|gC-_>6TaE@<7d zv_G6jGNRW}F724&W=~`n79OXL+k!p;qyt*PZ@&7yTE-8AK5I}*S2ZaYwLAo+5}xn_ z^tLWty)ZZ`O^Ux^K52l~mO?tg=%&MoA%36N_mLgXxyf^s@kBzw5zK)+L@Z=-$4WAv z+pX3^9<32LRkQOXgm(m`e5v;q`Or`-Z?fys#`S-sn$jaiPNGNo)Zt-^PD)@h zJIg}T-Dc<;XHn9=tnz-)G<#0FHCanRe_9@}kM5%SEggJ6mKe1{4=j>Hv#V;JNMmKt zx9QPNG-y_yDbRTs1V)(5`ay7v+t&|Xgf7UWT*Y_>9PtntU}qI~>> z&iM0GU#y191SzJ7S)|`R5A+@EIEy?LOQ9w%?~-)UrM8%GyHQ?J2Ycv9Dn!KQczS5k z#b1w5^dT9m|PUnL%!8|mW zT>g-kp*$=j>R?Qk@5BNS|Ssvc^2pH|VE9n>;Cqq&7 zeQ77SfDfyZ-U0jv{LOAMdHO$6ItBI)d7CVR$9j{+I+|Mvg5>U08}WU}~46}0h;K~K?{TrCoVTI10+cD2#7Y!h@EsWBtHX3u;par}>! z_fW7=GI86@y8MFX+U1;p4m%guOtGDyt;sXN(ksgc%0Zo|$2@2$Lv-`uOtwT&F~Gc8 zNq9*5_&BepZ8hL)$Q71J`#tW#Yl@2<+vfy)xOR@fAX*Y^f8|NzP;YVBQc;bDkO|-IhRmuvEc&E1U>_S5X*!J|R zd}g!=L6VxZvx+9YDImmquO{p(EpvbJIEo#q!ieGQ(;DeaV1Y<~R)p|q2fDL?*LO4+ zZAFm@YBD0boxFR!9)!P{JczQ5Wha<@Y(|VNk_C`z5;>S12D|wkf*dl5c-t}^i3$Rw zZtA~9Pm+R%_YvOyVejMf?qDU(#^SF}wTYbP3L z)y@@>pN2z8kiE2dj6hcTzPGZ;QSBe%gI-T_R<`--vEE0tIwXlXCsJSJn@Mmf?oC)6 z$9Y(J&U0S=9~J~&UJj^A&Hme(P*<&iW>Tx>TGBC6Z&-|VtqTf=a@nQ zwCKEG4QISJUVp$io#H(|E~I$lBwGXm5i3xc+F1Dfe{Eln2yiw$gXnWyX?KMV7{8FM zcOGWX3f<@mFT&qnaGI+&8MS9smMct!op)o3c3Ww&G%=yjwLJ8P%c?DdxCNb_GfF7$ zn=89@u6Tx9LJ!eF4^())aMo=j4o7_4xsCF)5I>7#S=g>Q6V2ntj8s*IjaUL1c6_<5 zm#^QSlux*5I-+uhWOp916d@8#YY{w>Qr$nh`x>hhrc*cQTuN_)Q$c}a2=)wFXTmMGHJTk(`NEP7+7!eYw3}5MM zv~8WX@|CC!cZ=>G0KgvUL}s|H9`p#@9Fo$3g}SJTeBzP`uZba*I24#@0wRxBP? z1x$A;r%E~Nf=?;(pb6UZ&I-N*>*=`&ieefc^?p}8=O5a?p|E_G&>dpm5;d7?bB|nR z90hA|ngA{)N}y3yF~3qr;zk2~{{7A>W}g+AXyE`pAwQ0EJ*A1SB>T=RNyhXm<+|G! z0=&_ZcdoXCF>~dW3rCO)O;rvQJqhtfTJ!aF3i0gtxyRp3FwewK@%E*1dmjr~WXS?U0~86&YV123AxpCSGnx zPiW|$BGR_u*h|?MveM;GG(o_O^#Jatfav|X!}1Jw^l)8d%yT4Kn&R|f<3jXK_GJQP z!39gVC#J}Y>mwOn{<={{L%B;^2EhEUfOX&*4jFu=zfD4@9Kj-1QD^GfGA-DA#Msiu zQsGUlHNUBgJ|`-IeLH!GOgwb{z_d9d|G5a{m1~|o%5uiO;Esxj2JrzQe^DyGkBG!z zN#OgLy>0#){LO_9d&m(^X|GF~(nHBog*$qhnSilF;?{!tZbS6=tTEf?{GwV^;CDnQ zI_odMVUC;ZVuBg$n+vo_s;ock0yt9f#iQdg0UxOvt!;aUTjzqur=Nmd=$K^Z!hVLOZ-_2* zaQmSw^Py{S^8J*rtai*}wtrBf<7*|J4bg6R9~Rlc{}>Sq>rt#9piOwBZV{N0-tk$7hX%!(Alt3mVIq~a@70F1TT(yxVWa?YpQu_x{+9u^e4 z(6i%Ub^GC2aBCGsfHgNf>P{_SNX2+fiX}li;I(^?#Wd?7&l`(6ZQx`cOF~Y$wn=hS z;QZ;l%trT1qvHorD?&Ca1!8UeW1#sFTOdu*QG`5CsDl?8c66YW-KeO>Om%Ecyo z;a0D~5;S5FJpFAg&c%sSl=SeS&CdQ|JsI#)3H>%9tL=7I6nT1-ZGG?4^Bon}O#pis z#?}urC|ho$Abx$=>H8{=RORaSF+c0sYhdf@sgp_3t|&9wr@Am+sqk&ottr z?%Qzpf|xePavvzgX1IW>dacLeVhHc#+n6|SRIP{1z%7+z^sfR?HG^asg<8>?)S5at z!zO+(W<9crf?Hm~H8TSJFwU|5*&TWQ{9K^ry2fA{(-xT^JLoW3> z&|&)iLQZ?GtKpAJlG7+U1E=++DPTxtf#OhVI^AjpHmc&`67Fj&V!TbLx3FM#4UKMo z^JcLq)fQxpM&R`#_N@#&UpX;~IC8vSwo;$Rl!S--SotZaY%HR#?Vr2nf+sBd)FMxW z!Q^c5C(rtH6xWI^r7Z2XosiTiWUC2suhkAVJS8!b@Hk(o;Day+No2prG#I1?g7J)i z0}~KBZ7cMf?D41On>53mV|p-R^n}H^Xx}rB0yV}>+t$w7Di$3Dhyg2XX@ky{w^wzVQsr9~uL9SrLAsAE{?*TjZqf>bH&*Uf-EJ>;Lkqd;$(s#rE{ zc*kX18msC^908howIOJ3-}#!p#3u$pN*k zl!y+%%fV6daGd<%qnkM3QW#2_X@f7|0Yx#O#*_`<`w7wN!cxk75o_+>p*6qpsv+mz zgoDKb^P3^>kr<5dBh(7oa(IS9aSOUJ%@^Zb&=l7%z$rpNDjiHSlam}Xhlmk2AwGr( zW6tG$L5N|qs@}fjQW=rLNa{;#mkZBJGnVy*o<0-H`-QgZAeQnJ2C=%6#YABj=8ub2 z3H>C`Jr+B9Nvta@v_27NZV+&s64>k%BNCV;8%37MIaVkGdXl+GJKMw&`2J>-z|Qc{ zevB5dgf1Ais&VrXeQk7@eHf>_2pF9MY=@QB-#rbpq$=|Pya#C+T+^cqEs91|06}M} z0n+MTcTe9~IZ>^o+uI)aC@^4Uf#t@vTto-{m0p_6IYgn}DH>L92~+f*$Bk2WQRn0$ zMRq+#6s}L27UAoMB`8}t5onKxwHeG}Lrs3{mBbs6L>xvPJNzuW9%jyr7mp8AFazL5 zVflFC^}`hKjSlt(h9wUX_v<(mRe_Sp?!q>S!TBI;u7s>5Y?DoKQu+^nP7+cfhkLSS zSBIM+qOA+XCO<^YtQLrMP^ma>tO_#h$4mmCRy_DAx@xt_>Pe;5r)6q9utf1GrtePh=wx;&y`Xf*@^9 zm!tVzv_Hz56Ni4_G0lKjY-;%J)gyi^QB(cv>y*i zE@+eb@k$?8rlW{CZ(o0|g0D74x9W07XIMcgm*$Iva~ymO->ib)%@DaC{jg;=o@=1} zt~qaB1#8&;^Vh?g2UI^W-cdA@C8S#|uPqC|W6~IW$|=9PWahl`u~BKv_yP-xeoU^b zKIrq8>4Yt<^dtB~tod_e?Ib~cW7uHcd+HUNo>|aZB@HOh+AzJG(y~9;yA4_TAsrfD z6ykd@OZ=&Nb7Ga=ou)w!E&wsBqfg8k93_WyI)3?HkP%V*>!icDlKj(I-VDZh#`$md z_2j6A<2-bNok5?l2BD-ccDE>UvcCQe?!(uJL}WZ{hQ>Tj%gkg%&esx@l$4MY`Qpsl zF;h5>Vi6O3KZP&$W<)h$Lz(P}5F4kgezua5sYn5Etcrl~i$lf4iL+QpCB#*x4dSDK zas>p--#gux*84~sX5?sDnq146jn~4w6Q^Z9i$5P?o^f z0a}Wm8f)CU9Axqp=Z{jx=Ca!-Zx7ZTFE`ir-Cg#*;ew-<5l*K4QxKHU=@nJ;OP)CC5BBcx<{$DNFscsv zD88#8`0(1K;E&xaE^zzEN1=}L5I&;0JDGNv4bo+E_r`WBM%2I|+p(jYnQ_v!y6vhe zs%hS3$8nRymxt$tYP_L>{@4dKSs`Vi-&&Wi|E5yBk@5eqJm=Kl}KW>LW~Fg@)kHpYflU zCugOxpn(hM=2R>@kjhjyqES$#O|IyLTkNeEhx7+YN?8`mZw`FC4%$b;b*_?(8tE!H zqPEULkL+&pi@>@JE>34^5y~loU|s1Is*dMQzvY!GoLb(kude_2S18yM!WsOSED5fY z(R|~7HQ#7XN<(1A$dmODw@Y_|IFYdI4@Ho<-qQh%v7N0a?_J6qbkaiLz4(nkq4^3Q zsn!kCw12X3wu8R0Cox)D`xah>HDA&?|t{0koTjavfD8`AdAo zs_J}_q_!+>Rl>QXQduwCy|?v8hai|}FO`Gplb>3R;d9SL-qk~c&LK8=ZnzTI;JTD7 z=JNA}ellK*`rqNjd`=iXd9nz*Yjr(GOLnSQM@7F*nEr~)=ztvf(|sh=mnA04&ANR4pcT*t<`{=pE0%$vj#+2oy4FPajX} z)OV6UiTWECTVQ^*v%qN-Z9n2#xE-RND&6NTfvg3`;c8MO0g()vh+SE33rq z+9tUXeJyeCK8pV)fzR%JZ7(PIt^G0I^FeBuROS`uGRBoPCC9|NFr%F$F#NKsYSwQB zvQtrP?wUE2_IsT1<}tBEyUSqGdG8GE*B1EF_XQghOGy0h?9ECE!M+7Wq5f#in7&(y z5O$HgCBj%F{K&8be^9=GCI0cZ+f_HhDk+JtmSU=sj^D4oQ=4$;c1^rFJSMA2?5Zd` z0EJ*qY)kLbB1}dPyMEv7`eQK>BDRR2d=~sZYxQA~M(Eu0Xn zYB=}H*?TDb;!nbqn3jAcG3Mx*w1sCv+7*ex;SH|om}~vsb|0*z0fY4`VWqY%ycmpq z6Tc8Q@Ej3#)@o0F;7T6_`k0@JH`^txlP-B!c&698R6V6pi_Qic&8qY6KYYWA|t9{4NfA zPYfe_vRG_vzbM4tJIksTRsySbeXbZwtk))%HWKp15|5ov`s^}m*Ic$b6B6Qsyx_s+ z*@C2I+l)NaM7|31c=KmJ@mjb;eD|!l2eR2;?uFsm5EaLq2MS-58*Z*Et1oA*Yd`Sy zlZ<&sYg}Ob@PNLDGML0_H!KrU>4nmlITg1QhlVK`n+>yD!)Nd4BQGU(y)U;uzY~!( zefRD(L8YoU=MWd7xEa;y8shy90;qzT@k?+lewTl9Bs+aXQR_!XLjV#dC;&%KH=mCp zkjiWv z$Z3_7vVY6@eUJnGH3~t-8XGUjW4qsjM;@SpB$)N|yh-Nwel0rBLvR&HihT=nd!cm= zi=x9iw!0sSI^D=x`b+LF_KXwOH?Ophtk%@>4{ma{VjYpd-hj2zY$NY&wph=}anZ!~ z&PW%?p69abBE{SNyJ#r@(eS}e zI(-irdU8#-#4KFPZ%9z@N0RO<5ZBWFGG2q&k3tumstt;UM-^f#rAu>UeicIR4z5d= z((iHKMNd#hnB-lIor zSZ3MB2Q5%26nFFjzsTg9xki~&y~yNIq1)bFWq(*HRK_i!4aaarhX5>x<`+!gBKRfQ z+Nygp5gR5|kO=9!!rnX;yTyZ-9=A>8wKu@|n=G!%_W2m(xG$L`MG6J712IJ2d3A14 z+WP)cH*8aG3oP}B9YoZ{*SJ-E=m9bgS-lR2MMiYMDCY%e_^6s@sJ&3+?Yx#K7J(YBAkh)(Jy9%4`~cWPI!*kQXfYGS`gQ0STV_ zUJb8D=H1NnVBD-H{+Np7Qq4T{$o9sD4Kv0@1u1u?*jZ65(Q0m4(^c{ThF%{B`PFyj zWoq6j8!zB}Cm+OzAK$3#)`$xwuF7B{AnG z-Ao(0Uu^^23O*ZlgViw!*@v!$QK(MY@|-K|OG zTdMbEw2@eF{UTLh$V^0guk#T34jvWImY=1@JDjQ%(3o@_nq0Ee=Kn%z@VZIziArj? z3k978f%S6saXCEQWsM`D-q6m)PqumEwmTZQuCWVdpWuEiR+#bQ?f#81w8*t<>E461 zX1wi8u1`gJh{H57qf$v=YM9apm64j5TN}5&ZY_&-3$7f|$1bU3zwzHE^sZ7rhs*jM zP-pf1?Y(&+<}Q-WnZ?6&_pi=|x9Fj9FMdGGCFp;M(NoR#Sw&qi;rU|JJ+Jk#ZPK@2 zt%mWNWOU}SLhYJG#hx;oiK31)#6Q?5ksryZ-nh3t_5nuWwM3X@y|r6HHP6+5 zDt_>yCbHGy>(6(~Z=PvoV{k3uZ@*HB*l5T`3lMum?W>r6qLp_fu3PJCS5nVr5KMMY zZC7SGY|;cpT}SHPKi%`WOa*nT2@YRaF$=D*b!=kSgcdW-9R4i@6EYwmpwWF5_vOo% z$xmB$Y-rrIZi;uTXa+B;s)R2paD3#+h(2%tO&8XLTmuR$7r@_#Cob*DgnTQ%kpgdu zeG{5q!E*~ZXzUp%9TZ<5ip8peTpu2}1;uDASZK=$36Awm0J-BnyDbu# zXMD^b;+M22b8=R{qq|Wa42$#M#B$kexq41MQVGp+krfCK2joNqRh~H~MLkorxd3ld zOpQ`wg!?Wi%5IO7NrvC~Lu1bK71KrbhLbEez$w>4*7Cj3m-H%bN@91uZlZEES2V4% z2l9)xdlZGt#V;rK^9FlyXb2dkKX^o@+Acp^OY+ze)$|#NqRbH>q`vX#HCraL781B# zFpqA=Z9^D;)c}KQ)swc4dNmk^9x5&9v+Y7*2{MZYIdW$=`_H~2W%{of?xGVW_W#wh zDg7y+gZ&7ixiNO(`ox%JsP)5K4{rp)_`lO%)m1_EUkb~Kzb+U!x4B)d zV2`?Mx%JUcTmiC=>JDO6a}AABeA?~W@B0p1^&z73=3ed_AL=L^*Me%n>lz?7=B$NL z-Vcf{Db~y8TiQsizi(Gx??Br0*K4!kRfzQ%>7zvWbe*aaMKF*o<$75$(^uE^8 zc^uDeUPHQBf~&}pGjAJTVM#g8*z;@U??5$gMoySDMRH2}H{KH5evQLHzsd)n=EQN< z{CvL$!&pc57YEj^Wm|jQKo7^Ue0LsjG|$M}`-({C_W=c|XP-V-G&$K>E$5wvAgNz& zKmV*_TA)q4Sry{GLULL_J7(z=QiKYQieI$K%&fKh6^NYJO1giVSC>qT^;{wixR$c$?K;`GE~kwu)awHBL5(&~r_DLrEV1crGsfOx73qVb7*uoG5a+;4RggH*bDptt|e!eTnjMGQFuu zF`I*2y+tH}1XM3Coy@oQ3xn``s)3)*l$rPAeyrJ8yeWm2yqxlg+6#()Pct^88hOj5 z-V(Tx*}HW>@Pi2}Qjol|o#`rs1u&T0kU+ zk}qq_{hQ(+6|{Y+C*JyKKJKNWF-o4xhyICMytE@=P~%1euDyR!PSJbYz=MFPPcgj2 zLRXmQ<3>#(Y}N>(f)7l8a&m;a_SJHg$Fp~)M`|kG2wpd7T^SRbw@CKo>hm4XZvoZq zRFs`0df!VTqB9oeBvT75V2>E)9~t|)kW)`kHZ1uR?R}@zzUfbYW@&?|c=-4IJPyUu zro@x>dU;7OQ*eJle2JRx%GuBTs-r9EPw3F!Dw-_k@xIEDFVb`|(+9k_-s1}2M@^{` z4yYp~mH@w*MW+vbpUo%w<1EkQ|I1faW>|fpqvG}H@==)uM!rA1uB0wl_DN*RGl48K zo4}?(^?+tzfqVw4g^D%CQq^Ry#RziddRwmCm^UhJnNB}p*Ch2JO4=R2c(%S+g>}{~ zV5)j$KxE~J^*!yHj8@khz)vAVAn%?!KLbiGQ0-%9;d<%u6#+XV&hW^|)$xwI=+~et zv}C4wG`ROKAV(Q|Bfess9D)HT_6&^oFT5?O?ANBzBdm)Ft+A$*9Q975`6(>6+RFi4 z|IjZ>(Bdbc)IfqVCEGvI27T~mqAjm|(d+u14!C9O?QOc3I@WQSVWHYE`tiH9E#bE`3zi zi*6>`hwYcZ?d2t9xLhB=L$$k+IAr;T^2FNZ>nV32mxVU(X1JAZ4C{bG3*(s{rSQ6{ z?4aX*=rqA$*wfa#%B?ClohXTtagb(IRMoqL5Y96OD;wkAZSh%(Od@WwN#}=A<=W-> zf#lEEcPPbHP9Udg(oQzQ_Hni&5-xr=<151#)*8*x3_|v>U^SzckidOoKS7Hf@dp>c znrd4#18sA^hn~w_dcO$Qjb1N7X@Qny(FH8}M6&mJ@s8BpB~(^;xy0mY{Iz=g^(((T zj}LH!m!Sub(a*-7FHQn@sS7vPJF@4~#z4+4fvRJ@ z4=FkaRw4iqdwr_LeYt)j%b?nbs5KM*?!}OYI{oYfve2Bccn0Lnry++R?^D;j_n6YR0^*0ou{jzZ9d??3>wM!gUN;UD?V+is!M^0no( zrgv@h{VAJ8WQIyPX3|nH#Xs*WX0W?WDNFa%`aZ8=xKZl2)brov=jl=Yk7xs_(2Ea?F9K>8pd`$CVow22I1x8OiQOB-)9^f-DQ+U^O;f{uLSAUqw? zR_NnDEsR);seAi4MIP(JEHUq)yVeKApJs_BOHl|@eT;DD8AIy4_K9O@O)Xw@dE}?^ z?APUWWP*q3K)>+&iF~TfNeJTuNN`wA4M8sOtH|ufd#iL7=(Va!xySl*?*ZcoA(ZE; z%MFvrX=5Z=P5@!8!2%?Y0!6n*GxGWjy=~Lkm@!@QZKOx{s)VwNBt@V@R8q4i)h%Ah zXR2{r>Iy%br;tQqQ(PZ-jJ%8y{<3CD3Ur^^8;9NVcsJQRl9{WX1!Tm`h_dvE4x<bn-GN>tjxn`qgJg9anXpr?6%!%vhU5`K|ID@M?_zELGyYpG1rt0s_N)cq`p zW7Wy@C{FX4iTX?avPdyOr+TdbG0sushz<9W+EfMdpz$r=g>%axyYwuxl9%mg!9|A3qDtH z+aU7Aft@p%tFQO=6_t+~C{FnBkj|YEsL6zqP-srI&E^)9lmBb`?_Y42M-TT3E!f#| z+06XDJ*a&XOflmsrRu{H1~8SG^);HSrX8#Dz^JmCWML26_SiOCR3OACMmvJ^RcLxk z7x@CluIOw9S#CO{oyPYETbqh}SVNfPsQ&c=OpIGy_nek52DiE#Mk9YFXr;bve4e*q z*gqmlSQt&N*g$!u>({U&Pc2Sz8d;FnxJl8WQSe^8(T=RyFz;`RdeajEF=4Kb0=daQ zlaR^2$=S)nS+F`K8USGZUjWBIIKR9g6Ag1O;T~RMKP>wg8Wd!}t1ut`$q&N63PP~G z&>x9BHIc`@C;iW5{@fgpJ|NPa0;Zomdlugig!|~Dk1%uQOw67=TS>B?Jb4l~-gqN! zzWHVh8#W9>h73WA7A?@AK?BtnrEiK(cA1al@>6*Ph?hstPu6pj#Mk}%&&~ZSuf5(( z)pLR7yZ#L@FYw(wA#;MXiFG?jdU#obleRS6*1axrQ@3M+70BMS$51>zWeOf2KNL;P z(f4EGKi49ceMh9^{mg4O6UbhtBSwrHN5Qx{<`kRePQkb54gJ{p*}vdLE0Dd%(D9fu zZ3@Q6bu`Cc{_OMk{!$*Q#g0bbzMahe@^TXeF!~L;3-!Ft;Syfp2BG2_ReV~F8TLPFx~ zS@5_8WDjYIq2nIIlqrv6Xb-E_U+^4Ooy<|2Z+Sd{XKO+b$GhLAPkY3|vmfT-u{U?B z__AS_#|_}p%%C3iaB25q#94vtZR5s}hQDJ(58a#OB=zYVzsE8!iYelEV4HrXO{wp{!Oy08N?W^rx6Q0}y|_3~ zDLHT@(AMu>fy2Arca$h6f$Lm90pX6?1u?_#LO$oMQ#xnxmNiU=2NA{=^ydwsiP`Rsedvq;Gjcb947be z>fjsEW(+28yU5+|F6daV1&`DmUQ}>=BQcqJSLQ>i`+Ja`Jin|2USLKM+1cJrlb8A`VE5xu^3c_Kpm{U`}967Mea z=i-2XT>>bH-gSAubm@}PyDk#!BF!#9lY~pZ5|BJ3Bt(I>p`oE>wJ*}&^n9c@_q471 zqu63}zx{J@`M0U9@KPS@@h^O*A|Cpi%A~D-AEXU*dq`-B9HBCsCU|DSE9go)yxf1Mjq^Or67-!Xbq3jDA44n+X#oci?~tPomgLi#?6MUVA^Ay3o= zefxLE%R?W-28Nw{cP2L8{Q^Lrr}7GS}^5sP1Li8)Fe{+T85gw=>?j zn;8Al zXH1le<7cUws79Dsvfjw44xU{27W&kY)(2wVi^dxdk}t&e6W@6W-wupN*P5OvDzsuU z+u+$nGtr~2FD{-ui=fCN3Ld>-WXJ82N8VV70gX+q+NCFY-1r6_oGCPH!L0XpW6`K~ zC@DCASO4#GQ<`>o@!gjY6XvewwyrnbfEVt27Q0QMS|5F}Rj1Vewg$14+Y}#}{G}8g z@Hw~XZM>n`)8i5RXYxR`q2uP(6+JsO$A|}JL%PR5KirJlCr4xXG)q1H_WFNuTT|l{ zZ`ZE9&?EK@JTOCgz;~GS-fk?o&mx#4<<)7?mSI2f*@HbNGSISFFyCFq#Q_%wDmw=% zG0=ViAALK=fqy+?{(?r9jYo$2qHp)PfP>nK%Up_TlXDyfSnC+G3X6!Q#5{ra6;^&o z*@a^ScUoHk>MGd77X=x|v2WiV>^QauyH2d9$`4yW&N>_G_l%1_QV!qFg!M-YE3nlZ z>em`CPgr7hmg^;;8W&NHrXZ(msn~)XePxIxZF2HPEdTB#@+ldmjkjkYk`I5Ooo0K> zv6Fz6UT||jxn4ZcsMT=Hr`LBz_Gtyy5M6247gFAj3O<3yu)GukLKJ`(5aF72@JR`8gm9 zs3JaT1lUul3^_aiVGKv49SoFoJk5oC9hi%$8qhLS{UmK7((PhI?|wm{F=vRo`S3Al6Tt6Mewg}G4UijE1eJug$B;ojIC_z)TH6LF z^+8gG7}&*tWo^eoc_cjVUp1)brix)|OcL-p}e1 zYJ)E&NY69XGVN=;`$aqSkB&r$FHgRE`s1#-FCn%l2=(eV;N2^QRP|-3eKYbi8q!ed zl#YE{OfYf7CaCUZ%-7NvM3P%=y6AT2Z13h<1vm6;h86E~yJYK^NJ#C2_94QP0_^kk9R!(`3u2a~1;sRoVLy(@8tHR|^^AY!>T_APw zr)MLGZC-qAJw70eAg1MgQ_-UqVSUnJw5c0{WA4Fy4L`&G*r5_X-;RsJn!)4PPQulO ziJ`=K6OT-$F{IN=JRw|U}`(Ab#tjk z%agyGXMWjhic$*?-`4}|I0{&ows5b~3FG^Gh9yble~l6(AKzrD$GuZjJyJL>#K^b} z?Jxpk8n48{L#)TDUC6k%wfXX5iQ`|p85+`-adY}~DvG$-pm`wSF5}{WivyLD1Cco7k&Zg3 z;a@RBbIw;UU`+dYj+;W}!^c#aJ=2-F~pl3Za zH36D_;kRH#ABsbGTBFI#?eI*DKDgc{Wn7Cz7+=2%(_4&e+Jr|owRl|atpdjju`37T z%a7kuO^`E5^bF|lFsJF9jb(3tex%x)U<4RzAp3xdb(gDuS8S2x%8i>3jwS(5ZyxIC zU5Zax@;#*_ae2Eqa7{Q+NU)%Qs$!A}u3zGlMX7e#vaBMy*a?h=aWNb~kll+qc4hG< z3o>o11+{50ZV$j*lghXXz2+Ry{Xq8(=`ZrGHe?9G`}6Up#Ay}wD|hxi&!R#~wtFDW zlfVG6vs3|V7#_?pxg^zP{&*Y^a7HB81+)`eo{JYRA~Q2nNv~&RWho#{`i@pNB9iMG zXqWzE-LGUmlK!RVC9&m^c=EW+m2e;xTUM;bDY=xx%DB1lx1dR1ULslD%GdE#^W0+V zI7VCO-wz3weqe7}Hmj!kg^nkEBqXXk?rGdv#ZNwQ6#J4dAnDvWoIRF^J=?b9n0ftW zT*^u~(<0H{pdn9U6*8@{5vTxK=Bdvx7IXel;IVHo8ju)bjL6vaF{bcAU^rqsc0kjJ?~$xRR{_?^=QeoMV$LBMwR7J1AT^!!cF}-C*-8! z@PU1}a6Sp=&YVHwp*`5X`v4Zf$C* z{8W9IpR{dge8PHMtjf99$W-F{__$${bPX{lf5r*!Bt==T1wFmY z!je0FNkVZ~?l`!NivunWRC*4SfhJvB+P7@|Z>a>VCQ6yieCah%Jzhzn`_F;6{1*OIpn*v4dC{&>kj+gT;2v&l zrnz}xy>BfXbMwXpf=5+|xW{>AQow|&LQMI(IQ+-pfJm>4MDWi){|u3)+_7Vay4X}M zI+eMlNs}h1RjU^I^y#Cd(*=mBH0E2u6WLH z4bU19jWH*`)Kxo=#uLZS;MGs|t04K4l(-Ea$g_}++8yxRi&N02iS7DG%F=yAY%_3k z7jp|Tcdj)_u(wC&=E2-p&4jOM z*y?O!@0$elHr=4S@7{T=#Y6r+F!Lpex@1V&dSPE^O1EM zNFNkF>p8_zS=K0H7e4>oR5hp838ZG^AS_$N6~tsw)gA>EO!pAzdKG57nb))WGF0m} zD=MJgc=lx(2V&$sU!VM-0EF~e4>w*8O+htfjvPQbc~zJ5qxBe*@`Q(C=W)ZyY%+et zm&RcNvzP75air()?3tf|ve_ZwnvAit#D%PXms0i zg`l~|C@4ItmU0ofV=XEu8_YIFx7f{>J0d2y@qNAafv5ai9kyOwc z)yitSGE%#OTpaklI3OT94|P+D-2)dX`9xfRlI_%?&11Gi<}x{8ug0 zB`X0AR1zbd@}~#kB*vv$0A`>TxRR1l6nj_(?~4^O7|TfaX912is3o>M8nn>;#+&@~ zAb&y$8VGCsXcv=NX~IvkDovqSJeE&(O1R5hpBxa7N5B{X6y-gGIza?zPfku&Dq^y! z8bsXEM+DrG%QS<6f>4EOLo|r4`;YV;iKA_uMr`pDTOOCWdJdF$_^JD4PCCh=CXb%V z9Q$i{e7Up|E^VoQZ-k8G9sk9onIe0wO%R22TA)ruJv46K3_0J<#o{d@)^3q9HcNri zSMt>PN)Do`N*no|P`?|jmwU|)cy`GWoH_U-c5?&h`%Rmyq~&?=dig7OKKOOKe0!v6 zKGV7A{CFX_KmGV|UagA1DpSvU(*KnyN(P1Um638~Y|yeXLaU?`qiDAZ72e1l7UD1I z?TsD>_U)wV$jBun-ne1db6C>vBzEuIiS66BV$<$pmTb=Kzzbvk4|P`m8!e?Sc1GnR zU}xvkJQOio&Y*xQ(&n8ywB=fm5s3sHj2Rdn3}zoxE`NO_7ZM zJu(jOow4K^RI>$IG^vexOO_Wih1nwXSmkt_?%l_&Xvr$-E&((eD$)Guu>; z!YA9*2KJ{E6rW9Vilj4}wu3MxCWIRx_H7`SgS&B^>W>-(*3I7+sLHY}eNgg|HuqrO z+UT~lra|ZwW9B#C3rgfG%`nBBON&^JZU%(chcD}CJ#KVdoriV)Q1uAqoNTn0F3*jd z-9A2~%_|gj$UE^D`W)Jo{yL2=$7Q=X;Nn2#Y`sbv^+>2QyWjmNxSwRz!CQ^Ui% zF1qjz9D7wTQ@5_hlkJ9^cc2|p=OFK@5W;KU?v8b#Pe6zhbwy5N+Io6V!?C9(;k*9? zV+17^EG7YQ%RWTT__BCtiG4l1RpfZFKAorQzUt?F7#A6nrkHnMFo?Nhe^8{A`zSAz8@ubSk~H zV#{J!j0b%s#EY8|VtcYok<;`fk3{u}Kfz(S?33C`Ra`%APRKM7;9`|b5aB#4aQg7eBz0$WH z9v-gd8fj;FLNy~7C2dyd*%Ys>S%Xwc%(~G}&x5k* z;X!FxIgui~7PlBZ&ab{?+ms%Z{FOGB6AZ?jk%_fSHOL->yPtU(gWK0c5$O=x1i=38 z3zO~`(l6r)U6B9)KmbWZK~$SL@{e7^FqAeIS_1+C6x||0UtG-JGOJ?Myu>>q&Yn7j zG`g!c%@)hPUU9E&$_I|quId?XKr-uOEx(s6ji03WfW%>p` znvjTs``RLm_0auTXv@5K9tTXxO8BiQ^cQMLW%_%18+DidFZCCRYn!?VZ^(ZMllps8 z<(4w0ddfzXgvsWWS*qNmhD3hXa?{S^)XCGx4r+jQ)Op^%+dzyj&cTHvzu?Oc=i!@u zBEpk^9Y-_KvZZWu+8V`Qswc_S-UQ?2_fB{y!XDH6*?^D z1WsQpLTvQ_HJ?bH(tistrI;_ijc#03wSgrK>%%^%=O6L+G^rhu_gikh1zlSBt3E3E zNqO=VXP(FLONFRbtr~x?@+onyrN@t9&#^3YsU52Nf#e}{7ha$qesP$LuCI1`dGgD{ zl($i*JRI11*c5Ect;PI?3D8Z%%Y2L3&-KY4`Q4Rti7Th$=>t;Jmpmr#7 zPGX(@l`+Tai7~xwL3UBc@zb#ca|~xMzSW`-=MX?Zd%NuW+M2>M8wj$CN#IT5^D%Yj zcntk~G7dP1BCANh>QBvDEgaIX;O50~2j4<`wQMllruV|Q#n0gNwacjsZWqp65lvxe z*T_j5CXK+Xo%%=4pXOz$2wGU#m>q2Dp`N5Sp^pz#;nhs%ul>**vVFzRl;lY zdR6^fNTm+@?>zau_eR9=6yN`$6UawzOv@q6?R;91Nd%LYfYD^)8$zNgc3vTH3 z`ZzrE>FZd&Z5?)h|RkNslsx2TiQSI9-5oQTgO9Fi#;ZUnhgx!Ao#JDH2Ui&QU?MEa9xpO<%T z7>Ip=VQFrjNavHmO@d55*?e+5c!8a}C-P|*auF?05#Niwe2mA_2PFhJis@g%J3dM% z#V>B^%Q3l)uwsTunj(f5dHJH4;rYDKGmrOm*X6>%3)H0{`qu?yPvb>> z3g{x(DFC?usv^A}5)y(iUi>Jw0HOkN>HcAFOFHRC+SdK)kI2hwQ6~Y-ipgJf38-l^ z!~7dy0^@#5y4(mJn#&^{V$7do4F}`)=voTcHPeuhfYp0VE`r<+pr2{pvCdQ8?ahTB z-onFA{2jAq{{!D8S=3?ph1NvOfO|2xp(#anrka;kzcXi|0cKxRt3gREF-P9V>I0m) zO#M~&)4g9WH>z<%^hz3`)y>F}e~of%>cTk6&=$#Y9cGkgPH2j97K=(?kN zTlA@Egsgo1E$mdU9dTn*V{<^IvrTHlp8@v_MlF9^X_tfz8_Y#{@^@6|<$tqT^4Shb zep@8HrHox)d~62iLjw&}E9L}zvF4-r2Sm0;19_qO=I!OUXuPVwir*zC@Xzs&VahX4 z;i0jw<1~*8`qgVbG-D}+L4g_7LN7zkhU}}Hg+IL}9lNsoL(6;YR{NxP`nLGuxFH?lW>41Fd zT)a%0QiG`q!QSLvb!JvMRp5AuYIQr>Y?yBc7OzMDH-=!@h&M5$vB9wn&P`JncEcPi zGW6@xL3fmK;zGx|2KLqVZ|?@UiNj~U%^u|nbbOZ=`rJRVu0QzJRfo#%yiT)zFp25gP zj8~{+;3!c}K*O8(Nj?4%iN8<}0=Qhj2J%Qc@AyJ>Z!dznyx)$JVgj>xQXEmF?v%v3 z*mTQs*OO_*mPd?${<3bDQvd??irN98xsbdRiTFH*7t)=XZ`f(nw<~ISGN>dtpAjxn zS}~8dEWRb4R8k}%{Y?_?q1@Q;rhhFe@*#B3BkgigUomPDcqaUhY7&p}J(O;nikBc& zmHZw4NM918bg+pIFLmNaEPQE>S>HC2vye&pVR(D{0_FVQ(@(;0 z`V8C=TNiGHY1s1qOicUus3}Pg^l0iR8Y8OO^cawc3aGbKMvVi#39jZ%bC_lGNzzH3 zEK=p#T@xL82V=clK$`ep{O7CcxVu+t1bLSr{p?Tpo>b zFro%Vq~{BAL=?a<>sPKu_XqkQjCeg&n@!B zKmOXE7s>Ku&HFFng-=b0b;RB1)z~8Ct<$)>8>eNYzX%_T16tv(K0%ndhSL6-8}QT% zK6q*JJ!o1r0Qo70@X?&<&^2oDmtlzX@kD`pQw-}Cgx9}in*H&Z@LE;Om@ok0K6&_Q zjWHIRJl2fX>zbFkp-{Ejf0`kik< z_~P9Pv|s+*-%;(=85nXyeO@$s7OUQzg80)$#F*Q=Ai%da?&uMSIU6{jGS}nr=lt;U zaC8Lfyf}k^cSoE)MexQd zYs>3cjJqMiaC(Oz(h^{LOTdVQu{hOZHu{9;WApZzm?*%r6SMiqX@X2qrVw8Y>`zJL z|2Tr^Q`fY?)HOX37nY0_DYoFL&URX_0ly8n zF>#+QFsA78CNHl5WS2C)_3p;9x}&)`+>?6LEyff5ZhHegU+ZSuhx4c6h3|%AUgt)Zs19#_=+tEqW?V?;5?f3ze8OU{ zx^D%A_r#*M1V8mmQ|QB^gyn1)2imMXDsa41?T0s7^h7s@dm&a}=+YGgZ#nk7IrcK0 zk3B@Lz+JU18*A27R)+U0?%#@NFH{L_!;L7rI-eTBalZ=FTDQifQ%f-?$u7K%3Kuvz zGZeo$2JxGXL_K2@u#PIJT6Z3co&?%$XU+8(y08jHHkpgw6t!QweInxX9SfV*f5=t$ zRma?1J}wSa90vp>PNfuk3ZV6$|Z6^MB9}VHNyn(mh|*iz&kA! zM{Ie7P65}&&Q}1GsG~6ecGFN*S(twXwRkX$qfAS2pUM9 z+#Dd}KpG4Hom`+NK?QDp2oJJKa@FR+1!gYwDH1*TOixYu)c4MM6C)CsNU5%2IEz*E zOI@xfBYjK&84a9!vYp(xu@Xdy_7a|`@Gs(SI4hOPc--CKDFLHx{KyMwFPFLY9FX=C z6%}HXIs^icBEc@|Vnm`{RK^HcCT%Uixk##u7-X8^VV=y*pL#Q$49(vtAb^N--2 zpm2U;C9#j$WtXuNM=w2q@T!JNFdZu8>N=@BIki~eTT`qWH{yW~z-v3`KDPm{Jh8zP zU~Prtfq^Z}K(3{3vf*nO?>aZ*`4PMD{L&xkaTae+z87y>XQAGMFz&SnP)qRTQjE}R z)Ze-;mU475Gw;YRIf_C}ei7|EBalG5-aEOuYg2O=rxOR`*RUfx z;X`a;NYbYN;vqia;cWdR`^E>h?LCgMH#v%N{k-B;+`-4zgk$341Gsr>(5C6raddwX zx4gF>`3$;los3OF`0=mt?D(&BR4cnX)w48FZZ|hy+%)DHZ2NXHw$bOz>bdv}A8TV> z166)NGxWWY!bRI9)=zltvC(+VI)T-@=BZmqG@cyW9Z$^P$WZpTC!F=SZn&>kee$^o zZSQ#koql*8>PmxSU*ef@dUM4pbM(Z+gQHac6S`z>FX4B|{%yY)k%D3f&RfTKy}LNz z;y`8QfQG*+2aQFESh$nGmYr!6IRb6W=hy54ZJeIyanqx~!O=FpAY@biOHmUL?^HtC z$a3<)XAq@-pgkJlZi3Kr>_BKy>GJf3-gY!NIunx4&{G5L1et3O$G2@iEalSF^#s`+ z^H4tFOEIZcIW{6nNq#x+uD8C4Nqd?*O7iB%W8&ucQj@1(fh9p@+r1YRn;xq|+!s^W{T zH*(YOs1qQa<9_PSE3a=zP-K~jf)y0TDD1m$_L+p{YaGuG#2+NUez25m!rsQi?HoV! zONs6Bc5&c$)Hr7X>1nz-5ENEb0rVBztZ$0txm8@}fJDloJq47FQZ**FekF z5~{%h(U&gZ`Z-Fh=MXABTIfOfGIt!}<$Kw*&(JNvg)ATiAaCwV{iCF(vA+-eDS;U4 z?*(7l)d-l2qDb46)H)8olhmH)}M|H z(RD4-#wSmnRG|8lnuLNBQualQHjbR_XQJQg|-?FW!1@y@NavN?z+F$5xz{9E1rEdJf_3;cV|A}E#T^w%c5-K}`w-n$STVfmf)_BN@T zVWa>*<4a!h5#z><=g=@w#GS6n@n&5#2rO}^xy&$CS-z>B;vr#P-X`H2Kh~0MBCXg{k_E#O@WwQpJb zBk~9^TOVlm3ebW-Vesc~wFiH@rA>>%qR^*4@aYLDt|vNEqFlZ^jB?noOv&W8UN-6A z8G#A&rt)OKYgl{Cfaf*H9u(OP_mAz5jL%-ff-T(O+m?Xy4|e45qs0Way%E*+Y{xN+ z7izX2fCujzhN!B>#-70|OR7MAYbf55E}>c8eM6!!VbMIaSn?+3t=wakoADYwa2y`E zyANt9>3oC#C7w+%;Xg0o)BpYl%eGll5QKhr55Sqv-$w$gB&X;M}&FUqg*=AVoL(a)VnQ!MwUrm^*8pLw{@8_aQto>Som7sW7<= z)jO~W9(wC_v|hHrc|C3$k8vaV^1H&Ocplw5h7CK|2WcX2?Ti|nr}bPdbhwO*11=6! zUJlp~=W9yog}uw1f~)s;iY+&g@MnIQCZ8{hId5)OWb#G+mn;GW644qlFLegB4T+vb-qld!os_MkgKOZ69a3P3n<=aaSnKZqCOq8uK1YK;1=@T*C+zH(qMvsmBii zZ#Tx%Mp{nQKL+%rY_F}Ep>Cbm=;MLOHAwY3Krjrd2`mkCFiiH30M~SG=|EG zs4P==E|yl^ff0M+ZDQHiqfyJ-(1zf-6X;&0TXFsrC7q(xNs6e>v#~WJ8y(9#4|-2R z!h_?~0?vjAm&3(@Ys>)wd$TD?ok^fQjUc<|0k2O*062zW%3nvY{Tq)@q zc+Tglp0FvU(+OmVO+Z5qhKr=OsDjBBo#qloTcBhw>OU7iS)`ejWV){>f(TYu@$;s- z7kAY3^@1Pm8WiakfW0a=21HUlh;F@1y;52$HojhLmQ9U->LQ7s;!#MT-JP2VlnSPw z01a6T5PkHXl!6bS?M8oMnKZ&7s=Og@5ryv6Bq9WFp8w#RI=n_w1kRv>jU#KpZ^_1Nem-*c|Ab^wf z(foY23jwe~o;(ns{nDjNN`hS^+eLMZ#1Q~SB-TaWdey2`70@p8i98Y}ww7GC2XG}! z`hyF+`Q4hsP#A)E0jEl9rr#G* zMsLzy@AX#si+UxU zhr|=f=ziP)=q{JjGH#$%IpS_Q_#apvM*x*TzurVP(z&709nazYzJ{)Ee^WZ)Rd8f1 z9-bA02hx&}nvtVUU`bn5sUEC0L*(RFX-3L4o>1I8sz2W4=~DdsId&%Bg)Wq}ll-L( zBm4gy@AaixG}0yUrM}gxS6A=1J+}aZOnxD>WfM;BYPfL04FjjdV;~Ec6EKpOq{$2F zg8Qa5#?X|D$RMsF>UF}xtD|xKx;TFPIPxBtf$rmtjUX?k4Q|>UPd*ueS+i#0<~#1k z?T z4DB=)Q#aVci@+E7k*Y@;RdXx?;6;*8y^7S0QweZ8N>PFrv97r56$~=#2e!(?BMLA7 z-)TgDOhC$+`V{oUj2_eRSoc`0dAF^d-t0NpdS)7;>dDT$HlFC(ZyqkS8HQOaAI0?2 zR5N&w#G?MsVr080Cx9uBdE1#F?y5UC<%}B|i+TTd9=ET48#A{}wNvY0$vG}^E++RK zjaVKp`&EXL&{HOq;NHZQnD@;TOt)8IVV8N&@JYD0!(VV`j2x*euc-{}?&Q)*Uiem~ zAvof2QKhpq?3>RFoQ(N)mv<)YTZDxCUWl&1&4@DT{Y?WaV!W6L^lpTwCKTY##4oX6 z!yJx1C-ad+<$SQT?^77w%K5gPmyeSk7!c%SoH8$mHV7y9W1`1wc~NMmje{DOVEcEgg|!~t0- z3CR9^p)W2GXcxWbzQu)z&PhTj!FEx#;>r2kUMI3F;;zKVqF6xoEH1v}GS34=K9s?8 zrv_JFN*cQ3R533QWq1uA4+Ib}Z|Coe+O)d{`zZbD^*HH>6kDK4LS59q$U06g%H%7h z5^0kli)6Y%-IY;!B{lkO;)=w*7;kPIh^b9oC}azYLU`dS?RqWXfs3ANNAh(1%+ufI?mq)=>gwO0*wTA{HACEK~-5J+i%rB}bQRZJ@I2Qo6Q zr9E;8^a+^0ckfTs!JdZ1#6$(G3#cYQS+i!%6u?%eP93FVTO^~UZM7u3z2v%t%N!$d zq%Y`xaHaFkABff0v?_W$3Lq{<+TEGCT>I#JbbCs>%Wsh0Sdno3NdCGH$nTFB{X3*} zgUe=)#1XsLBN)|diFCKgNZ&NktNWMUxY2Y-bWp)JK;s{??C_FJaX> z4Mq>F!UkUN8vo^P#13yQ*}@|=nE-Tcbzg&~S2o-{jvqwb$YzRuQ7rL)5k&t+jT)giaSv*@jKeWDf&UFO`enR~QO;vuvBjHOG_8s>Vhft+nL)ETwDRjP@ zA8OVNQ*;;nybBGQw#JboM^rs^U42t7z&|h;C9G>w62HyDd{uvljI4*E{ku`Wc{?0C zMu47uSjv%mE&};2SDcor%9Q&tJOg=}gS@L>zdmqaFKV!^=gyr|?IM5VL{asSU=$ud zjG7TmQF!Wdg?PIC7AGnxEd_zN#gjkl?D9JW67DPra96)z_{oOB9lS=j`Aybuyr4~@Bd zYy3>wByw;R=3Ky`(?=-n#2;)=e}n|pL>;Epz>!0Oa(HBS-jUBvV>IJ`}v*9}t4N;SmURfQ5WOw8Gw*r z@|ZipL6S zwLiVL{`r@WdU<3)EsGohbqu%u;W0iFWg?R7UljN$z31nsqQ#HW!nbD~gIvU^7t|U+&h15glFacQNfgu7o3FsyDlz6$^P>{GeIoWDn zlex_v)RLT?jYm<_^6|$XtNE{2uU<<3dE>^775H`~`3oKL2)#>|EKy~R88b%ZCv&QV zrKhK>IavU9(Z4R2oC@7iACbas)vA@6dj;Uppq|jI0YD9eYFqquxYSS2+0N=I;V$$0 za6sC+EOWW`(S1pNzomc3I2J!iC*Ys}m(teK{^Boyw9)@_RI&0qB=IzWoR^oU`nTjG zV?!Qmqx07NOatDMrx+LYAU586vIv)W$C0vYz3%z1X ze0k*i#5x}d)A?BQL6;%)h_P0!Bw+${3N2zwdf|_*ljJXamU!0nmwY6?gbV+~X!I9E83)1mp{*jDz>eCCH!s0nn|#lH9IQgBz>F zLEP@`iK6hj@O)RSmyq@MIq)7nUa7E>xk@h7E*y3P%guoA zvhQ)wEd>6#m()gJ!AI{PVA?~-8ud8*o_s}Z1PH%Fcu{)6n7-TyIDnGH>rqg*q2inH z0N<~L$6p5{zwZ!u&3FfA2+)hE9UhLHhljv->l)<0N%g4FofV*6s}_H!4*d*|(Kn-@ zMSFP7`yA&})6_VS&4KLa|AxSi79jiOMeyi1MAcQK_We^Y0)Od&qM%TCE?JLc{C)AC}1Rt3fK!x+R}E}Uf#a*_>G;j8zu|~%9yU*1Ykt`8M$Hi~?6?N>+0~`bm7W0~fBdnt(W6+p z9XD>A^SeRMP3c&leJ^+#*ij9%3@W;#QnUj4T6^zA?z~2`TyKJm@hR=myQlWgn=fQ! zcSb*6tt-SMC!3%;uiX`rGpJi#&D>yX9rtGL=6=JjIX^G)d=_-?%xf>!UUSTEr~7dk z=-#zoGwFiXw7I=`HC?g$!dNuDw;tu*t^xgc*MXhI!~!qeoyNWKMt}A`=l@T$QD*(Q?mz?~B4B<59$2$A?datTjNt(p4%y%TpZ45{dW{W0I zaFD3deVWEm@1Z^5{EHi8z(WSrgJTz zEn!YF2T3-)Yn||l4+=QlN3UzBR&GN_=0}ky#^(28AGx>tIBdU<;G=Uj!nC> z>;EDCNa(X3u`?YayY>nNj=mXqyAg@s-0p0<-dyCMdSrF41DFyU)xv@dvC$dOGbo9 zX5I{ejnBuX6`f~qJll8ueq1yut={JT)hsOf<<6{?mJ?GgmX<#^Z{;?xHj?Bqk9JGidumh1BlAo+oVk0gp_6^(RcL-WDANmZ%~j#2&5 zhAO=Q(%sT{KGbKz>-JsKA*G=mpHPKG*(ctt9bGyfuiIV1RVM^fs+7^Z9Tl8$cKh#2 z(S5yeDNc;n(}~(EHv+MF=0}E_GM;3-GX9vQxyET@tqr5AvjtW(ZFHN{rH~-f3}{>7 zNn$Nulb13W)hG00O$~}LM(6xqUzH|NEq*6FDymJDyRpPS*dgsWH~WfvS9e9?R$buF zr>O7wL3^i}aB-A0oBFIqDohL)jBWm zGG?SW6WT`+k4mW5KRv&?57!4whZV<<1DvYn+wE}y6nyveLO~eTj;3#*w+1Z@rLAe~ zhK>X0U#iTJn$9an_jFtkF1}u_;CEI5EQ!OtO5Ix`vTW5pZ@EB%h=Dtz1?YRoJ3h!8 zZJF3gb?3|gw#d{Lt5i;og!r-kd9$^cnQ2>d#O_EE10~XPbNNPfg*;pL^{``bWRZXH z*GfB+Z(|h3(aqDrQ7&RnSgBxQ0`PuFFUYL_W*RWI+JYJ2>@m5qOOd~6H=T65$Dkz$ z5T7;k<+2=Bn>=nJ`Sh`7Ma-Jz^Amy)@_6-?<4^=i$G2nhIB_mN%(b*1iokrcFBW*U zmAg^clA8y6uuBdcf^$fndKAe7Na>mi;VGj{t6{G)_}sI6Ipf)AW{oA}*j}n@dO{Cs zo%^?RKqV)MtfRLY-s|tuH9JyjxwH?{;y+iXJu8R*zC}0r@|iyLXl?bJ;&whqj)$ZL z?GIOwH|F=JQ*u(E64UedTqf@&UC)mF&v!}B$PRm^Y(LE(=Q@2Uw^#fMO0>mj<@wt! zwPlH)1hN_LH}u6Kx`MsPTAxl-F`aI*#WDWWpY+EZo~qb)7HX|t@Ncd7;@x1VCY{aF z-hQ60-RYN`F8(YnxdKA{yfAm}uE(_flzH};+R2&`^kz6w0^6k66|zEJ|De|EWE^=K ze53b+74Df6{ubPFWc}sgQkP}z#?gsdippqz8WNI;DZ?VxeeDp8F1zg#wAO8BIUJ!9 z+24L5;R5WIk!$z=?~zK!kk0GyijQQEFGc>(ga0{LH588WIF}xVzxWE!-I$PvU4mMO z;rU-Xi5I|F8Xs^W1+GXDJV(rJ6@ASWCFyg$le>PjPc-klMbAw>Ku~NFhu1CRp07Y6 zzUmE+B_}qeSuhh^-_Pw3vuBUHe`F2`e4B^nyr&3oGfsuUWF*jBHr%=`Qy78{qyN&Rd|-RC{i`5XSZS?p4YdA> z<=nr2kGt`XoTg(ZDGV8^OHXAam0Sz2ZO-?#d*Bh3)rD?;JD>?HYEN^$i`ue!czsXu z!EU?1I^Syiz4H|f^id!q^iAtKL_*%RLX&CeybXgn;<#RG27-eEu33A_g z){e8eaBS6g9uNSWArCt$NjFPeNe?r`0shmtU#9t5WvPr3>Bf#vky;=3F-XsYjx}9D zJ_yeHbD9(OydaU!3~e+c!7m0E_%c(;ZQ-u3D$EVd{TG-}YtHZXyImcJ+s@gW@|RGL z4m-4K56adoXcW97zVrI(!uEOgY|3u`&TdCi1y zCBm7u4B6s&EiZL@Bn?|J_hi|=${M0dzZFSfY=5V(8_2pGvM{*tU6BjUy)%3x0R?RD z0dtP>-Ty641<4mL3BNjqveSScDx+m1i3?NW>@}B5{Lj5 z(6#NaTb(O$G%nX?u7Cw;t+aE1)uQP4bH0`YSA1GMsduv)WCH*RF%rvW_@|Eyh!0zZ zwd8Up@9f6>YyXq0EA>$C879Wjdlvm*pL&%L?KPT?T{Jg=em*z%sHq|Ug5m-Tg}Bks_L7~*hxyIKJ}Civ z=~3Zv17q7M%f}<{eLya0;?u$hux_Gq+VXq1hV;LJJXl_Ne<%ziG0?s@z>L!{K<#}P6@i0t5fkTJMwc^5$}UNp)6PoDm4}$ zo{CC3qkov)-f~m+&K!;drr@sXvw;m9`lQyJ*&}C!gMXQ5A{L5|#jZ%6-M6rcMK3we zJ*@#vi2FYUXX;o+gCt$g5)j=E8-L*OVgo(_-Cqt)~t+bMmA zas=NeUF<~h34Ink>8tj6Do|KT@f55p!`PfJ4vj}sQR zurnK=Aq*a~svf)TAW*G#p<9Ynq|G~&2gLNLC=s%|dIh2QrGi@Gip;gMqnw4ljHV!5 z|E*%W1&JdSXZbTBA+sTTs=ohRnPQ8mqC8;o4LM`Azg>k%?H>9WWo3x?-EUKdv`?B(N+c5HqxzX?VkhaxCD@v=M0lpo)16M9zmn&CX@Wsx`Zg7 z5bclL3MaMuq}N3F6-K?I9W2<0ws~gAh?nX~#7x1z0RH@JAE^#5*72+!Wvs~=>Pzcd zvVI~`Ckrn#%zu52k?bS?vonSqGDG%z;f(CDH7&nH+^pQRch^%z-SUm)l0br{qSI?0 zfxXOjiLh1qy_~f0s<%*r@d+9NWaD?w$I3H7+{<5L;!{O3S{iMRm#viRhMi>kfpD!csRf~|Jv8kuqy2c z=(z5DC*-C~zCE0>-JHYouZG!HQIRlYs+qEKhAi|%`7|MwVT%bg2IjD7;a)?xY_SUU zwk4|8UvI^?BFhfwX&Xt^g&2&|@b5Hw>g6E$_8lbd{NUasb5v5u`#zaj+`cqr`Y42C z78pgo9GIwk@?y_1pDvsS4VU6@&UNl{$DWVbx_G?3Sw(g>Xl>5opKmi z{Iqy`%-d2)*Eb94R}fqTPF%tWeS-f3%>gsi^8}&Sx-#@RZvO|Y|ECM?GA%=vqjkfuvqWpv@C$>fst|*-Y*F85vfM+_W0^ zdEy{U*VJ^T{Q63yNN>vsIrtRw>d>C1DvkB0Ux{<3JbfxBQX^b7IC%wd?N6fo`{BDp zf0#5eHa;m)&G^S+f~RKcqSUE6lgK&ul#;kvSf;rDn`V0w#(8=uBK5BXt6WW+Rd=N* zZA~bpD|O{D*Uzf;<(Y@nBNQ~xV|;^O?pTH*Tr%uv&;>tlOKr{tFp%&V0Tb<=m^c z$^EFtrJEBbJM!ww^7x6I~-poAUrp$mRe43Xhs zN+v&o-zqn$gVWLd4GUn@Z1E?S*4zIFOAqZsLW z*r;{YDe?7|Ogs3m8$^u(2t1DyGs3NTeKOZ5TJokVio{znV|pY$D9j1kd8q=qm)BPH zV;*80M(0&`m6l1x$yos^I5$=#Z<2?7Q{}rlSSi*}h|xZCXs`^fMrb4uW8J#4o-vHz zg}gFsxEB|L{nkW9{pJ4SnnpL-Jchg7<~Tb$`*?3Pgh4|?g}wG)%O%fXB{|1}dZLY>U3@}ZIYpwCW-#JKu+hnAq;?5a1 z?MtQYi{$Zpthv^y8p0H0asOau|d(TrD?O!{&rBr`?Are7iVY^`gA6DOKQ zM#MWdglG0I6`8l)!`G_-z0EkzC%;%YPk<>z@8cYmC&Sk|6g0rt$%`~1vOu2^))|o# zH`iOnoPpw*37G7Qe!RcD6wgF9Ajpxt@FqeN9<5hgPs z$ealfY(gA$Q9_wPDJ-|lL(0&>*0)yXaYh)Z6FMb!DR43Q!Sn20@{vB3?qMF$9Njg` znI`{9ELy$Hmr{-66?$5yt_QIeIa>wjn57{KfFsO)ZEwp3k#A#IOSgXp^%^PE`f!>Z zbCH`@2MiVK%eUWBPPlln#$Kt^9=wT~J@?}CD`rL(i1cB2AjzX|YQAr{bLTb=WF0>t zlKwJnQgkn$M~m}h!`S9A?DO=N5Uf{b^Oe9zeLkL5n#AhxEhzQ zzBkF8g~yVyGe<`~LduBmGd;|zwi-~r^7&a5S}oy`{pV>Ml1uxpMratGA1svfJ#%<5L?qcyvB*e zzz^!LK4Wbf`^w`>66woHXUf+2nwKEVZ{)Aj=q8XVdu%2XKBFA68tr9fGy3m~QrBbh z+&RR|PNj~DeyKDKn;#A6-S&K^?jUMeFQtsWL}v=Vtm54LeZQYj@@rkDdvD&_aL3(> zRa7Kph;R>*U4?BqVf=o>ij9y02ZeOB{xvv}vU{mRL1JMYdC<&Zi%aP?wYm+zL`ugJ#yY%F23@cS7$H1EVch`NTh(ZQW1-k(|EGl~7bK2(#Nh(YN`oRBQdI!lSQl zRL?1(LQZ#$bDiPJ%#(bnqowSVj7eR9o=JTOMN$bMCLVeax|&r;^GaxfNu)WtPy+DiF=sz?Ula5N(Hu&Ji`*(u50$s_ z_%llPdRlkZ7|}Ki-AOlkZrK;s#F9aMy&b21>U1wGW zXtniW@4r0x|MGiaOT78l21w-$X&M1l81Y_Pb|eZLo7C(GolT#H z+>r^Og`3Nk?+71{@C?>W_){tG8&WnBC~UTF5`RJkQuAszM_zEaaq_Jw_EU@{bv;RV zx{cJp_aI|hdgRapG~feTa&M2>yOH;Z!1PpUs=mFP$Ump^g zAE1l+kEqiTmZ8~o5DG@VvyMRj*3`wwulpU}pi=R)O*o&5o@|lLWG+NSO=ml21LNwH zs~uR=5MTGD@4@f96VWVQI@vL}tU!nPI!)~I-;kur z_Qwm{P*Il7>5^#OJx8XV7zZp@ULxfgwl(7@tDx>7X<08hRo5Y-KzsL=J-ai23k7_R z>tym(UWclzsmj(kL4Lz;;IpijK|;pu+g^~=G8~@cal5~jqJRUE2w13P-1`J%jh)l< zvz~t0Pl|m}f35l(8L10QAbOBm)cbagwyDyW$`miyPh{*g&X_mVrFo1g0oiDw%SScj zvcxXHo^Tk)xB;4$IL6|Iu%R6T-r3)<63$P(=C7H+TU6HGPxui#8yKBCBSxrIyG`ZH zlD$o1V(n{f=vjhkaLeUfo73bH#1?tZ^V_)flas>DroHapf;m=3GwD&qRm<%Wzo5sv zD}0|973a?bs%GTJJ1S=FA5Uk;SE(~KigEUf!Qimdns9kx9iFI8#G*sb-XBKO_e=L+5z`7h})76~Z|OnPsY zce(te$9uISm-bc^SYNCV>=$SKG|x8KiU}VCC9kQYw4s#o5IV#)!P=MK8@7N5yx`2Y zKU|C=B7ix4!E$R+s^hsw%MePDJ5t)|^6Y7`p-nD>SoBj~1$##HNL-o(rA0I`axrdS zirfl5rP3F_P<|`AoS-B3_Rg{-no_4GBQsnn3*m3yH3KD3Z=X`c;O96%xJd5ItA!QJ zjeo%LzsV-kj{DhR9KphTj%7X(!wxsFhoPT#Bo^mZ zzeB!>*8coGd8tw?fmVW8i2!>|%kU8{9iZCbNh41Clgq5T2imTgZ4H|s8SiW95rjW2 zjcBD!U^A=W^|n6C#t#^1R3;ZO-_Z-;mS%b4E)oS!EjwsEr#haUJn+$M3*A|dlbj34 z{JZY-Basb&iHw?@Nr;{SGqaZInE4=-h4^LtQb0Kc14BDnF-u0lUkLxH{_g)GqcdT1 zY5;lO3VV6B3HF0%^e(cwzY*496NG3XvZI>9-}G>-a$sj?juh`WS~T;uo@1VU(8_HV z!_*1Xu>E1T5in2U+z5z#opI0DqtR5pk~U`_j%7ME-6flc+%3_oNSDrFgVtraHJ6EV z@fM}p`Gw9<%j@8(c0wuTl->vXGED}`hqe{spy|<;domz{OkR0@-E8*T`1U?T_A-@E zK3j_QEZylfa^)-Z;ls8Pb$O}o(lEvedM$O307NnELL`_lATi(dMjj!GM*KOJ!4q+~i(ono9pF=9|XzafBjg%km>AGy>WHW?F-C5+Fjp0~vyMgySn zmBl{TTz2BDEF$Ci6txu~qZUwaPHOq`xRFs?V=}UmY#)A>iS1iLk{8Q9k6*3mDNKEl z9WKyuyAbh%gVJ$+3mJlrtj`m#M*ah?3MpKhUZ{|h7@HZF6P^lJht8s+UQXbMqw|Sc z?OmCyQ!hx~Ve#JW83^sF4SJtbCGfUrR*GX<(USOK9YFAIjQxnXx zV!2uz4rp!^d;m@N8U8rCbW-muCMfBA;7B-L_t4iqr*e_o{oGDyVE;tpb2llsIsf4~ z(ui05OG*??FcUe8?6Sr+gVA-piNy7lH$G5p7#Q-{ix;`^pN*Z=~ zVDBuSsY&mEZ#Y^lM4nr|E@mfeXxnbAsvUSDt@(}KZ6}`Xq7#LU=bnP49)|aCVcP?u zF=XPBr6-j%eiM>bdD5ow-{@R|NpkM4SRUZ@`fYBSb5-2EqI)co?Lt6Y$K&mLo)FCN6Re4R9GB?XTt;nhe-GwsZSYZt9#2`p^Bt z3+ZF~uZy=(yN}D3@!~WEWKblal(yM2S;bdwvNKf>fUIkSg;dYwn?B@XLKM3ma^sr9fSv6*gKU$DJ1`rD_`||j#J+yN6o1H>? z!=u(}>Up{T7MmEA>G>CYq#6 zr+C`$wGY(`a5cmOz{}Ov<0=-Kh+coL8#Dj0lt&+72&Lz0O+ zVCCGc`uNV2L%f@rIufnS{&5gh%h6|@6H9Ag0#8|eU^VKas!~>BTAe3_wsw1(i$#pq z)eH?9n1ty5O4{z=GYdvvr@pt~U_4Fr!{3rA2Hmv40Bd-FES_1}dWzStv-)WR#E|Muho2eeBzmU*9hZL@z^83tK?K<71F{Ad-x zlTZYltM%`DXO~xeS3SbL#WM?BUdfrN6Lgs_iASv)DKJ55KPPoWwXpNzY%z39%JI~> z2pL{~uN4W#lEY`%jJ;7kyQ+6(4h~@O#J@6Yz4cy=WyB!xq>KrYLGhW~H$H3oba$T4 z)7`+E)G-IQa=+t8SW<0m0(7Mf?^8F>zYn8vQC=@^przADP zQI{{~%QfO7gsS?1kGt!bE9L7#7TLs4`vIi9vVn|ch=(9Q)0znM8?#8gNZH7gHn$@m zj+TgCpUgz*trk>ZcEdX?>r)Tm{#*gVzTEZ@iPLJYveDcip;fv^(r-I%O1d7V-fT|~ zHc>{iNXFiMKLKV}thUc<(^~O};GD2rHtBDX{d5PTnS{k8_Tvl7o}LVEgy#fA*H*kd zT@kWoOgR}7Kdu@2zEUt8{JTG@mMceHsQ0rtiYP4vMN$SxJVzLs1j6@te6}%x{H=xK z=D$7nSrxydjw?4w>db~6sU%V*hC0W>|5^cQ>5y>T?#*^!W_olm{-~1RPUD7%u+Xb3 z#OTXLM|*5(mBcLK0r6PwR*GW{rRK*-&z+<}7%xf7@`;-&r2UDSG`_@Ks`)7F2>sup zGGR;&kl3RH^!UpdOD^Fk`M2gQf zrjDXT52Fp~9=Z?5tM89D$5%RT+J0vyJz8XpDyOy$3db2e_P(4&@nE78|8v0dD8n7d zq=xL#Pt3k?jsn6d@rj|6r+m63YJ2j9W;POLyrC)%hwhKj=(IZkhH`#pMbR48;#kXt zK<1YwUoTwrK_XSYV1!DtB+AZgKlRlDCkF&{TkvUp7`0mM{EQsHQKe0f|bp(p3n zydkDhEuAr4tpWHW)T#lE<7<@KSEk^Pu0_Inxmv=qzUjveRT_FALNHt5YX?uVYomxg$?=~aL$ zmb`8&5Wpr6NA3MbrY~r9Zu4ibkQ>ty9I94p#M6NIZr(a_+Bv7tO^b2@wO>4S_;N3S zqt0uY{kx~^;2yU*HSg2NU z1+2QCSz9wb(`P(ogHi%a&~M^i4IA6Ze}14StfuN>H#OPw8NTvaFWY#(aUm`HWQ;3! zX={9uXU*MPHl}KCS z^7Xw6CsID&0*&#)>i$t9O`IVKT=tv*6#Dja)lmafZpgDjfXB{P%B*9ThLcs#?cqP! z_ruyPV;hpxt=EVx_lMkld;jBHIOAO@&w!kreuO~A)ura-&-SYpN=Z-BAh#o>jI(=+ zc{HzvAesj{kNL*Kc;qzNfRnM?=^~3LzNIKHo1gFbe8yR)x|_XbbKN{k*Z*E*3B6k{ zsjvPDZnW+G?DRN9Xk9CFH9&XiTmA=ImLv>;cYz3c`b=sp^Y177i-alzUj!u{B?vPp zc!+U;lF#i3F1`~bw^7#CVsR2~o{6_F@ODf2f2fLi8$6%R1du@l{nwTJP#rVbuQa)| zMt0fRXqo3<8oblbtWT zsj5g`ggs~SxNoh2J_Ajn7o2L4tt&_qN{eYO$HL5sl`!1hKyjUgVeO?6(oqujw6Hx} z4B$aw&9tA;7~qQDj~Y1&p9hOZeHeQ8LTP`XfrP%$qGhu|>dO%t1^}L_o=?gu!I5Y? z`@n4J@}|mCCa z2QSAsB`RJ=c63o)D=F=o#ImV?hi9q2nUk^ec(Us!9dWcYT4nk~n9js|L$46m0U|&t zJ_>S2;YP%x0_swx$`Y6Bdligk1KJN1KVJ|uLbqfd$!i2y8b527p z{Sm+oFclURLsK6Ke*DYU^h`t^REr*Y8QqthGQhk)U7aNQAYAlmRdH^YwP6>@O}J}v z9kE*MS^Go1<44=2AfcROz3cnVKLBWT&-aGq0Q`thJ@sCuAyy7<@S04vdp{zuW{*?S z`@`l_E+i0e&+N`yZl5OfR&nwbg|R>rwvXX2T& zY^uHnoMqF~v@3^%))6vS2oLI;|6ZxqXgcK0bcu0Yi@s#j*R(`g)AP*d?OWc$6QnX} z!Qt3w3%jib$EYE-ZBq9~f!h6`+~bFFuOsGGcdC2ed8ajrccwi59n&&KwH-2j%eD45 z1MNru1cek&9MFDjHD1a(+qZaf;M;?fzlE5ie3vBC22wW#@R&Gn#y8Isr@GP;N9nX% zy7vc{6Uv?^6JhQa(x$+Wy|6sH`1jeg+2c!4&O0?(!^VMKvg)Ug?Sy}vW9e<#Zfg$y2Yy(QO&%HZA;{Kr;mK7iKux$pb z|Cm{0@hDkggkVkMy<^Z#3%?t%YeKRPc9M`aqV>E9L)BEQ_H^+GPrA$-w-UHK@OR^0 zB5(;2PjC!VZ>0-yqMvoLDi;4%RW<6?Z(MilnB98l__{gz^`c=HWFRzwi&su;+Hsfg zaDMiW)%HguzblYnpPqgk`?Da9vjZA4rjk0Y7@%pqi6t$l@fG95Dy>4@-J{KI5CR|k zjCk|!QU;AZ=bo7s=s;!in{2(UT5g~u9x-3zLI-*9#(ugP8a9L%rLUKKADQY@Vc6lKi;Vnc($cPW+aV3Q^o? zNVrCaJ!kGg$Z2?qk$N%8k`RfNdyt)Bl59VYfD006EWfUYPxnggz&UieE%2HtapWA=ISq%mQDwA{|gWQZM^-D7kb^B z?^_20)VO9-Gcxv?6Oy;fjR)W3H*ufbzhC|M!N?!-m(Oi_9`3`gS{wvwk4+(P-Tgs1 z&k#{zP+yT->jwx|j1whYct&PUM3)VSe^PylN9W=l_+!$zQJJ1D@dZ_bDvb8vkA9^1 zzgbudKH9hr2oc8aHUJ*z9?hg5<#6aoi9dKmNkYt0t2qV*T#gTaY=crJq{e{n-PE!1ttI$^oI@q zDQwQnbC~+BKEP_?yIEjOknr0jEy$SWe#5)ngd_QTu6f7-x~lG`8E82yn0IwTOGqO~ zM_HosS^L)aH&Ul%Sa&hSmBJk;^3B1gJCDQoA@g;N8c#mLH@}`%eB;mIdQz^8y+48y zBGxbr_{@y<1`E8&!eIL<_e%H{Z?M4k9GPP@k%Rvo#CvIk$tUDTkd7tTH?z769YOv^ z@+UImf{tKE_xsn_VXTB=hb zS1oZ5Z9f|Wl=!&k)c%7vYD$F})vg7@j#$y?o6h(@(gnRq>IhVQ-8xm*Y(=~RBDAaV z+Rph7>~u)i-9?#4O$D?3vnZ}JuC8(m@slh-nSny77Mp(W$mg+h$e6$`1Syh6Tj?GR~ALhd1* z7m{LHwU}#$_Ep~0fPzV<8xp0vL1`bXyG@aRaYH^{;{@$@Reyo_9yy^FD%3l`@#&%}q_pM>Z}_iMI#KcKd2luhrEO-mG1O>aFc{5SFBy%2pXpheTt2ze zb}1i?(Vp0VKta~(LXnn{+FMLMs~MmB549ri4_2cem%}?vWU!wN?XE)N5&|c0*2ni2 z%Ng+PJJF|4AEbg|0PzFv$3s$*<)aCojMYEqe%+N~p!~vMov4CfP|l2flxF-gI()45 z>mHYvgc)G)Fby`y+(IgxGroKu6kqGHYFrcUX?Jdv^WRJSb^AOC;R!)`slY6Igk`nP zaaNR_+vHc(FKO%+dr@kWi3C$RYqE({uCm|k5kT76N0NJHzFo8|uPs(_ z&>7=hoRjozOlPmQ>sXf=#D@l0o`w-kqGIXL=_s(jB{V5hi&PD{BGuIavS&FK4IjJ) zAG_0!ADY~d3~rSd1tf$Y0ZVgs*GF>#ip5FeN?7lZ?%ac}vO8fL(46k9?^+D=F3*>6 zN-;OuegcO0@$NBWlG12*pS2d9#9`7q3mSoZB>_QfKdvundY0<~a(|N= zH<!Y1xm1@_wIRyU2+HhE{p3hv;p__U)_Dg&9HoC}ZVKQy$s6M>8YO zIKc)?4(FZj+zWwV00>{Eax0y6BPrxXR~944*qb8ki0b{I)D*lpXQGPhv3`0+jkuE) z8;Qm1XgdVyJou3NA-V|d_S$kLhG2tlF>1Ja)pId6-|T6Al-95=w^y7Z+V?B%*=p_g z#Bb8z5d1isH!t1;i>2ae$FjeyCDV>QH4k7jB4Ce_RW+Z0yWP2O45(nS&ytmyXk0Y=dZ zUI9-NmeKa8i7QoVZ%d!P^E8nug$8_|e-Wf3XtMp8<14}|E3)iVSyu4y4csJ$&z?xz z|8?SP+NEe>uk*!X!+V#^SnuOS@vcshTA^BaYF}fbV8$USm|+rd5C7?$G|h(EXEyo= zeL)DrBwVUW28T}DC&OPSRp_RyGq?wNfz^mn_+4xDqbh92<&K?cd$df&D5#u7Qbhi!MS7~aJMPwVzMTh)p(uj zoE}=A7Kc|7K&SsdvSc3YuM%k}*R9{qq%i-x$!Vb|y?ggC*#~aOm1LvQ5XFQ27=ssX*Rl9;q^ahDr36s{g5@i z%>2w`N}qpAl*G1Q+ORc8sUkCb_CnxP%>34GPivNS)NIYqH-B3AK4Rj!zr0&gP==fY zK-a_(fd5b6=ZNe$9vexi6>jKtR_ouIHeCAc^M}~ru8h9=p7U;BA-ni0`>yt-v*Q`D zjlAr|EG)b7f8*eNuDc|^n5@i4_p=aI@laRr9FC6J`l=Xv_?Y&jbN~nPMATTqVkugNN01O;>MpdOohjgB`iLllR)Y`D1hY#t`!=IN z!nY49Goa?@S|ESK8pT!5=TX!ALySu~bMl@AJ#`5*Rp6O zCWTqtup{UFln#=WzOAWW`bb5Xb*EJl#8@aIo~Vp#hfh-||FGOBCQO@kOPeKbJcyW| z+ZhMx)egnZ+&&MG;hv%=3`{Y!ouhSOGLa|b1=I-jibKY+1168-U@_mEUm7DBEmL)d zQ!u<_yUn=2*A@&@j`ePhb?1gT5@APzZBQI{CsrB}+^ z8W~y?5}h6eps5RKjDF`2pj<@tVJQ(Hu~02qW}@#K`jx6Qo^Uo?^UtvNH&KSlhEhCof$jq(=j3XL{WkQGeUHt+1aUdqr9F!8=kk!!CFx*D&M0 ztKR*ZkzTgaE|!RtuP-zA8!@MCadwpHBPL32fFj|QNPF$2rLjaD)ahWzcRI=__#!tz zXSdLDY0*MDd^v*?e+q3sqpT8ELbT3nf2e3}xVCfBigJX?D{t^SfH5gd_U^IOgRExX zM*3nBehU{cp)D!n%315Y!(2WCU4PAg{t*f9YB3lO1j|_nvIswbs|wYWa+&=IZA^!* z%`ESkshyuEa$mfc2+AB0;Mj4+(l6H|#j(W~J2S++7YhvBz{4_)&R-PXGrE?hMXQ!S zchM+lK$&l1)eqH56l9t1ki%^b|BXO> zw+UednuG)4ZaB}u5DkeDR-U~jPU?owAX&}=&85mmBc@N zvD%8|O>iUAZNSwo2s_%iTKPj`dK8K(NxHFF<#ln>7SYKv7?hh?0l)ulAMga~?awts zotu869|@C*239;x|CSvoc4W&NI88H$d*Qh*&Q@=4A056h2T9)?Z>zpBN_Bdz;uaB+ ze)tIN-8qzJFjR2WUeK&AfkiB6H1uZ&~7~OrP_2y zzfO-kFd{D~6qw|IYcj$@=YgW&Z&1gu4JJ|36UJvKz0N@}*E|*_1i%W)Ffz*n=k;?! zC~&Zo>iN>qUL`B9%$Hx`efW*2isf6Kwo&I!>nq6*3tH8z;*_Vk%{zUK%=pR+$yjN* zJ>WmQ9`$rc;3~IEoHr=-L)sZ3OrMiWt{(b{a*#}(P?B7SH{lRk&$(Bpx_x37HdQxK z&4K4`^z_EmP?ECHzp2zl(qx$XnJ{rRh(XuL%4nqd&j7QKkC)W7^%im7`0k@n)5ELd z69~>O-DEsX_96Tr*R`PbUCmVTE^Q=k>591_)cm5nI*XBz_Kg+GPfrojBS(3c?p~Gj zFm8zBNCq24B$SDHT=A}cnvUztm%upjvg_1+1KzQBQu2z6T-FGu4)yW2-<4o@VShHc zNBVN$Wlm_)(WjxuuW0hu8j>9s!GxjIMpvQ0(K`2~#D;N^T%ab&qTCNxRgltE)9I|J z#0^4>e>3J2_^(}BH5NnXt={sD-!Wi9Gl=v4cYNsj(*H-(H-%T$1>1I_m!IeZG0Jdt1=dC*+HkseuctgaQP3ba9FG_93f-pjltVm`IxZzx|2!=Ao>!#?9HX!#7 zX!`FN_zAD~2&2}|S`_oU-sam_I7 z_8>Ef(sIje`l@Xgj|;>KY^RU9KQ&kRtB=C>u1?T!1=Ja zwHPk7@x*FbYZ`#G_WjLG+Rv6GaOw+|kFs;s%^gI;Sbs%Gxl}Z!iA0>vKYc z?Lse?+dbcDq?$C?8c@}4!Fl-ys+NP-zc1nOo(D_9^ zX3SD{h3=c+8HJy>ZP)@CesTWu(F>W$_FUdUvu-q`EgIY9Po4<6w-4=o{cCBJe+mqD2?!W98n8Z!LGlWBMXREyxv6(=VG)KgZ%T|zfA7xr z{o2t}Ke@k4R&bk!*XH$q7^Oe(p!?ss_ zwYa*Ipq`5AAna4WyyTP!L^8O?mZ`YsDWd^~hdKy^iwML|NP51Ljo=R~?@ifRZq8Bq z%HAYsMX0l7*@@%5yi*6QqTLp?<8A5OE`HE8v7OtDHmvLbDtAdl(`5UH&!>X zeN3K!uXrw>PaRvb8qM~A$(a~jINiB=rBqiblaLyXp81GVVY|Q52wDtydF&;HH~hmh zrGtI9G|fyy;|WEc(nkUt@4&j}Q#@Jjp_-{yh?*%|g>_y;nwUo=o1y4@3E(BUHo?Ys0UhNfideCd&FbSHDGUtpsUznT9t~1hH`M15}=X_r~T+Y;ZVsLG#+Cx;r)5k}7v0@O@ zg4BeH=RNEVF~d#=@JA~35Dk`L2GV~7K8$D;Y#~l(l;PZ5){f$bfBoxxG}59c(b%}T zk8WAJ8nt_PlkmAGKoxD2X<>|6m=bY4iLn!7+pR)W8t;mrOK!H`QZcwNs?pWwD9|l8 z|M~f0^+BLH_xEu7b-lYqPhZBi@REzFojcCd$xB0fSlS{#DROWh!xK1paSQD3t?VSc z(t=uh0c%eKykGVp3fB$igsOPAF30t_pDEMue(b&)Ok9-9vSZZVVR5_&Pap4_=$Zcw zv5tzI%!nRQdt~58ELewX(RA*Tc7cW8-ez@Y3@@Jl`qohf6ilOTT)*}lVl!fzcMpO& z>Y!l_BYHzvJZvmCoRt^3KKs7roE25@2RG))bv|~vbKx=X(4uZ)_5xPk!M<9LtiX$_ zhxYT_kSfvMJ(y>ZOOwPpyS+dYqUs`M-99DBqi5bmWSx=%tzpS`)~oM% zdbDHU!!%xoatDSdO*aCX^y4jF0GaQw!#nIgDC44{FvdB1X0L`=73n^KCYJ>J2Xa(N z%JKJ=SfCZs5QauhYq{sPhc0LD^z^hTW&Spx%6^3Z)%vsZRPOLRcJwz^STtO`NZwVb z!XL6;@e;f+#YZ8gv+#&Rhh>X7h2h*kOXnjko_dEr_lq|1UM0bX60;h5d9l+% zyJ#Ekl9Ra*$)ND4mYVse{Cs35DmsN`Q8(h!dvv+3bhGs^UTJkn|L=(TrVXxtuYyKugdbWNEYJUAY^{*s#*5?w5>2i%;Ghs! z7DsIEbP0^v5DSXC;}cxdWph&-@{WlJU-_l%J;+!wJnu3#(HW@c6L2qtggey;gM^Fj zB~@e>Ksd{dA8Zh{?jLXHK3mP$K9Mx0d(j)R0t!E8y(Y0X@OtQ_Oct&MKdnwKJIQL% zuY!|;9K3+Qq}E0Y8;ZMqdTn@m!!v&nFyNuc)aXf6gjj;?6jNuDsrqy3!TT>1H%t<% zk#tmRoV%aZHCgOqnBIE?j3yTVOcW618p_-xb>^Tf^M;GMXDr@oDQGZ=fsMp6hpNRkpy_05c1{fceT_*F&r8B8-3hbT9&E)^Iky7z{ql0N!y3}CIBK6Hs#Y6Y5m>nZx^G5KC_C?Gs9c}OMQ#{76*CX?+D}fsILyDvM?2jzMKVJ?IM2Ur>p6E2~XI8AmzXYZ|)Bj zw?yZ-h5`2Lv73mD0l!`~C8^y3=C4WfT|Bqnk2dc)y;3g9C0T+CzuQ0P|DaCqOXxJS z1njSiZ?;Bis}cB|OqJvb!CtOqPs}?%vniK4C--nL4ilSrWu;SUj;ogLL8cs?EPsS0 zcICBAe8A4u#%JWfP#$#39%X?2_QQU4EvYfmhW8it$S=^&_!eT?!>=BT@6_)r2js|m zh-!3Fv|}cCLR-H-^;obUf@e}u%3OKS8BcL-idzT?xG85Z1EqK#7Sm+ZVsKQn%&aF6{k7lCiUTUs~BdY}c+#cKoYw>Y~`l=Ed!u(e0rF zRTLEo8Ez0U?Tl)>NN=OV^0cdS!V)ND3%&8MJRtA%4$~Ob1ThYF_vTcIo zx4?_hzK@)ux6|iielbq3f82XkMDa1xEOMozUvO96sWHui^mIK&T;yT=6ODZZ-E^LW z_kixs@=MJh24_S6D~K`$fnVe;((rXtP(x=;Vz{A`s)mrP`Ue@!mr5x4X*p4fp<=16 zJi~;H8DLg|IjXHf#`h$}GwVVc{pM+{W$dtZVMw6teJ8{FTNEKx8Mb}}4L`6#CMVpC zUZ!Fqf9X8Lf^CjB2$1Ao!7+ZZ8$buuorbJ@3%QNVLEpme`6Aix_s&42`^bL3;GO^8 zSks^JaPLYh>J*_L|ErILrcA`%u(aAy8xbx9pCs(h>HTSI`BrC`m^Oy`qUetcc-z&R z^DOIxLZf{~^(RTjd z8}(Yevbau4(1wW3YrKtP^R?iY@jpGAnmw57gI;$nYkW2= z)w@Lu1;wCftYePU@)YSc&Wd;71D}xiob>%=fUxi{Wrn5^3+E(!ja3N;Xcj~>fk>w}Kj4rzr@(wGr z*$y#NauSj3`G{>+qF8)-(H7g=yx7N%HM03F_}JF1m^e(XYpaaBUPPLwV@$jrq&eJZ zjP}=?mTjd*Cq{xUo85XHv(a}ypUY3#R;vSdM|I!GcmX2ZLJ*Mtb})KGQ`mMIHT{kK(PKl>%gt!;5S`^Z-3qy0$q?2&O}P*HE5!=Y{xGA|PVx!qbP$>I;wp5LGYa zITE2XMpmqtH19rZQjFqn^C5l~Dv~;Kta8FNwVTrFQ?(Eg`EUy6+dl)2kWNe#8WWGV z-76mzMR7&U#A-jzF4a_pwhekd)}i>dF5#hP@7#Ms6x;phDxN?YP{IA7L+MGt>Dk2{ zEdI%hFMg1Ii8jJ0l4P}t^04EAKbv-+D>)d;!uzjOm-s=l=;m;P$vnJ7pgTVuOXb)= z_|>$pJcSx29X3K7rfGuuJK6kbe$AWk)mv)f>EVxq3Nhib%zS-~{>kfOm{!TtWH5ud zL`{FpygW0I>ab42SxACsOWWOE$Ump1Zl5Z%6gy9aqVxUiCC9+2bc8@2a;V0;5HLo~ zH!NvKrIlDzA2F#5n}e1pcrE4z*62ed2%61>mK-62{F}avV#+)ctKlZqN z%(y=D15*?HaE$r?a@7FemEeBq2Nmm}_Elz5xjFq@K`HrL-vLighg zWN?nsh3DImQciojYdV?sY9}pdyC57^95(FcboF*qGPyEpTVo8O@_O#9Efs4j#q}QlQqvl?k+WE=2ilaMR9zD>a>t^MnWPtJ zMFzS)_1aa9vk0l|^~IS?^Yud#+IC8d8^r;LujUkpU4Ee z%;PT@le3~ndHB)#6|o&h>XwLcw+9(<0Vu8)sUk9`f0jTHMUtf&hzj(&Ay3%C_+@__ z=tkJe)zqFokoEmGfhA(ruuFx13JPsR$tN`4R=fCPHA=-r-qzlP*@Uv9+<8Aj3vg0? zfIJA0`LN&oQg~5zp;@)?gud40@@dlSQ_QQZjC}UMig#?qqiwe8)2_)m5N9DWrJZTf z*8j(ZB;>+Gxqfo}q;lON%-KVR*K?gXR2N$_ViaRMSci!B0x61iEQGsxusCZa1#|2~ zb!T?0D$9oFt1g^nc{g&6H75P@bApsIM?^#5hRL=_TZDe#znK!Pd0&A6UOXwtq1Q_4 z6YUg`UQjio-<7K;ibr%SE^>yvV2VQuhr>EAKZhp-G?OBx&rZ3eS%v|(NhP{3kC9BD z*4@Lh_uqz|1$FjC7ff6Qah~}OPdcYNdS$!yrO^kAWZ!vdB??K6p#uSyA0^Yd(RZMb zeW^5a_7|VJrCC}>l)hKvUU|f|1Bj(V&5RW#`ZCVk_1ns5S*6Iy#ytV|=YQX3Nsj$*aJ@)^=P#Nqj;8_Sj*X!1~uX6Gq2$AH2m$QKSEi@BPPtceqfltQqg*01mx8> z(rc=Sq^$>~K)Ybfh)$asWOgOfm-)BW8S2o2%Wa~G^Vv5VJ~S@UoA<0ogvDhFGDU;_ zb8$*1P_0IKAQ6rG*|J$#0diNS+lZ0E{%&MktIqf7_njjM3RF)}hgp?2w?}>#l8}iA zI}IjqZ}!=WS*mRq=UPIe2(ocC;k({}jL|llenFdo=O^6U$*7%3u5~Z!`m-8%%TISOJ=Za=N|n zX4_aFE4rn?{~TCoWk7O0`Q z%_@nc(Fmf1hgewTsX^*|B7WDo{ID2GkvvP5#FuNf#pkb+ui@5zTAz<&R%a+6z}u5X1VI*^tj=;Ef%JAC=M<(K`*d+(gikial*f zrNNZFP1xccHN$hS%DG8?OYTjS9zTE1Xx~)8!;o77$mFIZ*0-+Iy(FJz*rNKfjIku& zePc%VQ2~Er#r*v5-F%UFFav6LE@^AkOim!(75d5LvR%W$jee>t^u z5iqB1zNvn?!9eA+!J~9MQ>c#!QP?&-caaHe9{M^3Me5^WYF43BnTI?~o$595S}Ice zgw4A}XGZ*ydw%-`Ny8FHP6 zudNKvEVlY76sj(jZyRbRW0_Feuy1mRV(;U&rIz%-r%V41qguN6?^$t?%TdA{%QU`{ z#mbM1KDl8LO)NiWT!3uh|9Jsa$G81G(6itHX9^+opk_06sa*I83o9>Ii)PWS6Y&1- zd0?Y@rD~_5c%tOkbt@}mi{Fke)skMI-(RhJZ7pZj zu`Pa#a8)}p^}unlQLm?b{G>PwDt&(26~rqsMdhWgF1A~LpDHd>Nd{M|<`k+80tCwZ z&-VU2il8wX=c#3|T1+C8$nh5l)gl6(xeIb+?_@o2GcKISjt(J}M zfqepHa`*BI#?S&3HweAN@7zB`CrsqiAnUN4#}Buv6zW$d@($wQYlj(H0^+29_Y@!h zSKU5i+AHuEOn1|_jLXco{8FUC#D9q~SIOunq_KLp{sip5#W~Cv(|5%gU&jgx&W+CV zt>@N-aszuYkKiz6E6l9D;pf%~C!3Fi>Bydkr2`i4{1|k_>UK!o3@P%ICH4=@EY~ur z3zj0U8)i2&07pa!w=n2_J}VYYb2Hj^a$xaCWGH;aix5#NFp0ju%j0^U{poyp1*qOm zK_)I_$s=!~9^$cKiox^#?6P-@hJ7oUxIbq2&$a76>)>37{}J;+STPREC(1<)_YP2^ zB_^dV34kE{V_T{W4b+IhwK$bum)tP^tJ$v)~L70FfL9>w!Z7KQTfqTFjLXL z=gtg4G6mZ0$~SHG!|*D{ftHT{UKQ7SCys;0fpywzha;PCPBnCd5dR-G#~Brk`YYGx z!_Bsxj`u$6pw;Oy#a^1zAxBJ26!|ei^;a>Z*?%IuIkTvys(Pjal_#N&&5KKO4}G`8 z4iPZ**_4HPm)|p*l+j@$v*(7#v9%iqs)ppZ4_BMSL(%NSl01IrvLC=4)dpjDZLb%y z#N>B=j};JOkvONBgG|g|7}7D$0K;D(AcBl~pbemx?fo{IF7OuGE1N^LRPFiI`K6eQ z5#iwSir|dC64P$>GGClaHCvr-%tH3PARR}*)627R4W$;KlS@{+lQn|bKbBTF0Yle9 ztW;y;1x`~lIwodjBy{}X{C?s>R?E=+bXK8%$XSywoJ8Eu>A1VNpeTkYby_h%&mt_7>zs&#*z>AtR5$1+;$7 zRlbcDnp#dHj#Obm1Xp-*y!pDW8*!!){0fvkiO{ui`{g`ur-uphvL9jkAe+k@O-*>9 zDJKvv@YBlLmUYK`XLZq+UiS>}2R2nbRsnskyYH*L%XptQ*WwNUbhcS|;n#V;TXS@_ zxdU`XwRl*#X)Cm`<$P@&wRre&vTYrq#8&D8E%vMUJ^)5F8k}rB(pU5&{9Qdf8vI|L z%GtCd<2SN4HE&l|?j27()-zS!7j2sW`*SC16CaO^?aL}$0H4>@iK=_-d7$y8-4U9W z9b}A?^+MT)gn zKjM)rXJr#7>h97_Sj+`4N=(eTBjp|fjjS+6W|q{yY`pA)*W_CWZ!|nFY;|V7;up$W zkC}VYN^cjPCwwtS(BfBhT-rJ_OqX}?4Jv$do79lJg*KKgdbm^jOTp`{o-niy*x4>S zrR+y7Ab-VA_h7ct+?{Lcl}yp0i#K;CxA(n%DxZRkiiqXvkwW98`-}5KSCUQHUZTup z>$HwqLusXJPyH*1=i^bor27PdeKHo#K=n6c&Ud5kHaD~`OUD{xqj_b|_I9dOJdBG* zTCa)YrTc=g=jVB!y+OmAM+~F9Je!0^C)~JTd|rCdt1XQE&v(blty|u1wTItFHC1U} z-W5*M5>#R3gBk{mcQG*q39z|WlE6z=bKlFxSV1Vu7s0+#F0>2^- zeG!}nD}b`KMyybIDzSazifpwFWExBzOrs__1@!_t1{z5XA>v{Ecx*| zUnV?(d>z^p^w*ytdDAp@0>5;a1VtzbrKM!l5S-7aii><8Z|zNQtQT&x%>}9jJ5HP> zMz6Hf>oEL=+Mve{#eThvN&ewf>$}hNxRIOV0CSrKVdH(tW680A8aNRy3{+?S2)+u9kbboQhA z!b3a7@nCuAJ6imH!Y|%y)coE$syuD~j@u0H?{M3?v~vG)z|mRw9@HkN#HHRQ7;L&> z9lMKZIsG_Xb$o2!1h{rQ&8R4F;n*wy9H>XWHilOkw{>1Wv}(kweL9m0yreU(S^|BY zJIboqdU`aUHvTs-n~MAfz#1K1thMw@BF*NZP|E7Yg~_QI&Z014oz`BGd~2&FzPz4Y zXMBdVB*m-7g}#pB)m8;^yUfVi$XtCM)i9g5U{P=$1w(>2=NQ{K*PAb+8_j=ANvKt1 z1{V0VWcWVuHo}9hePre!n9Tv~d{UP;Wk)~*y?z^$>#9Z+Qjy`>2PTIS?B`jFYsM(%yo3wznu1aS%=Xd{H%EV?o*)XN4Ent%4b(m9l z2Er`NE+ayJh@y%pS-8bAHfb7vl6&SqJS6r1(sb81iq*PA?a{e<%P39M%c0N%f1wH@ zkt27G9EkwR^bJjU_Sz{Edpu|)si?bBp#$4gfXnpNib`3I)?#Aa-Gtox63$DL)b!?E=)aVSUGi_jypEDlDch?!~$vUUn5c`)ou zGn8v}f!>=AU3K5PEC-`MsQ<(}kOV18omfYy+%D2tSL=;n(djiqwK|-MGnpgC#2@UA zUXHOOAC_q~aDay~e6PPeo-T)EE4=eXJUFJgj&TbM3v<2yArli5n>RjE(G@GmpOq-0 z{)W?QG;IXi6qQ{*!FdBU)|vS6fdW=C{MoTDt}Gf)uogWa1F?Pg;$p zKT=XGd;9xsF^umI>A9{~T`sXX31P2f9?(n}`5LPT!n0=`rcYF0tpSZJm6&xd@ubL+ zkiX5x#cTexe^4Yk7^%t^AQaZ@76zXff#0s=>9(zN(hFp-6QU-A>q#Q{xjf!E8SRed zat_|!+JfJ0MgLw8f37E9gry{F@rZ3BKrgiw4CiuG*eq|w=7_{Dz1V)u8|Pm1RA%jA z`|=xqX*yhecea_}o&D%L;$FRHT{7^oe{|_0Z~MGaj^5Ou!hV}7cI7s+zN3=aMgd(N z(yQs(GWwrt(!12VyXZ^ct{^EgXLj_(7a>agLh}KiYe7TQ$|(9kaN^s{O3pP1D`;hh zkR{2tE8AA*#-mT59sp6I6Wko9_UyUK68X{0gzFPZw(Q>AC}X164|sY7QFz*m$9UbD zx|4m7K{u@Nz$)-|%pIjzM#|SZRy^#!@vsw`2n?uJSzsIZ4*w2}^P)S2ZCmw`I%cdzV#t^g|JYB7Bl@IytkT2I0!$8lnZu5sI&{%0d{34XJ?7SR zOn0~OPPvx}`I~=J4GgdeT_pc5OX1;0pG)ZuZ@~6&A1~%?E#y+id>okb_U)`ndZd1= ztg^(@X$Ohv{ujmk?+T8#uNrY=U$h5f^qG@h|M?(d7uEx6gtWkqJ;$c(vGw;fkPYPN z^&!h+JNuhwqFRS6+&^R&pKQ+?qJq~ z|5qYP6*K{+aHe!(~jh-uuXMEAk%q$F~e}L0=DVNn><|dIo zGq(#5{0eA2e!C@QjLKnucXv10hTBBd)9W*q)WFLvP%TrvURf1-&k#@X6E_PM&gD!; zpXMD_Rfed=zaJD8X^}mbG3CV~m+r+LmHLOa=i|#-;X)XVu)r@0v_F&)x8vcab9Cl{ z$?`I5AelU@hq2mZz8Y%sAZ5m;o7VH!yF->vjDd2rd*(5Ff%3mwXsTCrB~PwXHqOt1 zefF;7kAn+tm91+c7$hI26*6B=by3z?n((dI|I<7@6@YZ*px$e*)VZwI%xKYv$cfj>c+H}RXl+E!p|9;$paSY}}U43ILepiM6CH~8@pUC&T*!NkRBLp)8 zB&+;Hmj;r>@droUt;;D#63+#qqIgkghWkCYBVnefgHjnmwm^&zKs>Ha)qMfyM7srU zq@B8;C-OE@8|t*Q@1NPDNE*}O ze$H9cRO2{7vSy%&&L@}x%Z-w>YNIo(;C=HJ^%7Fnq~;Hdx3B2Z7Ws}|bDi>v6BFXa<{ zxxvTzr*to8K=Iyi%zHKkH=SSrV7u1~j80_x09=NLWZ;HIMj zmnISVr5+OZ`0roN`D!;F;Pjb=KSz1>06PXQTvYBa<=)<&e2q5oKj_kkvk=OGPAd%xxUe`}pa4S5P`_0P-$ADqp>9C2J$QGN6SWS-tozToorieM@87Qejy|^(Y9r9ymU=n#j@`4jS6639g z<9W*}V^R7Ve;klTIbP1-Szh^dd|QE=_x{(U2`%_S1X@|UFfEwx^*+8j-@v=8h?l%7 z&uYIe8iP2$eb`o>tzF1=clGf)vt9$tcNjn}Lgq_X2PD{0GCir35j$XL`EO@)aV5}v z_Ibc$y-^iIF%=x4*TI2q9vwkSxt-rwbcW77q(a7ek z_Y;u7=w3FXpY;fySQD#{n(j(3wPqF#8r1uS$+S$@5KKu<6Ljhjyf`k+u+|ImBir|K)ktIma`EM!n?hZ$?b%w=W1pIktCYl`954soQo~mXE=`jjF1olTV1iWep4rq99yLM?g2#qa~x) zb5c@FDtjO{ObSa32FOA857@SaOY(ynMd^R%9dbgoajMx3+nFa6XF%H?@3-C?^UyOU z(sT#tjwxzIYW{1qBn#<0u--WNaT%{6{G&T?$B7VX;-$?klnArC%}!{!;MWu<5n+V8 z-xW4#hV#uvvjz33{Q5>vgJIx`vKwU0Q5j}MJq&kCH_iS%+(7$!agmS#3xPH6x zj=mCz6Pb&IR2ME7g-h)y98i9$%XJK?)TpHqxXtXTSIG>`BdYZa>AZ9eQnY{kR?$9PnX7WMOGvw*!*;kW8Pv zr_lTVy8eQdfg8wGp&;lE$1OjYZBAu7ggxM0ErHx=D&HxisO&8iYBGKbwV1PG;X*Ne z^5dRLf~oRQtiOkoIrpFI+{-~2GbxBbB-Ck2ehVWibrtmhP2*Q?7ONU=K7aq(Qxoa245obmdMb>E>!4Gji5xe|9t&;wk$&8wG<)4F8cP}%K}&*h45l;3aB zm6Nhh4hwANc2OP42{jd9w>R3>W@{IX(~_17;zM@2>9lP3Ta=0)s3@Yk!Kl9+g$~1b z{MWz8N;m7V?!-V<<}1N;pVdXw=oIj;<$@<+QU=mDJd^gdUK1LdO2f^7X;Q48J9vtw8I6 z=Fr5h>6yvl`u?X#Nb=x$4lt_61}0-Nli9ozwzVT8!-QpmU@Ybvsms9K;0DEuIxt1G zjAInTFZ(fM1H7->33~mRY~IyovxGWf3h{6=Z?(2>nYWB6I0Tg5;doLg$@a@@lSN#5 zdhlFHG;3}v%|Vd@r(x2=OMeh-h-vLePQ(4`P&&N_qRKkLz|AoE!x^ME^o{lW}RVO)8 zB7Z`(Oq+J!k8?{#!3X5`=lvK+O`pR<47eew{b6Y-x6LZah#pevhozK{l?YuRk1u^DIo5f`wsg z;;}R_)3!6T>%oTVb?neo%pW|N6(HdC7D)lG!A+(uOdDPQ!8bhw@F!RZ-d^79PiQ=> zA8It~st*H|W4<@fcdRJ|D*LANy>5jpg|9~MJOq+=0A1CUMG~cR!IT0R{UBEQFSO$i zhfBV@5q7f>nJ(F0x8rD`8uI8KQdrz8W&>7?iHU`}=c$@G%X@~x+a*Qq>=^bMyuc0& zCc79Qqs{Buy*Tl3Q~P_6g7g57{Nvk@Pfm|B8)S}0kGRtz3c>!obsZ}#=g*qdZKFQ& z&ikF7n*-{I6O$qSL4#g34EWP7Kez^V*9R;Shl|PU4c`RZU zU*%RGt+(C;*rlCYN59r_kvj=Y!k}oF(-PsrIehb}Xo z;OI%~X6fb!AL?0ei3%UPyR;1}Hz_I_=XhQ+t0CG+WZ;H_rKs3!JSb0^9Vj;DNq?M( z+^M!Y9O*AlX>%9&6_Y|fxQePo*_SqSk`iZc^-*Qj!r$VJp-H4h^;gYU+t-~Xxt`^a-3$Kl_u`wlxr)v2nNPQS4 zz)`*HYO4XKqxq`$GOHspsf!%UFput~93Z0&5Sry9VGfw2Eh==#3t|JJ{mJDEy-3iO zNELf=j56)!RMt>B-t2H8rr?GOdV%;`nxc!_KRiT6$i;@=7uRSS6Dg)E1qMD;OiY+Z z)FKL9y)tO&w*j?e5!kzm3O-0msQSea~r~cOs2GuqZ(ETvb?K9A8$z6P4_d|$@%`JxfLXO*X=Lz zOjk1WD=zyBm9PA)5R}y(p3P~L-AhgQcRrS1y?t>8T#k_~F2G`&T5@)9Gnhsh2~%H8 zE|xpUD%dBG7g#plfY1`U8|~%3ReZ^t;nd5dJ?}RZ1Oet|U9neuL-eyZ2@5+=&1c zei1z}T5t5!>cHX4&^fo=!LqgF7O?$SJu_SCTc%1sA0E0G^H%dwrS953ZORj%mrnBkCFeu>qu zmUD8B6tRzW50UFiGyVR&w*-bJF89y1l^%#c5P!_usvyK7iZD?!XTU+MC8#S5)i`7` z*rrs8bL@t(hV4}o@kc-@DYS2iLF<4&$7?eXh_^${!ZANDjh&0pT%%B0oc6nc+32y! zB2UM2V)*2_T`;i6GwjLhd(<(6F79vBc#%eLXec!-zV($eMpcy_^gC`t;b^O(u^)9I z#d<5%S$}8785NneIvfwZWC&H;%}8C!!$c9QD<)o4P@)eb-~G<8u}aF;>G1s_MX{{tF_{380+$BUA*)b}=a| zFri!pH%cezF3mur-8#x4BrQpz!b{-|_FK%5yJ*NLRQ7MLQ35&kN8{D(+#-D<`sVaZ zZ=JB=pFGq91NyyeyKwYk{K*hv6DUw&JOg4>HFhCF`(<8;5#5_&rU}+Bab~4lM z-|(IYE`|Iefuqg62VXje)za5ej0h2n!;w_Md&H{F0XJb>1oN1wk z<}i8b>ywN?>-W6}75#S5>HhEz7Cqa>uVd*h#ap0MFKC#GT-|HwW zzhV6UeZ1rFKH!Y@K6Thg$r|4Za+V&4h-Jfja>I{LKwCm z^PU}U9^Vw=1|I=W?pDBFH|-PN%rLc-s9v8l)31Nqv&BzNd_4w-1!AIj>%dJe?|0Wb z&y>FK*NM7U&zg(3tMQ6Tzk;{l3T?D+9Fil{%pSTwM}(6=&Gen;{dvcr3|Ag%lDp-9 zr^&<{ywmbkkiNhZ`^Wu~mCFKLkp+$v=FzrnOH*nEm?;T3*PF8Ad;tZVf5#$FWUw76 zLnUDi@NJ$#KLOXNm^p}HtTdK`QPyk=$yyGZzi<>ijFfeh{}AUeZy@l2lZ09QR{s28D4A+>DuN0jbaZukmjq*v4%U0nJ4^ zPAL==0t1>A5xoQfLPdL_G+8jin^qelN&tJ!TmBd=!U{(nRmAGO4zJgp<$pZQBc>Rad?9+q2 zlB99CJHNV5e!cT!K3!yZ!}xVi#Z(|4i?(8yI>9Gi_Otg`{vlP-CVbP7%HVBQ=lkTp z2kHPtIzYAlKlYmY2Mi=1b9aY12W(H*T5N()F-1rBFZ1 zj1j&FyZ%&nz6kyVgUnTuNClCGeR>ZJ((w*}sLwP~Z8{M0jBUrdp?%HQ$pEK_AEdPC ze*NWe>=hLSV1|nl@m#pXWHn}SZ>lxp5g&8m{Ytos#fc9m8sO63Kfc=ei0K}LOeT;4 z%LOSI)>WK2BEMOm*&e*Jq4zDr20Fi=n(Q@d(@wGi_+k{VWKg<|nc8NnF((C&vMmk8 zI4jI54=cf-p>nU%N{~%`rVn&p56u%B3vv=f_z81R@}*<=Vek@gVI4JmZ;jqNF>^lR zqi;upO_A#`CNOBOHMN5gk2_P!Fa?PCz}wPr7`~d;p3h1)DR&0l{@8oK z;c)tMIm)R4@L?4(N^1HxT#ojD1nl)6?d&%ve7dN^nHv$PhJ1PPd>#70v{C9)_-L>z}I2~GA#c6g`o=ZjWIqhQ_OS+w2&UYX4;$>BMk>6>}_O< zdy5M#tPYE*dav;odxXY1<*zd#BQ=fd1`30QOFX3fE23tNzVW1c5vn2GA`;ZMuL zIdB`E1&RW<5p7JyLi#;_Rr+N?>f!KnTrn)Ow}Z%R_m!uz@rR{|ySMKlq0wLWMO)No zS5g8m%l|>WwvDE}$5zgbiZ60!SkczB6t^$8!by{|Gu0uN_uyp!86GlfP?# zYjN~FNAh$Ds0z>;5yXRzgLQ;g@YaY09tv)i$GzeW(D8Vq_7fvR`D*)mCMG1Qvxv}A zXj1mSdFP6Y6*+$^95`nZhFM#74{B8C=TvtqwCz39v$KcVQ!p!Dt~+goPUQxh6X(r! zI)Qn5d5yU-ER9U|{@4SLlFQlmT}Pr+HjIAmCSjmg6jkePX}Zi7lOTG(Xo&(;+2_Wd zo97hIF#jJ{Zy6L<7j5n0?(Pr>65QPh5FCQLyL+HZyFX6XB5`ZyOed zZq6I>yWfKSztIV7vNsG>Z@ykn&?C$Kpd1NpN;Ask@pN|V2!ZeNW`onz?~+l2J7$Nr z{lloQ6NbavDR=jkA4X#jT~JIxbzV1sFi{m@TFGt;6*se6&u^dqb1BSVOQ7FnBW)yy5)F_@FCzjske`14B=L7vP zR}Yl8f`L^A4_Dk@^7>cQnGMxK0V#W7>p+fFtl=TktJq}Jnj+weLq^buJV4qMWvg#J z?(5x0vbCoxdGxxWU;SR@(aVVQ(aDJK&r_%BxtZSbnVs!@fnPPJ!XMX1*{ugxEuGFF zquq^`>m7R`=b7k_-fO&R-X+akbioyvQwE0Zh&IQmVdxgr!Ngoei4`IracwYsgQif^ zDXFCdp}Wn#WcdB~lz;D=E5pi&%nP2ybSwEKGXR5w=0S2SPO|m1d1SIUhljJzbYDj| zA?JFjGk2CqX$da$w$}T{uEFrxxj{xO z^(O5crgIjvmw&D-@%Aa>hUa#9#2(L_GgCH@GBM(#+aHXk=i&dFkC7z4=d%x?zbSmtqa1m!YIt5My}Jw5 z=a@W9gzy?WA!0JrLf}JuB`^s8bnmR$n?56CvgD5ZylIoS7yOq7xAm=#W~%kGHUE3; zJUtN2_J*L@VQ6baAn->_e*K-;BrjvBvEbH>1GrI}P* zrtYVBuh(Um`;+lf{L}q@{LLwCt`QT{9FKjTe~kONReJZ*bqkN}76~mHX;fdZQW;cz z1!pq0yb(VoCDPPb=vIv2QeYn>UC9;Ft;_GkAhR*yaC?Eu8(k8hvW1Gt*NEL( zU(gxqPZMrp$3z`*%EFCF!*e;SC+^iPx1Ky2}|xHaz=4%a?!)XJ1zEOv}!i+(>^QRbY*!LHx}^w-b#6nEBgZ zaS!Zp>|y;Nxe@iGF1KgHE{Ye#edZyUTyOg=D>;l(BtE5F>xSSBLp^9%D-QRfNhWS>u0pP&}FO#+E2hr@DE=Y4n##u=R+agled(U$!vz`x^xqViY zBn8KfVAbON>@nPgOYuw3HYU>z)u9_q6eExi_1%zfpS!b#4LxuqwzrLp(mOo^6*bZc!rec01SGCMJ5 zPgXS(jf5O?r;K{b^)Egy?#m3I5mF1Z6h@&lWa}DK_!adJjN8OteP{{P?ta80sUOT$ z`_wJ`tSdpCP&?-3Wa2v5BYQK1A*m`3h>&|&sK2=Aj`2q8o6^m98&5QMaD%Riv{G1^ zQaSNcDOp2E4q#MG{xRbzc7N`y#8&UtBko8iUtJ znD-|_gEa_Fa#u*2+?6ee72>T<=$gpQpf3)CTu%+1u_H$#vLHT_Ziw*>YC*1b>>aH^ z*m4#MIts;Rw7OtTQI?xfiNaO1mSP$VOjev2spH5}e1v1}<(DP}ZPV^xx#itCk)C1R z(6FbT5wbpnW>w>zUQFKI&KoY8M?YvGgMX3ZPs{y7svx!7(7&9qPJCOo^GJ1gW|0SF z=Mp279Cp--wJQL4MKaq!Uhu)EymC89=H-D)!x70 z6z%t4r{T~@f~q~^N47<#V7k4KqMLJ~G$su5Qz2+$TET>`+W|PPKMl6X2S#FgO|h9r zNN7f8DiZX$XMHr)+*oEjbyxY%y2M##I&W!fwq72|<1l^6cRyiX7WP_W0N%Lj86Rx> zhdwGvF;Oj5L1E)Fy!PcRUYgq~EiYh%=jrLCrq*SI{s|gMX>0> z`kYw0Kz(xkcUXuQizvb_$TdSj)NUBh3|9d5uM$YWfm89)6>Dv{14l}FkGs|UKgj9& zWhtOHSk%GToO%O6{3mYU29Tak>MP`G5o=JJIQ6*}Mnn|_G6SqOE%J3)?ho2rwmM$N z{?M=reV@ZU;q7i)0lxLSI^%sQvpHG8I5!4ZHw*d<{J@2}1LxQ~EynTw=^%Qk*5h2# zL7UoiG20t?om%8UydAP3cJtF5ne#gmepV)GmUa!KuY*IBY+`yqwqchU{G^-YOQB_} zSlu#Fx8@NY#iL{XIL+wO(T12EFPL*;LQ?Zjf-Ls_i<99!SDU=9sB{@MObYgoKx`S> zs?842cZjb3YCUk5Oamx2w>FdHb|RI9C+j`~24ja=dqjus;O&{Z6anCZjG?I!kalYl zQm+~<`{S{#9pGDe+G`(Q5!hY?`NCLLt|)!RV-2Y*v@peyG$T*jDA5pxd6dp%!=%uE zwp!)=Eq+UYJAQusS>wot#*u^61)nX~j4n54itGTOWg}kBY6Q_3w0Z z^mNT~hj6f5edeAy;oz1@lZ>ffH=Dp&S3NniyL2mwPj`YjcGk>B^DG#n?z|9A0OBQ0 zj6}z*UH4;Fb6CrHG+J6Pd75}dWs$(C(FdoQ!1e5druX5p+wA3D`3Z@(iH4h`J5h|{ z6T~*#4RQ@j|DnDVMT9-pFaG0V^SFqdfjJsp{7`V{=#juIqL&`5MnJ#SCeJf{T??Mj zb*ZaBK}h?=!HeO8qL3eF2DX*qM9dH@SgI7Z0-yi&CFa~|j_V{>&E&O+_ZdYz5fk6E zf8>tM6*Yf;WvjPd4p7h?FWjM!72=WLwP=~s1`i1t>$K)YdOgHw!xZGuR?MT6J0Bup zAA(e?`8Zp5#LR*w)-o|IHmGEaIp>H|u*$+NxUE3XQEzU^SsL5EX3!h?R9Pun^c!6y z`K#4qsx-iV3=osp5)@ekec&}?x3Ontho6(cq#f5v~v2&dHn*lCPbw0<0`J9wXJVSG#gzke-}u8i*2JP9P*AVTldJpwFxGSNA|j!Q zBYd3A1(v0_NMo3xA)^<=auAQQ{ggo@Rd=o}d?orf8f1w7orTiYd2CTbm9A(K3N7ju zA?+ZsfI48`G}$(#r-6}!Ha{>4GY=BC*mWUKb4Sv>uUQ6=T*aRbNddnL6%6g=8fBf# zvgKazxbZ_kbZ1yco4zXOAk*cOYWPqA;sQd%qydQE;eFv;RG$6$o-n4{7myEViTknRs?E;( zVIP#dn{~Qs_&s2iPmYuij*0WjME!+;G%)#kpC5}P9CGCD#Bk7SMg51(#^`QCvpcGc zT^P4Ki|$EWm1+#*x?;g#@>;%2QM@53k=yW4s$ob%HfaIwQZZu@lOiEeG*GxCe}}f= zW`Jv8j_ogrmxzCqr(kn%X^x8^DomCP>b zk_NMaxU`S9CVM^)gcum1(_Q6rTp14kqV>FWV~+vDVR)zvPkmkBUrO?$=K9*hyJ%AW zfik=|_sC9A6AYspk*FeX0i^3Krtqhi8dchXEU7h*8WQe5bN-Zh@OHsp(^-EexPj5Q z6)wW0Vz$9c@K4K$&oRc?v5KN#3B}BSzDeD*Cfn`XZv$ujx+jn%W!u)Se&KT`oF^y^ z$v9>v`)kQh4y){p-kzAv-sbj0(5ls2Hzi9(*QIQhXs!-Ku2mKvx`|7U;Z~r~t_l9++MPIU^45+o=l)@|O#kmd`yc%KKLn#;MPO7P zvb7vt&;Xwu@oS`$$Yg~M6a{G?Q$jCviU@+pE<;_O5q4Wc$xDJ zq6I6SwA?MFPBJZZX?{fTHWM*V0w^o^G{OmLCVzBltc1&$Iq;Dk+5}jPt->dP+tLp4 zh{1=%$J6M(y3FUR9xGa~Q^cku0JMO|-1d3YI%p}3 z?P+Jcr1@o5tqTM&UzTmBfeq*v$ir;-#RN_jW=%GOlyV3$NSF|a?*5p8JR4~TcJ;< z5+7d;BD$D5PO_tV{iVVbwqhHrJ2Pt*0)OMx|3o7mZ_|7l39`HFB)kZQO~#FYY-Y^n zY@%AZVaN0wTRMU7h&t+t&=t>7bRrP-8l1&PmhQ`z)Oj&!aixkt>MIZklQ4sbocKulr5$h4H<$euEj@NY+#b>)Y_>i3P(o2rr~ILQKlY;C)0N$B^kJMssSpMT zDpbO{EjP)@Q@5a*O%n@)DE9G}tG?RZML1ib@A>M=Wbphbl-9J^T>d;SYaY_;otz9? zczmK7P+X7%#vo_z!8&e19TvPj$i#3(dggMr@~?bNE#2%e1cyPz_rSiy9#mwW*&Yn) zyzw$z1$Fh5?p*V7I&Q-=34jJp@#UpJpxm3Y0nZ)iJM8H}Y*!7V!KIPR7wf~CwT#H# zl%a-hL9c{!%wxA!;{~W%3{+W-omeGDyqt( z2KV=$LQpv~2<(#gT&__@d}_0UVA~737GK-Hx%Yp7{{Qr7dllph8_N&A4CwH^dCH3t zcMT_u)P=jk?KI_;KpHCu%HupDn9tV7CDhyRwqBs&7ewvL5dxryRtL&Lz*ox_Qa|i5 zu#~#DSgAr;&yh-c<2aX0IG6FxFkvWvSJp{6gPJ+K=BzV-8b|mEbE{>I^o=2Rr^(6+ zCMTM#455gP-bPV;I@L)9tH3h9L5>=Qk#_(3pj5672;7^7R2Pbx z{9IA5WPC}T%(618pwmbidowf6h2TWo^dxM%)^pek+uCDfy2ZeCzftvB(Ut%TKsSwQ z+4)ay-VkrZ;-=C(^mQP58w5|`f2vb>QJ{;Wn>t3K8lJOWnMA{_?yK+uzng_QUM3O~ z*aejtn5l8;+P`eS31~5W?aEER*AxbEOYwkUH%KrHXS|3Td+( z2F5hn=c?(O2xLY075U87>^`9T_qQno@@=0Z&AvB^l^v7ZiWD{|*8QIsz@ecCYz;|( z&$rPnRY%Yp$qyh>g<%hAs^Uu;E^(~1rC!YU?iu%79Fp|F2)XlsiT8I3D3;uy$0yxU zy=RX33GZ7!!hh>U_8B{~C}-XO0H(~>xlFc?njb^tz@u?2#|W_(2h!hbtzXvM&LXnB zKKdJ?g@{arPWD?fYT;)TyRcaJHW&5cWvt`T?1}P?qd_NWDWU$G4@uHMMPf7!o+X?! zsWzggDb5|J%bxVJ&7l9gmi^z3D}h91ymzh%bmxWNw!fWEz$A!j@%UO`poS<#Elrt% zXYBq3f!DCr=}X&6%U*1%A?0j0J^-OPTDLPo@rQZ(re`k%v{j9E!&<;mMYmy~JfA2v z1UWHrcw9C=)gPnB_N5Bc*2>+~GD*uo;@X%bFO%S1)qgGEG zCj;PSs{{DO7ieI2%x(gC+qKprZKFff?r|zh2C}kStxtC2N`Pr3#(081Z_fs^vz`wf ztK|Be4uqp}#GQ)|P)<@fLo*%3atS~_`aoUQbI+7Yw(|y3OTV5$^)8%ta&jcb$p@3z zzNDq`@@vH&w(Zv{HL`~un9oeh(*EjBh49PI&p| zZKZ4@jbb5mMwRxvT7aeZh|R=y z^SCL?%vVADSxtlA!KB11#_Q9RjTc+6v@I7=Y}ho3N2c(8&#%yepbdD8qxa2MM-^TW zZ%ATIjB70oj0=_qcDcEED1wC-(s_@5WPu!jBF?C3D;8W?$+|M@CWW^X*eIk6DAC&n z;gewUooq9|8G^cR1RQ023w$Y0Z6dx0u&Wh`hTU(fH9iVb|NJc1^;8xz;)_!I@0k>X zDm|}NkIb@?&MJx2^sD3_7;c#p=;H;TF6kz?7+`~gR*jTKrT{$fO98_llG6jD`@L^-n{7m8*hIB6v-~QAEm;4*_u}>fNHj>R zIfEz!eyOQl2%f|pdord#q)38wwn0aIP0LyR7)E~5sl4F7di=og-MTXE=J|2o^CN`~ zZj(EUaAc0$xT8TwGF(D)#%W0=Y>9yOB>P2|Ujk*{EQ!r`y8(~P<7;6|nLL^H9EbM1 z&8u0CySJ~Ta69OTh+F}_Ke4PeTP1(`@rv0);u7hNU>0fek;pb=iR<(OzaCC%NYhkZ zw>WH0+9Mx2uw%n(Yr7T5R(Lp%MjPrcU)M{en&&bC)p+!g^G?+;0`au-Uv%HBi)iT~ zm<)OJDmu06qtL%xQ=EGSBK_5s|D4Mb5xQ4O1$6+}YI6sD20^ddjf?p1v9NG{=Pz%z z<1G2Y&Cce-Xb!u@p%FASP6&`5$n|od_kdVK)$Yud|1M|Gxblu22y<;yx+QufApclU z{eI954G7?j$zEJhef8MzxLf$AruT43t9W0N(}rB*MZ#Dxb9l1d3)Pu(ao!9F0U>Ea^8I8h zU0eOQkKX@S=Ga4FH_7xcYyk)b1j{?zWFER-1XV1k(;mrwU4@2*HpxytW*n^Aa)0z) z_?}a@f%JfMT`QA-_=je5S&!)V0mOWIpWyI!p$U>NhbZ^fd%S;}tj+tbU=)8!^J14D z@i`X4(!PLrQXH!RF4m7*gbHePfrQVW+W`kw3i%aflq@cHibe=X!RN>m$+ib^rS$8Lr9d!&!C zs^*9Bp8-gqXS(Dl9ldmEt8$1Jd~LYiM$I^#Q-=`K=;4W*-#`gTOgJHiz8XWWKe&w$ z#vim=Y5|d)3et{ofNnfXul~DJH)>l|ImG57f)_9+^ zdn~xPT>@Sw;u&H)f4H>U&L$@B?;4a+Dcp3DZ+EZ|eWl%xao*R;{T|;iM`4P~91Tt= zROG(r2KIcARZTY3-gW>Lx-@(m7DrK0^FwjZOJpFls~^dEBZ6-!%-E|c>-#CKo$*Pn zD-pQ3XQn4bxtZ=HR*yy_tQ3g`slxD{i7*&&0A-Hs91T|B9LZ=-z7|w=H@3MswumqG z@{EBWwN6LzQGjKc#XhD=NYUM@s(r3Q$ZMMn;t zl5mfA!ofBB+Da+^rmHZ9JhSO)8Ry0`TDAxK6$7nSpLkKacc26tpD?LjPtU8xc1S&U zUi+#r_5K9T7l{n7xGE9#0nWHJ(Hg9JJC) zKj~$U3H8z}nK+za5ba)k2WzSBud1YfjVMR4ECDzlVi}tN|;e{g1MbKlKMq~M2ig~^G*6eHE-C6(N)Q_ld9RF(5n1B5`VzL@Q zgo4f7I@{0^3g_xWtXk!N=$W}1ij9X%ly2?=)ok@|E=b-N{0_n!y|Z4an>9UhXEc7R28N|0-YlwPyl!2<)W=61Hg$CzYV@Y-TrTATzc%Ey3odm10U z>hvoX6|G)`v9AE-H-3_~4j$vN@S1&p^3mmO#9#mqn{NSQ-Z!aunLZxJL}%ZZiKiaG z*?x|#qS#$6g`MFIeylVhi8^?BMAa`c$KE zto|EwgE9{^=KjA22X8&mtkDXJkM%BzYz63tOVffWQ8O+MvBN@hU4COb+rR6Csx&3* zdNu5sRyw0WE`$DObF&L-R{a2RsHr`g8KbP7K=yd54z2m(!NFllF0LhreW>5aG7AON zr(t(`6cF&SPz6K#IntH|=8bzZCB>ZwIKRX7^UKmSjuutKgXnZkeoM6zKwd(vns+g8 zT04Es6K!NEs7p{?`fh+usoWJ!a+`rMTJq2}4-4-+#(qdR;if|k=e+92YEE!$5A&09 z^o~pn*=tM{K|{wNb*Xc&snKBL-a~Y_fR}D*LgO;*G231} zqPXGTYmngA!33hxvJf{8!b(H7w#(Ds$-QoDRo8N|2WfJ_zS387x4Et;qtq+_?m!R< zDAEqfNaX^b?dy-_u3CNpowfbtg-;o(1o92;L2&8Q_&OH}^!lS^C8dHtIdx?O_T!mh zS+SFtK|!+FXxW#i&s0Lc*ru>VI0iFTD2cvvt>nYXXnBBU^K9iKHoEPSgOism6$C@} zlu`bp!5PSvtC1)On;LHoj6@Gw`Ne64cmjGvnupgZ${;@Mf7b=qt^T%~Cw zg)1}HdcPH&Af@QKZsCYE({&wN6M)39OM*ePxyR(Iv$xnmMs1#kS~Pb<@>ei4 zV1E44c`umF>Xu2%A@b*tYrc0{golSJ1RmgD@;$p=uHfkO$GzzE?r(?<@og|&*6!7U2~#h5JQk*i zi5FVeU*CI~sHwvk9yt@$_AZ1sw!}tO}h~m&ClnqE+mi(K!uy?fkggyQH4x4G_>(L&?}N1E(N)J!qv=TA@t72 z0_^>8=pp)umdS@4_wefhAC7|I&KFp|%lfw6S0U&q;ICJrKmP3;t#UDifPPriCb-Mej?l#>ADu>wK-=YsRzrsq~MtY?^W7=5@Vdsd>k0ZHHi zolcbJ2e!64l9xcV|I-1R)=b30!V zhZ(9B8{s0_quBPidf2+39oR6dMIsUe711e0`qUFypT4 zzze_Bp6`K3JfO?iCLdn>Nc{Sbb+Vg>_u_FasRrr_vmk-gz9Ga#Y=k^BMo_VMU)BcRlTlt$+_Q zoqVf<7|l=JJ97Ka-OPx~D3{t>3Q2yx4YY>Ccy03-PHgx0>W#KycdvE2^%6W%yO-#B zvp3;rTqV*w+|DhstEVSMAGaJCx34(H{@}og6TguW`cnFae$H@c8#m^uBtAFcxmWJ! z`ecCV$?Met{nY(5!GVzdCL+jfo3m@mGk*OV?chrJem^2N$B`V@J@vv*LTlj5)ZO)9 zm`6bSU0}y0ypCsAWNLfF6kRx$oIY9My6J^LkN;iX?Z+1t*lp-PxUwIl+~1`&w2TA} zLw=>R^;@Yp0JG;SrV(>u@(rGe7|Kmp_{Q-SiAd^4T%*O!=rq-DHY*>0KL;ul>hXg% zC;n*nxwMVApTyFn9S1aID@m<<|5;GScZFg}$D1o#&*=bI2I)7*CuF<`K@|g60r@>cc#G%MD6oX~(h5UhLEQRdMyQ zOiR00*!opGuw6s>W$`f@YG9JDrhW@wIEG4{gn7l+JY?@`N_uexVFu~zvDzJy{-%T{IPMr?yIS|v_JL*K(c-Wvl za{0xEiV4Nlg&P5Rtl0mp%5!&GAm|882eEz}eJ!8;GB2ax1{wvoq-!XH;R1;b{7>-_~DlUwu!(EV{OZlFZ zxwVDE;(1xvo4>v~`(QWquIpNt#Bx7f+QeC=-Eq~l?mmoXc5{NqgTmV9PGE$3wc>h? zh?(hp4fMPB=E6Om3bZCawh^YDSM4*aftMil(fQK6}`4CNbp}3phX~uT^Q*_ zyNn0IU=j*XDw!rxo;mgNxCdzmiSydO1cG6_hh-@?V`wmMPkfZ{&g-pGVLPoIwaNw> zrV&FipnxuFL%U=PNd$^+$=W@TeKt^+<*`Ixa;C+xzbPfLw9|C0=9!{iz?@B}SR+AR zIzc_Hrzw$Kxb$1VtyFfvRg}ELTPD_$?{a=-v!nGHzK3=Swhi=n z@u`2ITD8zE$tm<;K&}pVpkn#{sDiCLdAz@+7fxB2g*}_#V=*#2Rx)+mqhuuz#7QQh z(rf7_yq!IyXPo@KVFe~KYCgjI=;n_OXj?xnI~_T`>|cT8rR$_GA~L+O_qqlwI2 zzGag_g|AbvL#yBO#}hNAqz;`40wuByOWsD2B9-&Y0#!nnExwB^i3quI61IeNjCTuN z&srJA+`ZXkj&HqJ{HeiHSip62>*Wx>LK5AekA&zi;*Hn#eb)K$+pcQUWYYm>TK(wM zFy|%`NC4klBR|MKQQ^F!WySt^zH+m+_AJ6!AG6QnX<7I5Gtuobi5sP{(r%mn z0J<7IH(px#-5Rn2)Y`ydh_`b}na&@>Rp*K$q(A{2caOa>P5kbW_MG`kwb$dN#9G&L zv;$m39CYcTZa}Nfm&pa;B|`{zWpQaemle0H%sS#zuPmn{*_PRV1mAB`#5@ndUVQ<=byEV2pXV5AR((6rm5r<-mZhJsX6x z5Adr8#}`IK=$SSwiQ$h(24qz*U^V)4Tdje7b~Lcjdu|tUm^>bZ(F3?;-`%1n-NeEA zHGxwZ-DG1C%BROA*;?958Rv|4-V?Q_BFU0A8!o!LW=bY?r!D^&V|3OlfotVBO-CF1U;gYNbe?P>5)gGLC-6TvYt2LR{J)pfaXu4_OP+wLpkc`kgrU-J$ASaz^6PmQb8=9fJYdHs9=go{?kj;&&>8r ztEI_$VHhP(r^&_|PO>8wmOIcn?s6&R*L<6mvLjp^34v_@U4U+!@Zm4c#5gt59f3a- zKE5aftfK+bP?y$*V}0uFgiohhiZc5MqMvIt7);>q-f;hyV(H}4TRKWS zZO`Uu!Fp;){*Nw8T}}eSIiL?~6X3=J-EU-o-4rA09*W1LoAHZF8KM>P)=PicCn~Y2 zE8sCO7(Vkh3>TdTY*t+JjFh&rqb2r$9p7n&PU`nZrY5kMa6?XbK_&+@4rqi4ywJUk zs*8MmGXXq0$ZU*}CS~gs^X|7opl<&*I#c)PcX(dFVyds}V7RfdN_kyL@f> z0mtPW7q{35UTx!y<|Qm=N>V6Aj*%hC0{ss|kZ*wxzXmCaW=naEX&rIjV{FiYY}^@X zg7Ws0zKRcoZ#133Wy2xE)^^lCC|`Ps(?wy}p^CUQH-%$70(E}C)Yl^sK<*{iSRELr zRKfaUtE2^dX|RHu)i|VedmBwgcE8V7DIIkx-R=o`UwuC9luty?>GY142|L@>?Vc*_ z+DkSRJN9sV&{pgf?(Q~CVAh6&&r=#x(K*$=b@wlV*UgDrGUWCJi3DAwXE_pv9_&C3 z^YPESZyO!1`uYj?-4v+}dOt^>)wtX<<#ml~yj|iYrNx{KMn%FbO=z+-i>!N}wO=m$ z4cCv%`&?gVciRMYyp6TI{lFGJ9LgN?e9yBspw-D`m|ixl$5nGY>6``Tj`q!=@uD z5@GHUw;Lq#6-^NFOO?)C+pC7KGSX93V#wgNGS0LQj10b>q%2w0v@6^cyB%HT2eUX4 zk93E}r94?O?(}=ssAETeX=#3+q^i8PFL}xzsS>$IZiIJd!tozXE0rSkXnRUIuX;-DBBH3*Y18Xh62!6c^V-CCP|ui{2k!i z#G$dBwn{j<-G&Ouv`+kXCfxB!JrxW4K=znEvAOyrOmao@FXAHM1U5=`D4!L$K|}J z)Jk%*RO}Xx-8CV zdkw_AZ%7x{&zl7^YpSozku|>yIew!XS{7FH*;oL!nxhyaE$t+z|lfvqN1+#7F6eBs1CQHAobqx@cGT^ zJlkFK;4MS_25bA5xBi3d22<*7FTXh&~`EJp>VOL{-FPqqy~8SLEbU zaXnra@pJ)w@aghC|IUZKju=3Eok;;0{rap=#~`*lP(xg+zw>LML?OX{!92oypYUKN zGq{L$o5OpNCsz~{fE-TY17to(6%5@+`0Kddj2hcoU$?Wnjw$rUjMI5dfMw~2#B9pB zovb1WrnBsFp6a4=oST+LzRIr%ER=wjCI)%BSuIr^s+0%z)0dncgbF349QX&zA=J;@ zO#yxc2|~3Fu8Wv~r58b7_AU9%=bjY&<>&Tg_Svns{{bIm>Zn%Ng;QvYD7EruN!ax* zCg`aqJ;Y?;zh8eg#vqC9M_}C7kVlSg|B(o*()2aYFU{+Vb^N`r1S?RsP)Da#Z~b|^ z+k*C>bV&Y03;z*3gz6|Ppt9b*c&w>}So#Y)9DGeLKEX;`URG@AmW+&l&eF^|-WhAk3+p;$9}WZaRyfEvr$_7(8A_YM=RBmXl=Me$+s62i8Mb(mgl!Z}d%B zdg(qZo`GhndA8BctI^J}=g79+^|taYDflh2=0BWM>8@2L&a+>n z?61ewPK>pGK;5=e@X)|4Y*@~)uF9BJ4Zq-3{if3O1PFh&?x;)@8`Rae;je70L|>z+V9y%=zd_cGbi|Zh1e-!o;$B34`8-Ab8)}!n{xto zq*@(G!fDXEV)~1tQ9ljTV(RGP|1;HpQ&vhrVUj@^-#> zkkP&QVgVv2?vS`t$XRA}aG+7gw;!v@^R%}0qz=wvlbU+5nG)7_D(e&SJlXireNNKonz4f&JGEUQdHFJ#yC`Flh+r2`&Cgq{R< zTtmO-{xQfHNwa>QG@8s?VJxiQH6vU|E<50Zu2GMx6Q=E7&u6@oI8>);Bx_dUTZ(2E zY}6lTzs?A(&wr>D3_h~;s!}?P{rx)U*0r2=NYE%ExYGX1ueO();LP(P{N3KakK136 zsPQUn?l?D;TfDx)A!*KoKo!4h+m`e%xv9MfCmN9D+{}o$a(`v7{5f>5_c?vKooa7k zj$-<>=Ik#xhAU)QqRpu*tpwiOOYyCVTy`{kMOr&~C5^o@*PU*8QG_%17)8B2_f|)# z{X#cy#g0~x4iwr2Z@j%-O0^G~*m<{r#1aPJ6TjbMfINjES2vZuua7uN7yQ2LPIAMU zps64Z83jdS4lQ@&N;j-ChXMA92AQ8A)@irmD#fT2dWxvR>^z$;V@63c>)ss~b{&65 zqI!yiYqaP?Jsw(8cz{#?`7Zyu;5~LJBzoK&!>RAuc?yis2cs^_r1x7?)vD~J_-8`Q zP)lHRd1ZfDZ87wv_3;s-ed9liiyX~;!NbCv8CuhQk9`GgCiu&HHu~orF|PWnUD0Y5W<2tmc1 zH!%`3I><(YRKG{Wfn1j=L){>l=t|;{xH`-1f2&eYsVYgbm+~6$B~RqYJ@9Xe31OtT zM%3XWAwZK#jyuy^hFdoFSV~8{mK>Kn)_@T4$5S@bIzu!ezDpy-L|JO!f1`qe67htE z&ITiw0yl5*Qp%q4Du2}mFdV`OZ`6naCH~A{8-0r^3zFvE%8=TMi#Bv#$>ez9_ zzZnzFXt(H;*b3<6g5=JTSL7DB&{#?@ZQ6&|;!==%K0mvBx^|IOeAZvxv?nOn zx>9ppLwnqLCK&3=BC>-YiUP1_>-R;QkRrQYu;EeP@YsiFg|wrT+cH@tKIJ+K^K+5Q z0jDGKOwvc*QZ{xh^-Fgk)nsaHC19P}AJaS_3D?l1YV>LTjPMhN{?fU!#YbviqIdNB zqL5acr+U-<=og;kfM?FgT8|KP{O=2t(8w_@z>BJt>&o{FpfhE&fitm)Dx24;uM&^1 zJt()M?rM2Nh2pD#q0;w3T9ZvfGO_a)8Y)U|v~|aKe5^uqVR$FRp*}UU4=orQKByF{ znnP%})(UCj%ME+vn~Tm;Hs3e-N(8r7-Qn-@C%%Ntmz(Q$8!i-R+E4e1E46FH;k3t)BVzv9V+Hsl*9{Y$Jy+>X0AWU-&@RRy2t2~o}UN>e6P9tuD?*t zHr7k|v~BMnvfvHuuWRjP>bsA5L&p12I*b(A7v1+iC3E^YB3|hXE{BfXlrm5atd3%BFY&;3z-@iRua*J*1zCIr>A*UBq z&z(byd(bU>iWDftf&`~H6nB^64lV8!DNx+q-5rX%v``?p zQy_S;;toNATX4PUIq!SF`~90IPj>ci&mLJbYmJf24iDx;&0brPb5_lH&p~l)+r|_S z`8|bE-h82KSBshM)1|>JAo)X^#7pk21*>h_iKx_&gJ3uILNxj)0r@IXxA1e&`DisS zOwazog734XKkFmJIowNcJlhLTOYBxK8UW|h9P~T;gd2zYg-o~~q`33khE*8*K;Pog z56N6`ib?i8r-V~A%NkqLkL{8D#x>*{9u2YTvP$d>UhuaH2ywm!Q>md~*Gl=25tj66 z^?x(;JWbJO|1vd$*w#XIXsz@t7q3ch+5@)UGhROzAi;Be3T_7~zduGH3z;mg!9G5C zkPef+&bJuBlsPS^LSJWY!%Mknz&88t5Qu&JA#<_=al*x8e6WQ!1?x9EBg|>K=uXAx zh&vXqsIyL^-e!Wzpsm5RGh{_X0q59aSa`7sYc3#wN2dBs1IeUqm?g5-kwMVnsYtLQ z)F7fXz?V=2(e|O{AjFb7@8Q(kplyxvvM)$yGcy6dXq37VX>o1>vIt}QFnZ9kn-o2I zP-A*gd5>M)(Ak8lgB0(HB>NWGmG*s*bPN?t4u$IN&%BzjOdN-U$OOj^~On!>z z(S^%Rf4R|W04QD?wdyAk?8}Cv(n!|1{1&)Qx2@R;`8r#VIzxqYsaxP=p$T`lO!_K2 z@Aa#xTB9!-J1-#7BigAoS6W&sJAGyV(EZV_1Aulx^7OFKqCq515{Bt+l;kUYhfn*#B=6Y z*{RBwocP@akMRw^2@$y-#o^{h8e`6{f@ZJFrnNha4=`VXf6dUJAnS_lG?M9U)Ej^v zS-%KE>CKec;L-#o`%9P)`x%^@SDCKe>Ok0~B1SHBsruu>GYECY8HstyXcT3yR*UTr zlX8)1uMldT^SJQ!ci!bI8T0VOV+g^vxY*u>fUEAT=BGJOEY{D0oO=mriHZI&@zP*l zOwG~&Pm7uC_xYdF-dGGNc{iFdCULR}Cv3Eo3z@9PX$yh_h^|s~Sv#=u@*PKK;`z_g z1nZ@20%0Hq36{P^jGme?tr7b7!N9at7`7N!1mEF13%VsH#=CntD- zghZl?$8AF(+lh?B=>82Y+^RupNS|JJ*(BaS6{3ZXCPPnkF>=&XKcf4L zsTb)lQ*umTA6$gbV}3S*sEVXoS$2h=-R}V+g6z0dAIk5o1W5Fe4iFG#O4LQc?$2s4*6SiC{j)~V{^#YD@q{SM1}CM z(VxOh_=&yp?>PSK%J3wu8ztv9{$n=cfZKdj2Zx`{4Ey$H910ZEFsnn(y`Dm$jfS0Ds$E zdT6Km$Y(}Nqv=B{?q-m7J7bepY2)N6c9``%JFETh)5%jru$t8Jd3l$|jLd_j9HcjJ z=qC=1fhQ?tHa-gA!&5^$?ddB$SR?sQ?ZauCvH8o9rau2#mZwgDkf#pks zOyo6Yg$e~+oB?Ueom{PyMlN5q6W6lIE+}}_cmw1yxVazLeGS-`J)6g3120=%G0Wel zSubGt%&z3(FWFU@T}ns$)+#IS7FavTlFT=l$l=SBu6*}Ut! zsyl0}8;()4k3-|%OJT+blK3>8O2m(Xc;V?_giIXgJMuBxiY-)zsqZ(=oz+{bbqCt7 zUd6Y@`!JkWTUSNHLd^LR)V%T{%EERz3=hq`(k%S4$lt%{H{h<3FE{j1scB=^Cakn^ zeu}8fBT_Y8F9}=*?3+#RN}kUcn=8%)xC;f^=3m|owzM3#qB^}(Z&#Dw4+KdHMQYyO z+|uM&O|+omsuDe9ANtP^{cP!!o&HBfnd9$nS{N)?o-1SkZ$LG)8%~N8Xv0s0rXzYk;l;h|RP?_NHf5Lh9 zc^DZ@tbYd*(gV64mPI^Y%&T>wbq3e@$$#{@HN~yvT#p$rUG&$C!Q+mNk?|AwTF;b8 z?PhSO(7m6KOY`u!%4+L$cb5P2VG|JC3A^TzyK0^6Nr{unG{H#5#TXtJLFTu-GmVVV zYLu#EdTR96gWEbj^)E}nO7{6dL&Vwd!PbH!mj)WCP6tN_i> z>FHM#$~CVK8|iW^V*OW5g^xsf-sk~7t7fNtCGUo-QA-O{+t0l}`Wp<3-nz4bFF%nC z32KUZO&RuJXekm~M>7qwU<=)x!FR}Bn}@-<1cwUOoAZ3;Hno2(m>a2uVKVF$ni6?~ zZ36IxeczvrGa4P|m{lCaMDcKeUduHRHdd#*Lm=QREX{yjl&Uq%tAq4DCQy}5sj&8_ z9SjuW6Is>U=y5;q&pUk|D7&kh)%?%ZUW!N^p?uvPBg>)yoBZzzx2*BSktXXMW(0zu zwxB>U4lVU^@Q42#KFTJVwA>1GjY#)ED6j= z9S=9Jg>p#F(8-nfgH%G zDHhj|>BQ&8uU!A!GXD;WwG<<(c`UQnmeBa&Qofqy#(zc|ep&QO5zmIM#ko16btuU@ zrTnwTp|inb$>pHk@$1je2Y01-w_nBx3hZRQbNhUT*J|(@ff~VpkUbG>zQOY|hH$!0 zgp(7?8j#=mK)+4-Mr@Z)+TE6_dKgBU@KYOLJde3p$ATp^SwU0l{V=2oD{CZcDg8{rxuXjHEURu3`7!wGD6Gr zB+?fHl^fvy;D82UC3KRF&11dYkBZR^r&yIw=I&noKhjxGa4=ChoKW17rfZGr5u>-G z#mq?HHUWoY45b3i>4q_Bxm(Z8ORx#8S%IdH(sGw={jx$E?}sFvlCYL9c6zi)Lw-WU z_%wOtm5XOVwa1`qlD;o_NG=F+_oOcqEo-zmtqYXxt z;w6hL|AK`oIvq>|&)!Y`miC{JMj)GCzyHws*@qR>8{#?)s6lyE?yS3$IG-7}q+RXn zR(dq42^7#Dhkw696174ahvmidEGtyVOz$&?O7gAejZsS&gWd5`#L6E3@9Rd~_2Y9- zH`(M{+k~Fb;^ZVo|K;UUFy+7niJR4HQbA4mI;dAn67?WT`$erX>UFa_!7ojn$T=Ur zq9|KWZ>2RAn&I&BEAtbulr*)bFDS)CRSvt3*<9s^#X;fp+}vM0;&Bc&@tx&{wno2z z{<)%f(40v7J34w4feU8SNvMXD6wQ5}$UkhtXRRgnJsFiAg{pdQcv)XwuQPwrzqR@4c#oStJvSjTx^+g=s#*$V<|tA~tm)*`cNMSc37Km4 zLsTDl#Oa3R2i%KXm}QwvemAV$AH6xB>Zk#6EXUnmrtk3`F~1e+io-Fy*|F`oTtko} zfNYU$g}TUW9T~}H3-=e6m_n7V7MqT^nAeR0=|o-^BKr4|)bEbp@DpU*0v=jrO5PNVvg$6&EV#s@-|P4SHgw%B;(_Pkhf z4VhlLsRxuPA%lqV5LvU!@3~}s^f%-K+eeVF+zmCf48?&@gFjk@|+ z6@JD$B5#;KynwIaawId-a?y+zpnZ4IWI7dmns}=Ei{t5j>zjA{Ll@aP3u?T6LaN<2 zi`|MNHEfE60T#i+^^L+WqG9nQZZIE@t>sn zF$Gimh}_$i7u$J=c9<@9shU6L5uQhpSYDsW!OMc+Z%yT3!R%7KKb`Ig9?CYdOR*?V zH$FS&5jwS}z#>0=e@gcj;i0h!$t2m8zM)5X0Mm7>>7plJZ-VDtT`W>N`Td`Wxg&Te zV@8NzLr?%bY#kJWTvOn(dB(Y6v?4W9HM`bjTZKqw-t#U_>weUUsp;Va5D2J(0p(QX zndwWI_~lZF$FL?bx!`v($b3Mcm}(|YG6}ySKQQJ^LeF4;DEV@?FF}TqKf`niMO#v? zv!Z8G*tdrhr4Fur^4N=xD1A$27Fu?X6Xp(nf8w=oE6p1BK|rr>e$)9JeXgb>uNIF+ zFCQFwG!p^>(dfB3@B9N;*Iq{R*+n|Gl7&1@s2cWR94ePAgX?Vhksf{RZE&^4Tmg*1 zgl>ui13rdrCKJA0m3D;Y-;bbFOmyVaOx08xi`W2C3S*xRn(fV}f;HMZ;bP36J^``u zHeWWea8Jcdd}A-GGC5_)IF3X1_m(GhJ7#Ityu>=$VCBJ}_5-6>R$s*CR37&&@&P98 zd=rhar#q3A^Ms7wcU~Cl5)Yym0^yh4<|}2z$h^Y=g`Zi!DCdVir{wqd0o^gawD3*z zfj{z5zqRPGrBf)=K^K1BKjf#J)e}Pss;B=OHPZfKBWW9Ty*x3$Y^xTwWF~eCYyRXC zxI5e;!1_pH>@*3LW-J+JfQRp!BbEu_gCxsz^I<8Ah*hdm=IB$*|&ZQ9tCQVpX-a9f z-o@(>dL(=6u%+{LAEsyg>13bNeA(xWe|>SoFva$Tx-1@$Y#3XqEO!_puZ{8HPc^>hmjTx`#)nwOK=4&=vd3nBz!!*9^YphzJJHv&zqkTP_B1dz<_m zl>m|bqu{mc_wliTF$%wE)xo!{X(KU>#=^nhAMFSU;uYSfq+zA}2pZ+QS3_)EdjDT? zhW(0lyERf9l1ozRoJ2s6ZU%tvhuPCao?iQPNq|h^1Np>I!kitJaA~`o`Z0zG?ld~Y z4aQI-qCL;sb4gjMtxHJx+8Q>`K8PP7$6~o>G!MbublSHQ()H-6kl#u0hFRntLAwjJi+ zYz`8RAid#IAv<n-zXg5ww2y`pTaa z_ryo`qWc%(%M&IS>pm;=@O)%iOu*ug%*LW_fRvdR(b#0ox;5e4&3enxn#az2l5F?g z-uz5V<9F+*g=8oGcQ}_nDC}0EF6|z$S@Q+whb257kNdy#gOVhFV;$c+#CB&ND;hqO zRL_ma3E1-&&RcNd2`8{u-WT_ z05eILnWY&k5fVvyR_x?9O`InyeF!*NKl`=g_q#C71UmP_LJ5c zm6|%LEm1@gTQ{z$RAQLN7sE(-N-gB28N;V>X3A|d{Y~)Ou_;g9`;JFvM#Y;|o4h^> zQ6^=?%1Z}gOTVbu);~q`=r()~xI)Q-gd+!%8Yqv2hRukATmiF;gpU`83j{Q?E}-p{ z!zuSQYLp==7Xq2IbIrb>V4lQJS`=Rk^z_5vZ59(8wW5@g^0xB=Hy=||s$_70yNt~1 zgV{<;$m{Qnd)pI|sy8Y*&2B{FYnLphe33PiZk0}l2|{N9mF}DO6xSQq`PZMP@(`*< z&I!d%3rDqLJXfUyFEyKP4cC*Q{Z+?zp1ytlIo-BER-Gm?wxPeR23&l|mZ{%8ULqO@(5av!73-HTLcbZe6#SIXSwqRaML`=9ZS!Pod{6 zEpaI?(+}$=`5ji*cv6MX9q3l9e<%C>$Y;5=ZsVm*rFgnA`Uyc7S?J0TO1@GuG6MOx6sN z>W$tiYPz_SH`?Mgq~{*_&PT0;0S)S`50&{K^|O)DQD98K&qEyp8m%ehKPcKxR0p)X zrkk=b#LQeLSZInG&6OJ4wDHTwmd89$(Bqir&eJLAGy?fTp%=}srjzuz=pwkq9Dk;7 zq5DV2~>jd{@t0`4{{195ap}F@y=nxNjnQ;qt+ruRl zN?4v6H6$c`hj%9|qaFX@1A8kUU7nwkI7hOcw$wMkEV^u;>^jswDN zlz~uKjnM>%aQ-6?Ycj@B{}@F#A&0RD|BEQk{C`M#0Toh}w2w6E;-h4^INx2Hf0%Fx zw1zy&+XJ)L?PvQ|`&9zz9_N6SBN^*~M1U=^sT!Lh>RJ_k)&^K&rCj9++r#ZR+E3r8 zms+UrLQ#<-i|KAtTBXy~lXy_%()FD+P5Rm+ueZXBja`KV#q0cY_SecnJAqEw-QI7r zl#yRs#DdqDop3c&EG_e0+#>^eokM@57=*!rL+^iq!&+3*x36f$typAZ+>;9o_9{=W zk#rvi@B3@oZ*4rHeq&-8j^Y)wvjO|3KnwbK^aLl6zEymtJAay}Y4>|aRk|d$*IH;0 z(NK4-2KVMdcj{k28n{f49>wk*IM`6d-OIdQ-3h;*Wu6usry7}H(9%g>5{-3@dft%JY3CiFl$Bta}lM zPi9fk&z^5cEZA5>QTynQDtl9%JA`Il?F*w7Zf25P$*kb!{5W#R$Qfle0n|5(fOqiavgRNA47J4jf0b5{}k3_ z+zwc1!y|FtVG7)%Vx*|}sfohPYvC&C%69Q4r(>WzpAo5kT2wvO9x&K^@O8-iJI-C} z9RsAVCzQ9YmFWS(KJ>#(X_vzksxm14N%pMt14nd&PG$TnvK)*>-D)Ff58?Xf!2O34 z44Yc#8~Y>ds%E=eG^TXlI1De97*i^Tp>3+3Y6hJ`&;0LnYyGdD6z*xSDZbg1-xvmu3^q zZN1T0`EqGuKGLX)IXirs@L`brVy8c0{bS0f``3D)q<@`v1^3^n0d5Vg+A^GX!$yGJ zt)wfMB5U9H3GtI<19KWd!ZUqrN)7gCfrAK_Q-t6lljbg8Tcon&VRY6Ty$>2Rw?aGG zrC9qR94_02R=f}KoN2jiG(KLNyM;FF5~BFRG==Z|_-WoOD6qgWRBEDESNrmcX0MZr z4JMNup}`5ELcsF|sx9aeTAr1~=g38U|hxng)}+ePhuxQ3JS+)?6$1L9dksbPwaBtAbK(vf9-L z*1{-tmy_NhN%}2v^J#g)W(d6s0k~1yaCWVSRqMEbbWyH(vR=)haAhZ|&e&h1_X2x^ zE2t;q375R(As@00`IYGZfh6ZoXV)<5%gs3S4Prh$Ha+22KN7P1aNgYe0=tYZerT`! z(CRt*+U>mIttVOOjoCd46UR}O(e4@m-&K*$ymJOfJ3(}yNlFv2zx%#-#pkk1os`Ig zS=aF7J$1tXbYw zl^I;8i>P$j{?ZliG%Xcd5)WG5AY{3*gqB*0hEDTRu1g1;jHA2LS@{{ASFTGE#4ydn zas$!Dn4eb|1Kk#AC&J@1Y3FooyMN6i9qYfD>iW37Bk)1d_udNYfUK4rdWtcQmJ75Y z7zcN%8xO>bA6v-y-~0g6A+L;bwuL5BwVgcqlL4w2Q4bZ^Ffb-9>3)Zd8R0CHwcYILo6R(tx9OfxGF zziuucV*XeyVJOZ9as-bk5v4r55fxO$ggE*sPR>@Ceo6OC`ycDqWu zne@&~r7lz|5I`R*V9B4+i`Hyh+Q9NJdRh(d_W;a`&Ad}Y*}3E0e69)XOZX&t(+GNqhfHbo z^Gefe+;S@HcM1#H!ovxQ<=vkKUx^MollMp@7ccPN?BLPRDiksMlFXzi^GNd@jD8EP zMw5Q1weECLK)>=H(;Mm+6cBtTIr2&C%3MDw=>en(#}zPHX}uNEibF)HD+cpw@_QTA zjpwNVj}vSdWya1W5>(XZ5OPod>kwks5anYM0&lIvTyORzoD3=}mZfiInPZ<2d(uA(7l`rtI)B!|&y-L|RRNgw z)FvWuFfif2r|v^|8tjP9FzQ0s(#o(KJKlr{-hR;kPB+q0kfyMo2sqY}OD{>MOu=R? z{*#vS^D9NX=n30d6Ga?W*SC0CbKzYZm(Stzj&s+W-gY96eDwu_YRHaH9TDetq( z1|Z8h0}Qdszf$C{!1wILWGUdZEa57Z@kqEg2Ujb8?GRq4sw!>&@YMVuInOSv6|;V2 zySU(**`c;h3l~KNsr&}obRapQ;nJt4#ogFT)-_`nqhG(%GO8#Uf}Xfyq`Xg$oC0yk z#j5g}zhco4(B|HK+%)*(b-8@?yU84m@{+zs@;s$a+KaJIh}a#IY+8Atrit5syDOc& zf^J3m`yohgC~PH&&Om!HEa@%?7&h7=yXbsIuSJL4hhCM2H8bp)uXRW{Bu- z(f~2;YsN)8&s*5lh2~Zb!ak6#tuZ5P+bRZd|5zerxtSW)AeC;ugUn0t>OwO01l3|T z4a-PGjwS_){B>#`aAWOoT!#8iVB#4zS`9gd!Mnlt^3I$h+)^W_z9(M2qPPh~kmNA^ zSl=h@{Y7l~6rI1(*&@U##n+kdq7?0Z3TBhZA~-Mt&#BLgi)}tA zBlDk!zNJ>h;fnWZO4Wl3F5HHAS7NSUg&*Sqp{6G=g}QP`u>NWP;vg^G7Ae?gO6Zj$ zpWk=N9SKRd^W6ndf?$NMrAE~THQ||;%<9w0!bXPY6z3OGz60goG2ldTL_<`Z7qxkg ztU>+?Cn;`7iGMCdRb@qsIoZ)nN+|bu0OitrKs%rsU65Mx%ygq0>!9N>gR?7@4{)9E zUqB#6p!KNxkyihQAm|4r_AV&j&1`FD0Abo3Nl8b)jhOX`GjE~p-zQz)_N5%Q>TIkEYyjGla4hO;R$$r>2_UN6rh6B-7GB$p3*^_}Ci~$w3@g zv})ELED0j&7K+2#DN&?YWe74Kr8?ydqVXa}lU5}boZ;)XW2R6U>znz>MJbm&z;7}k zalEXufSu_V-=xA)9|`kce+-8_nv`NZUPEgH8Uua={_}H2{$+Ba?GYGo@6-i7t7AZ~YkGBRl`_P$KF^=^M%wtCC3^t%Pe zZl_qc(=zdF`|g;R=whyn4j!T6EVKxC2*jKni}LUY5h(Q4e<0TG#iT#E!kMAD;WVO z`iuZaxmrJzeN%D3mho5(;JC@Ccbq!^zU!rW!~lC5bbvFqgX|4SsJnttewOrNICoru zANZSZ(ySPq-{Rm%-(I9?k1o87nxj0!!?O5Wev9DIpT)A9IeE(1jCLrpJRZoqPuL7u z@4(I5=cpLbj8{A@T7e!`i--rwNv0ycO&keqPOe8f@%w`Au_0cs-YU!*j%XRWAiQXm zFo#&vJK6)3UI3FIAEsAcA+e)Oo-T{lExX%z#lYoD+sp_{g4af(-z5I`T@d}7?2EcL znJZIaX9uO5wyxE+xyh8u-Br)B8LzrCF#z-b;E`=tj23$PMasyuB_24>gsXLm&b%J` zi3u+Te{F{U1I&JVZ!HM)d`qL%`VZ}tf@x^FE{)L4pZA4{1itp%qeg73@?eBo9fEL) zqEjU$pyG|EZ*BQ2VGgGSrg_tY_D01~-9)jupLd?RQEOG55l2m2@l(!3>{?J( zFEC|ix4nB^_mXIn4XmIBelr8>F$*0`()0DIBg0k7W*I{XfMGW+Y$wNTI2D7-ah z1k!fusTvPWa-yLe31Qb0{o7NCTCX zkNcz6;=p4RuubtQ*^sxrV^5eHIM3ll-mDd&x}F*GoRJ$GE)UUO>jaLjXUEGz?ceNL zM#ZqaYn}09<1D(2kqtoAtu$vpEj&`UQmg8nmb6e9cR9bERablY&Pce>Bzn>8AM;?} z!6g@+FXn`rocf4S*E0^&_iF%-*57S|fE!V?;BBor()+f z&2uPOM+wh%VW|lx!RX-@rUwxcd#9f0Tdbpfso4Vsnx+K1kCwzncWn(BZsqxu=8YzO zmJy=#{A{->YoU!4kSBZ*js{6dSYz*++q&ufI>iUH~GKxb>g{i z6#9~sr{mdQ2Vd~iKz@Fn%`7Eb2d)x|Pd~XpSK~JZr*tIp%b>pb^gQ+IBKSlpzNCP+ zxIu$nc?(^qDn*#S=On3*sq!h;KL6H$t+xpQ%Q@6!JuQX4+B7<;f9yFrI>|^Q5_PeP zRe{$y27mpKs0#u75Cgtxo4=Kek1LvR0~y3Q$uT{d=b2TRBM!6q}*f>lM=p=PMe8$BYG{fR2^|4)bx|YYQ{h`XZ4@bO2S>F>oMs(4~@U#^?5XEa@`Mo*J;WYmq)&DoGzI&b!4h*zBqYrD-(XkGsO0&GqtSus1w|jcjEX`e-`4xR?iR!G8_bD?lIyqac#&M;*A5_fuA~(noZUdnVdD72^|WiK8-9&>?Q!(>D=a z3DlSez~?YOQ_3cKR%u?TL(Ngg4poD7q_QM^iugcOCSLe#i2fJS^xbm0^5Z-?lc8VWC$Sf6krNE z0bGcDdfbq07FM*zLRh!7*nNw<^UeSPzG7`2Bv!_SU<8-{N0kmCVTgezF z3RPUY3zhgq02|`NjtcZ!U9!J;%mHP=HuWiD#{^7phWOhA@)|n#+yjCI;VtvVrt(o} zwvjhNe)k>mm$A$Q&K4W)6K-tF1W^)t4_iFnB%3@IJJ}6Y>h!H3k1lj~iG~jv4O`t= zQNvO3jSUBuC;G$3PFLy)g`<@(J?fMejmdd^^q(OM(}|2&fMQD}CWEJ%ZmRvD!Lxmt zLFtI(qhXeZ&yd-^B$0<#zjZoDT8)4Hf~%${QdR45S73sF37FTD(C!Y!L zbSmE@k>VZEgxwLzJAzH2gYt^aXJtNDdI}+;?rr5`gt1MH4|ub939F zg|rs=Ezi9LJM7klz%+kXiL4G9_LJEp(hB3>2#M+Ba%s=?$WJ$5{!hE12Y5`!9C60+ zJc8H49ghnfo+sy)?3S9?7H=~>1q4sC;$+vJ$*oo~mHWx^|HrJ`P^E@2#@3{N#N@G9 z{_7(B!%NmHY>JP$AEMn=f{om=@6u1jy5(?x5w+;N+-IHB7cx`dAPDxzZb^2E z!N0mLd5bp!oqeFj{cYMD_05J>R<=;qne-D_%u_K6H`L038Of&u`wW9dOopJJBT3>J zl~Wi>EWnFBeT$f% zG|pp3Zu&vpT~^MvXLQ_Q1{rIG&~}+?~8wi??<&c{-gD73Wu({ zvep+2a$!OuA|k8hrk_NIKW0?@ywui##=cVxVone#wRxF1QqELwfu6ezJZc);(|+Gq zg94Jh(}@bn18V7ADqauG?@=8M2tB)ZM6M$!{C{=%J=DS%(sPaD(kZdSqS?eH&ubJ& zQfJ+75z?Tf4gdeJvNwb|(>I!7!+-y2y9)+&Ppy0001Rva_gI8%ddOPa~eRp72nvvm@-? zt$8K!S1&F|d-J-*zNzNu4A@o^-!@A=@#JU6*ZYsCQ9yRj%{Rkd9uSR|85Eix5zE%O z42<%2O^v&b(g2KbCem9e6vvG(#muS(I!0H@%E~;)^(=GB1`Lw1NYX8J*TZ?59W_$# zI2y?Tra1xg_ri(C{2=Ek=tB7Yp0ll`IUQKR3otltFa%%AQr)^KzqMozR!%T`bb>6& zUYa}z<{b%LnQg>)?w%WI9*y;pcSUA6$!`lMK1Zqw7MC9wt9#<&zU~jIKCPg?g4~`7 z$oY~=zJfA|j2zHa|E9K=arhqN^;XGJq)O7P%;_1Hbsuq^zf}WEakkCq>2#qb@NLJv zT6U9cp5rVpt=O4SokaqJ3Vs2I1&Wt`^*HVy65r`J-XK< zxSuxgIX{1Je(ei!rehN&lCEpPhtkVrxNd3!!btg-TJj6~+aY;d-4C zNswiluQjYB>GF!|=re7wZt4(yWF-)=|6L*#jQTwo?QA*_;~4lI0a?__2Ruc z))N_KfHm08B0QQWDz8i@*>1S=JJ@G{+%)8wgV#FoH|3BDjY*m8{SGFMFP&BSH;emPE9C=^&l{uF(c zUzyDlvubBWu_(QAK#9of5$im8)Q7(Nkir5r-(eqYdbS^jvniH@ZW&DwL^cgPKEIPJ zU&e`VD9fexG?$C?5oa0xXsv5%#7;{WiZ3}p^Hei|fHgKxNFjZg6)=Lcd^NV5b^oiz z#V~C6)Wd9Qlne2>FJ-FXJM2@CoArZj#y5_*h;jOOkECr21)|p@FkL7vQFzuswx6Zr zy|v0%>p1mn%^kwMJ!J?7LS70;(Is&2L9q z_V-^3e4KF}i_oE^9@7ZIvdM-L3K3dzbXxyTAQA+9rGoQbz9Lx0=)wnl{%^VjANtx0 zP6Oq+ObsK`10&uNbLFDcJoy7oBs@`de{&bEDz}yEi2B@&&ZBiPP_FiS3rQSArG}Ly z9ve|^tk%_*OZ#?BZsS0@SJ{L-+su11Swq_tGV$v zGE-vHznP+-o&Swq$RV}-OJc=}75zoLij!DKMBRGdOuU)~kQ@oTh!FPbx>YW>@DAEj zJ-qT|&7tjcLQ{_wf#~OkKCn{4C8b*V`w$06zgJQd_F69#hFzHu%e#wG=;T3}%bjBxalJ$5%CFeR zymnJ{Hf@MH#l9N;pct(+9N}O$M6_rmwP;|r@Tx|sVH_;*ij^E?R~_Z5W{+ee{tD2e znD7d{&NvRg*0pHBIL4YLsz`xrz^KtJOw4EzPU3k=^KYyOs5|*x%jNhu%hI$pZ^CYj ze$A68@rXxk@^=+-g;zzmg*W{?=D?iQUY_@zO=zskD9OncfIHvYrb z1#A2fU#}`Q=i6V7RyFbZzkUe*Uy2`oLsixP>$rl}v0oW(?Dy)Ju}ToRf@QnW~|IPK~$bsU~;?+$~+No=gk zfw8xKD{tu8XiJ>#)@j`)C33Snhk=(vIJC+MM2dNj4wzo08_cy~NC|iz>{F08be}kj z`!9q)u~U5{rv-ZjGZ`BR;!#(t8Bs>s;trSBT-ifdDP>IWwKC?hF6sikQO~h^)*@+h z45m#*j0_IgMU0rq1n3lOohT>zI|oQ9j3NLCpOh2-QKJg9{_pjAMOAQfH&Rr1(SgN? zHf$MaR2jw(BTP*+7d`zDOZ08RS!t_vuK(W4N?>jMGHYiB^39sLW`by!!%Co$ri}FP zi=dfkz62tS=geSvn)|2fY5sjqF)89hpR*p8VSg>z>$0#6tTD4L&v#A|>ApW{<_;Ml z2cdTRez{K@Jcpl6@*63bYwkI}2u@2l&7PZ(72fZ8dUaxOoK$C@E$-ZWZlCFf9WLHB zla@-dt-$785^?CWi(_bo-KOpvUY1nT(cx3@v-cGt;c8Dj%7>qy-lp|uAG2Et95)}7 zAOE36xhnS6^<-czt4kEq>*~076Rkx(oIf`eFq#)S0X$yK(B2LT>owL4Mz9HoV@V!B zt!vKJ*~?tKzYCNehes_b{%1<8a7>F0NSys2!{vY6P=PyD0lL_L_jni$u%gX#hX=+z zCtV97=E_%$LH}o~(xB>BKUGc;XKMM5P?S4|dxW-cdkl)PWCIYL@i@aZYFqHl8b64p zoVLTZVT8CtqQT|6-3ll&r~as{H1qS6G)06XbimG_mwt%nJ9@@#1eAvlKq~?PjJl@d ztJamR`L=GjajYYp1xzJz$a~BUk3B4e`a0T2UhCK^yjK`Nox*gYN2FY$3>BI#-dF5g zptgukI&jIZlQ$h!QM{6y(#3W&ZH)(1`k4-4T(z{&cM)?L!dWJC+*(xPW@Oa+|68pN zU~hDWp`7y25KErEc%Nz$(S}}S)#=68?RLG-+V@jDwR~Tr0L%IS^QyMcOm{n*wAqaS zOfeUKIGjFI>(zh}RvO(is|(mk!WPrh&2qDy3qV z#74i0HIg6Ocs^9|X#jT*Jwuy@UDKxdt9QP8UNmCfet9PKt=kql8geEI7OMVib9v;2 zpmH>;B6kf?bUXN@CGrE&Y*$^IwnNfw(j{%$1`bE)Jm?d3=!PG2q(&snX&ZGhG6zI9 zk600#^((*{M3k{+b1#+x^6HhOq=a9u@RJ>f`vrNqi1vF_di*+TNvbPa-*X*9n}11V3Gy_((;m33@`hh%Cy;^kcmA63RqT!B23u*y$$&2Kfn=9c(F9^H2It!fpuj<=6jMQ3Z9jmpfsEGJ7Y+xzT*0_d_a=VQFA3w;?0QZ zO;%lgt#6h#uLJ*SZ;30Dyb+r!XFXP7ErGbc9Uj9`YaegJk5zDyk4Jou=e13NWs6m! zpFXCS_#*5I<6b|V!7-eVH(=0~S4%bizA5Zq$mJjSvun>q`TspROk6n0o~Vm|TO!#@ zxTSXHn{&r50vC5KdO-c&+m^g29CVl0bGU8nZhP*|kn!L&ueS;-51~tRnnr|_&jF=& ze;}zF0-wm?iK5d~XWQ zn?BB^*~c?xxVj0fuZG9aYzzy2^qMMW@DHZ@wxTaIyM$XW{xp)Bjhq#n+&^x?0-;%h zE@?dUp99{1m4%J=$>wJ)4kA!{KIQXty=lA=5zRz=A_kX*t?d(2zpL1U4JVN%Z_-x3v)cX4# z&3OE48(IF?MheCZr~kb8|4DJhvVJ9%kMRtK(ix(%hU|@pgmgYiJ+o@+v2UW@HV`>C zz&x9$f8yzVUP`>?%Sz2;_sf+s7p1hhdsH>#Il`puX|l@^%=>w5G4!F*%LE!?I)sYQ znp^+Zml6Ew!dBTmf(1r4OzF^-ZFUrYPc~Fxv?@xDf2`N# zZrK`XNGJCfVya}C_3Clvlk^psBWTr;?!&GX_UkjF*0hkUro!6?diV&N6qGMi@me;; z;*okFSN(n<&${BP(*-_lZJ8GTSCa;Cq`&w#_$#qF6>FyTcVV^?m)}h9XpN=x|G^>S zk&CSOP_^o9rR{$U7CVv0k7DDI8~7gizR_1j*`-FS!j?&Q-nzZ$jc58TejV+SlXXA- zX;J1xsilgQc;KVnHoZY@3M{)+)+7qPzx2$s_nq1XIUm%mB)w;~Hs){E@as0~*JC&2 zo6gTN!~wl5YeI?;QJhged0(yxUeWAXM6{R*5W0I~^MJ@w{MdzY!|xpbdah4F9H=do zf7Sdir7o%Ll&~ih@QLlj=hj~lOrK0LqQ1z!sj*M!c`Wy}a+znZV)EX`3teajdjD|j zhR}gF=V42Zzav)-?`S(zd*E;TV;wtXfb960zT;$b70&i5z!6^4BC~SnE0>cjhwfR| zHg2ps9J03dF@4-iPk5)q*>gsEq$d=O*yDhmzX%en#K?&wB`&4v|F$6$|G`rd3;*q1 z{By&2a@)+ppP#fp^$#wI3RIo6PZ@P@r+(9NAYqhk+tfzJ*X2QGU|po<7w}!ICbRB7 zKh)8t?9|WE;vZqfLkOjBEp5$3XPrq^1uNcpw;}qX~+?;6Jo{dx-~ng;5(D*8f+K zDG?IHFQ$#zwO6HxOV*=MpS0(|Td%`)WC<9#aAebjTsXy|{wXAkc@p)3CXY#-wj^2H zDMr!`>6nM}XCfGcF8wp7IHgY(4<5YYPR1Nto9%5n4=5>h5BJNjZor^r~|N=)z!W@jL#-c6Zot$Wy8~& zTThjb(36~CWld{ktnT)o(FgxS$j9i28iLCEY<( zsbu0vF%~#lbsC%Al@)BUc)ojivtiuJh`SW9cW~DF=+@m`A5b!%!RJhhDC^lER`g{^ zU;r#X+CSq@R|URALmn)@yWO~)R4x2pZ$~`rG=S)}KY&*&DeyafLEYbq&Kt@hflnziaicWVqzCpW8qRbu0zLr*8`0}o#EsNRso;u zFKXGihh-_|`#fR7VtaY~;g5JbFORNB7wK5hO9#xWG%ZzV4IT;Lt)xdk28rQcHYDI6 znK%_k0dorr)l%ApyAr0sQf2av~YoU=p5}DVC9> zBIXP9YEhs#XFC(7&d%iDbbNH%g!yEc2hZDE^-PM2UNwJ2oa$0_vlwt-E-qhp5`8`H z63h7Zxo67#U&oSoOE0IDbGXLye=+Z`9SC~$o09$ch6$wWx@VC_Qbx?|tdaZ+?(Pci zXPcg^1<`XslJk0xl`{5r4@ifPaRdrFd_Zrg@n->;(MQ@+5mv+5ADvxY?A5P`5AQv@ z#Z~A0{-1u*tO$F;kF;;3cY?52+VgW9>92q){w{aa03K}jIFo+EL8mKv zb$kF-zdJAfz-EQf9R9T$JwKUjW^-t8WD?AvI9nUk3Z#GDKX%7^<+>7|E9%;*#ETIv zfl-(<1>NB3J%?KTLCv9D62-jDuQ;pni<>)eQ2@`R^CxB8FWAgA{DRQ1r00SIo^ec> zP^$p?ZE2gU^$dE(fV1g z>zL%3crbG=Zx>xB9s3G?c`RJM> zYPY`yF|`%(DhzggnDf^+oCx399a#r~=cK3g2~7(fzv0P$vB&*z+O%3d9;evlfs1#S ztvzts`e@)8ixs~!3-+T^st4na3Piaag1?9XFX>U@M@ z%!E{K^51tfD4NY`ng2ys{_!Y3sD`@Wl?(|tSm`&o0ODf=ls^wMb)w5A1_qIKrlDzL(xl@@cvI7p$%V0i1s^L{}yn8L#p1~ zaZhq@CXFfU)qcNi-%`)GsBnuA*~Ivbn&Dgd&i|{H;A;Ig8COh9v3|A~C6MmLrIu;q z;W$1L*h+ zJ|KPDLrAPlI5;C`lG`j zRq5U&Y2kb3aU0J2kuVX0;+#M&Ejp`T_;`BomsZ6`W&t44RoWDBJmPHk^}Nt7iXg$( z$7Wk`HG?`hlh*k+9*0Z3c2p(LrE=yXp=QURY322;K~EnG724~%N+=+sC~jEUqTd}w zSfdda3%)G_Z8hq$$Ostbn7R?R<_BE&6k$U1yelriui8@G;jgr%qa{4d`Jd{7rcP3E+ zmIq-S46jgNNF}z`|rjB!d*18Iet{~CX zvdbR1=ua#pb3;D1AY2|uBMO!l14O#Rkd2?9A{L^Td*TgPpXE1(g6zN zW;R`A`Ln0zsUV~rjf@dGKF-T~I7$qHUC#NsYDGxYNhw7L5S$O5J4ein{pC06k(keI;xl%Mpqf| z(9+IM&wuPuO~iC~@k;2IOW940lA1#4o18pvzDZOJb}8JMxa}s)*TLoCkL$ zIXS}dpzt`EZTc)aEo8&zJW93!wSQ! z%Zy{gLeG%ki3ajL|LvvxYiAJ3PNC;jq8p!Fbxk7TLFqRP_5WaYr!acS$8Z<(2hya@ zAe{2|d53xQ4fSh@;v-zGu3V`3M z)>%%itAYUu*zp+XZFc@-Uz~0+m8ZMg|`d}6>pSp=H}X#Z~Ct-=z%$N9HBaGpUIvN+}!k_MHj7- z`5di5GP&_c+Qvz!9QsH-8?&UL-(*yZ9UK?hH|1fQ5e^LFhcf79agS z=g`j>_I0XcE05`r0o_uwlL3J^mr+O@ddXJgnPU)bBP+<`li)BDbYs!0k(T zt59%t>7&uhnQ5AlX;y;ij(a+vp6i7^3Q?WCNz8DA)~IU`KWy}v%+llLu!bVi@-8ja z%MU}YQyzMR`y{sr7l}eCZzY6$sRidHdf+#0OB^?Xa3fhBSd(nr(V9eIN`1*?p;MMS zM6-Ti|33VlD6fD+Ox98G$tAJXt4gWkJ;9Aba993{|AXzPf(t?ZrX^(A~Kl|LtscBvVX0I>d zv(3Dxk(2XGm7+P;{X^P4MW!dLr(3Q+ZC4NkXR(o2$5a8Xqdy!yHWFRCyPO`lx-}!u zpeG8MGM-v>9pYkZp9PR-a8Jd^3;^V8+t_tO(aKBGF!ezC3JZCKiAI*G8!_(4!oHNv z%)~`$4y4oSw(Kn~GD%&&EOU?)?~k;K0rl|D%0SPwjIebU=0wUtE!f(+}YNA z{?F6BpPI^xMIFelqU&mrqv#n|&D*UjB{SyJr6W++g!ox5U1>Ve^Aa(F=4_zOrg+0f z?W#MfHm(aSds_OYJB(u+j8q=2zJY6tAEk&1^qa9>q)48^-c>jes)p|9pIx4V^``y& z@eLlYGmq;o%!eS`5%!SSDSSE>Avu_l0(dFyKI1Jpodc&c$GN($m#Reto;61;f)6J< zOTJws9RedNjEA?AHGEfG&im;D=;#3&ug^YNWm0mfUMXar+2Xo08@vaq#1_L{1+VC@ z=wt?B)HPh~5fcu>hYvs$baANGc%bkTZ#o z0uU};>y%288uZ`ixijaG(xb%pI{QF_W4TYCm+~D`e+*QKpcD#^MyIWdznOI?>hy7Z z8`hF++AsCU~|y!y_#7NXBp2b-2mR zQZWPN9TNq23E&6_FmfLSeB$uAm!YHE&K9etJfKo;s9z3@<_oc-Rso{#kI082nTQMH zM`L5;xy21fqwUxGdC@B@jj~k(uw#wN!Bul5ZmU?)ynOE4lRq4%Gn;b_(4_N_lreqq zM$|=e@pd!B&n(kaoTNAK65?C;#Bxlc$LjvGwW5<|xVWT=*X%sPzA$dhMslUbHB4-u zme#`2vyy+j@Y^>*9c|x%aPOAwHa*1Y8`)u%6diQe7cFDCVDq{#?jXQf2`VykMqB2FRzo zWjxOH{mHUdaGaHNu;iZ{K!uJAKQIZ@VRD^4-EcOw8%=)`Xx!@es;soEb6khSA|gjF zF*e%W)Z2>5_dH{y?+m9_whC3uG_v)zUkBTIiOm)_qh7Rn3>dy^c{1mAG46T=fyB~N z%i=D7+d|_60&oX=$viNcm2FWe(C3uS&5Qs|Tf_B2dH`T! zWQ}+%uoVE>P>U(znXsQ|+A}a@psLW*u`-rAn5s_ZUE_#Xx@CXA#-h*v*A3Tq)D(s~ z{IP|AKwdhj<4L`S_i}x{iwg}q6bB`*UOy3)o^JF5}kX2ypH`m8C7p1RU7fW=vMjeD%q?w*f$4*qj}3U%r)~% zCCokchV@Mny{U|o;T0`aO zQooL_s06t*?rVb|zpzO{$S7NjT~UoZj)WNgl>fER0F?pw>S)b}H>-Dr-qV$QM8zv) z@7q>B;}&q67X7q(1T`NKV`7O)K`Ame6m#`O3hG`6j#xxDTw=QxYYubtCy3x^`aU}~ zConvQ#v7m`E|CA3zEa#@RZ|{gvm&-8_x_&l@a4&|-yxWHM;a2>?>hGuZ zP|nPPB3Km}(wo9kCkH1VQ09kAVt&X7K?@|6>tTFYXX;^8;=(ywuA^#R1^8EzzBNK7 z!q=Yqz_r94d{Ki^jWR8%$@k_BCk5#ZGYg>-jc*hT_9T5<&po&M%E}VB8ewF$39{R4 zzD{8$fun;ITg)E_2nv-Os?8f;z$cfdw?6pbSDnZ5KmZVy&$QF0vTjSi8xFdA_NoIT zefu3yHzVhAt!AEN#rAPU7G4%;cSzi`N+aG%%Yj7oQW0)}`Bi8Rc;jr^s%68_r7K|a! zN_%F$$;pv!rFb{pmAGp<0<9f^hQO~%eQ>k3g1+6Ly-~%;<(AFbuC7(BNdraZ=~Rum zDg~i~`xay2tP$Gl>iHaT24NH>k>>^yH-YN&Kvkpt(9(%{*jU#@{(mYITanyr3;+^KuPAp21+lhHv_Ldo3j``OMWxnE6% zxvrkLXbS5xhRidqz0*yR4zH*nX2Zt*newOn4fB+%4Q$v9&PG`O}aQl`I>osCNE3|tU0e{(k&FxfML$$80^} zepR?alE@j&{SqBH)U~;_X#iG8cYqiJGpr`)yUwA?_P>;0V=^b3)Lc2psXywiRJ3q>A!=++JKAErlGNC>LxVb!=yTmLq;jzT~hT;@&vSJa`SFiXHQA)tGpJF)e z!cJ||WzSnEq`a+I?AQpOJ1+Mb%$MDGor)U_ju@{1ZnB)INIuZ{ z&=u~U9{$ZXt>jx}lc&#SCeE!fp!PAa!*xr1s{)Gs+|)!4ih6Flv~Ec2 z?Wo#Dr1HPcdwXhz_!PDGb+)WYbV4)rFMWL#?B4yDAe$E8zdC|i_!@htB;Aw}b+|H9!UnuJJXjn#Zsjb9=1s}fYxVQ2 zkv^8y|1gNrx!e3Ad|YL{>ERc0WlxI~7v=M1Zj!G@uG?Mwsv9Qi5ffU0sM~O0Kz^-7 zq<&ku(v+q&2a@e@fzn9Qs)k-@HZ1?pHpeNfRCys{IQ&gE7P}x=Q+4M;Gp;Y9mY#(U zJ$jlukl7)1=ChZxPHISnDxPfWsbX%NHS0F9Y`nAHYBe?VWl2h`Hu-3~*nYX1KVl&n zRg*7_sytc(d9QpC>|&;GE(CX0iO9ry)#VX$g(~M*mk0?p5R(ZkjvCsb1Jx8N)&&c4YfS$3_9}xc~B|$!?DB1jT?~COL8^3c{`-t*jEI5|OWI$O0 zm2+TtYZF1uxuIi^JZteeSsTYMB-%Tb(jZ?mxUW+<*+*AFDVKq+Z-8`p~Sg7J^NjIlc}e>I(-2GID5kE>HUG==D_J6+{alx;#5F8g;uw-iDz9OeGkmH zg~%!g55WcWC>A>F%qwio!_k91IJ{LHc7dK58b=RH0-}?xA=L8R8+(T}kP1V^*4uP~ zRq{54{0^mbeZ>K=1NpKyK1XKBGCal3Xa-&6+Ezt>Z!TYq0K3ohF1h3q_Wgmt)e+)k z$eSgG`B+whUxMs0%%w2%DUX_rzftX0h#M?Ny!A_itBXpe7%P$j1E#;n#mxvzQ@nrL zSLI4RUJuk4HqMfH{d_W7M$c1s`u~#QHljN6;F541st~F~O6$734iJzJpHV7D2mbm-Ti)SKN7|5tw!pVWYod#I+Hu@kRM**?Jgc!k^;6GV%-J{wg^z8lL*B5f+{SRGr1H6qMMg-m6n|Zj|9J_3AVW+hUW~28?Q7CLer#-S5R^ zg1sAd8#M=$Vri>{mkWX`2>n?^Mj& zDTMhs)%=tIHcQcaCOI|{Q zMOPcR$z>V`kkQMNa-TSy^}=vDP8Y)@H+`S0RJ@!$8NG_ormf%(9N4;I9t1y4faZdr zTN-DPxRhuu7JtedMJB;oq0&r=qe)Zemx0Fq^OK8NJ7;gt4=|9JW_nLv0U@{9B(cRl z(nVpQ_WDqg3Db-SFQ?N7+59bu+P$%y_3KaC8Nn=YZ?F8I?Uq>6fV&^}cg0+Ay^)<{ za5YEAqP1fl3#_ncA${(paJG+^kIJ02K@mwkXv2E%MLtC5Onzq6b$VtV0Jy{tI?Ex2 zQxC#y>VV!ysDOpBEaFGIrJx5j7j~G0v+D$;TNewwO=SPuf4(O)OuF{sju3DC)?2 zJCWDDA6X^T=Zkb-3<@~KtlW>nXd4)&AU}x)Nb|!nwy(OXkUcZ^OIbJjBJNkfc`lEO zZw!BN5Z2|q)-`L0E|BH_$)@8MKuI%YE%D_pBu=lwD^* zKcxJoP_B$L;cFq+D94wXK9|1ZQ-JgN!03(bGkb$msYXwp#d_IOEhD!M&EylB#gO9M zQrlWhUEklV8N6ofX{q|F)Nvd)mCud969T3O?WT+Mr@qJmL3Ge#aXye@HA?&DY=Ccv z^d9&$D%TTCv-o0;J@RZ+8=v5U#;v3q3WX)BUy1Ph4swjA^>D$Q6rzg{liB3E0*U0U zq(wKL^|y*&o@tbmjNRkO_I+}>#SZr7)u_pPT;6NE@XM^=ghS4*(8n514YkyzAbo;ok z_K~N3rH*^7e6)E`CC%fz9jy%Wv_JPmtn2-1c(Q%Ew}Kv(9o^ABIeOW3y(yM)3xV5M zpUjMt#?yfNiY|nlyBJ-C!EeH!+B)jrg}0mo&|!mg2<9VEE`^Jq*B8kOwOB}m=NNac zI^!&yxFPPd6Hf#Jum}MpfV)=mgl`2gRwE@duA-##`5V7fPxWTjzrns!B_JE5*tT}%H;zP$1Cd{?3} zn@P@EUshBJixLyZS1GwVAEqyaG%TS}lV6P0A_if5mnVnX6!}D?i}SL6a_5paB^q%J zCy@E{iFkg(-c{=Epb~;0Cf$@^K#fH_+tmZ9xMBj3a6V5&^rWCH9`3c~UZQc8X zd%VWPySdNQXXquTlASsc2c=7SIiaP#Zcy%RZ>$JVUc0M!^{vkg$i&DMcd0aM;`{R{ zrcW0+_KlgabWyH0oef$IH)49={x`NNj(+t5Y48 z40LcP0;%(XlvT1{4%9rn&EkalwU8*oPHf$E7Du-+iTU*Oos6Br;|7gmq*}8BaQNA3{PK|p zl{M0n z9#v=S-}gOS9UhNC$J@3#EK&t&LDewi0R60Z)Z-SvC!Ah$XJ1o~aNfw3%cHZsInCo< z`}fr*;+7Qm8r54?BICMvj>i@`l-SF{H;;hXlz^tqs)p4m=IYwdKtKe7mcXlX4jM_3 z4brQ<>vgl-m_`zEn0?ZT-|!hC`uYPj4n4Pf*|1sdT+#Q|gn^<@KQ$B;+nIH|zWy0` zvS$(5Mv`msLHim@`b5h1Er9mIqwMbztHOXd=YaKc!SY;vZFC}qEV~R26Bwfpj2UoP zkTmx|n7q-JwbLsk27J0nrcBy;*U3jhU4kMMvi>gOeo9Z)aU1igP0wCfS6+{=wx3Vj z?8^9F98{(yA$ND5JxY)m83t{U;`g^?ZTEWf#e~;yIYX937i6O|2JR_rol@ZCed%Sw zkD?PuJEho5tIr%U%aLOPf-+r93{Xp>^jZTya(tE|5Z%Zxoz;YPi{t_Z`}zE}^2)vx z>MKOB^~BI8jTTJaJptqUYLLdjWko=)Y3aqAFE{42yr&};fJh+xAoOI$0{bo@(iBh# z@it+4mo_5vEIBG;f8V>S5KkC~>VA{se7lcWJw?3V%Rc`dV)PT$IId%I+nCeN^+fuX zoDZeBaF)PrV)VZ9+&i-)f2l(sU#g5)-o?Q)0u$G6&%O8qbu|hRlTC#kZ#Vzn%8)o3 zFJ|qFS-^RE970rPFXae3I?8=<*QCQcgK_?3>4pZbLFYC~a_%%#0Aa{O#*~gT_#vG& z>wO(~I=b@j)dlft|I%nwV zW@+uH>CwQ+w&zZibeJcvW^TjWviqEmOOyZUzFwo7C5v}gG-Ki*nPaU`a6ju?u*6gf zkO?j>>}v9#bY-xk!gFvSEU9vj?ZHJzl~_vn+gZ+CM7Yb#&m77row~67WL}X>S0v+) zn8%CPZ0)joiph>E|KyGwlnA4Rqc!}^VEAny2RQ7avZy9@=a8<45h{%)E!N6?mw*E|UA(CLU1q6ic{y2i1Dr9l+@4gz2 z3rOk>Je!%%-e?+Clf>ZOLEC9fabX3*4UE>O5zRyf5juGrB{16SSdrjs$_N+odm|p3 zucGJ>mP#JWnV{d-0pZ<=Tr2L$h;{x7ko046!-n%0JvHk3>-3mQz3{a7LGd-EE@bb$ zHFsq&fB!|A816bDmyBme_U0(dCfghsU70v@+TN~*8cPSOGsuV-Pmb~$?DIfO5?+H| z4_-V68UWeL-4ckJsRZ3TstlC?wvEx4UY&yJ;c#E}oPd#&q`K5I)c_G0$&)E<;A8b; zc9#{{OU%ZNe4KW0J4fs==RS|=-jh6G9X`W*JR@!oc?pE<`a&4QYfg+8tn}WqMggrADlfkJf7| zEG>CWTAtmcxSzaVK;II}%-r!#Vm(GPz+vI1e6yvgl=z@an+Rrahy-%zfWNL-;N{T{ z8XrPETNy4y)H zqMKQ`z2VH1UMgB}!9Lr!b(Nn@_H~4&<^aX8CRmvy1g77xzA-XFAF?I-%l>gS3)#Gaw} zs#i^Q%TLksP8k25c(G7B4$?_gp!8ROHTi{|-;)C_ z*#?e0+kwl@zuou@qazcozQ6IagG^dYw4oJq6J#;7c!&?^XJW5e5VsSwlb9yqU6cE* zB&e(r+UNZz8$^dyg%b6wNbMctNso)oZNFl_BBp&RPJm!kOXxM-TiB`?DA|uFf9eM$ zT!2M=*s@wWbuJg(3ua3ywRo;b37R*5O1{w z#l&VyX6$he-7=P%{Ck;P$?^@`n&0K6>J-%clvJ{e%2dm-Yv*n%66@0vk6cS@;`I03 zI@#IPGt0ye*9esJK~gqtbf>*XJSap*EL$e-T|E|yOKnl^dU$Jq%lpG(5%#SB6ATrn zx;XB%Pp#h2VF2nZh>pku zxiqnnc*T_{sWE$1Zg?H2_{BK(F~AqmT3#-HQWz_Cj%Bq-EH{56x~91j=b4P{CSUX5 zcSrWfx_(jD-&)h9v`Bbj3c-$(?-c~TySWhAu~PwIJ1HT{>yX)E(3gF9HNrM2x`(bv z1Fb6??<$r06<_9QNb#;6PpLM!kL`$YbPdAmV}VR^*v zo1<#A+Ljj3dT~^n9)3M>J8u)fV@U4Q?Dp*Mk&KPG8NDVM0nYc|E)XhA=Kl$czudYj zx_aaGuQK<}el8*N6Z6g(yS>l~d$}xsY1cPlj@ZV%uC<%lE|PyN{1hbtehd^z1kF9k zXUH$`7C0*~R+IooImt!>y7keQ&Vd5zEX`2_LyaDeDYeu2Hqe2yDj+9Q3g<7LzmxHo z5eQk>J#`qH$j!_Kjl@#`Ps9YxA-ffuUli6pRM(a53wn73erq`~PL7hsXajF`XwZ*` zJ>$N=5O0wzhjR0uW;v}iG>jeL$+if6EbZmW4aSu^-DWp2BI#gJtGks8%hO;Kms^9h zD}CzFZC38y=eI2;hK*E1alhT8M`czFY}rye)n&bvq>@Tb3nc*+EmK}sKEL*wbsfLx zh*W(`51c|QL;);|m^cap@-(D9E}QKkBRyQqBHdgpk-2xKb1OP&!g$=xOt;obR54Tj8Jpe%rS2SyQPT%thHyVs! zBq-k)BHcyiKzK+aQX$*tHfsqs-hWDZurfHm5Hd_xK~D#lwCyvC6i{d)o#95ce1M2_!&~wFVu-_{ z1x4Z6wRV?VmP?Q8)lYxs`lSG76HVA29CdkGtGiBdkzs$v#U+9kT}xzTGQ>*fk_wwj zRP!`j8j4}lHn>7ud(=})T}-jV2-TqUy*f&l;(FAiwqLtlhqhvE9OKH=+pMVFI-m8} zume&R?Lo%CE3DXu*`Kol-&rLi!O*7>T@~xfSA5`Hzus8?_UYewAi+mqRy8|C$3O&& z+C}!Ime*nqE>aeISJ5v^cCnaZ@B7HE(jJ*#=i%q)n{Rpe#m{ znz>ikXL^Mb^3-LvNVCA%uQ#N{nUM24D`7Nq^!;dnL$vrkbWL2${^e%}iGcUiU+0Iw zluML0dI=%+cPC4ZjC<=u_}23`*w3h`wfW63svnKTqib-PMSS~8x?Vld!8=`>JgpU@ z)V#DN_F*2io{zvCu*K;@Ch>Odg7#k9S_=9_amA4CO7(`31?V8lDZacB&oU!cO96~z zZ*j-~iDPJ;Lra;d`IWe4s$QizNa)N}sWYizIZ{&`lSfO9FzjmBUf5<)R%om18lr>? znRqM<#{J4m4M5I!+FNaEY{i*t@k6~#EsKn&3VGK}E2>9|VUeXrc=GbNqCHNsHs*(+ zW2W{88w?QDBQ2jsUMwyDWmTz1N!+G}fJpn~q`G!R&s&4d@ZcU~7mM1;b4)U5`W=HU zLUF|qlX8K=rSx{u*(PVB=hllv*(5X8d!?^i6s_~Y>+60Tt^XpDs4!@4&aTM%kLPR? zc{nr|8`Q>|YOWLJ)Y3?Wu{v+NPSGcN60)JtT)IAvT+_pP7!CA-Y5Jd-iEx`593sAD z4xIhTkG)lXZW_DCDC<&-CKKq`u-h!I#h`;#gJkl^2&|y3*htIX6vXCqwfj8xV4HT6 zqg-^AOB26aOlPrr?_ls6WQ6_E!qUp85y46COnmfK4xz~pd;_&O?Gb>B1bTP%Kyg2q zbnO+_>^dxUkEj8l%VhPIB$I_&Y!j)j%ye-r)ZZ>_q*SJ-&?8d!v=;k85jMgD_n+35 zNakwp(cR#IpQC)3Ey7~b_8lEN7I)!iF8-&p=2lN^Q@d5tb_>Z%!Zs&xkx*$<>2m14 zql4dg?ks=%2gSYpWVNGlSH~8g4Jn6JDQ-AQBg4c?t75%y7j^hX*HAV&ch|raA~Y_K zd6c!;Gw``b*AJr$n_KLN!NBn?&xxiF`=iR&nx$Tc5*$rT7_M*n9ZD0@q0Z*S-DCci zP~Jz;43P=KoT?i2mp^NXX5KDzxj3x#LaI3;dVIstdKt0y2k#T_2lh;;zda?)m6#~_ z_koZ^lGJq@%klaN_6h7!v;`IecJ!Gidhit%GT5v{X+wEW+zL=>yx50j)2U$h{3;7J zcJvpI1tF9lXa4z8~pzAr*};^CQ+r* z+qNM4t#v)@2W$7iPn+EV5*>22{rQ#9( z>o@dZ?e$uI(}@?^xE?Nevw!4{EvUJ=y0!gO$_4P<4b#(dmw`}n$7A?95h`EA2`bSv z3sY}<0i2f&a|-?|kPdVNPLJ@L=6L3f+<1fEEKpBGbad}j5Z3N7Rd+i%N_joHzW4Fj zEryJJM0qn#Y?esn>lMwA4}zhcAs21VX;oRr=NhSd4>Y`e6*AkNP5Blc{{@DB@@Rt@ z--Api>=Yd!5}tO29K?R+K;=A*izR2g69dm{O5V?a6?!E*X>RdA{HJF|V4J=3UqJbZ zrWvs2U6jXti4^A3`)}aiP7EZe^-hhRkbI1w%9H#4Pz?emo7QgG?A1N_e9o5~V6o+) zD=-~DZ`_31%`VWDIIZ;(Inu)}Isa5sA9egWP!~ISOjdCgJwqXT?m~wuREL-tlPALZ zKtZ^Zx{BQ14Ot@Inq!FjTFT$nuQpDa-$;GYl%|LY)nfl>dsA2l_da)-QG^@AnFJ@G zqFhyIXNmp=P@j^W0;>(J0_|kr`wp%VF#s5JcZGG5KuD?>U8%lNy}Xt;_UU+?4TAj; zzD1>=e8(lm&c^0vS5x-*@Dondgw(CAvcUmT zOW)_6q=6T`)0eKXKxH00ktsq8e*YoTi!*39Pe6k@r-3ASi0@mZUCJx&*dxoRiz=Q% z&C3B8k=JRw^GA9ClT62V6jq9^>}5^KTfq50rI;r6w~|&5VxL|5Zu)n&-Y&lLqQh~(6=Qkj$TN2=|MdoRgGfCUk& zQ=$Ykt0V+58sT!7y)Z#Vw7PH0Muci1V=B>@`l_-~6f%}M<7`Z5P%1Ilrrc}M4$&2? z)K|VeNOU6g+wQEX{mgylR~<|_=mX_)&^PlK-^ zGZJ{qz*KS+_|l-l=uom9o1yoaO}@PgUGRnC;!BBtB(ng zcVSP^_R$@aGG2!W1KS=~X+C^hPELTM0V!6PXigp!Fq&#Wfr0EgRjjb5!;L7~{WfX_y~=gem#7Sx>h*RsJ*F#!^{YVH}wd79*9|!T&-(gNs-CCIRzZAIoD-q}Zs? zKfg33Lv-TwW79K$BkJb>YBPMZeEBPu_(D$HUbcH@Y8@SyV z0DaB(67a|V;lr8-@u2P5##7xY3YnAlEbu{zjWVgkEU*E4olu3ldRwZKHKAc-5WLdq zaYCKNU$F#b26Ympv&s=@$caKMsZ0oQpVP}crpywH^3e}-t!WHOvwV6ARIrVw;|dM` zu>COUxv$Eo&pWJTVWyfHRuzA(hqM*yMqs_tEaLWCA|XN+)LmstBV&`3x=@JmwA;Tu zy=@b$ZBu;$F)6(I38F|FmV~GL;7O~=oKY-Xx1JgwpBB_ zPkKDIGhuDTg0Yq@^BAbbe~3RY)% zgglmB>oJV+1*GHo_q?8nLD6Nu7P`l!h=Y(gJR~rNH0ZF7JeYd08Oz^VKUekvVi@3P$4b>iNaXNv;4##L*qru#M5jg_#Mju!hd9!g$14l69+4 z^+li1U>{pcMXB+$*I(8b(0utR6L@*f``y(0m**(~-np@)-@vvW&q)q`n5+`?pM0-k zB~>pJ%%|EmuUG#}W9DDwdlDS*@_BpvvIvRfFUQcW*t9~-)8w#cnBD&JxC6Y<-;p9@ zk-s@7KA+B~kS8}gEdPGu5<{?0F1blrs#CuHtUAlZ3dX*!H#MDZji|%hfSdOSkY-4u zi4WGCk{eY*e?G7LV`|`lK8KowelEF;>(ow^ z-jr^s)3LWYd3!&9{_Jykx^nRDCJmQb-$x4N!Ji7Uua>O-7_K%Q)~{h1;hT7e;Y%mE zgLvPEAB(^#IrB4~y}BXx=Kw?hRG$^zb|m6mN4e7EpNss*WI-sWj0A(ue>vXB8Bp#j zlXa90Ubp&}m_vO+>M7f6)icwnha+ad)c+bpWp^ zGkv0})x`Ky$e;h=@FfU2O>g4EDlq6dsw!#;2}|Q8b6+nmyjeZTrJkL6!;8h|DGPgp z&;(nkkBva`*o6hpR7xgeMsl_onrtK!`}%)5#b8{r%d4S*CXdm)XjM z=eJA~=+v2SBjV=b&C7yeIKR{{=&*8Vk3HCik~ zvPZT;*|#hWi9{kL?#0M3gOTjZX9-!d3|ENAS}BnRk!)8(Gxq&j#=eCZA|%3hW*Xi8 z|L=L`nP;AvdCr-0-rxIs-}iUUoC8w_r#t#^gqumEh%oU7taesCY2ou8iodjuid7Mn za+SVAerv75b!2h>)6w!I00SP;tV4<5rs9UBnvLP5z~lq#Kmd}xkfQdbc0cOduyS28k z#hf{LcnfBq`wU&ytUG(#bQ&zU6vvI#wRoX#Kh&K+EO(JDJAHLZgwIBO>sdmU6moh^ z16~+Uwa;oy&?2Sz+vY;aK~)Wu2+vaan5KwSDp4%<5HM8TQ7?3{T%c^Ptj+{>!M0%4 zXQ1Tk_>Diq0dUUNb(+#MPukqoO9t|jySg4?c3m&tq0tQZd6TRqNoULMwwH8NHa!Xw zx}6TL+#!MJVwpM~%jwZ;j+w^t$jST0j}Ecu{9XEOXgaV=4) z*$SKDIb0N$a)V>c@5He_A{DKQ20=0paai%3^VkBc9;cOnI^ZLKt(^Ku3*G7p~X;dBulUGo%>+C_2{X($_(d(mQsUK_nTk-Xna8U@`HlrXcm{EqG z#`Y^yjph~@Vg-S4HQ}ZmtJVrGXt4k0W$EWL;xB9eoonQl5=I zcqD{_prz89E6>5B%qUp8rP8%a{g(B9D9a%hh|f4Ta`t0JML8#djNqO~tr^%k z8rN+W(O#OQ+-lv|;>|c8lT@D&yBazvzxZkYu3Ruh8#uieWXWbvXUT$h;N&TolSS4Y zSz1-6V$y7(k_6U9*OjlDn=jJb{3#NJ3(euqRpx!hIh=NbRQ5t#W3=bJ-Etd=TvfymF^oyFQ=7T(?; zkL_p71zVE7bnb82X=|d)g9BCGetZ3OnoN(Mh)mJiRv9Lw%|4|@7-JOmK#yfhg%ssG z^>x~GU-A&6)gWbhX3U}c~$kX41HV=ouHI~E# z4FmGaq~Z^X{k{9ptr>uLQ*|FMBK3WnOj_(TR;QAvGd;iLKU7nrM-_zB?iQBPv9_uL z8DbYYich-xmr8GpdRSlEQ2udtTix%DlLsPvoJv8$D}ZiFc*NuVb9IIt!u=xg7OCYj zx<&K!M@+ZbCGI%YQ;PV2wG(R*#bZp5SJvWoOcr(P-JNJWyB!=vyokHcTcT$q713?Q z_bsBV^4NAVfc%c1^RO&Eft~e)dVDqb=;3}g-a^BboQ&k)fJsE0AWacG3mo*7zq6cO zP4I`}pNN0V`$4UyfqC)!K!Ft=yR0xWCiiKqnf1-v!zFhirO+N-Vd11LmaWg2E4R`{ zNz|`)^ueSv91J9u<~40q3g48ruJM%vi!cwlFFm3=U9Fd(z=t4$$YsGG3EsaY3ld_b zRX%#JA0IWhC+(yvU~QHsJ`Nnb(Ap>JM=S5FO<n zB2lMiLZn9`lRNxQ*Qqg~Km{cw%rbxHa2qYCBnJ^WamM{9UYqXn8~ihh@6=|L_F2kZ zCD2fo30x_qsBnGp*IsVNd0gqze9#!5`N5sP{iWpoYGYOOW_NNTDY7yFE27%r| zwoj`qwfuCiOvqPy{PXJOP0f>?(JR>9774WdNpjM9j*!7t`lTk8U(72}lEo ztqIt%@Y_SI2~=3i;gOgPyP#@&^Q-EBs6dh>^___K#0YuzXx@jA#!R zI+hmdUSBX#Zq}^c8qq$-9?OSrE;*>?i`RW0nrEnTf0s(HwK^;v{h?CEe3>|za2H~k zo7$Gw?<4tN-%Elv*|32Vlii`>yg3!xK8y=1Nv0D?vE380No8Ifa9*xQPa->zLiS8j^yVO+dGq|a$P1wl4_Jk4rW)dXOFFDH(d{4-C&290Owjxj z<1kD&r%9UO95orB!wXQZ6g9Z_efPlb^?!}^FM+V2*o(@#7H^LCds?JfX+44#u5`)3 zD=rIVA5n$RZ?T`g)e9t$KGu%%99KQ&!wTmdEV_%j@&zR^A3-?&cdj}vz|8EIPR?|` z1|Xq6f;BV|S_z?I-{H@MU-l{LjKdonBVLM(`*#RL-b}~5mHV1Lup*9`Zc6$`VdQ2q zx!B){+FY6z0u|*y@KW7W`IH|Dg<9)VIs6kC^u9Wj7(qvAGp(hH^o{~AWvL3jDvN(M z?+k7a9n8y@Ko_5wixzPNh50M}p2Y6nN0T({_TK<3s81yxCn(p}2N z>mo!n^Uv@HZj3wfCz+ok(b~f2K^46WZXfM_;}L$0r#H{c$zf>FtTO%R_n$S|&u8m2 zfPtD;qe_82ln0;H%QAf+9=cXx0fii2b}AUv2>h{8B{lA--&wvRr%W{VLOx|{}X#D1em$)@F2|FZ)qlT;d``JD=47Jh|D=3!ZuMk0TU}Ibasj1 zQ94&=Ob))-L{&v+DNuQ|Aif02PwL0;CYb#3{r?&qzYPpNx-(O&Kt~qX_5g#xb6B!m z9?X#0MQ=L}S9q%62-7o&T^b`=ArO;EuLv?IZn*(hUb)3NO~FY{635`BmYUzZfeCnn zAIaC{EZrMGArgXk-4OakxQI^<-qTEA-?A%m6pC`cPPI(GIFvJirKb|!+B|tnC1ZPK z*)%T&_j`r!Bo7jO9$X98ulCBs_SK8EZX~jV#g(7?V7b;EV>Hq1aQrTf1HgxjpX`<` zuy5titZhZ;YBpjG5ivcQamQtVj&vFt4}ew|8GpSK5XC@v80sE*)e&KCFNTX9Ai4fb z9jYUPv{#G!3tgFz1l29>ubkb=Wpt^A<|ec1)U+zYef(b-03j>X9sL^nX9@Z$$9P|% zWm-Vw&1Wd&jt?k0$x;zS!Y-KDaOkDBo6?;FsQJOu|mCQ io.snappydata snappydata-cluster_2.11 - 1.1.0 + 1.1.1 ``` @@ -36,7 +36,7 @@ You can use an IDE of your choice, and provide the below dependency to get Snapp ```pre // https://mvnrepository.com/artifact/io.snappydata/snappydata-cluster_2.11 -libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.0" +libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1" ``` **Note**:
@@ -71,5 +71,5 @@ To start SnappyData store you need to create a SnappySession in your program: If you already have Spark2.0 installed in your local machine you can directly use `--packages` option to download the SnappyData binaries. ```pre -./bin/spark-shell --packages "SnappyDataInc:snappydata:1.1.0-s_2.11" +./bin/spark-shell --packages "SnappyDataInc:snappydata:1.1.1-s_2.11" ``` diff --git a/docs/best_practices/setup_cluster.md b/docs/best_practices/setup_cluster.md index 34adf32838..a212b17630 100644 --- a/docs/best_practices/setup_cluster.md +++ b/docs/best_practices/setup_cluster.md @@ -16,7 +16,9 @@ So, with multiple concurrent users, it is best to avoid running such Jobs using !!! Note This above scheduling logic is applicable only when queries are fully managed by SnappyData cluster. When running your application using the smart connector, each task running in the Spark cluster directly accesses the store partitions. -### Computing the Number of Cores for a Job + +
+## Computing the Number of Cores for a Job Executing queries or code in SnappyData results in the creation of one or more Spark jobs. Each Spark job first calculates the number of partitions on the underlying dataset and a task is assigned and scheduled for each partition. But, the number of concurrent tasks executing is limited by the available core count. If the scheduled task count is larger then they will be executed in a staggered manner. Each task is assigned a single core to execute. @@ -56,7 +58,7 @@ When you add more servers to SnappyData, the processing capacity of the system i ![Concurrency](../Images/core_concurrency.png) -### Configuring the Scheduler Pools for Concurrency +## Configuring the Scheduler Pools for Concurrency SnappyData out of the box comes configured with two execution pools: * **Low-latency pool**: This pool is automatically used when SnappyData determines that a request is of low latency, that is, the queries that are partition pruned to two or fewer partitions.
@@ -71,7 +73,7 @@ Applications can explicitly configure to use a particular pool for the current s New pools can be added and properties of the existing pools can be configured by modifying the **conf/fairscheduler.xml** file. We do not recommend changing the pool names (`default` and `lowlatency`). -#### Controlling CPU Usage for User Jobs +### Controlling CPU Usage for User Jobs You can control the CPU usage for user jobs by configuring separate pools for different kinds of jobs.
See configuration [here](https://spark.apache.org/docs/2.1.1/job-scheduling.html#configuring-pool-properties).
The product is configured with two out-of-the-box pools, that is the **Default pool** and the **Low-latency pool**. The **Default pool** has higher priority and also has a **minShare**, so that some minimum cores are reserved for those jobs if possible. @@ -82,7 +84,7 @@ The [**Stages**](/monitoring/monitoring.md#stages) tab on the SnappyData Monitor To configure the priority based on specific requirements, you can also either permit the users to set the priority for queries or add some pool allocation logic in the application as per client requirements. -### Using a Partitioning Strategy to Increase Concurrency +## Using a Partitioning Strategy to Increase Concurrency The best way to increasing concurrency is to design your schema such that you minimize the need to run your queries across many partitions. The common strategy is to understand your application patterns and choose a partitioning strategy such that queries often target a specific partition. Such queries will be pruned to a single node and SnappyData automatically optimises such queries to use a single task. For more information see, [How to design your schema](design_schema.md). diff --git a/docs/configuration.md b/docs/configuration.md index 0c061012c5..f2000df2be 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -8,24 +8,18 @@ Also, each component can be configured individually using configuration files. I The following topics are covered in this section: -* [Configuration Files](configuring_cluster/configuring_cluster.md#configuration-files) - +* [Configuring and Launching a SnappyData Cluster](configuring_cluster/configure_launch_cluster.md) +* [Configuring and Launching a Multi-node SnappyData Cluster](configuring_cluster/configure_launch_cluster_multinode.md) +* [Configuration Files](configuring_cluster/configuring_cluster.md#configuration-files) - [Configuring Locators](configuring_cluster/configuring_cluster.md#locator) - - [Configuring Leads](configuring_cluster/configuring_cluster.md#lead) - - [Configuring Data Servers](configuring_cluster/configuring_cluster.md#dataserver) - - [Configuring SnappyData Smart Connector](configuring_cluster/configuring_cluster.md#configure-smart-connector) - - [Environment Settings](configuring_cluster/configuring_cluster.md#environment) - - [Hadoop Provided Settings](configuring_cluster/configuring_cluster.md#hadoop-setting) - - [SnappyData Command Line Utility](configuring_cluster/configuring_cluster.md#command-line) - - [Logging](configuring_cluster/configuring_cluster.md#logging) -* [List of Properties](configuring_cluster/property_description.md) +* [List of Properties](configuring_cluster/property_description.md) -* [Firewalls and Connections](configuring_cluster/firewalls_connections.md) \ No newline at end of file +* [Firewalls and Connections](configuring_cluster/firewalls_connections.md) \ No newline at end of file diff --git a/docs/configuring_cluster/configure_launch_cluster.md b/docs/configuring_cluster/configure_launch_cluster.md new file mode 100644 index 0000000000..9a0808550c --- /dev/null +++ b/docs/configuring_cluster/configure_launch_cluster.md @@ -0,0 +1,175 @@ +# Configuring, Launching SnappyData Clusters + +Before you configure the SnappyData cluster, check the [system requirements](/install/system_requirements.md). In case you have not yet provisioned SnappyData, you can follow the instructions [here](/install.md). +TIBCO recommends that you have at least **8 GB** of memory and **4 cores** available even for simple experimentation with SnappyData. + + +## Launching Single Node Cluster with Default Configuration +If you want to launch the cluster either on Amazon EC2 or on a Kubernetes cluster, you can follow the instructions listed [here (AWS)](/install/setting_up_cluster_on_amazon_web_services.md) and [here (Kubernetes)](/kubernetes.md) + +If you are launching on a single node, for example, on your laptop or on a linux server you have access to, you can do so using this simple command: + +``` +./sbin/snappy-start-all.sh +``` +This launches a single [locator](should point to concepts), [lead](should point to concepts) and a [data server](should point to concepts). You can go to the following URL on your browser to view the cluster dashboard: + +**http://(localhost or hostname or machineIP):5050** + +By default, the cluster uses the following ports: + +| Cluster Component | Port |Memory Used| +|--------|--------|| +| Lead |**5050** (http port used for dashboard.)
**8090** (Port used to submit Spark streaming or batch jobs.)
**10000** (Port for hive thrift server.) |**4 GB**| +| Locator | **1527** (Port used by JDBC clients.)
**10334** (Ports used for all cluster members to communicate with each other.) |**1 GB**| +|Server |**1528** (Port used by ODBC or JDBC clients) |**4 GB**| + +!!!Note + By default, the locator uses **1527** port to listen for client connections and the servers that are running on the same machine use subsequent port numbers. Therefore, **1528** port is used by the single server that is launched by the above command. But, if the server was launched on a different machine it would listen on **1527** also. + +All the artifacts created such as the server - logs, metrics, and the database files are all stored in a folder called **work** in the product home directory. Click the individual member URLs on the dashboard to view the logs. + +**Also see**: + +* [Connecting with JDBC](/howto/connect_using_jdbc_driver.md) + +* [Connecting with ODBC](/howto/connect_using_odbc_driver.md) + +* [Submitting a Spark Job](/howto/run_spark_job_inside_cluster.md) + + + + + +## Configuring and Launching a Multi-node Cluster + +[Provision SnappyData](/install.md) and ensure that all the nodes are setup appropriately. If all the nodes are [SSH enabled](/reference/misc/passwordless_ssh.md) and can share folders using NFS or some shared file system, you can proceed to [Capacity Planning](#initialcapplan). A shared file system is not a requirement but preferred. + + +### Step 1: Capacity Planning +You must consider the capacity required for [storage capacity](#storeplan) (in-memory and disk) as well as the [computational capacity](#computeplan) that is required (memory and CPU). You can skip this step if the volume of data is low and feel you have abundant resources for the client workload. + + +#### Planning Storage Capacity +SnappyData is optimized for managing all the data in the memory. When data cannot fit in the memory, it automatically overflows the data to disk. To achieve the highest possible performance, we recommend you go through this exercise below. The capacity required is dependent on several variables such as input data format, the data types in use, use of indexes, number of redundant copies in the cluster, cardinality of the individual columns (compression ratio can vary a lot) and so on. + +A general rule of thumb for compressed data (say Parquet) is to configure about **1.5X** the compressed size on disk. Else, you can also use the following steps: + +1. Define your external tables to access the data sources. This only creates catalog entries in SnappyData.
For example, when loading data from a folder with data in CSV format, you can do the following using the [Snappy Shell](/howto/use_snappy_shell.md): + + + create external table t1 using csv options(inferSchema 'true', header 'true', path '') ; + // Spark Scala/Java/Python API example is omitted for brevity. + + + More examples for loading data from external sources are available [here](/howto/load_data_from_external_data_stores.md) + +2. Decide if you want to manage this data in a [Columnar or Row format](/programming_guide/tables_in_snappydata.md). + +3. Load a sample of the data: + + create table t1_inmem using column as (select * from t1 where rand() < 0.1) ; + + This loads about 10% of the data from the external data source. Note that the fraction value (0.1, which is 10%) should be reduced when working with large data sets or increased when the data set is small. You must repeat this process for all the tables that you want to manage in the memory. + +4. Check the dashboard to view the actual memory consumed. Simply extrapolate to 100% (10 times for the example above). +The amount of disk space is about two times the memory space requirement. + + +#### Estimating Computation Capacity +TIBCO recommends to configure off-heap memory when using the Enterprise edition that is TIBCO ComputeDB. This option is not available in the community edition of SnappyData where all data must be managed in the JVM heap memory. Even when off-heap is configured, you must also configure enough JVM heap memory for Spark temporary caching, computations, and buffering when the data is loaded. + +!!!Note + Only the columnar table data is managed in off-heap memory. Row tables are always in JVM heap. + +TIBCO recommends to allocate at least **1 GB **of memory per core to JVM heap for computations. + +For example, when running on **8 core** servers, configure JVM heap to be **8 GB**. +By default, **50%** of the off-heap memory is available as computational memory. While, you may not need this much computational capacity when large off-heap is configured, it is still recommended for reserving enough capacity if working with large data sets. + + +More complex the analytical processing, especially large aggregations, greater the space requirement in off-heap. For example, if your per server off-heap storage need is **100 GB** then, allocate an additional **30 GB** of off-heap for computations. Even if your data set is small, you must still allocated at least a few Gigabytes of off-heap storage for computations. + +### Step 2: Configure Core Cluster Component Properties + +Configuration files for locator, lead, and server should be created in the **conf** folder located in the SnappyData home directory with names **locators**, **leads**, and **servers**. + +To do so, you can copy the existing template files **servers.template**, **locators.template**, **leads.template**, and rename them to **servers**, **locators**, **leads**. +These files should contain the hostnames of the nodes (one per line) where you intend to start the member. You can modify the properties to configure individual members. + +#### Configuring Core Lead properties +The following core properties must be set in the **conf/leads** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| hostname (or IP) | The hostname on which a SnappyData locator is started | | +| heap-size | Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings.
For example, `-heap-size=8g`
It is recommended to allocate minimum **6-8 GB** of heap size per lead node. If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 95% of the heap size, and the `eviction-heap-percentage` to 85.5% of the `critical-heap-percentage`.
SnappyData also sets resource management properties for eviction and garbage collection if the JVM supports them. | | +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. | /work | +| classpath | Location of user classes required by the SnappyData Server. This path is appended to the current classpath | Appended to the current classpath | +| zeppelin.interpreter.enable=true |Enable the SnappyData Zeppelin interpreter. Refer [How to use Apache Zeppelin with SnappyData](/howto/use_apache_zeppelin_with_snappydata.md) | | +| spark.executor.cores | The number of cores to use on each server. | | +| spark.jars | | | + + +##### Example Configurations +In the following configuration, you set the heap size for the lead and specify the working directory location: + +``` +localhost -dir=/opt/snappydata/data/lead -heap-size=6g +``` +You can add a line for each of the Lead members that you want to launch. Typically only one. In production, you may launch two. + +In the following configuration, you are specifying the [number of cores to use on each server](/best_practices/setup_cluster.md#computenoscores) as well as enabling the SnappyData Zeppelin interpreter: + +``` +localhost -spark.executor.cores=16 -zeppelin.interpreter.enable=true +``` + +!!!Tip + It is a common practice to run the Lead and Locator on a single machine. The locator requires very less memory and CPU, the lead memory requirement is directly proportional to the concurrency and the potential for returning large result sets. However, data servers do most of the heavy lifting for query execution and Spark distributed processing. + + +#### Configuring Core Locator Properties + +The following core properties must be set in the **conf/locators** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| client-port | The port that the network controller listens for client connections in the range of 1 to 65535. | 1527 | +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. |Current directory| + +### Configuring Core Server Properties + +The following core properties must be set in the **conf/servers** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| memory-size| Specifies the total memory that can be used by the node for column storage and execution in off-heap. | The default value is either 0 or it gets auto-configured in [specific scenarios](configuring_cluster.md#autoconfigur_offheap) | +| heap-size | Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings.
For example, `-heap-size=8g`
It is recommended to allocate minimum 6-8 GB of heap size per lead node. If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 95% of the heap size, and the `eviction-heap-percentage` to 85.5% of the `critical-heap-percentage`.
SnappyData also sets resource management properties for eviction and garbage collection if the JVM supports them. || +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. |Current directory| + +#### Configuration Examples + +``` +cat /conf/servers +node1_hostname -dir=/nfs/opt/snappy-db1/server1 -heap-size=8g -memory-size=42g +node1_hostname -dir=/nfs/opt/snappy-db1/server2 -heap-size=8g -memory-size=42g + +// You can launch more than one data server on a host + +node2_hostname -dir=/nfs/opt/snappy-db1/server3 -heap-size=8g -memory-size=42g +node2_hostname -dir=/nfs/opt/snappy-db1/server4 -heap-size=8g -memory-size=42g + +``` + +## Configuring Cluster Components + +Refer to [Configuring Cluster Components](/configuring_cluster/configuring_cluster.md) section for more details. + +## List of Properties + +Refer to the [SnappyData properties](property_description.md) for the complete list of SnappyData properties. + + + + diff --git a/docs/configuring_cluster/configure_launch_cluster_multinode.md b/docs/configuring_cluster/configure_launch_cluster_multinode.md new file mode 100644 index 0000000000..ff0fff8052 --- /dev/null +++ b/docs/configuring_cluster/configure_launch_cluster_multinode.md @@ -0,0 +1,115 @@ +# Configure and Launch a Multi-node Cluster + +[Provision SnappyData](/install.md) and ensure that all the components of the cluster ([locator](configuring_cluster.md#locator), [lead](configuring_cluster.md#lead) and a [data server](configuring_cluster.md#dataserver)) are setup appropriately. If all the nodes are [SSH enabled](/reference/misc/passwordless_ssh.md) and can share folders using NFS or some shared file system, you can proceed to [Capacity Planning](#initialcapplan). + + +## Step 1: Capacity Planning +You must consider the capacity required for [storage capacity](#storeplan) (in-memory and disk) as well as the [computational capacity](#computeplan) that is required (memory and CPU). You can skip this step if the volume of data is in relation to the capacity of the nodes (CPU, memory, disk). + + +### Planning Storage Capacity +SnappyData is optimized for managing all the data in the memory. When data cannot fit in the memory, it automatically overflows the data to disk. Therefore, figuring out the precise memory capacity is non-trivial since it is dependent on several variables such as input data format, the data types in use, use of indexes, number of redundant copies in the cluster, cardinality of the individual columns (compression ratio can vary a lot) and so on. + +A general rule of thumb for compressed data (say Parquet) is to configure about **1.5X** the compressed size on disk. Else, you can also use the following steps: + +1. Define your external tables to access the data sources. This only creates catalog entries in SnappyData.
For example, when loading data from a folder with data in CSV format, you can do the following using the [Snappy Shell](/howto/use_snappy_shell.md): + + create external table t1 using csv options(inferSchema 'true', header 'true', path '') ; + // Spark Scala/Java/Python API example is omitted for brevity. + +2. Decide if you want to manage this data in a [Columnar or Row format](/programming_guide/tables_in_snappydata.md). + +3. Load a sample of the data: + create table t1_inmem using column as (select * from t1 where rand() < 0.1) ; + This would load about 10% of the data in the external data source. Note that the fraction value (10% above) should be inversely proportional to the input data size. You must repeat this process for all the tables that you want to manage in memory. +4. Check the dashboard to view the actual memory consumed. Simply extrapolate to 100% (10 times for the example above). +The amount of disk space is about two times the memory space requirement. + + +### Estimating Computation Capacity +TIBCO recommends to configure off-heap memory when using the Enterprise edition that is TIBCO ComputeDB. This option is not available in the community edition of SnappyData where all data must be managed in the JVM heap memory. Even if off-heap is configured, you must also configure enough JVM heap memory for Spark temporary caching, computations, and buffering when the data is loaded. + +!!!Note + Only the columnar table data is managed in off-heap memory. Row tables are always in JVM heap. + +TIBCO recommends to allocate at least **1 GB **of memory per core to JVM heap for computations. For example, when running on **8 core** servers, configure JVM heap to be **8 GB**. +By default, **50%** of the off-heap memory is available as computational memory. While, you may not need this much computational capacity when large off-heap is configured, it is still recommended for reserving a capacity that is proportional to the data being processed. + +More complex the analytical processing, especially large aggregations, greater the space requirement in off-heap. For example, if your per server off-heap storage need is **100 GB** then, allocate an additional **30 GB** of off-heap for computations. Even if your data set is small, you must still ensure the availability of some space in Gigabytes on the off-heap storage. + +## Step 2: Configure Core Cluster Component Properties + +Configuration files for locator, lead, and server should be created in the **conf** folder located in the SnappyData home directory with names **locators**, **leads**, and **servers**. + +To do so, you can copy the existing template files **servers.template**, **locators.template**, **leads.template**, and rename them to **servers**, **locators**, **leads**. +These files should contain the hostnames of the nodes (one per line) where you intend to start the member. You can modify the properties to configure individual members. + +### Configuring Core Lead properties +The following core properties must be set in the **conf/leads** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| hostname (or IP) | The hostname on which a SnappyData locator is started | | +| heap-size | Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings.
For example, `-heap-size=8g`
It is recommended to allocate minimum **6-8 GB** of heap size per lead node. If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 95% of the heap size, and the `eviction-heap-percentage` to 85.5% of the `critical-heap-percentage`.
SnappyData also sets resource management properties for eviction and garbage collection if the JVM supports them. | | +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. | Current directory | +| classpath | Location of user classes required by the SnappyData Server. This path is appended to the current classpath | Appended to the current classpath | +| -zeppelin.interpreter.enable=true |Enable the SnappyData Zeppelin interpreter. Refer [How to use Apache Zeppelin with SnappyData](/howto/use_apache_zeppelin_with_snappydata.md) | | +| spark.executor.cores | The number of cores to use on each server. | | +| spark.jars | | | + + +#### Example Configurations +In the following configuration, you set the heap size for the lead and specify the working directory location: + +``` +localhost -dir=/opt/snappydata/data/lead -heap-size=6g +``` +You can add a line for each of the Lead members that you want to launch. Typically only one. In production, you may launch two. + +In the following configuration, you are specifying the Spark UI port and the number of cores to use on each server as well as enabling the SnappyData Zeppelin interpreter + +``` +localhost -spark.ui.port=3333 -spark.executor.cores=16 -zeppelin.interpreter.enable=true +``` + +!!!Tip + It is a common practice to run the Lead and Locator on a single machine. The locator requires very less memory and CPU, the lead memory requirement is directly proportional to the concurrency and the potential for returning large result sets. However, data servers do most of the heavy lifting for query execution and Spark distributed processing. + + +### Configuring Core Locator Properties + +The following core properties must be set in the **conf/locators** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| client-port | The port that the network controller listens for client connections in the range of 1 to 65535. | 1527 | +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. |Current directory| + +### Configuring Core Server Properties + +The following core properties must be set in the **conf/servers** file: + +| Properties | Description |Default Value +|--------|--------|--------| +| memory-size| Specifies the total memory that can be used by the node for column storage and execution in off-heap. | The default value is either 0 or it gets auto-configured in [specific scenarios](configuring_cluster.md#autoconfigur_offheap) | +| heap-size | Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings.
For example, `-heap-size=8g`
It is recommended to allocate minimum 6-8 GB of heap size per lead node. If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 95% of the heap size, and the `eviction-heap-percentage` to 85.5% of the `critical-heap-percentage`.
SnappyData also sets resource management properties for eviction and garbage collection if the JVM supports them. || +| dir | Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth. |Current directory| + +#### Configuration Examples + +``` +cat /conf/servers +node1_hostname -dir=/nfs/opt/snappy-db1/server1 -heap-size=8g -memory-size=42g +node1_hostname -dir=/nfs/opt/snappy-db1/server2 -heap-size=8g -memory-size=42g + +// You can launch more than one data server on a host + +node2_hostname -dir=/nfs/opt/snappy-db1/server3 -heap-size=8g -memory-size=42g +node2_hostname -dir=/nfs/opt/snappy-db1/server4 -heap-size=8g -memory-size=42g + +``` + + + + + diff --git a/docs/configuring_cluster/configuring_cluster.md b/docs/configuring_cluster/configuring_cluster.md index 0d07634bea..601d11e784 100644 --- a/docs/configuring_cluster/configuring_cluster.md +++ b/docs/configuring_cluster/configuring_cluster.md @@ -1,4 +1,4 @@ -# Configuration +# Configuring Cluster Components Configuration files for locator, lead, and server should be created in the **conf** folder located in the SnappyData home directory with names **locators**, **leads**, and **servers**. @@ -6,14 +6,14 @@ To do so, you can copy the existing template files **servers.template**, **locat These files should contain the hostnames of the nodes (one per line) where you intend to start the member. You can modify the properties to configure individual members. !!! Tip - - For system properties (set in the conf/lead, conf/servers and conf/locators file), -D and -XX: can be used. All other JVM properties need the `-J` prefix. + - For system properties (set in the conf/lead, conf/servers and conf/locators file), -D and -XX: can be used. -J is NOT required for -D and -XX options. - Instead of starting the SnappyData cluster, you can [start](../howto/start_snappy_cluster.md) and [stop](../howto/stop_snappy_cluster.md) individual components on a system locally. -## Configuring Locators +## Locators -Locators provide discovery service for the cluster. Clients (for example, JDBC) connect to the locator and discover the lead and data servers in the cluster. The clients automatically connect to the data servers upon discovery (upon initial connection). Cluster members (Data servers, Lead nodes) also discover each other using the locator. Refer to the [Architecture](../architecture.md) section for more information on the core components. +Locators provide discovery service for the cluster. Clients (for example, JDBC) connect to the locator and discover the lead and data servers in the cluster. The clients automatically connect to the data servers upon discovery (upon initial connection). Cluster members (Data servers, Lead nodes) also discover each other using the locator. Refer to the [Architecture](/architecture/cluster_architecture.md) section for more information on the core components. It is recommended to configure two locators (for HA) in production using the **conf/locators** file located in the **<_SnappyData_home_>/conf** directory. @@ -31,6 +31,7 @@ You can refer to the **conf/locators.template** file for some examples. Refer to the [SnappyData properties](property_description.md) for the complete list of SnappyData properties. + |Property|Description| |-----|-----| |-bind-address|IP address on which the locator is bound. The default behavior is to bind to all local addresses.| @@ -54,10 +55,12 @@ Refer to the [SnappyData properties](property_description.md) for the complete l $ cat conf/locators node-a -peer-discovery-port=9999 -dir=/node-a/locator1 -heap-size=1024m -locators=node-b:8888 node-b -peer-discovery-port=8888 -dir=/node-b/locator2 -heap-size=1024m -locators=node-a:9999 + ``` + -## Configuring Leads +## Leads Lead Nodes primarily runs the SnappyData managed Spark driver. There is one primary lead node at any given instance, but there can be multiple secondary lead node instances on standby for fault tolerance. Applications can run Jobs using the REST service provided by the Lead node. Most of the SQL queries are automatically routed to the Lead to be planned and executed through a scheduler. You can refer to the **conf/leads.template** file for some examples. @@ -66,6 +69,7 @@ Create the configuration file (**leads**) for leads in the **<_SnappyData_home_> !!! Note In the **conf/spark-env.sh** file set the `SPARK_PUBLIC_DNS` property to the public DNS name of the lead node. This enables the Member Logs to be displayed correctly to users accessing SnappyData Monitoring Console from outside the network. + ### List of Lead Properties Refer to the [SnappyData properties](property_description.md) for the complete list of SnappyData properties. @@ -111,6 +115,8 @@ Refer to the [SnappyData properties](property_description.md) for the complete l |-spark.ui.port|Port for your SnappyData Monitoring Console, which shows tables, memory and workload data. The default is 5050.| |Properties for SSL Encryption|[ssl-enabled](../reference/configuration_parameters/ssl_enabled.md), [ssl-ciphers](../reference/configuration_parameters/ssl_ciphers.md), [ssl-protocols](../reference/configuration_parameters/ssl_protocols.md), [ssl-require-authentication](../reference/configuration_parameters/ssl_require_auth.md).
These properties need not be added to the Lead members in case of a client-server connection.| + + **Example**: To start a lead (node-l), set `spark.executor.cores` as 10 on all servers, and change the Spark UI port from 5050 to 9090, update the configuration file as follows: ```pre @@ -118,6 +124,7 @@ $ cat conf/leads node-l -heap-size=4096m -spark.ui.port=9090 -locators=node-b:8888,node-a:9999 -spark.executor.cores=10 ``` + ###Configuring Secondary Lead @@ -135,7 +142,7 @@ In this example, two leads (one on node-l1 and another on node-l2) are configure -## Configuring Data Servers +## Data Servers Data Servers hosts data, embeds a Spark executor, and also contains a SQL engine capable of executing certain queries independently and more efficiently than the Spark engine. Data servers use intelligent query routing to either execute the query directly on the node or to pass it to the lead node for execution by Spark SQL. You can refer to the **conf/servers.template** file for some examples. Create the configuration file (**servers**) for data servers in the **<_SnappyData_home_>/conf** directory. @@ -143,6 +150,7 @@ Create the configuration file (**servers**) for data servers in the **<_SnappyDa ### List of Server Properties Refer to the [SnappyData properties](property_description.md) for the complete list of SnappyData properties. + |Property|Description
| |-|-| |-bind-address|IP address on which the server is bound. The default behavior is to bind to all local addresses.| @@ -168,6 +176,7 @@ Refer to the [SnappyData properties](property_description.md) for the complete l |-thrift-ssl-properties|Comma-separated SSL properties including:
`protocol`: default "TLS",
`enabled-protocols`: enabled protocols separated by ":"
`cipher-suites`: enabled cipher suites separated by ":"
`client-auth`=(true or false): if client also needs to be authenticated
`keystore`: path to key store file
`keystore-type`: the type of key-store (default "JKS")
`keystore-password`: password for the key store file
`keymanager-type`: the type of key manager factory
`truststore`: path to trust store file
`truststore-type`: the type of trust-store (default "JKS")
`truststore-password`: password for the trust store file
`trustmanager-type`: the type of trust manager factory
| + **Example**: To start a two servers (node-c and node-c), update the configuration file as follows: ```pre @@ -175,6 +184,7 @@ $ cat conf/servers node-c -dir=/node-c/server1 -heap-size=4096m -memory-size=16g -locators=node-b:8888,node-a:9999 node-c -dir=/node-c/server2 -heap-size=4096m -memory-size=16g -locators=node-b:8888,node-a:9999 ``` + ## Specifying Configuration Properties using Environment Variables SnappyData configuration properties can be specified using environment variables LOCATOR_STARTUP_OPTIONS, SERVER_STARTUP_OPTIONS, and LEAD_STARTUP_OPTIONS respectively for locators, leads and servers. These environment variables are useful to specify common properties for locators, servers, and leads. These startup environment variables can be specified in **conf/spark-env.sh** file. This file is sourced when SnappyData system is started. A template file **conf/spark-env.sh.template** is provided in **conf** directory for reference. You can copy this file and use it to configure properties. @@ -212,7 +222,7 @@ Spark applications run as independent sets of processes on a cluster, coordinate ```pre $ ./bin/spark-submit --deploy-mode cluster --class somePackage.someClass --master spark://localhost:7077 --conf spark.snappydata.connection=localhost:1527 - --packages 'SnappyDataInc:snappydata:1.1.0-s_2.11' + --packages 'SnappyDataInc:snappydata:1.1.1-s_2.11' ``` ## Environment Settings diff --git a/docs/configuring_cluster/property_description.md b/docs/configuring_cluster/property_description.md index 8a984eb247..64a7689c53 100644 --- a/docs/configuring_cluster/property_description.md +++ b/docs/configuring_cluster/property_description.md @@ -2,43 +2,86 @@ The following list of commonly used properties can be set to configure the cluster. These properties can be set in the **conf/servers**, **conf/leads** or **conf/locators** configuration files. +!!! Tip + For system properties (set in the conf/lead, conf/servers and conf/locators file), -D and -XX: can be used. -J is NOT required for -D and -XX options. + +## Network Configuration + |Property|Description|Components
| |-|-|-| |-bind-address|IP address on which the member is bound. The default behavior is to bind to all local addresses.|Server
Lead
Locator| -|-classpath|Location of user classes required by the SnappyData Server.
This path is appended to the current classpath.|Server
Lead
Locator| |-client-port| The port that the network controller listens for client connections in the range of 1 to 65535. The default value is 1527.|Locator
Server| +|hostname-for-clients|Set the IP address or host name that this server/locator sends to JDBC/ODBC/thrift clients to use for connection. The default value causes the client-bind-address to be given to clients. This value can be different from client-bind-address for cases where locators, servers are behind a NAT firewall (AWS for example) where client-bind-address needs to be a private one that gets exposed to clients outside the firewall as a different public address specified by this property. In many cases this is handled by hostname translation itself, i.e. hostname used in client-bind-address resolves to internal IP address from inside but to public IP address from outside, but for other cases this property will be required.|Server| +|-locators|List of locators as comma-separated host:port values used to communicate with running locators in the system and thus discover other peers of the distributed system.
The list must include all locators in use and must be configured consistently for every member of the distributed system. This property should be configured for all the nodes in the respective configuration files, if there are multiple locators.|Server
Lead
Locator| +|-member-timeout|Uses the member-timeout server configuration, specified in milliseconds, to detect the abnormal termination of members. The configuration setting is used in two ways:
1) First, it is used during the UDP heartbeat detection process. When a member detects that a heartbeat datagram is missing from the member that it is monitoring after the time interval of 2 * the value of member-timeout, the detecting member attempts to form a TCP/IP stream-socket connection with the monitored member as described in the next case.
2) The property is then used again during the TCP/IP stream-socket connection. If the suspected process does not respond to the **are you alive** datagram within the time period specified in member-timeout, the membership coordinator sends out a new membership view that notes the member's failure.
Valid values are in the range 1000-600000 milliseconds. For more information, refer to [Best Practices](../best_practices/important_settings.md#member-timeout)|Server
Lead
Locator| +|-peer-discovery-address|Use this as value for the port in the "host:port" value of "-locators" property |Locator| +|-peer-discovery-port|Port on which the locator listens for peer discovery (includes servers as well as other locators).
Valid values are in the range 1-65535, with a default of 10334.|Locator| +|-spark.ui.port|Port for your SnappyData Monitoring Console, which shows tables, memory and workload data. The default is 5050|Lead| + +## Memory Configuration + +|Property|Description|Components
| +|-|-|-| |-critical-heap-percentage|Sets the Resource Manager's critical heap threshold in percentage of the old generation heap, 0-100.
If you set `-heap-size`, the default value for `critical-heap-percentage` is set to 95% of the heap size.
Use this switch to override the default.
When this limit is breached, the system starts canceling memory-intensive queries, throws low memory exceptions for new SQL statements, and so forth, to avoid running out of memory.|Server
Lead| |-critical-off-heap-percentage|Sets the critical threshold for off-heap memory usage in percentage, 0-100.
When this limit is breached, the system starts canceling memory-intensive queries, throws low memory exceptions for new SQL statements, and so forth, to avoid running out of off-heap memory.|Server| -|-dir|Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth (defaults to the current directory).| Server
Lead
Locator
| |-eviction-heap-percentage|Sets the memory usage percentage threshold (0-100) that the Resource Manager will use to start evicting data from the heap. By default, the eviction threshold is 85.5% of whatever is set for `-critical-heap-percentage`.
Use this switch to override the default.|Server
Lead
| |-eviction-off-heap-percentage|Sets the off-heap memory usage percentage threshold, 0-100, that the Resource Manager uses to start evicting data from off-heap memory.
By default, the eviction threshold is 85.5% of whatever is set for `-critical-off-heap-percentage`.
Use this switch to override the default.|Server| |-heap-size| Sets the maximum heap size for the Java VM, using SnappyData default resource manager settings.
For example, -heap-size=1GB.
If you use the `-heap-size` option, by default SnappyData sets the critical-heap-percentage to 95% of the heap size, and the `eviction-heap-percentage` to 85.5% of the `critical-heap-percentage`.
SnappyData also sets resource management properties for eviction and garbage collection if they are supported by the JVM. |Server
Lead
Locator| |-J|JVM option passed to the spawned SnappyData server JVM.
For example, use -J-Xmx1GB to set the JVM heap to 1GB.|Server
Lead
Locator| -|-J-Dgemfirexd.hostname-for-clients|Set the IP address or host name that this server/locator sends to JDBC/ODBC/thrift clients to use for connection. The default value causes the client-bind-address to be given to clients. This value can be different from client-bind-address for cases where locators, servers are behind a NAT firewall (AWS for example) where client-bind-address needs to be a private one that gets exposed to clients outside the firewall as a different public address specified by this property. In many cases this is handled by hostname translation itself, i.e. hostname used in client-bind-address resolves to internal IP address from inside but to public IP address from outside, but for other cases this property will be required.|Server| -|-J-Dsnappydata.enable-rls|Enables the system for row level security when set to true. By default this is off. If this property is set to true, then the Smart Connector access to SnappyData fails.|Server
Lead
Locator -|-J-Dsnappydata.RESTRICT_TABLE_CREATION|Applicable when security is enabled in the cluster. If true, users cannot execute queries (including DDLs and DMLs) even in their default or own schema unless cluster admin explicitly grants them the required permissions using GRANT command. The default is false. |Server
Lead
Locator| -|jobserver.waitForInitialization|When this property is set to true, the cluster startup waits for the Spark jobserver to be fully initialized before marking the lead node as **RUNNING**. The default is false.|Lead| -|-locators|List of locators as comma-separated host:port values used to communicate with running locators in the system and thus discover other peers of the distributed system.
The list must include all locators in use and must be configured consistently for every member of the distributed system. This property should be configured for all the nodes in the respective configuration files, if there are multiple locators.|Server
Lead
Locator| -|-log-file|Path of the file to which this member writes log messages (default is snappy[member].log in the working directory. For example, **snappylocator.log**, **snappyleader.log**,**snappyserver.log**. In case logging is set via log4j, the default log file is **snappydata.log** for each of the SnappyData member.)|Server
Lead
Locator| |-memory-size|Specifies the total memory that can be used by the node for column storage and execution in off-heap. The default value is either 0 or it gets auto-configured in [specific scenarios](../configuring_cluster/configuring_cluster.md#autoconfigur_offheap). |Server
Lead| -|-member-timeout|Uses the member-timeout server configuration, specified in milliseconds, to detect the abnormal termination of members. The configuration setting is used in two ways:
1) First, it is used during the UDP heartbeat detection process. When a member detects that a heartbeat datagram is missing from the member that it is monitoring after the time interval of 2 * the value of member-timeout, the detecting member attempts to form a TCP/IP stream-socket connection with the monitored member as described in the next case.
2) The property is then used again during the TCP/IP stream-socket connection. If the suspected process does not respond to the **are you alive** datagram within the time period specified in member-timeout, the membership coordinator sends out a new membership view that notes the member's failure.
Valid values are in the range 1000-600000 milliseconds. For more information, refer to [Best Practices](../best_practices/important_settings.md#member-timeout)|Server
Lead
Locator| -|-peer-discovery-address|Use this as value for the port in the "host:port" value of "-locators" property |Locator| -|-peer-discovery-port|Port on which the locator listens for peer discovery (includes servers as well as other locators).
Valid values are in the range 1-65535, with a default of 10334.|Locator| |-spark.driver.maxResultSize|Limit of the total size of serialized results of all partitions for each action (e.g. collect). The value should be at least 1MB or 0 for unlimited. Jobs will be aborted if the total size of results is above this limit. Having a high limit may cause out-of-memory errors in the lead. The default max size is 1GB. |Lead| + + + +## Disk Configuration + +|Property|Description|Components
| +|-|-|-| +|-dir|Working directory of the member that contains the SnappyData Server status file and the default location for the log file, persistent files, data dictionary, and so forth (defaults to the current directory).| Server
Lead
Locator
| +|-spark.local.dir|Directory to use for "scratch" space in SnappyData, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. For more information, refer to [Best Practices](../best_practices/important_settings.md#spark-local-dir).|Lead
Server| + +AQP configuration + +## Security Configuration + +|Property|Description|Components
| +|-|-|-| +|-Dsnappydata.enable-rls|Enables the system for row level security when set to true. By default this is off. If this property is set to true, then the Smart Connector access to SnappyData fails.|Server
Lead
Locator +|-Dsnappydata.RESTRICT_TABLE_CREATION|Applicable when security is enabled in the cluster. If true, users cannot execute queries (including DDLs and DMLs) even in their default or own schema unless cluster admin explicitly grants them the required permissions using GRANT command. The default is false. |Server
Lead
Locator| +|-spark.ssl.enabled|Enables or disables Spark layer encryption. The default is false. |Lead| +|-spark.ssl.keyPassword|The password to the private key in the key store. |Lead| +|-spark.ssl.keyStore|Path to the key store file. The path can be absolute or relative to the directory in which the process is started.|Lead| +|-spark.ssl.keyStorePassword|The password used to access the keystore. |Lead| +|-spark.ssl.trustStore|Path to the trust store file. The path can be absolute or relative to the directory in which the process is started. |Lead| +|-spark.ssl.trustStorePassword|The password used to access the truststore. |Lead| +|-spark.ssl.protocol|The protocol that must be supported by JVM. For example, TLS.|Lead| +|Properties for SSL Encryption|[ssl-enabled](../reference/configuration_parameters/ssl_enabled.md), [ssl-ciphers](../reference/configuration_parameters/ssl_ciphers.md), [ssl-protocols](../reference/configuration_parameters/ssl_protocols.md), [ssl-require-authentication](../reference/configuration_parameters/ssl_require_auth.md).
These properties need not be added to the Lead members in case of a client-server connection.|Server
Lead
Locator
| +|-thrift-ssl|Specifies if you want to enable or disable SSL. Values are true or false.|Server
Lead
Locator
+|-thrift-ssl-properties|Comma-separated SSL properties including:
`protocol`: default "TLS",
`enabled-protocols`: enabled protocols separated by ":"
`cipher-suites`: enabled cipher suites separated by ":"
`client-auth`=(true or false): if client also needs to be authenticated
`keystore`: Path to key store file
`keystore-type`: The type of key-store (default "JKS")
`keystore-password`: Password for the key store file
`keymanager-type`: The type of key manager factory
`truststore`: Path to trust store file
`truststore-type`: The type of trust-store (default "JKS")
`truststore-password`: Password for the trust store file
`trustmanager-type`: The type of trust manager factory
|Server| + +## Spark Configuration + +|Property|Description|Components
| +|-|-|-| |-spark.executor.cores|The number of cores to use on each server. |Lead| |-spark.context-settings.num-cpu-cores| The number of cores that can be allocated. The default is 4. |Lead| |-spark.context-settings.memory-per-node| The executor memory per node (-Xmx style. For example: 512m, 1G). The default is 512m. |Lead| |-spark.context-settings.streaming.batch_interval| The batch interval for Spark Streaming contexts in milliseconds. The default is 1000.|Lead| |-spark.context-settings.streaming.stopGracefully| If set to true, the streaming stops gracefully by waiting for the completion of processing of all the received data. The default is true.|Lead| |-spark.context-settings.streaming.stopSparkContext| if set to true, the SparkContext is stopped along with the StreamingContext. The default is true. |Lead| +|-spark.driver.maxResultSize|Limit of the total size of serialized results of all partitions for each action (e.g. collect). The value should be at least 1MB or 0 for unlimited. Jobs will be aborted if the total size of results is above this limit. Having a high limit may cause out-of-memory errors in the lead. The default max size is 1GB. |Lead| |-spark.jobserver.bind-address|The address on which the jobserver listens. Default address is 0.0.0.|Lead| |-spark.jobserver.job-result-cache-size|The number of job results to keep per JobResultActor/context. The default is 5000.|Lead| |-spark.jobserver.max-jobs-per-context|The number of jobs that can be run simultaneously in the context. The default is 8.|Lead| |-spark.jobserver.port|The port on which to run the jobserver. Default port is 8090.|Lead| -|-spark.local.dir|Directory to use for "scratch" space in SnappyData, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. For more information, refer to [Best Practices](../best_practices/important_settings.md#spark-local-dir).|Lead
Server| |-spark.network.timeout|The default timeout for all network interactions while running queries.|Lead| +|-spark.local.dir|Directory to use for "scratch" space in SnappyData, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. For more information, refer to [Best Practices](../best_practices/important_settings.md#spark-local-dir).|Lead
Server| +|-spark.sql.aqp.numBootStrapTrials|Number of bootstrap trials to do for calculating error bounds. The default value is100.
This property must be set in the **conf/leads** file.| +|-spark.sql.aqp.error|Maximum relative error tolerable in the approximate value calculation. It should be a fractional value not exceeding 1. The default value is0.2.
This property can be set as connection property in the Snappy SQL shell.| +|-spark.sql.aqp.confidence|Confidence with which the error bounds are calculated for the approximate value. It should be a fractional value not exceeding 1.
The default value is0.95.
This property can be set as connection property in the Snappy SQL shell.| +|-spark.sql.autoBroadcastJoinThreshold|Configures the maximum size in bytes for a table that is broadcast to all server nodes when performing a join. By setting this value to **-1** broadcasting can be disabled. | +|-sparksql.aqp.behavior|The action to be taken if the error computed goes outside the error tolerance limit. The default value is`DO_NOTHING`.
This property can be set as connection property in the Snappy SQL shell.| |-spark.sql.codegen.cacheSize|Size of the generated code cache. This effectively controls the maximum number of query plans whose generated code (Classes) is cached. The default is 2000. |Lead| -|spark.sql.files.maxPartitionBytes|Maximum number of bytes to pack into a single partition when reading files. You can modify this setting in **conf/leads** file. This can be used to tune performance and memory requirements for data ingestion tasks when the data is read from a file based source for ingestion into a SnappyData column/row table. In SnappyData, default value for this setting is 33554432 bytes (32 MB).| Lead| |-spark.ssl.enabled|Enables or disables Spark layer encryption. The default is false. |Lead| |-spark.ssl.keyPassword|The password to the private key in the key store. |Lead| |-spark.ssl.keyStore|Path to the key store file. The path can be absolute or relative to the directory in which the process is started.|Lead| @@ -47,11 +90,26 @@ The following list of commonly used properties can be set to configure the clust |-spark.ssl.trustStorePassword|The password used to access the truststore. |Lead| |-spark.ssl.protocol|The protocol that must be supported by JVM. For example, TLS.|Lead| |-spark.ui.port|Port for your SnappyData Monitoring Console, which shows tables, memory and workload data. The default is 5050|Lead| -|Properties for SSL Encryption|[ssl-enabled](../reference/configuration_parameters/ssl_enabled.md), [ssl-ciphers](../reference/configuration_parameters/ssl_ciphers.md), [ssl-protocols](../reference/configuration_parameters/ssl_protocols.md), [ssl-require-authentication](../reference/configuration_parameters/ssl_require_auth.md).
These properties need not be added to the Lead members in case of a client-server connection.|Server
Lead
Locator
| -|-thrift-ssl|Specifies if you want to enable or disable SSL. Values are true or false.|Server
Lead
Locator
-|-thrift-ssl-properties|Comma-separated SSL properties including:
`protocol`: default "TLS",
`enabled-protocols`: enabled protocols separated by ":"
`cipher-suites`: enabled cipher suites separated by ":"
`client-auth`=(true or false): if client also needs to be authenticated
`keystore`: Path to key store file
`keystore-type`: The type of key-store (default "JKS")
`keystore-password`: Password for the key store file
`keymanager-type`: The type of key manager factory
`truststore`: Path to trust store file
`truststore-type`: The type of trust-store (default "JKS")
`truststore-password`: Password for the trust store file
`trustmanager-type`: The type of trust manager factory
|Server| -Other than the above properties, you can also refer the [Configuration Parameters section](/reference/configuration_parameters/config_parameters.md#property-names) for properties that are used in special cases. + +## Logging, Metrics Configuration + +|Property|Description|Components
| +|-|-|-| +|-log-file|Path of the file to which this member writes log messages (default is snappy[member].log in the working directory. For example, **snappylocator.log**, **snappyleader.log**,**snappyserver.log**. In case logging is set via log4j, the default log file is **snappydata.log** for each of the SnappyData member.)|Server
Lead
Locator| +|snappydata.sql.planCaching|Use this property to enable/disable plan caching. By default it is disabled. |Lead| + + +## JVM Configuration: + +|Property|Description|Components
| +|-|-|-| +|-J|JVM option passed to the spawned SnappyData server JVM.
For example, use -J-Xmx1GB to set the JVM heap to 1GB.|Server
Lead
Locator| +|-classpath|Location of user classes required by the SnappyData Server.
This path is appended to the current classpath.|Server
Lead
Locator| + + + +Also refer to [Configuration Parameters section](/reference/configuration_parameters/config_parameters.md#property-names). ## SQL Properties @@ -91,9 +149,9 @@ node-l -heap-size=4096m -spark.ui.port=9090 -locators=node-b:8888,node-a:9999 -s -## SDE Properties +## Approximate Query Processing (AQP) Properties -The [SDE](../aqp.md) properties can be set using a Snappy SQL shell (snappy-sql) command or using the configuration properties in the **conf/leads** file.
+The [AQP](../aqp.md) properties can be set using a Snappy SQL shell (snappy-sql) command or using the configuration properties in the **conf/leads** file.
The command sets the property for the current SnappySession while setting it in **conf/leads** file sets the property for all SnappySession. For example: Set in the Snappy SQL shell (snappy-sql) @@ -116,3 +174,10 @@ This sets the property for the snappy SQL shell's session. |-spark.sql.aqp.error|Maximum relative error tolerable in the approximate value calculation. It should be a fractional value not exceeding 1. The default value is0.2.
This property can be set as connection property in the Snappy SQL shell.| |-spark.sql.aqp.confidence|Confidence with which the error bounds are calculated for the approximate value. It should be a fractional value not exceeding 1.
The default value is0.95.
This property can be set as connection property in the Snappy SQL shell.| |-sparksql.aqp.behavior|The action to be taken if the error computed goes outside the error tolerance limit. The default value is`DO_NOTHING`.
This property can be set as connection property in the Snappy SQL shell.| + + + + + + + diff --git a/docs/connectors/deployment_dependency_jar.md b/docs/connectors/deployment_dependency_jar.md index 4527fef6be..86b6498e7b 100644 --- a/docs/connectors/deployment_dependency_jar.md +++ b/docs/connectors/deployment_dependency_jar.md @@ -12,7 +12,7 @@ SnappyData offers the following SQL commands: * **deploy package** - to deploy maven packages * **deploy jar** - to deploy your application or library Jars -Besides these SQL extensions, support is provided in SnappyData 1.1.0 version to deploy packages as part of SnappyData Job submission. This is similar to [Spark’s support](https://spark.apache.org/docs/latest/submitting-applications.html) for **--packages** when submitting Spark jobs. +Besides these SQL extensions, support is provided in SnappyData 1.1.1 version to deploy packages as part of SnappyData Job submission. This is similar to [Spark’s support](https://spark.apache.org/docs/latest/submitting-applications.html) for **--packages** when submitting Spark jobs. The following sections are included in this topic: diff --git a/docs/connectors/jdbc_streaming_connector.md b/docs/connectors/jdbc_streaming_connector.md index e172a90e1c..a91b887714 100644 --- a/docs/connectors/jdbc_streaming_connector.md +++ b/docs/connectors/jdbc_streaming_connector.md @@ -25,7 +25,7 @@ SnappyData core and SnappyData jdbc streaming connector maven dependencies would io.snappydata snappydata-core_2.11 - 1.1.0 + 1.1.1 compile ``` diff --git a/docs/howto/concurrent_apache_zeppelin_access_to_secure_snappydata.md b/docs/howto/concurrent_apache_zeppelin_access_to_secure_snappydata.md index 4e4437d017..76d4a2958a 100644 --- a/docs/howto/concurrent_apache_zeppelin_access_to_secure_snappydata.md +++ b/docs/howto/concurrent_apache_zeppelin_access_to_secure_snappydata.md @@ -53,8 +53,8 @@ Log on to Zeppelin from your web browser and configure the [JDBC Interpreter](ht |default.password|user123|The JDBC user password| |default.user|user1|The JDBC username| -3. **Dependency settings**
Since Zeppelin includes only PostgreSQL driver jar by default, you need to add the Client (JDBC) JAR file path for SnappyData. The SnappyData Client (JDBC) JAR file (TIB_compute-jdbc-2.11_1.1.0.jar) is available on [the release page](https://github.com/SnappyDataInc/snappydata/releases/latest).
- The SnappyData Client (JDBC) JAR file (snappydata-jdbc_2.11-1.1.0.jar)can also be placed under **/interpreter/jdbc** before starting Zeppelin instead of providing it in the dependency setting. +3. **Dependency settings**
Since Zeppelin includes only PostgreSQL driver jar by default, you need to add the Client (JDBC) JAR file path for SnappyData. The SnappyData Client (JDBC) JAR file (snappydata-jdbc-2.11_1.1.1.jar) is available on [the release page](https://github.com/SnappyDataInc/snappydata/releases/latest).
+ The SnappyData Client (JDBC) JAR file (snappydata-jdbc_2.11-1.1.1.jar)can also be placed under **/interpreter/jdbc** before starting Zeppelin instead of providing it in the dependency setting. 4. If required, edit other properties, and then click **Save** to apply your changes. diff --git a/docs/howto/connect_oss_vis_client_tools.md b/docs/howto/connect_oss_vis_client_tools.md index 253e17efcf..2da97fb52f 100644 --- a/docs/howto/connect_oss_vis_client_tools.md +++ b/docs/howto/connect_oss_vis_client_tools.md @@ -60,7 +60,7 @@ To connect SnappyData from SQL Workbench/J, do the following: 4. Click **Manage Drivers** from the bottom left. The **Manage driver** dialog box is displayed. 5. Enter the following details: * **Name**: Provide a name for the driver. - * **Library**: Click the folder icon and select the JDBC Client jar.
You must download the JDBC Client jar (snappydata-jdbc_2.11-1.1.0.jar) from the SnappyData website to your local machine. + * **Library**: Click the folder icon and select the JDBC Client jar.
You must download the JDBC Client jar (snappydata-jdbc_2.11-1.1.1.jar) from the SnappyData website to your local machine. * **Classname**: **io.snappydata.jdbc.ClientDriver**. * **Sample** **URL**: jdbc:snappydata://server:port/ 6. Click **OK**. The **Select Connection Profile** page is displayed. @@ -139,7 +139,7 @@ To connect SnappyData from SQuirreL SQL Client, do the following: * website URL 3. Add the downloaded **snappydata jdbc jar** in the extra classpath tab and provide the class name to be used for the connection.
``` -jdbc jar: https://mvnrepository.com/artifact/io.snappydata/snappydata-jdbc_2.11/1.1.0 +jdbc jar: https://mvnrepository.com/artifact/io.snappydata/snappydata-jdbc_2.11/1.1.1 jdbc class: io.snappydata.jdbc.ClientPoolDriver ``` 4. Go to **Aliases** tab and then click **+** to add a new alias.
![Images](../Images/sql_clienttools_images/squirrel2.png) @@ -167,5 +167,7 @@ jdbc jar: https://mvnrepository.com/artifact/io.snappydata/snappydata-jdbc_2.11/ show tables; +!!!Note + When connecting to SnappyData, if a SQL client tool sets JDBC autocommit to false and transaction isolation level such as read committed or repeatable read is used, the unsupported operations such as those on column table will produce an error - **Operations on column tables are not supported when query routing is disabled or autocommit is false.** In such cases, connection property **allow-explicit-commit=true** can be used in the connection URL to avoid this error. Refer to configuration parameters section for details on this property. For example, JDBC URL: **jdbc:snappydata://locatoHostName:1527/allow-explicit-commit=true** diff --git a/docs/howto/connect_using_jdbc_driver.md b/docs/howto/connect_using_jdbc_driver.md index b87a90446f..49c94d1dff 100644 --- a/docs/howto/connect_using_jdbc_driver.md +++ b/docs/howto/connect_using_jdbc_driver.md @@ -21,14 +21,14 @@ You can use the Maven or the SBT dependencies to get the latest released version io.snappydata snappydata-jdbc_2.11 - 1.1.0 + 1.1.1 ``` **Example: SBT dependency** ```pre // https://mvnrepository.com/artifact/io.snappydata/snappydata-store-client -libraryDependencies += "io.snappydata" % "snappydata-jdbc_2.11" % "1.1.0" +libraryDependencies += "io.snappydata" % "snappydata-jdbc_2.11" % "1.1.1" ``` !!! Note @@ -86,67 +86,3 @@ preparedStmt1.close() !!! Note If the tool does not automatically select a driver class, you may have the option of selecting a class from within the JAR file. In this case, select the **io.snappydata.jdbc.ClientDriver** class. - - - -## Connecting with JDBC Client Pool Driver - -JDBC client pool driver provides built-in connection pooling and relies on the non-pooled [JDBC driver](/howto/connect_using_jdbc_driver.md). The driver initializes the pool when the first connection is created using this driver. Thereafter, for every request, the connection is returned from the pool instead of establishing a new connection with the server. -We recommend using the pooled driver for low latency operations such as point lookups and when using the Spark JDBC data source API (see example below). When you access SnappyData from Java frameworks such as Spring, we recommend using pooling provided in the framework and switch to using the non-pooled driver. - -!!! Important - The underlying pool is uniquely associated with the set of properties that are passed while creating the connection. If any of the properties change, a new pool is created. - -**To connect to SnappyData Cluster using JDBC client pool driver**, use the url of the form:
`jdbc:snappydata:pool://:`
-Where `` is the hostname of the node on which the locator is started and `` is the port on which the locator accepts client connections (default 1527). - -The client pool driver class name is **io.snappydata.jdbc.ClientPoolDriver**. - -The following pool related properties can be used to tune the JDBC client pool driver: - -| Property | Description | -|----------|-------------| -|user|The username to be passed to the JDBC client pool driver to establish a connection. | -|password|The password to be passed to the JDBC client pool driver to establish a connection.| -|pool.initialSize|The initial number of connections that are created when the pool is started. Default value is `max(256, availableProcessors * 8)`.| -|pool.maxActive| The maximum number of active connections that can be allocated from this pool at a time. The default value is `max(256, availableProcessors * 8)`. | -|pool.minIdle| The minimum number of established connections that should be maintained in the client pool. Default value is **1**.| -|pool.maxIdle| The maximum number of connections that should be maintained in the client pool. Default value is **maxActive:**`max(256, availableProcessors * 8)`. Idle connections are checked periodically, if enabled, and the connections that are idle for more than the time set in **minEvictableIdleTimeMillis** are released.| -|pool.maxWait|(int) The maximum waiting period, in milliseconds, for establishing a connection after which an exception is thrown. Default value is 30000 (30 seconds).| -|pool.removeAbandoned| Flag to remove the abandoned connections, in case they exceed the settings for **removeAbandonedTimeout**. If set to true a connection is considered abandoned and eligible for removal, if its no longer in use than the settings for **removeAbandonedTimeout**. Setting this to **true** can recover db connections from applications that fail to close a connection. The default value is **false**.| -|pool.removeAbandonedTimeout| Timeout in seconds before an abandoned connection, that was in use, can be removed. The default value is 60 seconds. The value should be set to the time required for the longest running query in your applications.| -|pool.timeBetweenEvictionRunsMillis| Time period required to sleep between runs of the idle connection validation/cleaner thread. You should always set this value above one second. This time period determines how often we check for idle and abandoned connections and how often to validate the idle connections. The default value is 5000 (5 seconds).| -|pool.minEvictableIdleTimeMillis|The minimum time period, in milliseconds, for which an object can be idle in the pool before it qualifies for eviction. The default value is 60000 (60 seconds).| -|driver|`io.snappydata.jdbc.ClientPoolDriver`
This should be passed through Spark JDBC API for loading and using the driver.| -|pool.testOnBorrow|Indicates if the objects are validated before being borrowed from the pool. If the object fails to validate, it will be dropped from the pool, and will attempt to borrow another. In order to have a more efficient validation, see `pool.validationInterval`. Default value is **true**.| -|pool.validationInterval|Avoid excess validation, only run validation at most at this frequency - time in milliseconds. If a connection is due for validation, but has been validated previously within this interval, it will not be validated again. The default value is 10000 (10 seconds).| - -**Example Code Snippet:** - -```pre -val properties = new Properties() -properties.setProperty("user", "user") -properties.setProperty("password", "pass") -properties.setProperty("driver", ““io.snappydata.jdbc.ClientPoolDriver””) - -val builder = SparkSession -.builder. -appName("app") -.master("local[*]") - -val spark: SparkSession = builder.getOrCreate - -val df = spark.read.jdbc(“jdbc:snappydata:pool://localhost:1527”, "Table_X", properties) - -``` - -## Limitations - -If you set any of the following properties for a pooled connection, it gets auto-reset to the default values whenever you obtain a new pooled connection. - -* **setAutoCommit** -* **setTransactionIsolation** -* **setReadOnly** - -However, if you have set any of the other properties (e.g. spark or snappy AQP related properties), it does not get auto-reset when you obtain a new pooled connection. - diff --git a/docs/howto/connect_using_odbc_driver.md b/docs/howto/connect_using_odbc_driver.md index a00e9afa68..ee4ef970be 100644 --- a/docs/howto/connect_using_odbc_driver.md +++ b/docs/howto/connect_using_odbc_driver.md @@ -3,10 +3,10 @@ This feature is available only in the Enterprise version of SnappyData.
-You can connect to SnappyData Cluster using TIBCO ComputeDB ODBC Driver and can execute SQL queries by connecting to any of the servers in the cluster. +You can connect to TIBCO ComputeDB Cluster using TIBCO ComputeDB ODBC Driver and can execute SQL queries by connecting to any of the servers in the cluster. -## Step 1: Install Visual C++ Redistributable for Visual Studio 2013 +## Step 1: Installing Visual C++ Redistributable for Visual Studio 2013 To download and install the Visual C++ Redistributable for Visual Studio 2013: @@ -15,32 +15,42 @@ To download and install the Visual C++ Redistributable for Visual Studio 2013: 2. Select **Run** to start the installation and follow the steps to complete the installation. -## Step 2: Install TIBCO ComputeDB ODBC Driver +## Step 2: Installing TIBCO ComputeDB ODBC Driver To download and install the ODBC driver: -1. [Download the TIBCO ComputeDB 1.1.0 Enterprise Version](http://www.snappydata.io/download) by registering on the SnappyData website. The downloaded file contains the TIBCO ComputeDB ODBC driver installers. +1. [Download the TIBCO ComputeDB 1.1.1 Enterprise Version](https://edelivery.tibco.com/storefront/index.ep). The downloaded file contains the TIBCO ComputeDB ODBC driver installers. 2. Depending on your Windows installation, extract the contents of the 32-bit or 64-bit version of the TIBCO ComputeDB ODBC Driver. | Version | ODBC Driver | |--------|--------| - |32-bit for 32-bit platform|TIB_compute-odbc_1.1.0_win_x86_32bit.zip| - |32-bit for 64-bit platform|TIB_compute-odbc_1.1.0_win_x86_64bit.zip| - |64-bit for 64-bit platform|TIB_compute-odbc_1.1.0_win_x64_64bit.zip| + |32-bit for 32-bit platform|TIB_compute-odbc_1.1.1_win_x86_32bit.zip| + |64-bit for 64-bit platform|TIB_compute-odbc_1.1.1_win_x64_64bit.zip| -4. Double-click on the extracted **TIB_compute-odbc_1.1.0_win.msi** file, and follow the steps to complete the installation. +4. Double-click on the extracted **TIB_compute-odbc_1.1.1_win.msi** file, and follow the steps to complete the installation. !!! Note - Ensure that [SnappyData is installed](../install.md) and the [SnappyData cluster is running](start_snappy_cluster.md). + Ensure that [TIBCO ComputeDB is installed](../install.md) and the [TIBCO ComputeDB cluster is running](start_snappy_cluster.md). -## Connect to the SnappyData Cluster -Once you have installed the TIBCO ComputeDB ODBC Driver, you can connect to SnappyData cluster in any of the following ways: +## Connecting to the TIBCO ComputeDB Cluster +Once you have installed the TIBCO ComputeDB ODBC Driver, you can connect to TIBCO ComputeDB cluster in any of the following ways: -* Use the TIBCO ComputeDB Driver Connection URL: +* Use the TIBCO ComputeDB Driver Connection URL: - Driver=SnappyData ODBC Driver;server=;port=;user=;password= + Driver=TIBCO ComputeDB ODBC Driver;server=;port=;user=;password= + + In case you want to connect with a specific server: + + Driver=TIBCO ComputeDB ODBC Driver;server=;port=;user=;password=;load-balance=false -* Create a TIBCO ComputeDB DSN (Data Source Name) using the installed TIBCO ComputeDB ODBC Driver. Refer to the Windows documentation relevant to your operating system for more information on creating a DSN.
-When prompted, select the TIBCO ComputeDB ODBC Driver from the list of drivers and enter a Data Source name, SnappyData Server Host, Port, User Name and Password. +* Create a TIBCO ComputeDB DSN (Data Source Name) using the installed TIBCO ComputeDB ODBC Driver. Refer to the Windows documentation relevant to your operating system for more information on creating a DSN.
+When prompted, select the TIBCO ComputeDB ODBC Driver from the list of drivers and enter a Data Source name. You can then enter either TIBCO ComputeDB Server Host, Port, User Name, and Password or TIBCO ComputeDB Locator Host, Port, User Name and Password. Refer to the documentation for detailed information on [Setting Up TIBCO ComputeDB ODBC Driver](../setting_up_odbc_driver-tableau_desktop.md). + +## Connecting Spotfire® Desktop to TIBCO ComputeDB +Refer [TIBCO Spotfire® Connectivity to TIBCO ComputeDB™](https://community.tibco.com/wiki/tibco-spotfire-connectivity-tibco-computedb) for detailed instructions to access TIBCO ComputeDB using this connector. + +Also see: + +[ODBC Supported APIs in TIBCO ComputeDB Driver](/reference/API_Reference/odbc_supported_apis.md) diff --git a/docs/howto/connecttibcospotfire.md b/docs/howto/connecttibcospotfire.md index 0e5d931d2f..a9975ebf01 100644 --- a/docs/howto/connecttibcospotfire.md +++ b/docs/howto/connecttibcospotfire.md @@ -3,14 +3,29 @@ TIBCO Spotfire® Desktop allows users to easily author and update ad-hoc analytics, applications, and dashboards. To connect TIBCO Spotfire® Desktop to TIBCO ComputeDB, setup and launch the TIBCO ComputeDB cluster. -!!! Note - The Spark SQL connector based on the Simba ODBC driver in Spotfire has some performance issues. We noticed an overhead of about 5 seconds over the query time in TIBCO ComputeDB. You can see the query timings in the ComputeDB dashboard. We expect this issue to be resolved before full GA of this feature. +You can use any of the following methods to connect TIBCO Spotfire® Desktop to TIBCO ComputeDB: -### Experimental (but Faster) Spotfire Connector (based on TIBCO ComputeDB Native ODBC) -An experimental but much faster connnector is also available from TIBCO Engineering and accessible from [here](https://community.tibco.com/wiki/tibco-spotfire-connectivity-tibco-computedb). +* [Native Spotfire® Connector (Recommended)](#nativespotfireconnector) +* [Builtin CDB connector in Spotfire®](#builtinspotfireconnector) + + + +## CDB Connector using Native ODBC Driver +TIBCO recommends to use the native Spotfire Connector to connect TIBCO Spotfire® Desktop to TIBCO ComputeDB. This connector is based on native ODBC and is a faster connnector. Refer [TIBCO Spotfire® Connectivity to TIBCO ComputeDB™](https://community.tibco.com/wiki/tibco-spotfire-connectivity-tibco-computedb) for detailed instructions to access TIBCO ComputeDB using this connector. + + +## CDB Connector using Simba ODBC Driver + +Users of Spotfire® 10.4 or later can use the builtin CDB connector in Spotfire® to connect TIBCO Spotfire® Desktop to TIBCO ComputeDB. Refer to [Accessing Data from TIBCO ComputeDB](https://docs.tibco.com/pub/sfire-analyst/10.4.0/doc/html/en-US/TIB_sfire-analyst_UsersGuide/connectors/computedb/computedb_accessing_data.htm) for detailed instructions to access TIBCO ComputeDB using this connector. + + -## Connecting Spotfire® Desktop to TIBCO ComputeDB +### Connecting Spotfire® Desktop to TIBCO ComputeDB using Apache Spark SQL Login and download TIBCO Spotfire® Desktop from the [TIBCO eDelivery website]( https://edelivery.tibco.com/storefront/eval/tibco-spotfire-desktop/prod10954.html). Follow the setup guidelines to install the product. After installation, you can use the following steps to connect Spotfire® Desktop to TIBCO ComputeDB. @@ -44,3 +59,4 @@ Login and download TIBCO Spotfire® Desktop from the [TIBCO eDelivery website]( 5. Go to **General** tab and then click the **Connect** button. 6. From the **Database** dropdown, either choose the existing database (schema) **app** or **default** or you can choose a database (schema) that is created in the TIBCO ComputeDB cluster.
![images](../Images/spotfire/generaltabspotfire1.png) 7. Click **OK**. The **View In Connection (VIC)** box is displayed which lists the tables from the selected database. +---> \ No newline at end of file diff --git a/docs/howto/export_hdfs.md b/docs/howto/export_hdfs.md index 74d9c13a58..c9663c5bb4 100644 --- a/docs/howto/export_hdfs.md +++ b/docs/howto/export_hdfs.md @@ -34,4 +34,5 @@ val df2 = snappySession.read.parquet("hdfs://127.0.0.1:9000/customer") df2.write.mode(SaveMode.Append).saveAsTable("APP.CUSTOMER") ``` - +!!!Note + Snappydata supports kerberized Hadoop cluster in Smart connector mode only. You need to set **HADOOP_CONF_DIR** in **spark-env.sh** and **snappy-env.sh**. Currently the Embedded mode(Snappy job and Snappy shell) and Smart Connector with standalone mode are NOT supported. Smart connector with local and YARN mode are supported. diff --git a/docs/howto/import_from_hive_table.md b/docs/howto/import_from_hive_table.md index e3dfc9adcf..b669868859 100644 --- a/docs/howto/import_from_hive_table.md +++ b/docs/howto/import_from_hive_table.md @@ -1,5 +1,94 @@ # How to Import Data from Hive Table into SnappyData Table +Using a Snappy session, you can read an existing Hive table that is defined in an external hive catalog, use hive tables as external tables from SnappySession for queries, including joins with tables defined in SnappyData catalog, and also define new Hive table/view to be stored in external hive catalog. + +When working with Hive, one must instantiate Snappy session with Hive support, including connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. + +If the underlying storage for Hive is HDFS, you can configure Hive with Snappy session. For this, you must place **hive-site.xml**, **core-site.xml** (for security configuration) and **hdfs-site.xml**(for HDFS configuration) files in the **conf/** folder of Snappy. In addition to this, you must configure **spark-env.sh** file into the **conf/** folder. + +The content in the **hadoop_spark-env.sh** file should be as follows: +``` +export SPARK_DIST_CLASSPATH=$(/home/user/hadoop-2.7.3/bin/hadoop classpath) +``` + +Snappy has been tested with default hive database i.e. embedded derby database. User can also use and configure the remote metastore as well like SQL. + +In **hive-site** xml, user needs to configure the parameters as per the requirement. With derby as database, the following are the ** hive-site.xml** configuration: + +``` + + javax.jdo.option.ConnectionURL + jdbc:derby:;databaseName=metastore_db;create=true + + JDBC connect string for a JDBC metastore. + To use SSL to encrypt/authenticate the connection, provide database-specific SSL flag in the connection URL. + For example, jdbc:postgresql://myhost/db?ssl=true for postgres database. + + +``` + +If you want to setup remote meta store instead of using default database derby, you can use the following configuration: + +``` + + hive.metastore.uris + thrift://chbhatt-dell:9083 + Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore. + +``` + +Run the following steps to test Snappy with Apache Hadoop: + +1. Start the Hadoop daemons. +2. Start the Hive thrift server. +3. Start the Snappy-shell. +4. After starting the Snappy Shell, you can do the following: + + # To point to external hive catalog from snappy session, set the below property. + set spark.sql.catalogImplementation=hive. + snappy-sql> set spark.sql.catalogImplementation=hive; + This property can be set at the session level and global level. + + # To point to Snappy internal catalog from snappy session. + set spark.sql.catalogImplementation=in-memory. + snappy-sql> set spark.sql.catalogImplementation=in-memory; + + # To access hive tables use below command. + snappy-sql> show tables in default; + Please note that it is mandatory to specify the schema ‘default’. + If any other schema is created then it is mandatory to use the created schema name. + + For example, if schema / database hiveDB created then use, + snappy-sql> show tables in hiveDB; + + # To read the hive tables from snappy. + snappy-sql> SELECT FirstName, LastName FROM default.hiveemployees ORDER BY LastName; + + # To join Snappy tables and Hive tables. + snappy-sql> SELECT emp.EmployeeID, emp.FirstName, emp.LastName, o.OrderID, o.OrderDate FROM default.hive_employees emp JOIN snappy_orders o ON (emp.EmployeeID = o.EmployeeID) ORDER BY o.OrderDate; + + # To create the hive table and insert the data into it from Snappy. + snappy-sql> create table if not exists default.t1(id int) row format delimited fields terminated by ','; + snappy-sql> insert into default.t1 select id, concat(id) from range(100); + + +!!! Note + If you have not configure any of the configuration files mentioned above( hive-site.xml, core-site.xml, hdfs-site.xml) and started the Hadoop and Hive daemons, you will see the following error: + +``` +No Datastore found in the Distributed System for 'execution on remote node null'. +``` + +If you have connected to Hive and Hadoop and in case the configuration files get removed or deleted, errors or exceptions will not be shown. However, you cannot perform any DDL and DML statements in Hive. + + +For more details, refer the following links: + +* [https://spark.apache.org/docs/latest/sql-data-sources-hive-tables.html](https://spark.apache.org/docs/latest/sql-data-sources-hive-tables.html) + +* [https://cwiki.apache.org/confluence/display/Hive/Hive+on+Spark%3A+Getting+Started](https://cwiki.apache.org/confluence/display/Hive/Hive+on+Spark%3A+Getting+Started) + + \ No newline at end of file diff --git a/docs/howto/run_spark_job_inside_cluster.md b/docs/howto/run_spark_job_inside_cluster.md index fb1562e342..7c6762dde0 100644 --- a/docs/howto/run_spark_job_inside_cluster.md +++ b/docs/howto/run_spark_job_inside_cluster.md @@ -29,7 +29,7 @@ To compile your job, use the Maven/SBT dependencies for the latest released vers io.snappydata snappydata-cluster_2.11 - 1.1.0 + 1.1.1 ``` @@ -37,7 +37,7 @@ To compile your job, use the Maven/SBT dependencies for the latest released vers ```pre // https://mvnrepository.com/artifact/io.snappydata/snappydata-cluster_2.11 -libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.0" +libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1" ``` !!! Note diff --git a/docs/howto/spark_installation_using_smart_connector.md b/docs/howto/spark_installation_using_smart_connector.md index 2cd944e47d..9e4fe8eead 100644 --- a/docs/howto/spark_installation_using_smart_connector.md +++ b/docs/howto/spark_installation_using_smart_connector.md @@ -51,7 +51,7 @@ Start a SnappyData cluster and create a table. $ ./sbin/snappy-start-all.sh $ ./bin/snappy -SnappyData version 1.1.0 +SnappyData version 1.1.1 snappy> connect client 'localhost:1527'; Using CONNECTION0 snappy> CREATE TABLE SNAPPY_COL_TABLE(r1 Integer, r2 Integer) USING COLUMN; @@ -67,7 +67,7 @@ The Smart Connector Application can now connect to this SnappyData cluster.
SnappyData package has to be specified along with the application jar to run the Smart Connector application. ```pre -$ ./bin/spark-submit --master local[*] --conf snappydata.connection=localhost:1527 --class org.apache.spark.examples.snappydata.SmartConnectorExample --packages SnappyDataInc:snappydata:1.1.0-s_2.11 $SNAPPY_HOME/examples/jars/quickstart.jar +$ ./bin/spark-submit --master local[*] --conf snappydata.connection=localhost:1527 --class org.apache.spark.examples.snappydata.SmartConnectorExample --packages SnappyDataInc:snappydata:1.1.1-s_2.11 $SNAPPY_HOME/examples/jars/quickstart.jar ``` ## Execute a Smart Connector Application @@ -77,7 +77,7 @@ Start a SnappyData cluster and create a table inside it. $ ./sbin/snappy-start-all.sh $ ./bin/snappy -SnappyData version 1.1.0 +SnappyData version 1.1.1 snappy> connect client 'localhost:1527'; Using CONNECTION0 snappy> CREATE TABLE SNAPPY_COL_TABLE(r1 Integer, r2 Integer) USING COLUMN; @@ -91,5 +91,5 @@ exit; A Smart Connector Application can now connect to this SnappyData cluster. The following command executes an example that queries SNAPPY_COL_TABLE and creates a new table inside SnappyData cluster. SnappyData package has to be specified along with the application jar to run the Smart Connector application. ```pre -$ ./bin/spark-submit --master local[*] --conf spark.snappydata.connection=localhost:1527 --class org.apache.spark.examples.snappydata.SmartConnectorExample --packages SnappyDataInc:snappydata:1.1.0-s_2.11 $SNAPPY_HOME/examples/jars/quickstart.jar +$ ./bin/spark-submit --master local[*] --conf spark.snappydata.connection=localhost:1527 --class org.apache.spark.examples.snappydata.SmartConnectorExample --packages SnappyDataInc:snappydata:1.1.1-s_2.11 $SNAPPY_HOME/examples/jars/quickstart.jar ``` diff --git a/docs/howto/start_snappy_cluster.md b/docs/howto/start_snappy_cluster.md index 466744a300..27d612cbf6 100644 --- a/docs/howto/start_snappy_cluster.md +++ b/docs/howto/start_snappy_cluster.md @@ -1,5 +1,6 @@ # How to Start a SnappyData Cluster + ## Starting SnappyData Cluster on a Single Machine If you have [downloaded and extracted](../install.md) the SnappyData product distribution, navigate to the SnappyData product root directory. @@ -15,15 +16,15 @@ It may take 30 seconds or more to bootstrap the entire cluster on your local mac **Sample Output**: The sample output for `snappy-start-all.sh` is displayed as: ```pre -Logs generated in /home/cbhatt/TIB_compute_1.1.0_linux/work/localhost-locator-1/snappylocator.log +Logs generated in /home/cbhatt/snappydata-1.1.1-bin/work/localhost-locator-1/snappylocator.log SnappyData Locator pid: 10813 status: running Distributed system now has 1 members. Started Thrift locator (Compact Protocol) on: localhost/127.0.0.1[1527] -Logs generated in /home/cbhatt/TIB_compute_1.1.0_linux/work/localhost-server-1/snappyserver.log +Logs generated in /home/cbhatt/snappydata-1.1.1-bin/work/localhost-server-1/snappyserver.log SnappyData Server pid: 11018 status: running Distributed system now has 2 members. Started Thrift server (Compact Protocol) on: localhost/127.0.0.1[1528] -Logs generated in /home/cbhatt/TIB_compute_1.1.0_linux/work/localhost-lead-1/snappyleader.log +Logs generated in /home/cbhatt/snappydata-1.1.1-bin/work/localhost-lead-1/snappyleader.log SnappyData Leader pid: 11213 status: running Distributed system now has 3 members. Starting hive thrift server (session=snappy) @@ -60,3 +61,48 @@ $ ./bin/snappy leader start -dir=/node-c/lead1 -locators=localhost[10334] -spa ``` !!!Note The path mentioned for `-dir` should exist. Otherwise, the command will fail with **FileNotFoundException**. + +## Executing Commands on Selected Cluster Component + +### Syntax + +``` +cluster-util.sh (--on-locators|--on-servers|--on-leads|--on-all) [-y] (--copy-conf | --run "") +``` + +### Description + +You can use the cluster-util.sh utility to execute a given command on selected members of the cluster. The script relies on the entries you specify in locators, servers, and leads files in the **conf** directory to identify the members of the cluster. + + * `-on-locators`
+ If specified, the given command is executed on locators. + + * `--on-servers `
+ If specified, the given command is executed on servers. + +* `--on-leads`
+ If specified, the given command is executed on leads. + +* `--on-all `
+ If specified, the given command is executed on all the member in the cluster. + +* `-y`
+ If specified, the script does not prompt for the confirmation to execute the command on each member node. + + * `--copy-conf`
+ This is a shortcut command. When you specify this comand, the log4j.properties, snappy-env.sh and spark-env.sh configuration files are copied from the local machine to all the members. These files are copied only in the following conditions: + * If these are absent in the destination member + * If their content is different. + In latter case, a backup of the file is taken in **conf/backup** directory, on the destination member, before copy. + +* `--run ` + If specified, the given command(s) is executed on specified members. Command to be executed specified after --run`` must be in double-quotes. +### Example +``` +// To copy configuration files on all servers + +./sbin/cluster-util.sh -on-servers --run -copyconf” + 2. To run “ls” command on all servers with -y option + “./sbin/cluster-util.sh -on-servers -y --run ls + +``` diff --git a/docs/howto/tableauconnect.md b/docs/howto/tableauconnect.md index 73e1203aa8..2273b7e2ec 100644 --- a/docs/howto/tableauconnect.md +++ b/docs/howto/tableauconnect.md @@ -63,7 +63,7 @@ To install Tableau desktop: ### Step 3: Connect Tableau Desktop to SnappyData Server -When using Tableau with the SnappyData ODBC Driver for the first time, you must add the **odbc-snappydata.tdc** file that is available in the downloaded **TIB_compute-odbc_1.1.0_win.zip**. +When using Tableau with the SnappyData ODBC Driver for the first time, you must add the **odbc-snappydata.tdc** file that is available in the downloaded **TIB_compute-odbc_1.1.1_win.zip**. To connect the Tableau Desktop to the SnappyData Server: diff --git a/docs/howto/use_stream_processing_with_snappydata.md b/docs/howto/use_stream_processing_with_snappydata.md index 92cc7f407c..41231df385 100644 --- a/docs/howto/use_stream_processing_with_snappydata.md +++ b/docs/howto/use_stream_processing_with_snappydata.md @@ -124,17 +124,13 @@ snsc.snappySession.sql("select publisher, bidCount from publisher_bid_counts").s ## Structured Streaming -The SnappyData structured streaming programming model is the same as [Spark structured streaming](https://spark.apache.org/docs/2.1.1/structured-streaming-programming-guide.html). +The SnappyData structured streaming programming model is the same as [Spark structured streaming](https://spark.apache.org/docs/2.1.1/structured-streaming-programming-guide.html). The only difference is support for ingesting streaming dataframes into SnappyData tables through a built-in **Sink**. -The only difference is support for ingesting streaming dataframes into SnappyData tables through a built-in **Sink**. +SnappyData provides a build-in output **Sink** which simplifies ingestion of streaming dataframes into SnappyData tables. The **Sink** supports idempotent writes, ensuring consistency of data when failures occur, as well as support for all mutation operations such as inserts, appends, updates, puts, and deletes. -SnappyData provides a build-in output **Sink** which simplifies ingestion of streaming dataframes into SnappyData tables. The **Sink** supports idempotent writes, ensuring consistency of data when failures occur, as well as support for all mutation operations such as inserts, appends, updates, puts, and deletes. +The output data source name for SnappyData is `snappysink`. A minimal code example for structured streaming with socket source and **Snappy Sink** is available [here](https://github.com/SnappyDataInc/snappydata/blob/master/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/SocketSourceExampleWithSnappySink.scala). -The output data source name for SnappyData is `snappysink`. - -### Code Sample - -A minimal code example for structured streaming with snappysink is available [here](https://github.com/SnappyDataInc/snappydata/blob/master/examples/src/main/scala/org/apache/spark/examples/snappydata/StructuredStreamingWithSnappySink.scala). +You can also refer to [Structured Streaming Quickstart guide](/quickstart/structucture_streamingquickstart.md). For more examples, refer to [structured streaming examples](https://github.com/SnappyDataInc/snappydata/blob/master/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming/CDCExample.scala) ### Using SnappyData Structured Streaming API @@ -223,7 +219,7 @@ By default the `conflation` property is set to `false`. Therefore, the event pro In such cases, you should enable the Conflation by setting the **conflation** property to true. Now, if a batch contains **Insert(key1)** event followed by a Dele**te(key1)** event, then SnappyData Sink conflates these two events into a single event by selecting the last event which is **Delete(key1)** and only that event is processed for **key1**. Processing **Delete(key1)** event without processing **Insert(key1)** event does not result in a failure, as Delete events are ignored if corresponding records do not exist in the target table. -## Sink State Table +### Sink State Table A replicated row table with name **snappysys_internal____sink_state_table** is created by **Snappy Sink** under schema specified by the **stateTableSchema** option if the table does not exist. If the **stateTableSchema** is not specified then the sink state table is created under the **APP** schema. During the processing of each batch, this state is updated. @@ -234,11 +230,11 @@ This table is used by **Snappy Sink** to maintain the state of the streaming que | stream_query_id | varchar(200) |Primary Key. Name of the streaming query| |batch_id|long|Batch id of the most recent batch picked up for processing.| -### Behavior of Sink State Table in a Secure cluster +#### Behavior of Sink State Table in a Secure cluster When security is enabled for the cluster, the **stateTableSchema** becomes a mandatory option. Also, when you submit the streaming job, you must have the necessary permissions on the schema specified by **stateTableSchema** option. -### Maintaining Idempotency In Case Of Stream Failures +#### Maintaining Idempotency In Case Of Stream Failures When stream execution fails, it is possible that streaming batch was half processed. Hence next time whenever the stream is started, Spark picks the half processed batch again for processing. This can lead to extraneous records in the target table if the batch contains insert events. To overcome this, Snappy Sink keeps the state of a stream query execution as part of the Sink State table. @@ -250,7 +246,7 @@ Using this state, Snappy Sink can detect whether a batch is a duplicate batch. I !!! Note The above-mentioned behavior is applicable only when the key columns are defined on the target table as key columns are necessary to apply **put into** operation. When key columns are not defined on the target table, Snappy Sink does not behave in an idempotent manner and it can lead to duplicate records in the target table when the streaming query is restarted after stream failure. -## Overriding Default Sink Behavior +### Overriding Default Sink Behavior If required, applications can override the default **Snappy Sink** semantics by implementing **org.apache.spark.sql.streaming.SnappySinkCallback** and passing the fully qualified name of the implementing class as a value of **sinkCallback** option of **Snappy Sink**. @@ -263,7 +259,7 @@ def process(snappySession: SnappySession, sinkProps: Map[String, String], batchId: Long, df: Dataset[Row], possibleDuplicate: Boolean = false): Unit ``` -## Limitations +### Limitations Limitations of **Snappy Sink** are as follows: * When the data coming from the source is not partitioned by key columns, then using **Snappy Sink** may result in inconsistent data. This is because each partition independently processes the data using the [above-mentioned logic](#event_order). @@ -272,3 +268,21 @@ Limitations of **Snappy Sink** are as follows: * The default **Snappy Sink** implementation does not support partial records for updates. Which means that there is no support to merge updates on a few columns into the store. For all update events, the incoming records must provide values into all the columns of the target table. +### Resetting a Streaming Query + +Progress of a streaming query is saved as part of the checkpoint directory by Spark. On top of this **Snappy Sink** also maintains an internal state as part of the state table to ensure idempotency of the sink. + + +Hence to reset a streaming query, the following actions must be taken to clean the state of the streaming query: + + +!!! Note + When you use the following steps you may permanently lose the state of the streaming query. + + +1. Delete the checkpoint directory. (or start streaming query with different checkpoint directory.) +2. Clear the state from the state table using following sql:
+ delete from [state_table_schema].snappysys_internal____sink_state_table where stream_query_id = ; + + * [state_table_schema] is the schema passed as part of “stateTableSchema” option of snappy sink. It should be skipped if “stateTableSchema” option was not provided while defining snappy sink. + * `` is the name of the query provided while defining the sink. diff --git a/docs/howto/using_snappydata_for_any_spark_dist.md b/docs/howto/using_snappydata_for_any_spark_dist.md index 33ce4fd56a..8ff338d428 100644 --- a/docs/howto/using_snappydata_for_any_spark_dist.md +++ b/docs/howto/using_snappydata_for_any_spark_dist.md @@ -6,11 +6,11 @@ Following is a sample of Spark JDBC extension setup and usage: 1. Include the **TIB_compute-jdbc** package in the Spark job with spark-submit or spark-shell: - $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.0.jar + $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.1.jar 2. Set the session properties.
The SnappyData connection properties (to enable auto-configuration of JDBC URL) and credentials can be provided in Spark configuration itself, or set later in SparkSession to avoid passing them in all the method calls. These properties can also be provided in **spark-defaults.conf ** along with all the other Spark properties.
Following is a sample code of configuring the properties in **SparkConf**: - $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.0.jar --conf spark.snappydata.connection=localhost:1527 --conf spark.snappydata.user= --conf spark.snappydata.password= + $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.1.jar --conf spark.snappydata.connection=localhost:1527 --conf spark.snappydata.user= --conf spark.snappydata.password= Overloads of the above methods accepting *user+password* and *host+port* is also provided in case those properties are not set in the session or needs to be overridden. You can optionally pass additional connection properties similarly as in the **DataFrameReader.jdbc** method. @@ -38,4 +38,4 @@ Following is a sample of Spark JDBC extension setup and usage: exec.snappyExecute(“create table testTable1 (id long, data string) using column”); exec.snappyExecute("insert into testTable1 values (1, ‘data1’)"); DataFrame df = exec.snappyQuery(...); - ... \ No newline at end of file + ... diff --git a/docs/index.md b/docs/index.md index 646d0c436b..396e7d90c7 100644 --- a/docs/index.md +++ b/docs/index.md @@ -1,94 +1,164 @@ -## Introduction -SnappyData™ fuses Apache Spark with an in-memory database to deliver a compute+data engine capable of stream processing, transactions, interactive analytics and prediction in a single cluster. +## Introduction +SnappyData (aka TIBCO ComputeDB community edition) is a distributed, in-memory optimized analytics database. SnappyData delivers high throughput, low latency, and high concurrency for unified analytics workload. By fusing an in-memory hybrid database inside Apache Spark, it provides analytic query processing, mutability/transactions, access to virtually all big data sources and stream processing all in one unified cluster. -!!! Attention - This document assumes that you have familiarity with Apache Spark and its concepts. If you are new to Apache Spark, refer to the [Apache Spark documentation](https://spark.apache.org/docs/2.1.1/) to learn more about using Apache Spark.
-The current release of SnappyData is fully compatible with Spark 2.1.1 +The primary use case of SnappyData is to provide analytics at interactive speeds over large volumes of data with minimal or no pre-processing of the dataset. For instance, there is no need to often pre-aggregate/reduce or generate cubes over your large data sets for ad-hoc visual analytics. This is made possible by smartly managing data in-memory, dynamically generating code using vectorization optimizations and maximizing the potential of modern multi-core CPUs. +SnappyData enables complex processing on large data sets in sub-second timeframes. -## The Challenge with Apache Spark and Remote Data Sources -Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and can access disparate data sources in a highly parallelized manner for its distributed computations. Typically, data is fetched lazily as a result of SQL query or a Dataset (RDD) getting materialized. This can be quite inefficient and expensive since most workloads require the data set to be repeatedly processed. +![SnappyData Positioning](./Images/Snappy_intro.1.png) -Analytic processing requires large datasets to be repeatedly copied from an external data source like HDFS, into Apache Spark. Copying data, reformatting it (into a columnar format, depending on where the data is being copied from) and moving it across process and machine boundaries can be very expensive. As a result, we see that in several cases, applications using Apache Spark with an external data source fail to deliver the promise of interactive analytic performance. For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Apache Spark to do the aggregation. The alternative to working with a stateful store is to cache the data in Apache Spark. This, of course, suffers from the problems associated with stale data. +!!!Note + SnappyData is not another Enterprise Data Warehouse (EDW) platform, but rather a nimble computational cluster that augments traditional EDWs and data lakes. -## The SnappyData Approach -SnappyData fuses a low latency, highly available in-memory transactional database (Pivotal GemFire/Apache Geode) into Apache Spark with shared memory management and several optimizations that deliver performance and concurrency for mixed workloads. Data in the highly available in-memory store is laid out using a custom columnar format somewhat similar to the layout used by Apache Spark caching. Query engine operators are optimized through better vectorization and code generation. The net effect of these changes is, an order of magnitude performance improvement when compared to native Apache Spark caching, and more than two orders of magnitude better Apache Spark performance when working with external data sources. +### Important Capabilities -Essentially, Apache Spark is turned into an in-memory operational database capable of transactions, point reads, writes, working with streams and running analytic SQL queries. + -![SnappyData Architecture](Images/SnappyArchitecture.png) +* **Easily discover and catalog big data sets** + You can connect and discover datasets in SQL DBs, Hadoop, NoSQL stores, file systems, or even cloud data stores such as S3 by using SQL, infer schemas automatically and register them in a secure catalog. A wide variety of data formats are supported out of the box such as JSON, CSV, text, Objects, Parquet, ORC, SQL, XML, and more. -SnappyData is an in-memory database that runs Apache Spark’s compute engine directly in the database, and offers **Apache Spark's API and SQL as its interface and computational engine**. The fusion with Apache Spark allows SnappyData to work with a large number of data sources like HDFS, NoSQL etc. through bespoke Apache Spark connectors.
-While the SnappyData engine (that builds on Apache Spark Catalyst SQL engine) is primarily designed for SQL processing, applications can also work with Objects through Apache Spark RDDs and the Apache Spark Datasets API. +* **Rich connectivity** + SnappyData is built with Apache Spark inside. Therefore, any data store that has a Spark connector can be accessed using SQL or by using the Spark RDD/Dataset API. Virtually all modern data stores do have Spark connector. see [SparkPackages](https://spark-packages.org/)). You can also dynamically deploy connectors to a running SnappyData cluster. -Any Apache Spark DataFrame can be easily managed as a SnappyData table or conversely any table can be accessed as a DataFrame. +* **Virtual or in-memory data** + You can decide which datasets need to be provisioned into distributed memory or left at the source. When the data is left at source, after being modeled as a virtual/external tables, the analytic query processing is parallelized, and the query fragments are pushed down wherever possible and executed at high speed. +When speed is essential, applications can selectively copy the external data into memory using a single SQL command. -By default, when the cluster is started, the data store is bootstrapped and when any Apache Spark Jobs/OLAP queries are submitted, Apache Spark executors are automatically launched within the SnappyData process space (JVMs). There is no need to connect and manage external data store clusters. The SnappyData store can synchronously replicate for high availability (HA) with strong consistency and store/recover from disk for additional reliability. +* **In-memory Columnar + Row store** + You can choose in-memory data to be stored in any of the following forms: + * **Columnar**: The form that is compressed and designed for scanning/aggregating large data sets. + * **Row store**: The form that has an extremely fast key access or highly selective access. + The columnar store is automatically indexed using a skipping index. Applications can explicitly add indexes for the row store. +* **High performance** + When data is loaded, the engine parallelizes all the accesses by carefully taking into account the available distributed cores, the available memory, and whether the source data can be partitioned to deliver extremely high-speed loading. Therefore, unlike a traditional warehouse, you can bring up SnappyData whenever required, load, process, and tear it down. Query processing uses code generation and vectorization techniques to shift the processing to the modern-day multi-core processor and L1/L2/L3 caches to the possible extent. -## Key Features +* **Flexible rich data transformations** + External data sets when discovered automatically through schema inference will have the schema of the source. Users can cleanse, blend, reshape data using a SQL function library (Spark SQL+) or even submit Apache Spark jobs and use custom logic. The entire rich Spark API is at your disposal. This logic can be written in SQL, Java, Scala, or even Python.* -* **100% compatible with Apache Spark**- Use SnappyData as a database, and additionally use any of the Apache Spark APIs - ML, Graph, etc. +* **Prepares data for data science** + Through the use of Spark API for statistics and machine learning, raw or curated datasets can be easily prepared for machine learning. You can understand the statistical characteristics such as correlation, independence of different variables and so on. You can generate distributed feature vectors from your data that is by using processes such as one-hot encoder, binarizer, and a range of functions built into the Spark ML library. These features can be stored back into column tables and shared across a group of users with security and avoid dumping copies to disk, which is slow and error-prone. + +* **Stream ingestion and liveness** + While it is common to see query service engines today, most resort to periodic refreshing of data sets from the source as the managed data cannot be mutated — for example query engines such as Presto, HDFS formats like parquet, etc. Moreover, when updates can be applied pre-processing, re-shaping of the data is not necessarily simple. + In SnappyData, operational systems can feed data updates through Kafka to SnappyData. The incoming data can be CDC events (insert, updates, or deletes) and can be easily ingested into respective in-memory tables with ease, consistency, and exactly-once semantics. The Application can apply smart logic to reduce incoming streams, apply transformations, etc. by using APIS for Spark structured streaming.* -* **In-memory row and column stores**: Run the store colocated in Apache Spark executors or in its own process space (that is, a computational cluster and a data cluster) +* **Approximate Query Processing(AQP)** + When dealing with huge data sets, for example, IoT sensor streaming time-series data, it may not be possible to provision the data in-memory, and if left at the source (say Hadoop or S3) your analytic query processing can take too long. In SnappyData, you can create one or more stratified data samples on the full data set. The query engine automatically uses these samples for aggregation queries, and a nearly accurate answer returned to clients. This can be immensely valuable when visualizing a trend, plotting a graph or bar chart. -* **SQL standard compliance**: Apache Spark SQL + several SQL extensions: DML, DDL, indexing, constraints. +* **Access from anywhere** + You can use JDBC, ODBC, REST, or any of the Spark APIs. The product is fully compatible with Spark 2.1.1. SnappyData natively supports modern visualization tools such as TIBCO Spotfire, Tableau, and Qlikview. -* **SQL based extensions for streaming processing**: Use native Apache Spark streaming, DataFrame APIs or declaratively specify your streams and how you want it processed. You do not need to learn Apache Spark APIs to get going with stream processing or its subtleties when processing in parallel. -* **Not-Only SQL**: Use either as a SQL database or work with JSON or even arbitrary Application Objects. Essentially, any Apache Spark RDD/DataSet can also be persisted into SnappyData tables (type system same as Apache Spark Datasets). +## Downloading and Installing SnappyData +You can download and install the latest version of SnappyData from [github](https://github.com/SnappyDataInc/snappydata/releases) or you can download the enterprise version that is TIBCO ComputeDB from [here](https://edelivery.tibco.com/storefront/index.ep). +Refer to the [documentation](/install.md) for installation steps. -* **Mutate, transact on data in Apache Spark**: You can use SQL to insert, update, delete data in tables as one would expect. Extensions to Apache Spark’s context are also provided so you can mutate data in your Apache Spark programs. Tables defined in SnappyData are automatically visible as DataFrames. By eliminating the need to store data separately in a data store and then cache it in Apache Spark for computing, SnappyData simplifies system architecture and reduces the overall cost of ownership while simultaneously offering much better performance. +## Getting Started +Multiple options are provided to get started with SnappyData. You can run SnappyData on your laptop using any of the following options: -* **Optimizations - Indexing**: From version 1.0, you can add indexes to your row format tables and the query optimizer automatically uses in-memory indexes when available, to provide better performance. +* On-premise clusters -* **[Optimizations - colocation](howto/perform_a_colocated_join.md)**: SnappyData implements several optimizations to improve data locality and avoid shuffling data for queries on partitioned data sets. All related data can be colocated using declarative custom partitioning strategies (for example, common shared business key). Reference data tables can be modeled as replicated tables when tables cannot share a common key. Replicas are always consistent. +* AWS -* **High availability not just Fault tolerance**: Data can be instantly replicated (one at a time or batch at a time) to other nodes in the cluster. It is deeply integrated with a membership-based distributed system to detect and handle failures, instantaneously providing applications continuous HA. +* Docker +* Kubernetes -* **Durability and recovery:** Tables can be configured to be persisted to disk (the default) and recovered upon startup. Utilities for backup, restore and import/export are provided with the system. +You can find more information on options for running SnappyData [here](/quickstart.md). -* **[Interactive analytics using Synopsis Data Engine (SDE)](aqp.md)**: Multiple synopses techniques are introduced through data structures like count-min-sketch and stratified sampling to dramatically reduce the in-memory space requirements and provide true interactive speeds for analytic queries. These structures can be created and managed by developers with little to no statistical background and are completely transparent to the SQL developer running queries. Error estimators are also integrated with simple mechanisms to get to the errors through built-in SQL functions or SparkSession API extensions. +## Quick Test to Measure Performance of SnappyData vs Apache Spark +If you are already using Spark, experience upto 20x speedup for your query performance with SnappyData. Try out this [test](https://github.com/SnappyDataInc/snappydata/blob/master/examples/quickstart/scripts/Quickstart.scala) using the Spark Shell. - +## Documentation +To understand SnappyData and its features refer to the [documentation](http://snappydatainc.github.io/snappydata/). -## Extensions to the Apache Spark Runtime +### Other Relevant content +- [Paper](http://cidrdb.org/cidr2017/papers/p28-mozafari-cidr17.pdf) on Snappydata at Conference on Innovative Data Systems Research (CIDR) - Info on key concepts and motivating problems. +- [Another early Paper](https://www.snappydata.io/snappy-industrial) that focuses on overall architecture, use cases, and benchmarks. ACM Sigmod 2016. +- [TPC-H benchmark](https://www.snappydata.io/whitepapers/snappydata-tpch) comparing Apache Spark with SnappyData +- Checkout the [SnappyData blog](https://www.snappydata.io/blog) for developer content +- [TIBCO community page](https://community.tibco.com/products/tibco-computedb) for the latest info. -SnappyData makes the following contributions to deliver a unified and optimized runtime. +## Community Support -* **Integrating an operational in-memory data store with Apache Spark’s computational model**: A number of extensions are introduced to fuse our runtime with that of Apache Spark. Apache Spark executors run in the same process space as our store’s execution threads, sharing the same pool of memory. When Apache Spark executes tasks in a partitioned manner, it is designed to keep all the available CPU cores busy.
This design is extended by allowing low latency and fine-grained operations to interleave and get higher priority, without involving the scheduler. Furthermore, to support high concurrency, the runtime is extended with a “Job Server” that decouples applications from data servers, operating much in the same way as a traditional database, whereby the state is shared across many clients and applications.
+We monitor the following channels comments/questions: -* **Unified API for OLAP, OLTP, and Streaming**: Apache Spark builds on a common set of abstractions to provide a rich API for a diverse range of applications, such as MapReduce, Machine learning, stream processing, and SQL. -SnappyData extends Apache Spark’s unified API: - - * Allow for OLTP operations, for example, transactions and mutations (inserts/updates/deletions) on tables - - * Conform with SQL standards, for example, allowing tables alterations, constraints, and indexes +* [Stackoverflow](http://stackoverflow.com/questions/tagged/snappydata) ![Stackoverflow](http://i.imgur.com/LPIdp12.png) + +* [Slack](http://snappydata-slackin.herokuapp.com/) ![Slack](http://i.imgur.com/h3sc6GM.png) + +* [Gitter](https://gitter.im/SnappyDataInc/snappydata) ![Gitter](http://i.imgur.com/jNAJeOn.jpg) + +* [Mailing List](https://groups.google.com/forum/#!forum/snappydata-user) ![Mailing List](http://i.imgur.com/YomdH4s.png) + +* [Reddit](https://www.reddit.com/r/snappydata) ![Reddit](http://i.imgur.com/AB3cVtj.png) + +* [JIRA](https://jira.snappydata.io/projects/SNAP/issues) ![JIRA](http://i.imgur.com/E92zntA.png) + +## Link with SnappyData Distribution + +### Using Maven Dependency + +SnappyData artifacts are hosted in Maven Central. You can add a Maven dependency with the following coordinates: + +``` +groupId: io.snappydata +artifactId: snappydata-cluster_2.11 +version: 1.1.1 +``` + +### Using SBT Dependency + +If you are using SBT, add this line to your **build.sbt** for core SnappyData artifacts: + +``` +libraryDependencies += "io.snappydata" % "snappydata-core_2.11" % "1.1.1" +``` + +For additions related to SnappyData cluster, use: + +``` +libraryDependencies += "io.snappydata" % "snappydata-cluster_2.11" % "1.1.1" +``` + +You can find more specific SnappyData artifacts [here](http://mvnrepository.com/artifact/io.snappydata) + +!!!Note + If your project fails when resolving the above dependency (that is, it fails to download `javax.ws.rs#javax.ws.rs-api;2.1`), it may be due an issue with its pom file.
As a workaround, you can add the below code to your **build.sbt**: + +``` +val workaround = { + sys.props += "packaging.type" -> "jar" + () +} +``` + +For more details, refer [https://github.com/sbt/sbt/issues/3618](https://github.com/sbt/sbt/issues/3618). - * Support declarative stream processing in SQL - * A unique addition of SnappyData is ability to mutate (all of inserts/updates/deletes) even column format tables efficiently without much change in query performance profile. +## Building from Source +If you would like to build SnappyData from source, refer to the [documentation on building from source](/install/building_from_source.md). -* **Optimizing Apache Spark application execution times**: Our goal is to eliminate the need for yet another external store (for example, a KV store) for Apache Spark applications. With a deeply integrated store, SnappyData improves overall performance by minimizing network traffic and serialization costs. In addition, by promoting colocated schema designs (tables and streams) where related data is colocated in the same process space, SnappyData eliminates the need for shuffling altogether in several scenarios. -* **Synopsis Data Engine support built into Apache Spark**: The SnappyData [Synopsis Data Engine (SDE)](aqp.md) offers a novel and scalable system to analyze large data sets. SDE uses statistical sampling techniques and probabilistic data structures to answer analytic queries with sub-second latency. There is no need to store or process the entire data set. The approach trades off query accuracy for fast response time.
The SDE engine enables you to: +## What is the Delta between SnappyData and Apache Spark? - - Intelligently sample the data set on frequently accessed dimensions to have a good representation across the entire data set (stratified sampling). Queries can execute on samples and return answers instantly. +Apache Spark is a general purpose parallel computational engine for analytics at scale. At its core, it has a batch design center and is capable of working with disparate data sources. While this provides rich unified access to data, this can also be quite inefficient and expensive. Analytic processing requires massive data sets to be repeatedly copied and data to be reformatted to suit Spark. In many cases, it ultimately fails to deliver the promise of interactive analytic performance. +For instance, each time an aggregation is run on a large Cassandra table, it necessitates streaming the entire table into Spark to do the aggregation. Caching within Spark is immutable and results in stale insight. - - Compute estimates for any ad hoc query from the sample(s). It can also provide error estimates for arbitrarily complex queries on streams. +### The SnappyData Approach - - Provide simple knobs for the user to trade off speed for accuracy, that is, simple SQL extensions so the user can specify the error tolerance for all queries. When query error is higher than tolerance level, the system automatically delegates the query to the source. +##### Snappy Architecture - - Express their accuracy requirements as high-level accuracy contracts (HAC), without overwhelming them with numerous statistical concepts. +![SnappyData Architecture](./Images/SnappyArchitecture.png) -## Morphing Apache Spark to support mixed workloads (OLTP, OLAP) -Apache Spark is designed as a computational engine for processing batch jobs. Each Apache Spark application (for example, a Map-reduce job) runs as an independent set of processes (that is, executor JVMs) in the cluster. These JVMs are reused for the lifetime of the application. While, data can be cached and reused in these JVMs for a single application, sharing data across applications or clients require an external storage tier, such as HDFS. SnappyData, on the other hand, targets a real-time, “always-on”, operational design center— clients can connect at will, and share data across any number of concurrent connections. This is similar to any operational database in the market today. Thus, to manage data in the same JVM, our first challenge is to alter the life cycle of these executors so that they are long-lived and decoupled from individual applications. +At SnappyData, there is a different approach. SnappyData fuses a low latency, highly available in-memory transactional database (GemFireXD) into Spark with shared memory management and optimizations. Data in the highly available in-memory store is laid out using the same columnar format as Spark (Tungsten). All query engine operators are significantly more optimized through better vectorization and code generation.
+The net effect is, an order of magnitude performance improvement when compared to native Spark caching, and more than two orders of magnitude better Spark performance when working with external data sources. +Spark is turned into an in-memory operational database capable of transactions, point reads, writes, working with Streams (Spark) and running analytic SQL queries or it is an in-memory scale out Hybrid Database that can execute Spark code, SQL, or even Objects. -A second but related challenge is Apache Spark’s design for how user requests (jobs) are handled. A single driver orchestrates all the work done on the executors. Given our need for high concurrency and a hybrid OLTP-OLAP workload, this driver introduces: -1. A single point of contention for all requests, and +## Streaming Example - Ad Analytics +Here is a stream + Transactions + Analytics use case example to illustrate the SQL as well as the Spark programming approaches in SnappyData - [Ad Analytics code example](https://github.com/SnappyDataInc/snappy-poc). Here is a [screencast](https://www.youtube.com/watch?v=bXofwFtmHjE) that showcases many useful features of SnappyData. The example also goes through a benchmark comparing SnappyData to a Hybrid in-memory database and Cassandra. -2. A barrier for achieving high availability (HA). Executors are shut down if the driver fails, requiring a full refresh of any cached state. +## Contributing to SnappyData -Apache Spark’s primary usage of memory is for caching RDDs and for shuffling blocks to other nodes. Data is managed in blocks and is immutable. On the other hand, more complex data structure needs to be managed (along with indexes) for point access and updates. Therefore, another challenge is merging these two disparate storage systems with little impedance to the application. This challenge is exacerbated by current limitations of Apache Spark SQL—mostly related to mutability characteristics and conformance to SQL. +If you are interested in contributing, please visit the [community page](http://www.snappydata.io/community) for ways in which you can help. -Finally, Apache Spark’s strong and growing community has zero tolerance for incompatible forks. This means that no changes can be made to Apache Spark’s execution model or its semantics for existing APIs. In other words, our changes have to be an extension. diff --git a/docs/install.md b/docs/install.md index 63d1cf5364..3b104e9b7d 100644 --- a/docs/install.md +++ b/docs/install.md @@ -14,9 +14,9 @@ For more information on the capabilities of the Community Edition and Enterprise Download SnappyData Community Edition -Download the [SnappyData 1.1.0 Community Edition (Open Source)](https://github.com/SnappyDataInc/snappydata/releases/) from the release page, which lists the latest and previous releases of SnappyData. The packages are available in compressed files (.tar format). +Download the [SnappyData 1.1.1 Community Edition (Open Source)](https://github.com/SnappyDataInc/snappydata/releases/) from the release page, which lists the latest and previous releases of SnappyData. The packages are available in compressed files (.tar format). -* [**SnappyData 1.1.0 Release download link**](https://github.com/SnappyDataInc/snappydata/releases/download/v1.1.0/snappydata-1.1.0-bin.tar.gz) +* [**SnappyData 1.1.1 Release download link**](https://github.com/SnappyDataInc/snappydata/releases/download/v1.1.1/snappydata-1.1.1-bin.tar.gz) Download SnappyData Enterprise Edition diff --git a/docs/install/concepts.md b/docs/install/concepts.md new file mode 100644 index 0000000000..62c3f865c0 --- /dev/null +++ b/docs/install/concepts.md @@ -0,0 +1,11 @@ +# SnappyData Concepts + + +The topic explains the following fundamental concepts of SnappyData: + +* [Core Components](/architecture/core_components.md) +* [SnappyData Cluster Architecture](/architecture/cluster_architecture.md) +* [Distributed Transactions](/consistency/transactions_about.md) +* [Affinity Modes](../deployment.md) +* [Data Ingestion Pipeline](/architecture/data_ingestion_pipeline.md) +* [Hybrid Cluster Manager](/architecture/Hybrid Cluster Manager) diff --git a/docs/install/setting_up_cluster_on_amazon_web_services.md b/docs/install/setting_up_cluster_on_amazon_web_services.md index d0d102e864..75ea78d793 100644 --- a/docs/install/setting_up_cluster_on_amazon_web_services.md +++ b/docs/install/setting_up_cluster_on_amazon_web_services.md @@ -281,12 +281,12 @@ For example, to use **SnappyData Enterprise** build to launch the cluster, downl www.snappydata.io/download on your local machine and give its path as value to above option. ```pre -./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem launch my-cluster --snappydata-tarball="/home/ec2-user/snappydata/distributions/snappydata-1.1.0-bin.tar.gz" +./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem launch my-cluster --snappydata-tarball="/home/ec2-user/snappydata/distributions/snappydata-1.1.1-bin.tar.gz" ``` Alternatively, you can also put your build file on a public web server and provide its URL to this option. ```pre -./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem launch my-cluster --snappydata-tarball="https://s3-us-east-2.amazonaws.com/mybucket/distributions/snappydata-1.1.0-bin.tar.gz" +./snappy-ec2 -k my-ec2-key -i ~/my-ec2-key.pem launch my-cluster --snappydata-tarball="https://s3-us-east-2.amazonaws.com/mybucket/distributions/snappydata-1.1.1-bin.tar.gz" ``` The build file should be in **.tar.gz** format. @@ -524,7 +524,7 @@ Options: ## AWS Management Console !!! Attention - The AMI of latest SnappyData release 1.1.0 is not available on AWS. + The AMIs of SnappyData release 1.1.0 onwards are not available on AWS. You can launch a SnappyData cluster on Amazon EC2 instance(s) using the AMI provided by SnappyData. For more information on launching an EC2 instance, refer to the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/launching-instance.html). diff --git a/docs/install/system_requirements.md b/docs/install/system_requirements.md index 7d42fab440..edef6f2769 100644 --- a/docs/install/system_requirements.md +++ b/docs/install/system_requirements.md @@ -54,6 +54,9 @@ Requirements for each host: * If you deploy SnappyData on a virtualized host, consult the documentation provided with the platform, for system requirements and recommended best practices, for running Java and latency-sensitive workloads. ## VSD Requirements + +This feature is available only in the Enterprise version of SnappyData.
+ - Install 32-bit libraries on 64-bit Linux:
`yum install glibc.i686 libX11.i686` on RHEL/CentOS
`apt-get install libc6:i386 libx11-6:i386` on Ubuntu/Debian like systems
diff --git a/docs/install/upgrade.md b/docs/install/upgrade.md index 53cf5ac90d..9f3437c9e9 100644 --- a/docs/install/upgrade.md +++ b/docs/install/upgrade.md @@ -20,9 +20,9 @@ Before you begin the upgrade, ensure that you understand the new features and an 7. To ensure that the restore script (restore.sh) copies files back to their original locations, make sure that the disk files are available at the original location before restarting the cluster with the latest version of SnappyData. -## Upgrading to SnappyData 1.1.0 from 1.0.1 or Earlier Versions +## Upgrading to SnappyData 1.1.1 from 1.0.1 or Earlier Versions -When you upgrade to SnappyData 1.1.0 from product version 1.0.1 or earlier versions, it is recommended to save all the table data as parquet files, recreate the tables in the new cluster, and then load data from the saved parquet files. Before taking the backup ensure that no operations are currently running on the system. Ensure to cleanup the data from the previous cluster and start the new cluster from a clean directory. +When you upgrade to SnappyData 1.1.1 from product version 1.0.1 or earlier versions, it is recommended to save all the table data as parquet files, recreate the tables in the new cluster, and then load data from the saved parquet files. Before taking the backup ensure that no operations are currently running on the system. Ensure to cleanup the data from the previous cluster and start the new cluster from a clean directory. For example: @@ -32,7 +32,7 @@ snappy> create external table table1Parquet using parquet options (path ' drop table table1; snappy> drop table table1Parquet; -# Creating tables from parquet files in SnappyData 1.1.0 +# Creating tables from parquet files in SnappyData 1.1.1 snappy> create external table table1_parquet using parquet options (path '') ; snappy> create table table1(...); snappy> insert into table1 select * from table1_parquet; diff --git a/docs/monitoring/configure_logging.md b/docs/monitoring/configure_logging.md index 37dfdfa250..2e8d8cf586 100755 --- a/docs/monitoring/configure_logging.md +++ b/docs/monitoring/configure_logging.md @@ -82,7 +82,7 @@ The fine-grained log settings are applicable for classes other than the SnappyDa ## Using Trace Flags for Advanced Logging For SnappyData Store -SnappyData Store provides the following trace flags that you can use with the `snappydata.debug.true` system property to log additional details about specific features: +SnappyData Store provides the following trace flags that you can use with the `gemfirexd.debug.true` system property to log additional details about specific features: | Trace flag | Enables | |----------------------------|-----------------------------------------------------| @@ -99,10 +99,10 @@ SnappyData Store provides the following trace flags that you can use with the `s | TraceLock\_\* | Locking and unlocking information for all internal locks.| | TraceLock\_DD | Logging for all DataDictionary and table locks that are acquired or released.| -To enable logging of specific features of SnappyData, set the required trace flag in the `snappydata.debug.true` system property. For example, you can add the following setting inside the configuration file of the SnappyData member to enable logging for query distribution and indexing: +To enable logging of specific features of SnappyData, set the required trace flag in the `gemfirexd.debug.true` system property. For example, you can add the following setting inside the configuration file of the SnappyData member to enable logging for query distribution and indexing: ```pre -localhost -J-Dsnappydata.debug.true=QueryDistribution,TraceIndex +localhost -J-Dgemfirexd.debug.true=QueryDistribution,TraceIndex ``` If you need to set a trace flag in a running system, use the [SYS.SET_TRACE_FLAG](../reference/inbuilt_system_procedures/set-trace-flag.md) system procedure. The procedure sets the trace flag in all members of the distributed system, including locators. You must execute the procedure as a system user. For example: diff --git a/docs/monitoring/monitoring.md b/docs/monitoring/monitoring.md index 7f2a0ce119..f7c979a7e5 100644 --- a/docs/monitoring/monitoring.md +++ b/docs/monitoring/monitoring.md @@ -3,6 +3,8 @@ SnappyData Monitoring Console is a dashboard that provides a real-time view into cluster members, member logs, resource usage, running Jobs, SQL queries along with performance data. This simple widget based view allows you to navigate easily, visualize, and monitor your cluster. You can monitor the overall status of the cluster as well as the status of each member in the cluster. All the usage details are automatically refreshed after every five seconds. +!!! Note + TIBCO ComputeDB Monitoring Console is not yet tested and supported on Internet Explorer. To access SnappyData Monitoring Console, start your cluster and open [http:``:5050/dashboard/](http:``:5050/dashboard/) in the web browser. @@ -33,8 +35,13 @@ The following topics are covered in this section: !!! Note When connecting to a SnappyData cluster using Smart Connector, the information related to **SQL**, **Jobs**, and **Stages** are NOT displayed, as the Jobs and queries are primarily executed in your client Spark cluster. You can find this information on the Spark UI console of your client cluster. Read more about SnappyData Smart Connector Mode [here](../affinity_modes/connector_mode.md). +In cases where you cannot access the SnappyData Monitoring Console to analyse Jobs and tasks, you must turn on the [Spark History server](#historyserver). + + On the top-right side of the SnappyData Monitoring Console page, you can view the version details of SnappyData Snapshot. When you click this, the name and version of the product, the build details, the source revision details and the version number of the underlying spark are displayed. +On the top of the dashboard, the date and time details about when the cluster is started are displayed. The period till when the cluster is running is also shown. + ![](../Images/MonitoringUI/SnappyData-UI-About-Box1.png) @@ -114,7 +121,9 @@ The following columns are displayed in this section: |**In-Memory Size** | Displays the heap memory used by data table to store its data. If less than **Total Size** then the data is overflowing to disk. | |**Spill-to-Disk Size**|Displays size of data overflown to disk| | **Total Size** | Displays the collective physical memory and disk overflow space used by the data table to store its data. | -| **Buckets** | Displays the total number of buckets in the data table.| +| **Buckets** | Displays the total number of buckets in the data table. If a number displayed in red here, it indicates that some of the buckets are offline.| +| **Redundancy** | Displays the number of redundant copies.
Redundancy value **0** indicates that redundant copies are not configured.
Redundancy value **1** indicates that one redundant copy is configured.
Redundancy value **NA** indicates that redundancy is not applicable.| +| **Redundancy Status** | Displays whether redundancy criteria is satisfied or broken.
Redundancy status **Satisfied** indicates that all the configured redundant copies are available.
Redundancy status **Broken** indicates that some of the redundant copies are not available.
Redundancy status **NA** indicates that redundancy is not applicable.| ### External Tables The **External Tables** section lists all the external tables present in the cluster along with their various statistical details. The displayed details are automatically refreshed after every five seconds. @@ -163,7 +172,7 @@ The usage trends of the member are represented in the following graphs: ### Member Logs -In the Member Details page, you can view the logs generated for a single member in the cluster. +In the Member Details page, you can view the logs generated for a single member in the cluster. ![MemberLogs](../Images/monitoring_memberdetails_logs.png) @@ -246,3 +255,21 @@ On this page, you can view the total time required for all the tasks in a job to * **Number of parallel tasks**: Due to concurrency, multiple queries may take cores and a specific query can take longer. To fix this, you can create a new scheduler and [assign appropriate cores to it](../best_practices/setup_cluster.md). * **GC time**: Occasionally, on-heap object creation can slow down a query because of garbage collection. In these cases, it is recommended that you increase the on-heap memory, especially when you have row tables. + +## Spark History Server +The Spark History server is an HTTP server that let you analyze the Spark jobs. + +The first step in tuning query performance in SnappyData is to understand the query physical plan that is available through the SQL tab on the SnappyData Monitoring console. The detailed execution plan requires one to understand the jobs and tasks associated with the query. This is available in the Jobs/Tasks tab. However, if the SnappyData Monitoring console is not accessible to the investigator, it becomes a difficult exercise. To overcome this, TIBCO recommends to turn on the History server for production applications. + +To turn on the History server, do the following: + +1. Ensure to provide a shared disk that can be accessed from all the SnappyData nodes. If you do not have the NFS access, use HDFS. Provide the permissions to access a shared folder when you start SnappyData. +2. Enable event logging for the Spark jobs. For example, if the server was configured with a log directory of hdfs://namenode/shared/spark-logs, then configure the following properties in the conf/lead: + + spark.eventLog.enabled true + spark.eventLog.dir hdfs://namenode/shared/spark-logs + +3. Start the History server. + ./sbin/start-history-server.sh + This creates a web interface at http://:18080 by default, listing incomplete and completed instances of SQL queries and the associated Spark jobs and tasks. + For more details about History server, refer to [Configuring History Server](https://spark.apache.org/docs/latest/monitoring.html#environment-variables). diff --git a/docs/programming_guide/snappydata_jobs.md b/docs/programming_guide/snappydata_jobs.md index 1588d1f63f..3381e557b4 100644 --- a/docs/programming_guide/snappydata_jobs.md +++ b/docs/programming_guide/snappydata_jobs.md @@ -139,7 +139,7 @@ For writing jobs users need to include [Maven/SBT dependencies for the latest re For example, gradle can be configured as: ```pre -compile('io.snappydata:snappydata-cluster_2.11:1.1.0') { +compile('io.snappydata:snappydata-cluster_2.11:1.1.1') { exclude(group: 'io.snappydata', module: 'snappy-spark-unsafe_2.11') exclude(group: 'io.snappydata', module: 'snappy-spark-core_2.11') exclude(group: 'io.snappydata',module: 'snappy-spark-yarn_2.11') diff --git a/docs/programming_guide/spark_jdbc_connector.md b/docs/programming_guide/spark_jdbc_connector.md index 5084849638..fc2c1c6703 100644 --- a/docs/programming_guide/spark_jdbc_connector.md +++ b/docs/programming_guide/spark_jdbc_connector.md @@ -36,13 +36,13 @@ The SnappyData connector internally figures out the structure of the result set ### Connecting to SnappyData using the JDBC Extension Connector Following is a sample of Spark JDBC Extension setup and usage: -1. Include the **snappydata-jdbc** package in the Spark job with spark-submit or spark-shell. You can download the same from [here](https://github.com/SnappyDataInc/snappydata/releases/download/v1.1.0/snappydata-jdbc_2.11-1.1.0.jar): +1. Include the **snappydata-jdbc** package in the Spark job with spark-submit or spark-shell. You can download the same from [here](https://github.com/SnappyDataInc/snappydata/releases/download/v1.1.1/snappydata-jdbc_2.11-1.1.1.jar): - $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.0.jar + $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.1.jar 2. Set the session properties.
The SnappyData connection properties (to enable auto-configuration of JDBC URL) and credentials can be provided in Spark configuration itself, or set later in SparkSession to avoid passing them in all the method calls. These properties can also be provided in **spark-defaults.conf** along with all the other Spark properties. You can also set any of these properties in your app code.
Overloads of the above methods accepting **user+password** and **host+port **is also provided in case those properties are not set in the session or needs to be overridden. You can optionally pass additional connection properties similarly as in the **DataFrameReader.jdbc **method.
Following is a sample code of configuring the properties in **SparkConf**: - $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.0.jar --conf spark.snappydata.connection=localhost:1527 --conf spark.snappydata.user= --conf spark.snappydata.password= + $SPARK_HOME/bin/spark-shell --jars snappydata-jdbc-2.11_1.1.1.jar --conf spark.snappydata.connection=localhost:1527 --conf spark.snappydata.user= --conf spark.snappydata.password= 3. Import the required implicits in the job/shell code as follows: @@ -144,4 +144,4 @@ The following extensions are used to implement the Spark JDBC Connector: ## Performance Considerations -It should be noted that using the Spark Data Source Writer or the Snappy Implicit are both much slower as compared to SnappyData embedded job or Smart Connector. If the DataFrame that is to be written is medium or large sized, then it is better to ingest directly in an embedded mode. In case writing an embedded job is not an option, the incoming DataFrame can be dumped to an external table in a location accessible to both Spark and SnappyData clusters. After this, it can be ingested in an embedded mode using **snappyExecute**. \ No newline at end of file +It should be noted that using the Spark Data Source Writer or the Snappy Implicit are both much slower as compared to SnappyData embedded job or Smart Connector. If the DataFrame that is to be written is medium or large sized, then it is better to ingest directly in an embedded mode. In case writing an embedded job is not an option, the incoming DataFrame can be dumped to an external table in a location accessible to both Spark and SnappyData clusters. After this, it can be ingested in an embedded mode using **snappyExecute**. diff --git a/docs/programming_guide/using_snappydata_shell.md b/docs/programming_guide/using_snappydata_shell.md index 2d31699436..b4f794d7f1 100644 --- a/docs/programming_guide/using_snappydata_shell.md +++ b/docs/programming_guide/using_snappydata_shell.md @@ -8,7 +8,7 @@ Internally, it uses JDBC to interact with the cluster. You can also use tools li ```pre // From the SnappyData base directory $ ./bin/snappy-sql -SnappyData version 1.1.0 +SnappyData version 1.1.1 snappy-sql> //Connect to the cluster as a client diff --git a/docs/quickstart/getting_started_by_installing_snappydata_on-premise.md b/docs/quickstart/getting_started_by_installing_snappydata_on-premise.md index 6e036b0d56..49361df627 100644 --- a/docs/quickstart/getting_started_by_installing_snappydata_on-premise.md +++ b/docs/quickstart/getting_started_by_installing_snappydata_on-premise.md @@ -3,8 +3,8 @@ Download the latest version of SnappyData from the [SnappyData Release Page](https://github.com/SnappyDataInc/snappydata/releases/), which lists the latest and previous releases of SnappyData. ```pre -$ tar -xzf snappydata-1.1.0-bin.tar.gz -$ cd snappydata-1.1.0-bin/ +$ tar -xzf snappydata-1.1.1-bin.tar.gz +$ cd snappydata-1.1.1-bin/ # Create a directory for SnappyData artifacts $ mkdir quickstartdatadir $./bin/spark-shell --conf spark.snappydata.store.sys-disk-dir=quickstartdatadir --conf spark.snappydata.store.log-file=quickstartdatadir/quickstart.log diff --git a/docs/quickstart/getting_started_with_docker_image.md b/docs/quickstart/getting_started_with_docker_image.md index 59d43056ba..751b824c01 100644 --- a/docs/quickstart/getting_started_with_docker_image.md +++ b/docs/quickstart/getting_started_with_docker_image.md @@ -20,7 +20,7 @@ $ docker run hello-world A sample Dockerfile is provided which you can use to create your own Docker image of SnappyData. Download the [Dockerfile](https://github.com/SnappyDataInc/snappy-cloud-tools/blob/master/docker/Dockerfile) and -[start](https://github.com/SnappyDataInc/snappy-cloud-tools/blob/master/docker/start) script and place them into a single directory. This Dockerfile uses the SnappyData 1.1.0 build. +[start](https://github.com/SnappyDataInc/snappy-cloud-tools/blob/master/docker/start) script and place them into a single directory. This Dockerfile uses the SnappyData 1.1.1 build. Move to that directory and run the following commands with appropriate details: @@ -29,8 +29,8 @@ Move to that directory and run the following commands with appropriate details: For example: - $ docker build -t snappydatainc/snappydata:1.1.0 -f Dockerfile . - $ docker push snappydatainc/snappydata:1.1.0 + $ docker build -t snappydatainc/snappydata:1.1.1 -f Dockerfile . + $ docker push snappydatainc/snappydata:1.1.1 ## Launch SnappyData
diff --git a/docs/quickstart/getting_started_with_your_spark_distribution.md b/docs/quickstart/getting_started_with_your_spark_distribution.md index 83c214c315..6fea08c343 100644 --- a/docs/quickstart/getting_started_with_your_spark_distribution.md +++ b/docs/quickstart/getting_started_with_your_spark_distribution.md @@ -9,7 +9,7 @@ Open a command terminal, go to the location of the Spark installation directory, $ cd # Create a directory for SnappyData artifacts $ mkdir quickstartdatadir -$ ./bin/spark-shell --conf spark.snappydata.store.sys-disk-dir=quickstartdatadir --conf spark.snappydata.store.log-file=quickstartdatadir/quickstart.log --packages "SnappyDataInc:snappydata:1.1.0-s_2.11" +$ ./bin/spark-shell --conf spark.snappydata.store.sys-disk-dir=quickstartdatadir --conf spark.snappydata.store.log-file=quickstartdatadir/quickstart.log --packages "SnappyDataInc:snappydata:1.1.1-s_2.11" ``` This opens the Spark shell and downloads the relevant SnappyData files to your local machine. Depending on your network connection speed, it may take some time to download the files.
diff --git a/docs/quickstart/performance_apache_spark.md b/docs/quickstart/performance_apache_spark.md index 423f1a1d61..e42d020fcf 100644 --- a/docs/quickstart/performance_apache_spark.md +++ b/docs/quickstart/performance_apache_spark.md @@ -16,7 +16,7 @@ Use any of the options mentioned below to start the Spark shell: # Create a directory for SnappyData artifacts $ mkdir quickstartdatadir - $ ./bin/spark-shell --driver-memory=4g --conf spark.snappydata.store.sys-disk-dir=quickstartdatadir --conf spark.snappydata.store.log-file=quickstartdatadir/quickstart.log --packages "SnappyDataInc:snappydata:1.1.0-s_2.11" --driver-java-options="-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:MaxNewSize=1g" + $ ./bin/spark-shell --driver-memory=4g --conf spark.snappydata.store.sys-disk-dir=quickstartdatadir --conf spark.snappydata.store.log-file=quickstartdatadir/quickstart.log --packages "SnappyDataInc:snappydata:1.1.1-s_2.11" --driver-java-options="-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:MaxNewSize=1g" * **If you have downloaded SnappyData**: diff --git a/docs/quickstart/structucture_streamingquickstart.md b/docs/quickstart/structucture_streamingquickstart.md index 7b5fec714e..e2b93a6a57 100644 --- a/docs/quickstart/structucture_streamingquickstart.md +++ b/docs/quickstart/structucture_streamingquickstart.md @@ -3,6 +3,8 @@ This quick start guide provides step-by-step instructions to perform structured streaming in SnappyData by using the Spark shell as well as through a Snappy job. +For detailed information, refer to [Structured Streaming](/howto/use_stream_processing_with_snappydata.md#structuredstreaming). + ## Structured Streaming using Spark Shell Following are the steps to perform structured streaming using Spark shell: @@ -231,3 +233,6 @@ Use the following command to stop the running job: !!!Note The job-id used for stopping the job is picked from the job submission response. + +## Examples +For more examples, refer [here](https://github.com/SnappyDataInc/snappydata/blob/master/examples/src/main/scala/org/apache/spark/examples/snappydata/structuredstreaming) diff --git a/docs/reference/API_Reference/apireference_guide.md b/docs/reference/API_Reference/apireference_guide.md index b719dfeaa4..aaaed3f6aa 100644 --- a/docs/reference/API_Reference/apireference_guide.md +++ b/docs/reference/API_Reference/apireference_guide.md @@ -13,7 +13,7 @@ This guide gives details of Spark extension APIs that are provided by SnappyData | [**dropTable**](#droptableapi)
Drop SnappyData Table | || | [**createSampleTable**](#createsampletableapi)
Create Stratified Sample Table | || | [**createApproxTSTopK**](#createaproxtstopkapi)
Create Structure to Query Top-K | || -| [**setSchema**](#setschemaapi)
Set Current Database/schema | || +| [**setCurrentSchema**](#setschemaapi)
Set Current Database/schema | || | [**getCurrentSchema**](#getcurrentschemaapi)
Get Current Schema of Session | || | [**insert**](#insertapi)
Insert Row into an Existing Table | || | [**put**](#putapi)
Upsert Row into an Existing Table | || @@ -98,7 +98,7 @@ Creates a SnappyData managed table. Any relation providers, that is the row, col createTable( tableName: String, provider: String, - schemaDDL: String, + schema: StructType, options: Map[String, String], allowExisting: Boolean) ``` @@ -290,14 +290,14 @@ snappySession.createApproxTSTopK("topktable", Some("hashtagTable"), "hashtag", s ``` -### setSchema +### setCurrentSchema Sets the current database/schema. **Syntax** ``` -setSchema(schemaName: String) +schema ``` @@ -435,7 +435,7 @@ delete(tableName: String, filterExpr: String) |Parameter | Description | |--------|--------| | tableName | Name of the table. | -|filterExpr| SSQL WHERE criteria to select rows that will be updated. | +|filterExpr| SQL WHERE criteria to select rows that will be updated. | |Returns|Int| **Example ** @@ -589,7 +589,7 @@ getKeyColumnsAndPositions(tableName: String) |Parameter | Description | |--------|--------| | tableName | Name of the table.| -| Returns | Sequence of `scala.Tuple2` containing column name and column's position in the table for each key columns (for column tables) or sequence of primary keys (for row tables).| +| Returns | Sequence of `scala.Tuple2` containing column and column's position in the table for each key columns (for column tables) or sequence of primary keys (for row tables).| **Example ** diff --git a/docs/reference/API_Reference/odbc_supported_apis.md b/docs/reference/API_Reference/odbc_supported_apis.md new file mode 100644 index 0000000000..4e899e47bb --- /dev/null +++ b/docs/reference/API_Reference/odbc_supported_apis.md @@ -0,0 +1,143 @@ +# ODBC Supported APIs + +The following APIs are supported for ODBC in Snappy Driver: + +* [Function APIs](#functionconfr) +* [Attribute APIs](attriconf) + + +## Function Conformance of ODBC Supported APIs in Snappy Driver + +| Function | Conformance
level | Supported in
Snappy Driver |Exceptions| +|---------------------|-------------------|----------------------------|--| +| SQLAllocHandle | Core | Yes || +| SQLBindCol | Core | Yes || +| SQLBindParameter | Core[1] | Yes || +| SQLBrowseConnect | Level 1 | Not || +| SQLBulkOperations | Level 1 | Yes || +| SQLCancel | Core[1] | Yes || +| SQLCloseCursor | Core | Yes || +| SQLColAttribute | Core[1] | Yes || +| SQLColumnPrivileges | Level 2 | Yes || +| SQLColumns | Core | Yes || +| SQLConnect | Core | Yes || +| SQLCopyDesc | Core | Not || +| SQLDataSources | Core | Not |As per MSDN document it should implement by Driver Manager.| +| SQLDescribeCol | Core[1] | Yes || +| SQLDescribeParam | Level 2 | Yes || +| SQLDisconnect | Core | Yes || +| SQLDriverConnect | Core | Yes || +| SQLDrivers | Core | Not || +| SQLEndTran | Core[1] | Yes || +| SQLExecDirect | Core | Yes || +| SQLExecute | Core | Yes || +| SQLFetch | Core | Yes || +| SQLFetchScroll | Core[1] | Yes || +| SQLForeignKeys | Level 2 | Yes || +| SQLFreeHandle | Core | Yes || +| SQLFreeStmt | Core | Yes || +| SQLGetConnectAttr | Core | Yes || +| SQLGetCursorName | Core | Yes || +| SQLGetData | Core | Yes || +| SQLGetDescField | Core | Not || +| SQLGetDescRec | Core | Not || +| SQLGetDiagField | Core | Yes || +| SQLGetDiagRec | Core | Yes || +| SQLGetEnvAttr | Core | Yes || +| SQLGetFunctions | Core | Yes || +| SQLGetInfo | Core | Yes |SQL_DEFAULT_TXN_ISOLATION - **not supported**
SQL_TXN_ISOLATION_OPTION -**not supported**
SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES1 **supported**
SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES2 **supported**| +| SQLGetStmtAttr | Core | Yes || +| SQLGetTypeInfo | Core | Yes |Supports call only with SQL_ALL_TYPES info type parameter| +| SQLMoreResults | Level 1 | Yes || +| SQLNativeSql | Core | Yes || +| SQLNumParams | Core | Yes || +| SQLNumResultCols | Core | Yes || +| SQLParamData | Core | Yes || +| SQLPrepare | Core | Yes || +| SQLPrimaryKeys | Level 1 | Yes || +| SQLProcedureColumns | Level 1 | Yes || +| SQLProcedures | Level 1 | Yes || +| SQLPutData | Core | Yes || +| SQLRowCount | Core | Yes || +| SQLSetConnectAttr | Core[2] | Yes || +| SQLSetCursorName | Core | Yes || +| SQLSetDescField | Core[1] | Not || +| SQLSetDescRec | Core | Not || +| SQLSetEnvAttr | Core[2] | Yes || +| SQLSetPos | Level 1[1] | Yes || +| SQLSetStmtAttr | Core[2] | Yes || +| SQLSpecialColumns | Core[1] | Yes || +| SQLStatistics | Core | Yes || +| SQLTablePrivileges | Level 2 | Yes || +| SQLTables | Core | Yes || + + + +## Attribute Conformance of ODBC Supported APIs in Snappy Driver + +### ODBC Environment Attribute + +| Attributes | Conformance
Level | Supported in
Snappy Driver |Exceptions| +|-----------------------------|-------------------|----------------------------|---| +| SQL_ATTR_CONNECTION_POOLING | --[1] | Yes |This is an optional feature and as such is not part of the conformance levels.| +| SQL_ATTR_CP_MATCH | --[1] | Yes || +| SQL_ATTR_ODBC_VER | Core | Yes || +| SQL_ATTR_OUTPUT_NTS | --[1] | Yes || + +### ODBC Connection Attribute + +| Attributes | Conformance
Level | Supported in
Snappy Driver|Exceptions| +|-----------------------------|--------------------|----------------------------|---| +| SQL_ATTR_ACCESS_MODE | Core | Yes || +| SQL_ATTR_ASYNC_ENABLE | Level 1/Level 2[1] | Yes |Applications that support connection-level asynchrony (required for Level 1) must support setting this attribute to SQL_TRUE by calling SQLSetConnectAttr; the attribute need not be settable to a value other than its default value through SQLSetStmtAttr. Applications that support statement-level asynchrony (required for Level 2) must support setting this attribute to SQL_TRUE using either function.| +| SQL_ATTR_AUTO_IPD | Level 2 | Not || +| SQL_ATTR_AUTOCOMMIT | Level 1 | Yes |For Level 1 interface conformance, the driver must support one value in addition to the driver-defined default value (available by calling SQLGetInfo with the SQL_DEFAULT_TXN_ISOLATION option). For Level 2 interface conformance, the driver must also support SQL_TXN_SERIALIZABLE.| +| SQL_ATTR_CONNECTION_DEAD | Level 1 | Yes || +| SQL_ATTR_CONNECTION_TIMEOUT | Level 2 | Yes || +| SQL_ATTR_CURRENT_CATALOG | Level 2 | Yes || +| SQL_ATTR_LOGIN_TIMEOUT | Level 2 | Yes || +| SQL_ATTR_ODBC_CURSORS | Core | Yes || +| SQL_ATTR_PACKET_SIZE | Level 2 | Yes || +| SQL_ATTR_QUIET_MODE | Core | Yes || +| SQL_ATTR_TRACE | Core | Yes || +| SQL_ATTR_TRACEFILE | Core | Yes || +| SQL_ATTR_TRANSLATE_LIB | Core | Yes || +| SQL_ATTR_TRANSLATE_OPTION | Core | Yes || +| SQL_ATTR_TXN_ISOLATION | Level 1/Level 2[2] | Yes || + +### ODBC Statement Attribute +| Attributes | Conformance
Level | Supported in
Snappy Driver |Exceptions| +|--------------------------------|--------------------|----------------------------|---| +| SQL_ATTR_APP_PARAM_DESC | Core | Yes || +| SQL_ATTR_APP_ROW_DESC | Core | Yes || +| SQL_ATTR_ASYNC_ENABLE | Level 1/Level 2[1] | Yes || +| SQL_ATTR_CONCURRENCY | Level 1/Level 2[2] | Yes || +| SQL_ATTR_CURSOR_SCROLLABLE | Level 1 | Yes || +| SQL_ATTR_CURSOR_SENSITIVITY | Level 2 | Yes || +| SQL_ATTR_CURSOR_TYPE | Core/Level 2[3] | Yes | Applications that support connection-level asynchrony (required for Level 1) must support setting this attribute to SQL_TRUE by calling SQLSetConnectAttr; the attribute need not be settable to a value other than its default value through SQLSetStmtAttr. Applications that support statement-level asynchrony (required for Level 2) must support setting this attribute to SQL_TRUE using either function.| +| SQL_ATTR_ENABLE_AUTO_IPD | Level 2 | Yes || +| SQL_ATTR_FETCH_BOOKMARK_PTR | Level 2 | Yes | For Level 2 interface conformance, the driver must support SQL_CONCUR_READ_ONLY and at least one other value.| +| SQL_ATTR_IMP_PARAM_DESC | Core | Yes || +| SQL_ATTR_IMP_ROW_DESC | Core | Yes |For Level 1 interface conformance, the driver must support SQL_CURSOR_FORWARD_ONLY and at least one other value. For Level 2 interface conformance, the driver must support all values defined in this document.| +| SQL_ATTR_KEYSET_SIZE | Level 2 | Yes || +| SQL_ATTR_MAX_LENGTH | Level 1 | Yes || +| SQL_ATTR_MAX_ROWS | Level 1 | Yes || +| SQL_ATTR_METADATA_ID | Core | Yes || +| SQL_ATTR_NOSCAN | Core | Yes || +| SQL_ATTR_PARAM_BIND_OFFSET_PTR | Core | Yes || +| SQL_ATTR_PARAM_BIND_TYPE | Core | Yes || +| SQL_ATTR_PARAM_OPERATION_PTR | Core | Yes || +| SQL_ATTR_PARAM_STATUS_PTR | Core | Yes || +| SQL_ATTR_PARAMS_PROCESSED_PTR | Core | Yes || +| SQL_ATTR_PARAMSET_SIZE | Core | Yes || +| SQL_ATTR_QUERY_TIMEOUT | Level 2 | Yes || +| SQL_ATTR_RETRIEVE_DATA | Level 1 | Yes || +| SQL_ATTR_ROW_ARRAY_SIZE | Core | Yes || +| SQL_ATTR_ROW_BIND_OFFSET_PTR | Core | Yes || +| SQL_ATTR_ROW_BIND_TYPE | Core | Yes || +| SQL_ATTR_ROW_NUMBER | Level 1 | Yes || +| SQL_ATTR_ROW_OPERATION_PTR | Level 1 | Yes || +| SQL_ATTR_ROW_STATUS_PTR | Core | Yes || +| SQL_ATTR_ROWS_FETCHED_PTR | Core | Yes || +| SQL_ATTR_SIMULATE_CURSOR | Level 2 | Not || +| SQL_ATTR_USE_BOOKMARKS | Level 2 | Not || diff --git a/docs/reference/command_line_utilities/store-version.md b/docs/reference/command_line_utilities/store-version.md index 5882039c3d..0c7c85bf08 100755 --- a/docs/reference/command_line_utilities/store-version.md +++ b/docs/reference/command_line_utilities/store-version.md @@ -12,9 +12,9 @@ Prints information about the SnappyData product version. ```pre ./bin/snappy version -SnappyData Platform Version 1.1.0 +SnappyData Platform Version 1.1.1 SnappyData RowStore 1.6.3 - SnappyData Column Store 1.1.0 + SnappyData Column Store 1.1.1 ``` diff --git a/docs/reference/configuration_parameters/allow-explicit-commit.md b/docs/reference/configuration_parameters/allow-explicit-commit.md new file mode 100644 index 0000000000..835802fdb1 --- /dev/null +++ b/docs/reference/configuration_parameters/allow-explicit-commit.md @@ -0,0 +1,28 @@ +# allow-explicit-commit + +## Description +Using this property, you can specify whether to allow the execution of unsupported operations. Such operations may otherwise produce an error when you have set the JDBC autocommit to false using **java.sql.Connection#setAutoCommit** API. +If you set the autocommit to false, the operations in a column table produces an error as follows: +**Operations on column tables are not supported when query routing is disabled or autocommit is false**. +To allow such operations, set the **allow-explicit-commit** property to true. + +!!!Note + Although this property allows using the JDBC autocommit(false) and commit/rollback APIs, all of these are no-op with no change in the product behavior. This means that autocommit is always true even if the user sets it explicitly to false. + +This property is useful in scenarios as SQL client tools that may use transactions isolation levels (read committed / repeatable read) and explicitly set autocommit to false. In such cases, without this property, the SQL operations produce an error. + +## Example +This property can be used in connection URLs while connecting to SnappyData JDBC server. In such a case JDBC URL appears as follows: + +**jdbc:snappydata://locatoHostName:1527/allow-explicit-commit=true** + +This property can also be passed to **java.sql.DriverManager#getConnection(java.lang.String, java.util.Properties)** API in the properties object. + +## Default Value +false + +## Property Type +Connection + +## Prefix +NA \ No newline at end of file diff --git a/docs/reference/configuration_parameters/config_parameters.md b/docs/reference/configuration_parameters/config_parameters.md index b4c1d199a1..bcb9f27c00 100644 --- a/docs/reference/configuration_parameters/config_parameters.md +++ b/docs/reference/configuration_parameters/config_parameters.md @@ -68,6 +68,7 @@ Below is the list of all the configuration properties and links for each propert - [ack-severe-alert-threshold](ack-severe-alert-threshold.md) - [ack-wait-threshold](ack-wait-threshold.md) +- [allow-explicit-commit](allow-explicit-commit.md) - [archive-disk-space-limit](archive-disk-space-limit.md) diff --git a/docs/reference/configuration_parameters/load-balance.md b/docs/reference/configuration_parameters/load-balance.md index 371b6e77f5..dbc3a5ee5a 100644 --- a/docs/reference/configuration_parameters/load-balance.md +++ b/docs/reference/configuration_parameters/load-balance.md @@ -11,8 +11,10 @@ With the default value ("true") clients are automatically connected to a less-lo - When load balancing is enabled, clients may not be able to connect to a specific server even if they provide that server's unique port number for client connections. As a best practice, clients should always request connections using a locator address and port when load balancing is enabled. -With 1.1.0 release, the `load-balance` is set to **false**, by default, in the connection string. So now you must specify the connection details for a specific SnappyData member, other than a locator. -If you want to connect the JDBC client to a locator, then set this property to true. The locator then redirects the client to a less-loaded server with which the client makes the connection. +With 1.1.0 release, the load-balance is set to false by default, when you connect to a server's hostname:port. And it is set to true, when you connect to a locator's hostname:port. The locator then redirects the client to a less-loaded server with which the client makes the connection. With 1.1.1 release, same behavior is implemented for ODBC driver as well. + +!!! Note + You must specify load-balance=true in ODBC properties, if the locator address and port is provided. For example: diff --git a/docs/reference/inbuilt_system_procedures/dsid.md b/docs/reference/inbuilt_system_procedures/dsid.md index f7b54c02c5..c3faa9aeef 100644 --- a/docs/reference/inbuilt_system_procedures/dsid.md +++ b/docs/reference/inbuilt_system_procedures/dsid.md @@ -21,7 +21,9 @@ count(1) |DSID() 3 rows selected ``` +**Also see:** +* [Built-in System Procedures and Built-in Functions](system-procedures.md) diff --git a/docs/reference/inbuilt_system_procedures/dump-stacks.md b/docs/reference/inbuilt_system_procedures/dump-stacks.md index 27125c3c16..0492cc8e31 100755 --- a/docs/reference/inbuilt_system_procedures/dump-stacks.md +++ b/docs/reference/inbuilt_system_procedures/dump-stacks.md @@ -23,3 +23,6 @@ This command writes thread stack information only for the local SnappyData membe snappy> call sys.dump_stacks('false'); ``` +**Also see:** + +* [Built-in System Procedures and Built-in Functions](system-procedures.md) \ No newline at end of file diff --git a/docs/reference/inbuilt_system_procedures/rebalance-all-buckets.md b/docs/reference/inbuilt_system_procedures/rebalance-all-buckets.md index ccac7aea99..d0bd8ce18d 100755 --- a/docs/reference/inbuilt_system_procedures/rebalance-all-buckets.md +++ b/docs/reference/inbuilt_system_procedures/rebalance-all-buckets.md @@ -23,5 +23,7 @@ For efficiency, when starting multiple members, trigger the rebalance a single t ```pre snappy> call sys.rebalance_all_buckets(); ``` +**Also see:** +* [Built-in System Procedures and Built-in Functions](system-procedures.md) diff --git a/docs/reference/inbuilt_system_procedures/set-trace-flag.md b/docs/reference/inbuilt_system_procedures/set-trace-flag.md index d827a0ba21..58d7cf0831 100755 --- a/docs/reference/inbuilt_system_procedures/set-trace-flag.md +++ b/docs/reference/inbuilt_system_procedures/set-trace-flag.md @@ -25,4 +25,6 @@ This command traces all JAR installation, update, and removal operations in the snappy> call sys.set_trace_flag ('TraceJars', 'true'); ``` +**Also see:** +* [Built-in System Procedures and Built-in Functions](system-procedures.md) diff --git a/docs/reference/inbuilt_system_procedures/set_critical_heap_percentage.md b/docs/reference/inbuilt_system_procedures/set_critical_heap_percentage.md index b69c653b83..d2b6c1f7ef 100755 --- a/docs/reference/inbuilt_system_procedures/set_critical_heap_percentage.md +++ b/docs/reference/inbuilt_system_procedures/set_critical_heap_percentage.md @@ -23,4 +23,6 @@ This command sets the critical threshold for heap memory usage on the local Snap snappy>call sys.set_critical_heap_percentage (99.9); ``` +**Also see:** +* [Built-in System Procedures and Built-in Functions](system-procedures.md) diff --git a/docs/reference/inbuilt_system_procedures/sys_remove_metastore_entry.md b/docs/reference/inbuilt_system_procedures/sys_remove_metastore_entry.md new file mode 100644 index 0000000000..a4faed9d83 --- /dev/null +++ b/docs/reference/inbuilt_system_procedures/sys_remove_metastore_entry.md @@ -0,0 +1,46 @@ +# SYS.REMOVE_METASTORE_ENTRY + +This procedure drops a table from the external catalog if it exists (without checking that the table exists in the catalog). However, it does not handle related policies and base tables; hence they must be dropped separately. + +You must connect to the server from the Snappy shell, check the policies and base tables, and drop them separately. + +``` +// Viewing and dropping policies +SELECT * FROM SYS.SYSPOLICIES; +DROP POLICY ; +``` + +## Syntax + +``` +call sys.REMOVE_METASTORE_ENTRY('.', ''); +``` + +## Example + +Considering the case when cluster fails to come up, and the log mentions the pattern: *"AnalysisException: Table (dbName).(tableName) might be inconsistent in hive catalog. Use system procedure SYS.REMOVE_METASTORE_ENTRY to remove inconsistency"*, then use the following steps to resolve the issue. + +1. Check if the server is running: + + cd $SNAPPY_HOME + ./sbin/snappy-status-all.sh + +2. Launch Snappy shell and connect to the server: + + ./bin/snappy ; connect client ':'; + +3. Check and find if there are any policies on the table which caused catalog inconsistency and drop policies: + + SELECT * FROM SYS.SYSPOLICIES; + DROP POLICY ; + +4. Call the procedure to drop the table from the catalog: + + call sys.REMOVE_METASTORE_ENTRY('.', 'false'); + +5. Restart SnappyData cluster and check the status. The cluster starts successfully. + + +**Also see:** + +* [Built-in System Procedures and Built-in Functions](system-procedures.md) \ No newline at end of file diff --git a/docs/reference/inbuilt_system_procedures/system-procedures.md b/docs/reference/inbuilt_system_procedures/system-procedures.md index 3a73636274..949c39a78b 100755 --- a/docs/reference/inbuilt_system_procedures/system-procedures.md +++ b/docs/reference/inbuilt_system_procedures/system-procedures.md @@ -18,6 +18,7 @@ The following built-in procedures are available: * [SYS.SET_CRITICAL_HEAP_PERCENTAGE](set_critical_heap_percentage.md) * [SYS.SET_TRACE_FLAG](set-trace-flag.md) +* [SYS.REMOVE_METASTORE_ENTRY](sys_remove_metastore_entry.md) The following built-in function is available: diff --git a/docs/reference/interactive_commands/connect_client.md b/docs/reference/interactive_commands/connect_client.md index dac4d17f6b..d0c7792b2c 100755 --- a/docs/reference/interactive_commands/connect_client.md +++ b/docs/reference/interactive_commands/connect_client.md @@ -19,7 +19,7 @@ All further commands are processed against the new, current connection. ## Example ```pre -SnappyData version 1.1.0 +SnappyData version 1.1.1 snappy> connect client 'localhost:1527' as clientConnection; snappy> show connections; CLIENTCONNECTION* - jdbc:snappydata:thrift://localhost[1527] diff --git a/docs/reference/sql_functions/sql_functions.md b/docs/reference/sql_functions/sql_functions.md new file mode 100644 index 0000000000..72104c1037 --- /dev/null +++ b/docs/reference/sql_functions/sql_functions.md @@ -0,0 +1,4 @@ +# SQL Functions + + +Details of **SnappyData SQL Functions** can be found [here](http://snappydatainc.github.io/snappydata/sql_functions). \ No newline at end of file diff --git a/docs/reference/sql_general_limitations.md b/docs/reference/sql_general_limitations.md new file mode 100644 index 0000000000..a06cb805de --- /dev/null +++ b/docs/reference/sql_general_limitations.md @@ -0,0 +1,105 @@ +# Limitations + +The following SQL general limitations are observed in SnappyData: + +* [For row tables without primary key, DML operations that use Spark functions are not supported](#limitation1) +* [The syntax, `INSERT INTO <(col1,...)> values (… )`, cannot contain Spark functions in the values clause](#limitation2). +* [For complex data types (ARRAY, MAP, STRUCT), values to be inserted can not be directly used in the values clause of `INSERT INTO
values (x, y, z … )`](#limitation3) + + +##### For row tables without primary key, DML operations that use Spark functions are not supported + +!!!Note + This limitation applies only to row tables. For column tables such DML operations are supported. + +In the current release of SnappyData, row tables must contain a primary key column, if a DML operation on the table uses Spark function. For example, functions such as **spark_partition_id()**, **current_timestamp()**. + +In the following example, *table1* is a row table without primary key. As shown, in such a case **UPDATE** operations that use Spark functions produce an error: + + + snappy> create table table1(c1 int, c2 timestamp, c3 string) using row options (partition_by 'c1'); + snappy> insert into table1 values(1, '2019-07-22 14:29:22.432', 'value1'); + 1 row inserted/updated/deleted + snappy> insert into table1 values(2, '2019-07-22 14:29:22.432', 'value2'); + 1 row inserted/updated/deleted + + snappy> update table1 set c3 = 'value3' where SPARK_PARTITION_ID() = 6; + ERROR 42Y03: (SQLState=42Y03 Severity=20000) (Server=localhost/127.0.0.1[1528] Thread=ThriftProcessor-0) 'SPARK_PARTITION_ID)' is not recognized as a function or procedure. + + snappy> update table1 set c2 = current_timestamp() where c1 = 2; + ERROR 42X01: (SQLState=42X01 Severity=20000) (Server=localhost/127.0.0.1[1528] Thread=ThriftProcessor-0) Syntax error: Encountered "(" at line 1, column 41. + +However, if *table1* contains a primary key, then the DML operations are supported. In the following example, *table1* is now created with *column c1* as a primary key so the **UPDATE** operations succeed. + + snappy> create table table1(c1 int primary key, c2 timestamp, c3 string) using row options (partition_by 'c1'); + snappy> insert into table1 values(1, '2019-07-22 14:29:22.432', 'value1'); + 1 row inserted/updated/deleted + snappy> insert into table1 values(2, '2019-07-22 14:29:22.432', 'value2'); + 1 row inserted/updated/deleted + snappy> update table1 set c3 = 'value3' where SPARK_PARTITION_ID() = 6; + 1 row inserted/updated/deleted + snappy> update table1 set c2 = current_timestamp() where c1 = 2; + 1 row inserted/updated/deleted + snappy> select * from table1; + c1 |c2 |c3 + ------------------------------------------------------ + 1 |2019-07-22 14:29:22.432 |value1 + 2 |2019-07-22 14:36:47.879 |value3 + + 2 rows selected + + +##### The syntax, `INSERT INTO
<(col1,...)> values (… )`, cannot contain Spark functions in the values clause + +The value clause of `INSERT INTO
<(col1,...)> values (… ) `operation can not contain Spark functions. In such a case, use syntax `INSERT INTO
SELECT <>` syntax. + +In the following example, insert operation fails as **current_timestamp()** function is used in the values: + + + snappy> create table table1(c1 int, c2 timestamp, c3 string) using row options (partition_by 'c1'); + + snappy> insert into table1(c1, c2, c3) values(1, current_timestamp(), 'value1'); + ERROR 38000: (SQLState=38000 Severity=20000) (Server=localhost/127.0.0.1[1528] Thread=ThriftProcessor-0) The exception 'com.pivotal.gemfirexd.internal.engine.jdbc.GemFireXDRuntimeException: myID: 127.0.0.1(835):23712, caused by java.lang.AssertionError: assertion failed: No plan for DMLExternalTable INSERT INTO "TABLE1"("C1", "C2", "C3") values(1, current_timestamp(), 'value1') + +- LocalRelation [col1#42, col2#43, col3#44] + ' was thrown while evaluating an expression. + + +However, the following syntax works: + + snappy> insert into table1 select 1, current_timestamp(), 'value1'; + 1 row inserted/updated/deleted + snappy> select * from table1; + c1 |c2 |c3 + ------------------------------------------------------ + 1 |2019-07-22 14:49:20.022 |value1 + + 1 row selected + + +##### For complex data types (ARRAY, MAP, STRUCT), values to be inserted can not be directly used in the values clause of `INSERT INTO
values (x, y, z … )` + +To insert values using Snappy shell or a SQL client use `insert into
select` syntax. + +For example: + + # create a table with column of type MAP and insert few records + snappy> CREATE TABLE IF NOT EXISTS StudentGrades (rollno Integer, name String, Course Map) USING column; + snappy> INSERT INTO StudentGrades SELECT 1,'Jim', Map('English', 'A+'); + 1 row inserted/updated/deleted + # create a table with column of type ARRAY + snappy> CREATE TABLE IF NOT EXISTS Student(rollno Int, name String, marks Array) USING column; + snappy> INSERT INTO Student SELECT 1,'John', Array(97.8,85.2,63.9,45.2,75.2,96.5); + 1 row inserted/updated/deleted + # create a table with column of type STRUCT + snappy> CREATE TABLE IF NOT EXISTS StocksInfo (SYMBOL STRING, INFO STRUCT) USING COLUMN; + snappy> INSERT INTO StocksInfo SELECT 'ORD', STRUCT('2018', '400000', '112', '52'); + 1 row inserted/updated/deleted + + +The following syntax will produce an error: + + + snappy> insert into StudentGrades values (1, 'Jim',Map('English', 'A', 'Science', 'B')); + ERROR 42Y03: (SQLState=42Y03 Severity=20000) (Server=localhost/127.0.0.1[1529] Thread=ThriftProcessor-0) 'MAP(java.lang.String,java.lang.String,java.lang.String,java.lang.String)' is not recognized as a function or procedure. + +For more details on complex datatypes, refer to [Supported datatypes](misc/supported_datatypes.md) and [how to store and retrieve complex data types using ComplexTypeSerializer class](/howto/store_retrieve_complex_datatypes_JDBC.md). \ No newline at end of file diff --git a/docs/reference/sql_reference/create-table.md b/docs/reference/sql_reference/create-table.md index 7e6478f4f3..9a0625fc2f 100644 --- a/docs/reference/sql_reference/create-table.md +++ b/docs/reference/sql_reference/create-table.md @@ -272,7 +272,12 @@ With this alternate form of the CREATE TABLE statement, you specify the column n If no column names are specified for the new table, then all the columns in the result of the query expression are used to create same-named columns in the new table, of the corresponding data type(s). If one or more column names are specified for the new table, the same number of columns must be present in the result of the query expression; the data types of those columns are used for the corresponding columns of the new table. -Note that only the column names and data types from the queried table are used when creating the new table. Additional settings in the queried table, such as partitioning, replication, and persistence, are not duplicated. You can optionally specify partitioning, replication, and persistence configuration settings for the new table and those settings need not match the settings of the queried table. +!!!Note + Only the column names and data types from the queried table are used when creating the new table. Additional settings in the queried table, such as partitioning, replication, and persistence, are not duplicated. You can optionally specify partitioning, replication, and persistence configuration settings for the new table and those settings need not match the settings of the queried table. + +When you are creating a new table in SnappyData from another table, for example an external table, by using `CREATE TABLE ... AS SELECT * FROM ...` query and you find that the query fails with the message: `Syntax error or analysis exception: Table already exists ...`, it's likely due to insufficient memory on one of the servers and that the server is going down. +You may also see an entry for that table created when you run `show tables` command immediately after. +This happens because some tasks pertaining to the query may be still running on another server(s). As soon as those tasks are completed, the table gets cleaned up as expected because the query failed. ### Example: Create Table using Spark DataFrame API diff --git a/docs/reference/sql_reference/put-into.md b/docs/reference/sql_reference/put-into.md index 2f20214b87..54beeae7c3 100755 --- a/docs/reference/sql_reference/put-into.md +++ b/docs/reference/sql_reference/put-into.md @@ -22,6 +22,11 @@ PUT INTO .
SELECT * from .
; PUT INTO .
SELECT * from .
WHERE ='' PUT INTO .
SELECT from .
WHERE ='' +``` + + +``` +PUT INTO .
VALUES (V1, V2,... ,Vn); ``` @@ -29,7 +34,7 @@ PUT INTO .
SELECT from .
WHE **For SQL** -```pre +``` // Insert into another table using a select statement for column tables with key columns PUT INTO TRADE.CUSTOMERS SELECT '1','2','hello'; @@ -39,10 +44,13 @@ PUT INTO TRADE.NEWCUSTOMERS SELECT * from CUSTOMERS; PUT INTO TRADE.NEWCUSTOMERS SELECT * from CUSTOMERS WHERE C_NAME='User 1'; PUT INTO TRADE.NEWCUSTOMERS SELECT from CUSTOMERS WHERE C_NAME='User 1'; +``` + +```pre +PUT INTO TRADE.CUSTOMERS VALUES (1, 'User 1', '2001-10-12', 'SnappyData', 1); +PUT INTO TRADE.CUSTOMERS (CID ,CUST_NAME , ADDR ,TID) VALUES (1, 'User 1' , 'SnappyData', 1); ``` -!!! Warning - **PUT INTO VALUES (V1, V2,...Vn)** syntax do not work for column tables. Instead use **PUT INTO SELECT V1, V2,...,Vn**. **For API** diff --git a/docs/reference/sql_reference/undeploy.md b/docs/reference/sql_reference/undeploy.md index 12c3ffc221..edf9bae303 100644 --- a/docs/reference/sql_reference/undeploy.md +++ b/docs/reference/sql_reference/undeploy.md @@ -15,10 +15,6 @@ undeploy ; The command removes the jars that are directly installed and the jars that are associated with a package, from the system. -!!!Note - The removal is only captured when you use the **undeploy** command, the jars are removed only when the system restarts. - - ## Example diff --git a/docs/reference/sql_reference/update.md b/docs/reference/sql_reference/update.md index bea3732510..af8908ef70 100755 --- a/docs/reference/sql_reference/update.md +++ b/docs/reference/sql_reference/update.md @@ -3,11 +3,14 @@ Update the value of one or more columns. ```pre -{ UPDATE table-name [ [ AS ] correlation-name] - SET column-name = value - [, column-name = value} ]* - [ WHERE ] -} +UPDATE table-name +SET column-name = value +[, column-name = value ]* +[ WHERE predicate ] + +value: +expression | DEFAULT + ``` ## Description @@ -17,14 +20,48 @@ This form of the UPDATE statement is called a searched update. It updates the va The UPDATE statement returns the number of rows that were updated. !!! Note - - Updates on partitioning column and primary key column is not supported. + - Updates on partitioning columns and primary key columns are not supported. - Delete/Update with a subquery does not work with row tables, if the row table does not have a primary key. An exception to this rule is, if the subquery contains another row table and a simple where clause. -**value** +Implicit conversion of string to numeric value is not performed in an UPDATE statement when the string type expression is used as part of a binary arithmetic expression. -```pre -expression | DEFAULT +For example, the following SQL fails with AnalysisException. Here `age` is an `int` type column: + +``` +update users set age = age + '2' +``` + +As a workaround, you can apply an explicit cast, if the string is a number. + +For example: + +``` +update users set age = age + cast ('2' as int) +``` + +!!!Note + It is important to note that when you cast a non-numeric string, it results in a `NULL` value which is reflected in the table as a result of the update. This behavior of casting is inherited from Apache Spark, as Spark performs a fail-safe casting. For example, the following statement populates `age` column with `NULL` values: `update users set age = age + cast ('abc' as int)` + +Assigning a non-matching type expression to a column also fails with `AnalysisException`. However, the assignment is allowed in the following cases, even if the data type does not match: + +- assigning null value +- assigning narrower decimal to a wider decimal +- assigning narrower numeric type to wider numeric type as long as precision is +not compromised +- assigning of narrower numeric types to decimal type +- assigning expression of any data type to a string type column + +For example, the following statement fails with `AnalysisException`: + +``` +update users set age = '45' +``` + +Here also, the workaround is to cast the expression explicitly. For example, the following statement will pass: + +``` +update users set age = cast ('45' as int) ``` ## Example diff --git a/docs/security/specify_encrypt_passwords_conf_client.md b/docs/security/specify_encrypt_passwords_conf_client.md index bdbb2f537c..cd0a881812 100644 --- a/docs/security/specify_encrypt_passwords_conf_client.md +++ b/docs/security/specify_encrypt_passwords_conf_client.md @@ -34,7 +34,7 @@ Logs generated in /home/xyz//work/localhost-locator-1/sn SnappyData Locator pid: 2379 status: running Distributed system now has 1 members. Started DRDA server on: localhost/127.0.0.1[1527] -SnappyData version 1.1.0 +SnappyData version 1.1.1 snappy> Using CONNECTION0 snappy> ENCRYPTED_PASSWORD -------------------------------------------------------------------------------------------------------------------------------- diff --git a/docs/setting_up_odbc_driver-tableau_desktop.md b/docs/setting_up_odbc_driver-tableau_desktop.md index 5e6e05ada2..b6a89de0c7 100644 --- a/docs/setting_up_odbc_driver-tableau_desktop.md +++ b/docs/setting_up_odbc_driver-tableau_desktop.md @@ -9,9 +9,9 @@ ## Step 1: Install the TIBCO ComputeDB ODBC Driver -1. [Download the TIBCO ComputeDB 1.1.0 Enterprise Edition](install.md#download-snappydata). +1. [Download the TIBCO ComputeDB 1.1.1 Enterprise Edition](install.md#download-snappydata). -2. Click **ODBC INSTALLERS** to download the **TIB_compute-odbc_1.1.0_win.zip** file. +2. Click **ODBC INSTALLERS** to download the **TIB_compute-odbc_1.1.1_win.zip** file. 3. Follow [steps 1 and 2](howto/connect_using_odbc_driver.md) to install the TIBCO ComputeDB ODBC driver. diff --git a/docs/troubleshooting/troubleshooting_error_messages.md b/docs/troubleshooting/troubleshooting_error_messages.md index 68a9a888cf..920bc7ab83 100644 --- a/docs/troubleshooting/troubleshooting_error_messages.md +++ b/docs/troubleshooting/troubleshooting_error_messages.md @@ -17,6 +17,8 @@ The following topics are covered in this section: * [Node went down or data no longer available while iterating the results](#queryfailiterate) * [SmartConnector catalog is not up to date. Please reconstruct the Dataset and retry the operation.](#smartconnectorcatalog) * [Cannot parse config: String: 1: Expecting end of input or a comma, got ':'](#jobsubmitsnap) +* [java.lang.IllegalStateException: Detected both log4j-over-slf4j.jar AND bound slf4j-log4j12.jar on the class path, preempting StackOverflowError](#javalangillegal) +* [Bad `PutInto` performance even when input dataframe size is small.](#putintoperf) **Error Message:** @@ -205,5 +207,78 @@ To avoid this issue enclose the value containing colon `:` with double quotes so + + **Error Message:** + +java.lang.IllegalStateException: Detected both log4j-over-slf4j.jar AND bound slf4j-log4j12.jar on the class path, preempting StackOverflowError + + + **Diagnosis:**
+This error message can be seen if application uses SnappyData JDBC driver shadow jar and the application has a dependency on **log4j-over-slf4j** package/jar. This is because, the SnappyData JDBC driver +has a dependency on **slf4j-log4j12** which cannot co-exist with 'log4j-over-slf4j' package. +
+ + **Solution:**
+ +To avoid getting **log4j** and **slf4j-log4j12** dependencies in the driver, you can link the **non-fat** JDBC client jar (**snappydata-store-client*.jar**) in your application and exclude **log4j** and **slf4j-log4j12** dependencies from it. + +Note that the **snappydata-store-client** jar does not contain some of the SnappyData extensions (Scala imiplicits) that are required when SnappyData Spark-JDBC connector is used. That is when accessing SnappyData from another Spark cluster using JDBC dataframes as mentioned [here].(https://snappydatainc.github.io/snappydata/programming_guide/spark_jdbc_connector/#using-sql-dml-to-execute-ad-hoc-sql)). If these SnappyData extensions are to be used, then in addition to above mentioned jar, **snappydata-jdbc*-only.jar** dependency will be required. This is available on maven repo and can be accessed using classifier: 'only' along with snappydata-jdbc cordinates. + +Following is an example for adding this dependency using gradle: + +build.gradle example that uses **snappydata-store-client jar** and **snappydata-jdbc*only.jar**. The example uses 1.0.2.2 SnappyData version, replace it with the version required by the application. + +**Example** + +``` + + dependencies { + + compile group: 'io.snappydata', name: 'snappydata-jdbc_2.11', version: '1.0.2.2', classifier: 'only' + + // https://mvnrepository.com/artifact/io.snappydata/snappydata-store-client + // SnappyData "no-fat" JDBC jar + compile group: 'io.snappydata', name: 'snappydata-store-client', version: '1.6.2.1' + + // If users want to add his own 'log4j-over-slf4j' dependency + compile group: 'org.slf4j', name: 'log4j-over-slf4j', version: '1.7.26' +} + + // exclude the 'log4j' and 'slf4j-log4j12' dependencies + configurations.all { + exclude group: 'log4j', module: 'log4j' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + } + + +``` +
+ + + + **Error Message:** + +Bad `PutInto` performance even when input dataframe size is small. + + + **Diagnosis:**
+`PutInto` operation internally performs join. If the input dataframe size is small (less than `spark.sql.autoBroadcastJoinThreshold` which defaults to 10 MB) then the join should ideally result into broadcast join giving better performance than short merge join (which will be chose otherwise). + +However, some sources doesn't provide size statistics and in that case the size of dataframe results into value of `spark.sql.defaultSizeInBytes` property which defaults to `Long.MaxValue`. In this case even if the actual size of the input dataframe is less than that of `spark.sql.autoBroadcastJoinThreshold`, the `PutInto` operation will always end up using sort merge join resulting into poor performance. +
+ + **Solution:**
+Solution for this issue is to override default value of `spark.sql.defaultSizeInBytes` by setting it as part of session configuration with the value matching the approximate size of the actual input dataframe. + +The property can be set using the following SQL command: + +`set spark.sql.defaultSizeInBytes = ` + +For example, `set spark.sql.defaultSizeInBytes = 10000` + +Using a SnappySession instance this can be set as follows: +`snappySession.sql(“set spark.sql.defaultSizeInBytes = 10000”)` +Note that this is a session level property hence all the operation performed using the same session will end up using same overridden value. +
\ No newline at end of file diff --git a/docs/vsd/running_vsd.md b/docs/vsd/running_vsd.md index d8b6942f6e..b5e1935ee8 100755 --- a/docs/vsd/running_vsd.md +++ b/docs/vsd/running_vsd.md @@ -1,5 +1,7 @@ # Installing and Running VSD +This feature is available only in the Enterprise version of SnappyData.
+ Start the VSD tool, load statistics files, and maintain the view you want on your statistics. diff --git a/docs/vsd/system_performance.md b/docs/vsd/system_performance.md index a10919c5a1..cc1e93f90a 100755 --- a/docs/vsd/system_performance.md +++ b/docs/vsd/system_performance.md @@ -1,5 +1,7 @@ # Evaluating Statistics for the System +This feature is available only in the Enterprise version of SnappyData.
+ SnappyData provides statistics for analyzing system performance. Any member of a distributed system, including SnappyData servers, locators, and peer clients, can collect and archive this statistical data. SnappyData samples statistics at a configurable interval and writes them to an archive. The archives can be read at any time, including at runtime. diff --git a/docs/vsd/vsd-connection-stats.md b/docs/vsd/vsd-connection-stats.md index 7f7e54d10e..c4d4dd171e 100755 --- a/docs/vsd/vsd-connection-stats.md +++ b/docs/vsd/vsd-connection-stats.md @@ -1,5 +1,7 @@ # Client Connections +This feature is available only in the Enterprise version of SnappyData.
+ SnappyData provides several statistics to help you determine the frequency and duration of client connections to the distributed system. The following tables describe some commonly-used connection statistics. Examine the VSD output for information about additional connection statistics. @@ -37,4 +39,4 @@ The following tables describe some commonly-used connection statistics. Examine This figure illustrates how to select the connection statistics for graphing. -![](../Images/vsd/vsd-connection-stats.png) \ No newline at end of file +![](../Images/vsd/vsd-connection-stats.png) diff --git a/docs/vsd/vsd_cpu.md b/docs/vsd/vsd_cpu.md index 6a6b0051ee..69b7c0cf89 100755 --- a/docs/vsd/vsd_cpu.md +++ b/docs/vsd/vsd_cpu.md @@ -1,5 +1,7 @@ # CPU Usage +This feature is available only in the Enterprise version of SnappyData.
+ SnappyData provides host and JVM statistics for examining system load. An example follows the table. diff --git a/docs/vsd/vsd_memory.md b/docs/vsd/vsd_memory.md index 0e280c3541..df2c13f2e4 100755 --- a/docs/vsd/vsd_memory.md +++ b/docs/vsd/vsd_memory.md @@ -1,5 +1,7 @@ # Memory Usage +This feature is available only in the Enterprise version of SnappyData.
+ SnappyData provides statistics for system memory, JVM heap, garbage collection, and table sizes. You can use these statistics to analyze your application's memory usage. An example follows the table. diff --git a/docs/vsd/vsd_overview.md b/docs/vsd/vsd_overview.md index 300a5566f1..9bf6f7d7e4 100755 --- a/docs/vsd/vsd_overview.md +++ b/docs/vsd/vsd_overview.md @@ -1,5 +1,7 @@ # Using VSD to Analyze Statistics +This feature is available only in the Enterprise version of SnappyData.
+ The Visual Statistics Display (VSD) reads the sampled statistics from one or more archives and produces graphical displays for analysis. VSD is installed with SnappyData in the tools subdirectory. VSD’s extensive online help offers complete reference information about the tool. diff --git a/docs/vsd/vsd_statements.md b/docs/vsd/vsd_statements.md index 01898af965..7546804200 100755 --- a/docs/vsd/vsd_statements.md +++ b/docs/vsd/vsd_statements.md @@ -1,5 +1,7 @@ # SQL Statement Performance +This feature is available only in the Enterprise version of SnappyData.
+ With statement-level statistics enabled, statistics are available for all DML statements. An example follows the table. diff --git a/docs/vsd/vsd_tables.md b/docs/vsd/vsd_tables.md index fcc5f0167e..a91f77858d 100755 --- a/docs/vsd/vsd_tables.md +++ b/docs/vsd/vsd_tables.md @@ -1,5 +1,7 @@ # Table Performance +This feature is available only in the Enterprise version of SnappyData.
+ You can get an idea of the relative performance of inserts, updates, and selects by looking at the underlying statistics. An example follows the table. diff --git a/docs/vsd/vsd_transactions.md b/docs/vsd/vsd_transactions.md index 9f07bca077..e8fea0d075 100755 --- a/docs/vsd/vsd_transactions.md +++ b/docs/vsd/vsd_transactions.md @@ -1,5 +1,7 @@ # Transaction Performance +This feature is available only in the Enterprise version of SnappyData.
+ SnappyData provides statistics for transaction commits, rollbacks, and failures You can monitor SnappyData transactions with VSD. You can use these statistics to see the transaction rate. An example follows the table. diff --git a/mkdocs.yml b/mkdocs.yml index 73df6f6a74..9aecb9f33f 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -1,4 +1,4 @@ -site_name: SnappyData™ 1.1.0 +site_name: SnappyData™ 1.1.1 site_url: http://www.snappydata.io site_description: Project documentation for SnappyData site_author: SnappyData Team @@ -13,23 +13,24 @@ pages: - 'Provisioning SnappyData': 'install.md' - 'Provisioning SnappyData': - 'System Requirements': 'install/system_requirements.md' - - 'On-Premise': 'install/install_on_premise.md' - - 'Amazon Web Services (AWS)': 'install/setting_up_cluster_on_amazon_web_services.md' - - 'Kubernetes': 'kubernetes.md' - - 'Docker': 'quickstart/getting_started_with_docker_image.md' - - 'Building from Source': 'install/building_from_source.md' - - 'Upgrade Instructions': 'install/upgrade.md' -- 'Getting Started in 5 minutes or Less': 'quickstart.md' -- 'Getting Started in 5 minutes or Less': - - 'Getting Started with your Spark Distribution': 'quickstart/getting_started_with_your_spark_distribution.md' - - 'Getting Started with SnappyData On-Premise': 'quickstart/getting_started_by_installing_snappydata_on-premise.md' - - 'Getting Started with SnappyData on Kubernetes': 'quickstart/getting_started_on_kubernetes.md' - - 'Getting Started with SnappyData on Docker Image': 'quickstart/getting_started_with_docker_image.md' + - 'Installing SnappyData On-Premise': 'install/install_on_premise.md' + - 'Setup SnappyData Cluster on AWS': 'install/setting_up_cluster_on_amazon_web_services.md' + - 'Setup SnappyData Cluster on Kubernetes': 'kubernetes.md' + - 'Building a Docker Image with SnappyData Binaries': 'quickstart/getting_started_with_docker_image.md' + - 'Building SnappyData from Source files': 'install/building_from_source.md' + - 'Upgrading SnappyData': 'install/upgrade.md' +- 'Getting Started': 'quickstart.md' +- 'Getting Started': + - 'On-Premise': 'quickstart/getting_started_by_installing_snappydata_on-premise.md' + - 'Kubernetes': 'quickstart/getting_started_on_kubernetes.md' + - 'Docker Image': 'quickstart/getting_started_with_docker_image.md' + - 'Spark Distribution': 'quickstart/getting_started_with_your_spark_distribution.md' + - 'SnappyData Quickstart Guide': 'quickstart/snappydataquick_start.md' - 'Using Spark Scala APIs': 'quickstart/using_spark_scala_apis.md' - 'Benchmark 16-20x Faster Performance than Apache Spark': 'quickstart/performance_apache_spark.md' - 'Using SQL': 'quickstart/using_sql.md' - - 'Create SnappyData Cluster': 'quickstart/snappydataquick_start.md' - 'Structured Streaming': 'quickstart/structucture_streamingquickstart.md' +- 'Configure/Launch SnappyData Cluster': 'configuring_cluster/configure_launch_cluster.md' #- 'Migration Guide': 'migration/migration.md' #- 'Migration Guide': # - 'Manually Upgrading from GemFire XD 1.4.x to SnappyData RowStore 1.0': 'migration/upgrade_gemfirexd_standalone.md' @@ -40,6 +41,24 @@ pages: # 1.0 - 'Concepts': 'concepts.md' # 1.0 - 'Concepts': # 1.0 - 'Detecting and Handling Network Segmentation ("Split Brain")': 'concepts/network_partition.md' +- 'Concepts': 'install/concepts.md' +- 'Concepts': + - 'Core Components': 'architecture/core_components.md' + - 'SnappyData Cluster Architecture': 'architecture/cluster_architecture.md' + - 'Distributed Transactions': 'consistency/transactions_about.md' + - 'Distributed Transactions': + - 'How Transactions Work for Row Tables': 'consistency/using_transactions_row.md' + - 'Lock-free Queries using MVCC and Snapshot Isolation for Column Tables': 'consistency/using_snapshot_isolation_column.md' + - 'Affinity Modes': 'deployment.md' + - 'Affinity Modes': + - 'Local Mode': 'affinity_modes/local_mode.md' + - 'Embedded SnappyData Store Mode': 'affinity_modes/embedded_mode.md' + - 'SnappyData Smart Connector Mode': 'affinity_modes/connector_mode.md' + - 'Data Ingestion Pipeline': 'architecture/data_ingestion_pipeline.md' + - 'Hybrid Cluster Manager': 'architecture/hybrid_cluster_manager.md' +# - 'List of Properties': (to be updated later) +# - 'Property Details': 'configuring_cluster/property_details.md' + - 'Firewalls and Connections': 'configuring_cluster/firewalls_connections.md' - 'How Tos': 'howto.md' - 'How Tos': - 'How to Start a SnappyData Cluster': 'howto/start_snappy_cluster.md' @@ -71,19 +90,6 @@ pages: - 'How to Connect Tableau to SnappyData': 'howto/tableauconnect.md' - 'How to Use Apache Zeppelin with SnappyData': 'howto/use_apache_zeppelin_with_snappydata.md' - 'How to Configure Apache Zeppelin to Securely and Concurrently access the SnappyData Cluster': 'howto/concurrent_apache_zeppelin_access_to_secure_snappydata.md' -- 'Architecture': 'architecture.md' -- 'Architecture': - - 'Core Components': 'architecture/core_components.md' - - 'Data Ingestion Pipeline': 'architecture/data_ingestion_pipeline.md' - - 'Hybrid Cluster Manager': 'architecture/hybrid_cluster_manager.md' - - 'SnappyData Cluster Architecture': 'architecture/cluster_architecture.md' -- 'Configuring the Cluster': 'configuration.md' -- 'Configuring the Cluster': - - 'Configuration': 'configuring_cluster/configuring_cluster.md' - - 'List of Properties': 'configuring_cluster/property_description.md' -# - 'List of Properties': (to be updated later) -# - 'Property Details': 'configuring_cluster/property_details.md' - - 'Firewalls and Connections': 'configuring_cluster/firewalls_connections.md' - 'Programming Guide': 'programming_guide.md' - 'Programming Guide': - 'SparkSession, SnappySession and SnappyStreamingContext': 'programming_guide/sparksession_snappysession_and_snappystreamingcontext.md' @@ -99,15 +105,6 @@ pages: - 'Tables in SnappyData': 'programming_guide/tables_in_snappydata.md' - 'Stream Processing using SQL': 'programming_guide/stream_processing_using_sql.md' - 'User Defined Functions (UDF) and User Defined Aggregate Functions (UDAF)': 'programming_guide/udf_and_udaf.md' -- 'Distributed Transactions': 'consistency/transactions_about.md' -- 'Distributed Transactions': - - 'How Transactions Work for Row Tables': 'consistency/using_transactions_row.md' - - 'Lock-free Queries using MVCC and Snapshot Isolation for Column Tables': 'consistency/using_snapshot_isolation_column.md' -- 'Affinity Modes': 'deployment.md' -- 'Affinity Modes': - - 'Local Mode': 'affinity_modes/local_mode.md' - - 'Embedded SnappyData Store Mode': 'affinity_modes/embedded_mode.md' - - 'SnappyData Smart Connector Mode': 'affinity_modes/connector_mode.md' - 'Best Practices': 'best_practices.md' - 'Best Practices': - 'Tuning for Concurrency and Computation': 'best_practices/setup_cluster.md' @@ -165,6 +162,7 @@ pages: - 'CPU Usage': 'vsd/vsd_cpu.md' - 'Reference Guides': 'reference.md' - 'Reference Guides': + - 'Configure Cluster': 'configuring_cluster/configuring_cluster.md' - 'SQL Reference Guide': 'sql_reference.md' - 'SQL Reference Guide': - 'ALTER TABLE': 'reference/sql_reference/alter-table.md' @@ -205,12 +203,14 @@ pages: - 'SET SCHEMA': 'reference/sql_reference/set-schema.md' - 'TRUNCATE TABLE': 'reference/sql_reference/truncate-table.md' - 'UPDATE': 'reference/sql_reference/update.md' + - 'SQL General Limitations': 'reference/sql_general_limitations.md' - 'Built-in System Procedures and Built-in Functions': 'reference/inbuilt_system_procedures/system-procedures.md' - 'Built-in System Procedures and Built-in Functions': - 'DUMP_STACKS': 'reference/inbuilt_system_procedures/dump-stacks.md' - 'REBALANCE_ALL_BUCKETS': 'reference/inbuilt_system_procedures/rebalance-all-buckets.md' - 'SET_CRITICAL_HEAP_PERCENTAGE': 'reference/inbuilt_system_procedures/set_critical_heap_percentage.md' - 'SET_TRACE_FLAG': 'reference/inbuilt_system_procedures/set-trace-flag.md' + - 'SYS.REMOVE_METASTORE_ENTRY': 'reference/inbuilt_system_procedures/sys_remove_metastore_entry.md' - 'DSID': 'reference/inbuilt_system_procedures/dsid.md' - 'System Tables': 'reference/system_tables/system_tables.md' - 'System Tables': @@ -244,10 +244,12 @@ pages: - 'run': 'reference/interactive_commands/run.md' - 'set connection': 'reference/interactive_commands/set_connection.md' - 'show': 'reference/interactive_commands/show.md' + - 'List of Properties': 'configuring_cluster/property_description.md' - 'Configuration Parameters': 'reference/configuration_parameters/config_parameters.md' - 'Configuration Parameters': - 'ack-severe-alert-threshold': 'reference/configuration_parameters/ack-severe-alert-threshold.md' - 'ack-wait-threshold': 'reference/configuration_parameters/ack-wait-threshold.md' + - 'allow-explicit-commit': 'reference/configuration_parameters/allow-explicit-commit.md' - 'archive-disk-space-limit': 'reference/configuration_parameters/archive-disk-space-limit.md' - 'archive-file-size-limit': 'reference/configuration_parameters/archive-file-size-limit.md' - 'bind-address': 'reference/configuration_parameters/bind-address.md' @@ -299,6 +301,8 @@ pages: # - 'tcp-port': 'reference/configuration_parameters/tcp-port.md' - 'user': 'reference/configuration_parameters/user.md' - 'API Reference': 'apidocsintro.md' + - 'ODBC Supported API': 'reference/API_Reference/odbc_supported_apis.md' + - 'SQL Functions Reference': 'reference/sql_functions/sql_functions.md' - 'SnappyData Spark extension API Reference Guide': 'reference/API_Reference/apireference_guide.md' - 'Supported Data Types': 'reference/misc/supported_datatypes.md' - 'Configuring SSH Login without Password': 'reference/misc/passwordless_ssh.md' From 6759778049b276062e25cba7f61fa9c63eca765a Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 21 Aug 2019 21:46:54 +0530 Subject: [PATCH 22/26] Fix degradation in concurrency tests - avoid any hive client calls for setCurrentSchema if schema is already current one - store the list of inbuilt keys the first time in SnappyConf when doing case-insensitive search - forceably kill ChildVM if required to fix dangling ChildVM leading to thread-bombs making system unusable --- .../org/apache/spark/sql/SnappySession.scala | 8 +++++--- .../columnar/ExternalStoreUtils.scala | 2 ++ .../sql/hive/SnappyHiveExternalCatalog.scala | 5 +++++ .../sql/internal/SnappySessionCatalog.scala | 20 ++++++++++--------- .../apache/spark/sql/internal/session.scala | 14 ++++++++++--- .../test/dunit/standalone/DUnitLauncher.java | 9 ++++++--- .../test/dunit/standalone/ProcessManager.java | 15 ++++++++++++-- 7 files changed, 53 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index 6f563e26a3..46b6f5a90b 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -1563,10 +1563,12 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { */ private[sql] def setCurrentSchema(schema: String, createIfNotExists: Boolean): Unit = { val schemaName = sessionCatalog.formatDatabaseName(schema) - if (createIfNotExists) { - sessionCatalog.createSchema(schemaName, ignoreIfExists = true, createInStore = false) + if (schemaName != getCurrentSchema) { + if (createIfNotExists) { + sessionCatalog.createSchema(schemaName, ignoreIfExists = true, createInStore = false) + } + sessionCatalog.setCurrentSchema(schemaName, force = true) } - sessionCatalog.setCurrentSchema(schemaName) } def getCurrentSchema: String = sessionCatalog.getCurrentSchema diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala index fb12f4b4c0..809e8314de 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala @@ -170,6 +170,8 @@ object ExternalStoreUtils { override def get(k: String): Option[T] = baseMap.get(k.toLowerCase) + override def put(k: String, v: T): Option[T] = baseMap.put(k.toLowerCase, v) + override def remove(k: String): Option[T] = baseMap.remove(k.toLowerCase) override def iterator: Iterator[(String, T)] = baseMap.iterator diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala index 138acee53c..948ca74399 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala @@ -60,6 +60,7 @@ import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOL import org.apache.spark.sql.policy.PolicyProperties import org.apache.spark.sql.sources.JdbcExtendedUtils import org.apache.spark.sql.sources.JdbcExtendedUtils.normalizeSchema +import org.apache.spark.sql.store.CodeGeneration import org.apache.spark.sql.types.LongType class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, @@ -411,6 +412,10 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, withHiveExceptionHandling(super.dropTable(schema, table, ignoreIfNotExists = true, purge = true)) } + + SnappySession.clearAllCache(onlyQueryPlanCache = true) + CodeGeneration.clearAllCache() + invalidate(schema -> table) } override def dropTable(schema: String, table: String, ignoreIfNotExists: Boolean, diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala index 3ba0401fda..38b4d8ac7d 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionCatalog.scala @@ -92,7 +92,7 @@ class SnappySessionCatalog(val externalCatalog: SnappyExternalCatalog, } defaultName = formatDatabaseName(IdUtil.getUserAuthorizationId(defaultName).replace('-', '_')) createSchema(defaultName, ignoreIfExists = true) - setCurrentSchema(defaultName) + setCurrentSchema(defaultName, force = true) defaultName } @@ -497,19 +497,21 @@ class SnappySessionCatalog(val externalCatalog: SnappyExternalCatalog, } override def setCurrentDatabase(schema: String): Unit = - setCurrentSchema(formatDatabaseName(schema)) + setCurrentSchema(formatDatabaseName(schema), force = false) /** * Identical to [[setCurrentDatabase]] but assumes that the passed name * has already been formatted by a call to [[formatDatabaseName]]. */ - private[sql] def setCurrentSchema(schemaName: String): Unit = { - validateSchemaName(schemaName, checkForDefault = false) - super.setCurrentDatabase(schemaName) - externalCatalog.setCurrentDatabase(schemaName) - // no need to set the current schema in external hive metastore since the - // database may not exist and all calls to it will already ensure fully qualified - // table names + private[sql] def setCurrentSchema(schemaName: String, force: Boolean): Unit = { + if (force || schemaName != getCurrentSchema) { + validateSchemaName(schemaName, checkForDefault = false) + super.setCurrentDatabase(schemaName) + externalCatalog.setCurrentDatabase(schemaName) + // no need to set the current schema in external hive metastore since the + // database may not exist and all calls to it will already ensure fully qualified + // table names + } } override def getDatabaseMetadata(schema: String): CatalogDatabase = { diff --git a/core/src/main/scala/org/apache/spark/sql/internal/session.scala b/core/src/main/scala/org/apache/spark/sql/internal/session.scala index 7aa26fb3db..11b01de8b7 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/session.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/session.scala @@ -25,6 +25,7 @@ import scala.reflect.{ClassTag, classTag} import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.impl.jdbc.Util import com.pivotal.gemfirexd.internal.shared.common.reference.SQLState +import com.pivotal.gemfirexd.{Attribute => GAttr} import io.snappydata.sql.catalog.{CatalogObjectType, SnappyExternalCatalog} import io.snappydata.{Constant, Property} @@ -37,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, InsertIntoTab import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.ExternalStoreUtils +import org.apache.spark.sql.execution.columnar.ExternalStoreUtils.CaseInsensitiveMutableHashMap import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation, PreprocessTableInsertion} import org.apache.spark.sql.execution.{SecurityUtils, datasources} @@ -99,6 +101,12 @@ class SnappyConf(@transient val session: SnappySession) else math.min(super.numShufflePartitions, session.sparkContext.defaultParallelism) } + private lazy val allDefinedKeys = { + val map = new CaseInsensitiveMutableHashMap[String](Map.empty) + getAllDefinedConfs.foreach(e => map.put(e._1, e._1)) + map + } + private def keyUpdateActions(key: String, value: Option[Any], doSet: Boolean, search: Boolean = true): String = key match { // clear plan cache when some size related key that effects plans changes @@ -229,6 +237,8 @@ class SnappyConf(@transient val session: SnappySession) key } + case GAttr.USERNAME_ATTR | GAttr.USERNAME_ALT_ATTR | GAttr.PASSWORD_ATTR => key + case _ if key.startsWith("spark.sql.aqp.") => session.clearPlanCache() key @@ -236,9 +246,7 @@ class SnappyConf(@transient val session: SnappySession) case _ => // search case-insensitively for other keys if required if (search) { - getAllDefinedConfs.collectFirst { - case (k, _, _) if k.equalsIgnoreCase(key) => k - } match { + allDefinedKeys.get(key) match { case None => key case Some(k) => // execute keyUpdateActions again since it might be one of the pre-defined ones diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java index a8d706842b..78327b3f81 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/DUnitLauncher.java @@ -489,14 +489,17 @@ public void init(Registry registry, int numVMs) throws AccessException, RemoteEx @Override public VM getVM(int n) { - - if(n == DEBUGGING_VM_NUM) { + + if (n == LOCATOR_VM_NUM) { + return getLocator(); + } + if (n == DEBUGGING_VM_NUM) { //for ease of debugging, pass -1 to get the local VM return debuggingVM; } int oldVMCount = getVMCount(); - if(n >= oldVMCount) { + if(n >= oldVMCount && n < NUM_VMS) { //If we don't have a VM with that number, dynamically create it. try { for(int i = oldVMCount; i <= n; i++) { diff --git a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java index df1e74548f..4731c1e964 100644 --- a/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java +++ b/dunit/src/main/java/io/snappydata/test/dunit/standalone/ProcessManager.java @@ -50,6 +50,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.TimeUnit; import com.gemstone.gemfire.internal.FileUtil; import com.gemstone.gemfire.internal.shared.NativeCalls; @@ -269,9 +270,19 @@ public ProcessHolder(Process process, File workingDir) { } public void kill() { - this.killed = true; + this.killed = false; process.destroy(); - + try { + this.killed = process.waitFor(2, TimeUnit.SECONDS); + } catch (Exception ignored) { + } + if (!this.killed) { + process.destroyForcibly(); + try { + this.killed = process.waitFor(30, TimeUnit.SECONDS); + } catch (Exception ignored) { + } + } } public Process getProcess() { From a7295f4c7bde3d0e6c3f368c682d7f460d1e21ee Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 22 Aug 2019 19:14:44 +0530 Subject: [PATCH 23/26] Use old HashAggregate implementation for single groupBy key by default --- .../main/scala/io/snappydata/Literals.scala | 4 ++- .../aggregate/SnappyHashAggregateExec.scala | 30 ++++++++----------- .../sql/hive/SnappyHiveExternalCatalog.scala | 12 +++++++- store | 2 +- 4 files changed, 27 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/io/snappydata/Literals.scala b/core/src/main/scala/io/snappydata/Literals.scala index adfb070a41..86dc59180c 100644 --- a/core/src/main/scala/io/snappydata/Literals.scala +++ b/core/src/main/scala/io/snappydata/Literals.scala @@ -291,7 +291,9 @@ object Property extends Enumeration { val UseOptimizedHashAggregateForSingleKey: SQLValue[Boolean] = SQLVal[Boolean]( s"${Constant.PROPERTY_PREFIX}sql.useOptimizedHashAggregateForSingleKey", - "use ByteBufferMap based SnappyHashAggregateExec even for single string group by", + "Use the new ByteBufferMap based SnappyHashAggregateExec even for single column group by." + + "The default value is false since the older implementation is substantially faster " + + "for most of single column group by cases (except if number of groups is very large).", Some(false)) val ApproxMaxCapacityOfBBMap: SQLValue[Int] = SQLVal[Int]( diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index 5972fd9195..4e2e384e91 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -79,28 +79,22 @@ case class SnappyHashAggregateExec( hasDistinct: Boolean) extends NonRecursivePlans with UnaryExecNode with BatchConsumer { - val useByteBufferMapBasedAggregation: Boolean = { - val useDictionaryOptimizationForSingleKey = (groupingExpressions.length == 1 && - groupingExpressions.exists(_.dataType match { - case StringType => true - case _ => false - }) && !Property.UseOptimizedHashAggregateForSingleKey.get( - sqlContext.sparkSession.asInstanceOf[SnappySession].sessionState.conf)) + val conf = sqlContext.sparkSession.sessionState.conf + val useOldImplementationForSingleKey = groupingExpressions.length == 1 && + !Property.UseOptimizedHashAggregateForSingleKey.get(conf) aggregateBufferAttributes.forall(attr => TypeUtilities.isFixedWidth(attr.dataType)) && - Property.UseOptimzedHashAggregate.get( - sqlContext.sparkSession.asInstanceOf[SnappySession].sessionState.conf) && - !groupingExpressions.isEmpty && - groupingExpressions.forall(_.dataType. - existsRecursively(SHAMapAccessor.supportedDataTypes)) && - !useDictionaryOptimizationForSingleKey + Property.UseOptimzedHashAggregate.get(conf) && + groupingExpressions.nonEmpty && + groupingExpressions.forall(_.dataType. + existsRecursively(SHAMapAccessor.supportedDataTypes)) && + !useOldImplementationForSingleKey } - - - override def nodeName: String = "SnappyHashAggregate" + override def nodeName: String = + if (useByteBufferMapBasedAggregation) "BufferMapHashAggregate" else "SnappyHashAggregate" @transient def resultExpressions: Seq[NamedExpression] = __resultExpressions @@ -984,8 +978,8 @@ case class SnappyHashAggregateExec( // of loop, causing the TPCHD query to crash the server as the string positions // referenced in the map are no longer valid. Adding it in the list will // prevent single hashmap from being gced - $overflowHashMapsTerm = ${classOf[java.util.Collections].getName}.<$hashSetClassName>singletonList( - $hashMapTerm); + $overflowHashMapsTerm = ${classOf[java.util.Collections].getName}. + <$hashSetClassName>singletonList($hashMapTerm); $overflowMapIter = $overflowHashMapsTerm.iterator(); $overflowMapIter.next(); } diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala index 948ca74399..9e69eb4030 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala @@ -238,7 +238,17 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, } override def databaseExists(schema: String): Boolean = { - schema == SYS_SCHEMA || withHiveExceptionHandling(super.databaseExists(schema)) + if (schema == SYS_SCHEMA) true + else { + // if cache is small enough then linearly search in it since hive call is expensive + if (cachedCatalogTables.size() <= 200) { + val itr = cachedCatalogTables.asMap().keySet().iterator() + while (itr.hasNext) { + if (itr.next()._1 == schema) return true + } + } + withHiveExceptionHandling(super.databaseExists(schema)) + } } override def listDatabases(): Seq[String] = { diff --git a/store b/store index 2d8864c001..8a44bd053d 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 2d8864c00164167f5d8c021c263405411d540cb1 +Subproject commit 8a44bd053d9090664aa800c6669ab33c1c0cd801 From 6a60fd20ee058e9c35e1ed2e2e081f44659415af Mon Sep 17 00:00:00 2001 From: ahshahid Date: Fri, 23 Aug 2019 05:54:42 -0700 Subject: [PATCH 24/26] =?UTF-8?q?temporary=20fix=20for=20HashJoin=20on=20s?= =?UTF-8?q?ample=20table=20with=20non=20sample=20table=20not=20=E2=80=A6?= =?UTF-8?q?=20(#1437)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * temporary fix for HashJoin on sample table with non sample table not working. There is still a bug where by if the sample table is broadcasted for Hash Join , in my test I see 1 row missing. I suspect the reservoir region is not being broadcasted or something . so if the relation involves sample table in join it wont be broadcasted but will continue to be used in hashjoin. --- .../scala/org/apache/spark/sql/SnappyStrategies.scala | 11 +++++++++-- .../scala/org/apache/spark/sql/collection/Utils.scala | 4 ++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala index 8976cb2f1c..0b1fbcfc0e 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, Exchange, Sh import org.apache.spark.sql.execution.sources.PhysicalScan import org.apache.spark.sql.hive.SnappySessionState import org.apache.spark.sql.internal.{JoinQueryPlanning, LogicalPlanWithHints, SQLConf} +import org.apache.spark.sql.sources.SamplingRelation import org.apache.spark.sql.streaming._ /** @@ -366,8 +367,11 @@ private[sql] object JoinStrategy { * Matches a plan whose output should be small enough to be used in broadcast join. */ def canBroadcast(plan: LogicalPlan, conf: SQLConf): Boolean = { + plan.collectFirst { + case LogicalRelation(_: SamplingRelation, _, _) => true + }.isEmpty && ( plan.statistics.isBroadcastable || - plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold + plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold) } def getMaxHashJoinSize(conf: SQLConf): Long = { @@ -392,7 +396,10 @@ private[sql] object JoinStrategy { def allowsReplicatedJoin(plan: LogicalPlan): Boolean = { plan match { case PhysicalScan(_, _, child) => child match { - case LogicalRelation(t: PartitionedDataSourceScan, _, _) => !t.isPartitioned + case LogicalRelation(t: PartitionedDataSourceScan, _, _) => !t.isPartitioned && (t match { + case _: SamplingRelation => false + case _ => true + }) case _: Filter | _: Project | _: LocalLimit => allowsReplicatedJoin(child.children.head) case ExtractEquiJoinKeys(joinType, _, _, _, left, right) => allowsReplicatedJoin(left) && allowsReplicatedJoin(right) && diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index e6477a455b..ff521870f3 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -525,7 +525,7 @@ object Utils extends Logging { // if both baseTable and schema have been specified, then both // should have matching schema try { - val tablePlan = catalog.lookupRelation( + val tablePlan = catalog.resolveRelation( catalog.snappySession.tableIdentifier(baseTableName)) val tableSchema = tablePlan.schema if (catalog.compatibleSchema(tableSchema, s)) { @@ -552,7 +552,7 @@ object Utils extends Logging { try { // parquet and other such external tables may have different // schema representation so normalize the schema - val tablePlan = catalog.lookupRelation( + val tablePlan = catalog.resolveRelation( catalog.snappySession.tableIdentifier(baseTable)) (tablePlan.schema, Some(tablePlan)) } catch { From 748d7e57e0e78c49441465ea080d4f96585381a6 Mon Sep 17 00:00:00 2001 From: Shirish Deshmukh Date: Fri, 23 Aug 2019 19:03:44 +0530 Subject: [PATCH 25/26] Test for GitHub issue 1436 (#1438) * Test for GitHub issue 1436 --- .../PreparedQueryRoutingDUnitTest.scala | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala index 2f389ea428..a08fd48147 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/PreparedQueryRoutingDUnitTest.scala @@ -908,4 +908,37 @@ class PreparedQueryRoutingDUnitTest(val s: String) rs1.close() ps3.close() } + + def testPreparedStatementUnicodeBug(): Unit = { + serverHostPort = AvailablePortHelper.getRandomAvailableTCPPort + vm2.invoke(classOf[ClusterManagerTestBase], "startNetServer", serverHostPort) + logInfo(s"testPreparedStatementUnicodeBug: network server started at $serverHostPort") + val conn = DriverManager.getConnection( + "jdbc:snappydata://localhost:" + serverHostPort) + + logInfo(s"testPreparedStatementUnicodeBug: Connected to $serverHostPort") + val stmt1 = conn.createStatement() + + stmt1.execute("create table region (val string, description string) using column") + stmt1.execute("insert into region values ('粤' , 'unicode')") + stmt1.execute("insert into region values ('A', 'ascii')") + + var pstmt1 = conn.prepareStatement("select * from region where val = ?") + pstmt1.setString(1, "A") + var rs1 = pstmt1.executeQuery() + assert(rs1.next()) + pstmt1.close() + + pstmt1 = conn.prepareStatement("select * from region where val = ?") + pstmt1.setString(1, "粤") + rs1 = pstmt1.executeQuery() + assert(rs1.next()) + assert("粤" == rs1.getString(1) ) + + stmt1.execute("insert into region select '\u7ca5', 'unicode2'") + pstmt1.setString(1, "\u7ca5") + rs1 = pstmt1.executeQuery() + assert(rs1.next()) + assert("粥" == rs1.getString(1) ) + } } From 69af56451fbabdd8eeb7a833f66057b81a91887c Mon Sep 17 00:00:00 2001 From: Neeraj Kumar Date: Fri, 23 Aug 2019 19:07:02 +0530 Subject: [PATCH 26/26] Link with latest store --- store | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store b/store index 8a44bd053d..3f9f516a66 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 8a44bd053d9090664aa800c6669ab33c1c0cd801 +Subproject commit 3f9f516a66f84ca9cce998b25a3b2cdd34b63df2