From 611451ce5825b03716be68fa090c48c40d944e25 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sun, 23 Dec 2018 02:28:47 +0530 Subject: [PATCH 1/5] [SNAP-2790] check valid key_columns in table create (#1225) - call "pruneSchema" for key_columns that checks for presence of columns - fail if key_columns is specified for a row table - change Seq.empty to Nil --- .../cluster/DistributedIndexDUnitTest.scala | 6 +- .../cluster/SplitSnappyClusterDUnitTest.scala | 2 +- .../DynamicJarInstallationDUnitTest.scala | 6 +- .../udf/UserDefinedFunctionsDUnitTest.scala | 2 +- .../io/snappydata/ToolsCallbackImpl.scala | 6 +- .../io/snappydata/benchmark/snappy/TPCH.scala | 3 +- .../org/apache/spark/sql/SingleNodeTest.scala | 2 +- .../cluster/SplitClusterDUnitTestBase.scala | 5 +- .../scala/io/snappydata/ToolsCallback.scala | 8 +- .../org/apache/spark/sql/SnappyContext.scala | 11 +- .../{aqp => }/SnappyContextFunctions.scala | 4 +- .../apache/spark/sql/SnappyDDLParser.scala | 7 +- .../apache/spark/sql/collection/Utils.scala | 4 +- .../spark/sql/execution/SnappySortExec.scala | 2 +- .../spark/sql/execution/TableExec.scala | 4 +- .../columnar/ExternalStoreUtils.scala | 4 +- .../columnar/impl/ColumnFormatRelation.scala | 4 +- .../columnar/impl/DefaultSource.scala | 5 +- .../columnar/impl/StoreCallbacksImpl.scala | 2 +- .../org/apache/spark/sql/execution/ddl.scala | 10 +- .../sql/execution/joins/HashJoinExec.scala | 4 +- .../sql/execution/row/DefaultSource.scala | 2 +- .../sql/internal/ColumnTableBulkOps.scala | 2 +- .../sql/internal/SnappySessionState.scala | 1 - .../apache/spark/sql/store/StoreUtils.scala | 16 +- .../snappydata/ColumnUpdateDeleteTests.scala | 10 +- .../io/snappydata/SnappyTestRunner.scala | 6 +- .../spark/sql/store/ColumnTableTest.scala | 37 +- .../spark/sql/store/CreateIndexTest.scala | 12 +- .../spark/sql/store/SnappyCatalogSuite.scala | 2 +- docs/connectors/jdbc_streaming_connector.md | 2 +- docs/security/row_level_security.md | 16 +- .../scripts/cdcConnector/bulkDelete.sql | 18 +- .../createAndLoadCDCColTables.sql | 90 ++--- .../createAndLoadCDCMixedTables.sql | 110 +++--- .../createAndLoadCdcCol50Tables.sql | 100 ++--- .../cdcConnector/createCDCRowTables.sql | 264 ++++++------- .../cdcConnector/pointLookUpQueries.sql | 18 +- .../cdcConnector/scriptsForApp1/insert4.sql | 14 +- .../cdcConnector/scriptsForApp2/insert4.sql | 14 +- .../source_destination_tables.properties | 18 +- .../scripts/cdcConnector/tableNameList.txt | 16 +- .../createAndLoadColocatedTables.sql | 110 +++--- .../createAndLoadMixedTables.sql | 372 +++++++++--------- .../dropColocatedTablesInRightOrder.sql | 18 +- .../dropColocatedTablesInWrongOrder.sql | 18 +- .../hydra/cdcConnector/cdcConnector.bt | 12 +- .../hydra/clusterRecovery/clusterRecovery.bt | 12 +- .../snappydata/hydra/putInto/concPutInto.bt | 4 +- 49 files changed, 725 insertions(+), 690 deletions(-) rename core/src/main/scala/org/apache/spark/sql/{aqp => }/SnappyContextFunctions.scala (98%) diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala index 7a9ce1bd43..088165761f 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/DistributedIndexDUnitTest.scala @@ -114,7 +114,7 @@ class DistributedIndexDUnitTest(s: String) extends ClusterManagerTestBase(s) { // } // executeQ(s"select * from $tableName where col2 = 'aaa' ") { -// CreateIndexTest.validateIndex(Seq.empty, tableName)(_) +// CreateIndexTest.validateIndex(Nil, tableName)(_) // } executeQ(s"select * from $tableName where col2 = 'bbb' and col3 = 'halo' ") { @@ -164,7 +164,7 @@ class DistributedIndexDUnitTest(s: String) extends ClusterManagerTestBase(s) { // } // executeQ(s"select * from $tableName where col2 = 'aaa' ") { - // CreateIndexTest.validateIndex(Seq.empty, tableName)(_) + // CreateIndexTest.validateIndex(Nil, tableName)(_) // } System.setProperty("LOG-NOW", "xxx") @@ -228,7 +228,7 @@ class DistributedIndexDUnitTest(s: String) extends ClusterManagerTestBase(s) { // } // executeQ(s"select * from $tableName where col2 = 'aaa' ") { - // CreateIndexTest.validateIndex(Seq.empty, tableName)(_) + // CreateIndexTest.validateIndex(Nil, tableName)(_) // } System.setProperty("LOG-NOW", "xxx") diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 6127717cf3..772825e647 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -435,7 +435,7 @@ object SplitSnappyClusterDUnitTest snc.dropTable("splitModeTable1", ifExists = true) // recreate the dropped table - var expected = Seq.empty[ComplexData] + var expected: Seq[ComplexData] = Nil if (isComplex) { expected = createComplexTableUsingDataSourceAPI(snc, "splitModeTable1", tableType, props) diff --git a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala index 62053cb29b..debcad9efa 100644 --- a/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/DynamicJarInstallationDUnitTest.scala @@ -67,7 +67,7 @@ class DynamicJarInstallationDUnitTest(val s: String) var testJar = DynamicJarInstallationDUnitTest.createJarWithClasses( classNames = Seq("FakeJobClass", "FakeJobClass1"), toStringValue = "1", - Seq.empty, Seq.empty, + Nil, Nil, "testJar_SNAPPY_JOB_SERVER_JAR_%s.jar".format(System.currentTimeMillis())) var jobCompleted = false @@ -106,7 +106,7 @@ class DynamicJarInstallationDUnitTest(val s: String) testJar = DynamicJarInstallationDUnitTest.createJarWithClasses( classNames = Seq("FakeJobClass", "FakeJobClass1"), toStringValue = "2", - Seq.empty, Seq.empty, + Nil, Nil, "testJar_SNAPPY_JOB_SERVER_JAR_%s.jar".format(System.currentTimeMillis())) localProperty = (Seq("app1", DateTime.now) ++ Array[URL](testJar)).mkString(",") @@ -169,4 +169,4 @@ object DynamicJarInstallationDUnitTest { else false } } -} \ No newline at end of file +} 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 973a425329..d4421745b4 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 @@ -279,7 +279,7 @@ object UserDefinedFunctionsDUnitTest { def createUDFClass(name: String, code: String): File = { TestUtils.createCompiledClass(name, destDir, - getJavaSourceFromString(name, code), Seq.empty[URL]) + getJavaSourceFromString(name, code), Nil) } def createJarFile(files: Seq[File]): String = { diff --git a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala index e5418a2a2e..845321df61 100644 --- a/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala +++ b/cluster/src/main/scala/io/snappydata/ToolsCallbackImpl.scala @@ -118,12 +118,12 @@ object ToolsCallbackImpl extends ToolsCallback with Logging { } } - override def getAllGlobalCmnds(): Array[String] = { + override def getAllGlobalCmnds: Array[String] = { GemFireXDUtils.waitForNodeInitialization() Misc.getMemStore.getGlobalCmdRgn.values().toArray.map(_.asInstanceOf[String]) } - override def getGlobalCmndsSet(): java.util.Set[java.util.Map.Entry[String, String]] = { + override def getGlobalCmndsSet: java.util.Set[java.util.Map.Entry[String, String]] = { GemFireXDUtils.waitForNodeInitialization() Misc.getMemStore.getGlobalCmdRgn.entrySet() } @@ -146,7 +146,7 @@ object ToolsCallbackImpl extends ToolsCallback with Logging { } } - override def getLeadClassLoader(): URLClassLoader = { + override def getLeadClassLoader: URLClassLoader = { var ret: URLClassLoader = null val instance = ServiceManager.currentFabricServiceInstance instance match { 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 02b474fbe4..b651bb0ebf 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/TPCH.scala @@ -901,8 +901,7 @@ trait DynamicQueryGetter extends TPCHBase { Seq(args(i), args(i)) case (_, i) :: _ if i < args.length => Seq(args(i)) - case _ => - Seq.empty + case _ => Nil }).toList } 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 037bab1b61..b0d74ffd50 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/SingleNodeTest.scala @@ -103,7 +103,7 @@ object SingleNodeTest { case zp: ZippedPartitionsPartition => zp.partitionValues.map { case mb: MultiBucketExecutorPartition => mb.bucketsString } - case _ => Seq.empty + case _ => Nil } // each BucketExecutor must have only one bucket. diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index 4fbe03cd11..d951199642 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -341,7 +341,7 @@ trait SplitClusterDUnitTestObject extends Logging { // select the data from table created in embedded mode selectFromTable(snc, "embeddedModeTable2", 1005) - var expected = Seq.empty[ComplexData] + var expected: Seq[ComplexData] = Nil // create a table in split mode if (isComplex) { expected = createComplexTableUsingDataSourceAPI(snc, "splitModeTable1", @@ -433,8 +433,7 @@ trait SplitClusterDUnitTestObject extends Logging { } def selectFromTable(snc: SnappyContext, tableName: String, - expectedLength: Int, - expected: Seq[ComplexData] = Seq.empty[ComplexData]): Unit = { + expectedLength: Int, expected: Seq[ComplexData] = Nil): Unit = { val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect() assert(r.length == expectedLength, diff --git a/core/src/main/scala/io/snappydata/ToolsCallback.scala b/core/src/main/scala/io/snappydata/ToolsCallback.scala index 85e2c5a33e..739277ec5c 100644 --- a/core/src/main/scala/io/snappydata/ToolsCallback.scala +++ b/core/src/main/scala/io/snappydata/ToolsCallback.scala @@ -52,19 +52,19 @@ trait ToolsCallback { } def addURIs(alias: String, jars: Array[String], - deploySql: String, isPackage: Boolean = true): Unit + deploySql: String, isPackage: Boolean = true): Unit def addURIsToExecutorClassLoader(jars: Array[String]): Unit - def getAllGlobalCmnds(): Array[String] + def getAllGlobalCmnds: Array[String] - def getGlobalCmndsSet(): java.util.Set[java.util.Map.Entry[String, String]] + def getGlobalCmndsSet: java.util.Set[java.util.Map.Entry[String, String]] def removePackage(alias: String): Unit def setLeadClassLoader(): Unit - def getLeadClassLoader(): URLClassLoader + def getLeadClassLoader: URLClassLoader /** * Check permission to write to given schema for a user. Returns the normalized user or 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 d8892de2d4..61f76e8c6f 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -1093,10 +1093,13 @@ object SnappyContext extends Logging { if (ToolsCallbackInit.toolsCallback ne null) { SnappyContext.getClusterMode(sc) match { case _: SnappyEmbeddedMode => - val deployCmds = ToolsCallbackInit.toolsCallback.getAllGlobalCmnds() - // logInfo(s"deploycmnds size = ${deployCmds.size}") - // deployCmds.foreach(s => logDebug(s"s")) - deployCmds.foreach(d => { + 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 (nonEmpty) deployCmds.foreach(d => { val cmdFields = d.split('|') if (cmdFields.length > 1) { val coordinate = cmdFields(0) diff --git a/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala rename to core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala index 7e3a84b14d..a2e3107008 100644 --- a/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContextFunctions.scala @@ -14,13 +14,11 @@ * permissions and limitations under the License. See accompanying * LICENSE file. */ -package org.apache.spark.sql.aqp - +package org.apache.spark.sql import io.snappydata.sql.catalog.CatalogObjectType import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 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 4e2d4fc80f..7b7d31a990 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala @@ -314,11 +314,8 @@ abstract class SnappyDDLParser(session: SparkSession) applyTo: Seq[String], filterExp: Expression, filterStr: String) => { val applyToAll = applyTo.exists(_.equalsIgnoreCase( SnappyParserConsts.CURRENT_USER.upper)) - val expandedApplyTo = if (applyToAll) { - Seq.empty[String] - } else { - ExternalStoreUtils.getExpandedGranteesIterator(applyTo).toSeq - } + val expandedApplyTo = if (applyToAll) Nil + else ExternalStoreUtils.getExpandedGranteesIterator(applyTo).toSeq /* val targetRelation = snappySession.sessionState.catalog.lookupRelation(tableIdent) val isTargetExternalRelation = targetRelation.find(x => x match { 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 70106ce2f9..baab7912c9 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 @@ -310,7 +310,7 @@ object Utils { def mapExecutors[T: ClassTag](sc: SparkContext, f: () => Iterator[T], maxTries: Int = 30, - blockManagerIds: Seq[BlockManagerId] = Seq.empty): Array[T] = { + blockManagerIds: Seq[BlockManagerId] = Nil): Array[T] = { val cleanedF = sc.clean(f) mapExecutorsWithRetries(sc, (_: TaskContext, _: ExecutorLocalPartition) => cleanedF(), blockManagerIds, maxTries) @@ -319,7 +319,7 @@ object Utils { def mapExecutors[T: ClassTag](sc: SparkContext, f: (TaskContext, ExecutorLocalPartition) => Iterator[T], maxTries: Int): Array[T] = { val cleanedF = sc.clean(f) - mapExecutorsWithRetries(sc, cleanedF, Seq.empty[BlockManagerId], maxTries) + mapExecutorsWithRetries(sc, cleanedF, Nil, maxTries) } private def mapExecutorsWithRetries[T: ClassTag](sc: SparkContext, 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 2d85d056ea..989a92879f 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 @@ -77,7 +77,7 @@ case class SnappySortExec(sortPlan: SortExec, child: SparkPlan) }) } - override def usedInputs: AttributeSet = AttributeSet(Seq.empty) + override def usedInputs: AttributeSet = AttributeSet(Nil) override def inputRDDs(): Seq[RDD[InternalRow]] = child.asInstanceOf[CodegenSupport].inputRDDs() 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 99b7243213..04ee9b6092 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 @@ -114,7 +114,7 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { .region.asInstanceOf[PartitionedRegion] // if the two are different then its partition pruning case if (numBuckets == rdd.getNumPartitions) { - new DelegateRDD(sparkContext, rdd, Seq.empty[RDD[InternalRow]], + new DelegateRDD(sparkContext, rdd, Nil, Array.tabulate(numBuckets)( StoreUtils.getBucketPreferredLocations(region, _, forWrite = true))) } else rdd @@ -146,7 +146,7 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { // if the two are different then its partition pruning case if (numBuckets == rdd.getNumPartitions) { val table = relation.get.asInstanceOf[PartitionedDataSourceScan].table - new DelegateRDD(sparkContext, rdd, Seq.empty[RDD[InternalRow]], preferredLocations(table)) + new DelegateRDD(sparkContext, rdd, Nil, preferredLocations(table)) } else rdd } } 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 6f7af9ee27..5dc0cd8aec 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 @@ -492,11 +492,11 @@ object ExternalStoreUtils { * @return A Catalyst schema corresponding to columns in the given order. */ def pruneSchema(fieldMap: scala.collection.Map[String, StructField], - columns: Array[String]): StructType = { + columns: Array[String], columnType: String): StructType = { new StructType(columns.map { col => fieldMap.get(col) match { case None => throw new AnalysisException("Cannot resolve " + - s"""column name "$col" among (${fieldMap.keys.mkString(", ")})""") + s"""$columnType column name "$col" among (${fieldMap.keys.mkString(", ")})""") case Some(f) => f } }) 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 84882992fd..60157eb4ff 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 @@ -251,9 +251,7 @@ abstract class BaseColumnFormatRelation( val keyColsOptions = origOptions.get(ExternalStoreUtils.KEY_COLUMNS) if (keyColsOptions.isDefined) { keyColsOptions.get.split(",") - } else { - Seq.empty[String] - } + } else Nil } /** 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 dcbac37a4e..401753f10b 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 @@ -116,9 +116,7 @@ final class DefaultSource extends ExternalSchemaRelationProvider with SchemaRela val partitions = ExternalStoreUtils.getAndSetTotalPartitions(session, parameters, forManagedTable = true) - val partitioningColumns = StoreUtils.getAndSetPartitioningAndKeyColumns(session, parameters) - val tableOptions = new CaseInsensitiveMap(parameters.toMap) val parametersForShadowTable = new CaseInsensitiveMutableHashMap(parameters) // change the schema to use VARCHAR for StringType for partitioning columns @@ -134,6 +132,9 @@ final class DefaultSource extends ExternalSchemaRelationProvider with SchemaRela } else field }) } + val partitioningColumns = StoreUtils.getAndSetPartitioningAndKeyColumns(session, + schema, parameters) + val tableOptions = new CaseInsensitiveMap(parameters.toMap) val ddlExtension = StoreUtils.ddlExtensionString(parameters, isRowTable = false, isShadowTable = false) 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 9912fe166c..d1ac23c1e1 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 @@ -536,7 +536,7 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable } override def getLeadClassLoader: URLClassLoader = - ToolsCallbackInit.toolsCallback.getLeadClassLoader() + ToolsCallbackInit.toolsCallback.getLeadClassLoader override def clearSessionCache(onlyQueryPlanCache: Boolean = false): Unit = { SnappySession.clearAllCache(onlyQueryPlanCache) 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 477ef14be3..591bdde9c7 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 @@ -524,7 +524,7 @@ case class DeployCommand( val deployCmd = s"$coordinates|${repos.getOrElse("")}|${jarCache.getOrElse("")}" ToolsCallbackInit.toolsCallback.addURIs(alias, jars, deployCmd) } - Seq.empty[Row] + Nil } catch { case ex: Throwable => ex match { @@ -544,7 +544,7 @@ case class DeployCommand( if (lang.Boolean.parseBoolean(System.getProperty("FAIL_ON_JAR_UNAVAILABILITY", "true"))) { throw ex } - Seq.empty[Row] + Nil } else { throw ex } @@ -579,7 +579,7 @@ case class DeployJarCommand( RefreshMetadata.executeOnAll(sc, RefreshMetadata.ADD_URIS_TO_CLASSLOADER, uris) ToolsCallbackInit.toolsCallback.addURIs(alias, jars, paths, isPackage = false) } - Seq.empty[Row] + Nil } } @@ -591,7 +591,7 @@ case class ListPackageJarsCommand(isJar: Boolean) extends RunnableCommand { } override def run(sparkSession: SparkSession): Seq[Row] = { - val commands = ToolsCallbackInit.toolsCallback.getGlobalCmndsSet() + val commands = ToolsCallbackInit.toolsCallback.getGlobalCmndsSet val rows = new ArrayBuffer[Row] commands.forEach(new Consumer[Entry[String, String]] { override def accept(t: Entry[String, String]): Unit = { @@ -626,6 +626,6 @@ case class UnDeployCommand(alias: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { ToolsCallbackInit.toolsCallback.removePackage(alias) - Seq.empty[Row] + Nil } } 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 0fdd5d2db1..8d5bc8f37d 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 @@ -208,7 +208,7 @@ case class HashJoinExec(leftKeys: Seq[Expression], } (streamPlanRDDs, buildRDDs.map(rdd => new DelegateRDD[InternalRow]( - rdd.sparkContext, rdd, Seq.empty[RDD[InternalRow]], preferredLocations))) + rdd.sparkContext, rdd, Nil, preferredLocations))) } } @@ -270,7 +270,7 @@ case class HashJoinExec(leftKeys: Seq[Expression], } (streamPlanRDDs, buildRDDs.map(rdd => new DelegateRDD[InternalRow]( - rdd.sparkContext, rdd, Seq.empty[RDD[InternalRow]], preferredLocations))) + rdd.sparkContext, rdd, Nil, preferredLocations))) } } 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 6d93bb15df..863240fe12 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 @@ -97,7 +97,7 @@ final class DefaultSource extends ExternalSchemaRelationProvider with SchemaRela val fullTableName = ExternalStoreUtils.removeInternalProps(parameters) ExternalStoreUtils.getAndSetTotalPartitions(session, parameters, forManagedTable = true, forColumnTable = false) - StoreUtils.getAndSetPartitioningAndKeyColumns(session, parameters) + StoreUtils.getAndSetPartitioningAndKeyColumns(session, schema = null, parameters) val tableOptions = new CaseInsensitiveMap(parameters.toMap) val ddlExtension = StoreUtils.ddlExtensionString(parameters, isRowTable = true, isShadowTable = false) 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 188032a514..85943e69ef 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 @@ -66,7 +66,7 @@ object ColumnTableBulkOps { Property.PutIntoInnerJoinCacheSize.get(sparkSession.sqlContext.conf), Property.PutIntoInnerJoinCacheSize.name, -1, Long.MaxValue) - val updatePlan = Update(table, updateSubQuery, Seq.empty, + val updatePlan = Update(table, updateSubQuery, Nil, updateColumns, updateExpressions) val updateDS = new Dataset(sparkSession, updatePlan, RowEncoder(updatePlan.schema)) var analyzedUpdate = updateDS.queryExecution.analyzed.asInstanceOf[Update] diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala index 9ab5e0f18f..0a98815521 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala @@ -34,7 +34,6 @@ import io.snappydata.{Constant, Property} import org.apache.spark.internal.config.{ConfigBuilder, ConfigEntry, TypedConfigBuilder} import org.apache.spark.sql._ -import org.apache.spark.sql.aqp.SnappyContextFunctions import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.TypeCoercion.PromoteStrings import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubqueryAliases, NoSuchTableException, Star, UnresolvedRelation} 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 ba38f39ce5..3e5b19994a 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 @@ -321,7 +321,7 @@ object StoreUtils { val pkDisallowdTypes = Seq(StringType, BinaryType, ArrayType, MapType, StructType) - def getPrimaryKeyClause(parameters: mutable.Map[String, String], + def getPrimaryKeyClause(parameters: scala.collection.Map[String, String], schema: StructType): (String, Seq[StructField]) = { val sb = new StringBuilder() val stringPKCols = new mutable.ArrayBuffer[StructField](1) @@ -336,7 +336,7 @@ object StoreUtils { // since table creation can use case-insensitive in creation val normalizedCols = cols.map(Utils.toUpperCase) val prunedSchema = ExternalStoreUtils.pruneSchema(schemaFields, - normalizedCols) + normalizedCols, columnType = "partition") var includeInPK = true for (field <- prunedSchema.fields if includeInPK) { @@ -488,7 +488,7 @@ object StoreUtils { } def getAndSetPartitioningAndKeyColumns(session: SnappySession, - parameters: mutable.Map[String, String]): Seq[String] = { + schema: StructType, parameters: mutable.Map[String, String]): Seq[String] = { // parse the PARTITION_BY and KEYCOLUMNS and store the parsed result back in parameters // Use a new parser instance since parser may itself invoke DataSource.resolveRelation. @@ -496,9 +496,15 @@ object StoreUtils { val keyColumns = parameters.get(KEY_COLUMNS) match { case None => Nil case Some(k) => - val keyCols = k.split(",").map(parser.parseSQLOnly(_, parser.parseIdentifier.run())).toList + if (schema eq null) { // row table + throw new AnalysisException(s"$KEY_COLUMNS specified for a row table (use PRIMARY KEY)") + } + val keyCols = k.split(",").map(parser.parseSQLOnly(_, parser.parseIdentifier.run())) + // check for validity of columns + val schemaFields = Utils.schemaFields(schema) + ExternalStoreUtils.pruneSchema(schemaFields, keyCols, "key") parameters.put(KEY_COLUMNS, keyCols.mkString(",")) - keyCols + keyCols.toList } parameters.get(PARTITION_BY) match { case None => diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index c96a084ceb..ad5e4d5f0c 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -377,7 +377,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { var result = session.sql("select UnitPrice, tid from order_details where tid <> 6").collect() assert(result.length === numElements - 1) - assert(result.toSeq.filter(_.getDouble(0) != 1.0) === Seq.empty) + assert(result.toSeq.filter(_.getDouble(0) != 1.0) === Nil) result = session.sql("select UnitPrice from order_details where tid = 6").collect() assert(result.length === 1) @@ -390,10 +390,10 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { assert(result(0).getDouble(0) == 1.1) result = session.sql("select UnitPrice, tid from order_details where tid <> 6").collect() assert(result.length === numElements - 1) - assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Seq.empty) + assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Nil) result = session.sql("select UnitPrice, tid from order_details").collect() assert(result.length === numElements) - assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Seq.empty) + assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Nil) session.sql("UPDATE order_details SET UnitPrice = 1.1 WHERE tid <> 11") @@ -403,10 +403,10 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { assert(result(0).getDouble(0) == 1.1) result = session.sql("select UnitPrice, tid from order_details where tid <> 6").collect() assert(result.length === numElements - 1) - assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Seq.empty) + assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Nil) result = session.sql("select UnitPrice, tid from order_details").collect() assert(result.length === numElements) - assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Seq.empty) + assert(result.toSeq.filter(_.getDouble(0) != 1.1) === Nil) session.sql("drop table order_details") session.conf.unset(Property.ColumnBatchSize.name) diff --git a/core/src/test/scala/io/snappydata/SnappyTestRunner.scala b/core/src/test/scala/io/snappydata/SnappyTestRunner.scala index 4c4960fca1..abe1194e33 100644 --- a/core/src/test/scala/io/snappydata/SnappyTestRunner.scala +++ b/core/src/test/scala/io/snappydata/SnappyTestRunner.scala @@ -154,7 +154,7 @@ with Logging with Retries { } def Job(jobClass: String, lead: String, jarPath: String, - confs: Seq[String] = Seq.empty[String]): Unit = { + confs: Seq[String] = Nil): Unit = { val confStr = if (confs.size > 0) confs.foldLeft("")((r, c) => s"$r --conf $c") else "" @@ -230,7 +230,7 @@ with Logging with Retries { def SparkSubmit(name: String, appClass: String, master: Option[String], - confs: Seq[String] = Seq.empty[String], + confs: Seq[String] = Nil, appJar: String): Unit = { val sparkHost = InetAddress.getLocalHost.getHostName @@ -246,7 +246,7 @@ with Logging with Retries { } def RunExample(name: String, exampleClas: String, - args: Seq[String] = Seq.empty[String]): Unit = { + args: Seq[String] = Nil): Unit = { val argsStr = args.mkString(" ") val runExample = s"$snappyHome/bin/run-example $exampleClas $argsStr" val (out, err) = executeProcess(name, runExample) 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 e528ed8748..5abfc7c2f3 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 @@ -1403,8 +1403,9 @@ class ColumnTableTest snc.sql("drop table if exists test") } + test("Test method for getting key columns of the column tables") { - var session = new SnappySession(snc.sparkContext) + val session = new SnappySession(snc.sparkContext) session.sql("drop table if exists temp1") session.sql("drop table if exists temp2") session.sql("drop table if exists temp3") @@ -1420,6 +1421,40 @@ class ColumnTableTest "id2 bigint not null, id3 bigint not null) USING column " + "OPTIONS(key_columns 'id2,id1,id3' ) ") + // if key_columns are not present, then CREATE TABLE should fail (SNAP-2790) + try { + session.sql("create table ct1(id1 bigint not null , name1 varchar(10)) " + + "USING column OPTIONS(key_columns 'id')") + fail("should have failed") + } catch { + case _: AnalysisException => // expected + } + try { + session.sql("create table ct1(id1 bigint not null , name1 varchar(10)) " + + "USING column OPTIONS(partition_by 'id')") + fail("should have failed") + } catch { + case _: AnalysisException => // expected + } + try { + session.sql("create table ct1(id1 bigint not null , name1 varchar(10)) " + + "USING column OPTIONS(partition_by 'id1', key_columns 'id')") + fail("should have failed") + } catch { + case _: AnalysisException => // expected + } + // key_columns with row tables should fail + try { + session.sql("create table rt1(id1 bigint not null , name1 varchar(10)) " + + "USING row OPTIONS(key_columns 'id1')") + fail("should have failed") + } catch { + case _: AnalysisException => // expected + } + session.sql("create table ct1(id1 bigint not null , name1 varchar(10)) " + + "USING column OPTIONS(partition_by 'id1', key_columns 'id1')") + session.sql("drop table ct1") + val res1 = session.sessionCatalog.getKeyColumns("temp1") assert(res1.size == 1) 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 e7333fbc17..5b1b08e1ea 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 @@ -126,7 +126,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { } executeQ(s"select * from $tableName where col2 = 'aaa' ") { - CreateIndexTest.validateIndex(Seq.empty, tableName)(_) + CreateIndexTest.validateIndex(Nil, tableName)(_) } executeQ(s"select * from $tableName where col2 = 'bbb' and col3 = 'halo' ") { @@ -366,7 +366,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { executeQ(s"select t1.col2, t2.col3 from $table1 t1, $table2 t2 where t1.col2 = t2.col3 " + s"and t1.col3 = t2.col2 ") { - CreateIndexTest.validateIndex(Seq.empty, table1, table2)(_) + CreateIndexTest.validateIndex(Nil, table1, table2)(_) } executeQ(s"select t1.col2, t2.col3 from $table2 t1 join $table3 t2 on t1.col2 = t2.col2 " + @@ -386,7 +386,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { executeQ(s"select t1.col2, t2.col3 from $table1 t1 /*+ index( ) */ join $table3 t2 on t1.col2" + s" = t2.col2 and t1.col3 = t2.col3 ") { - CreateIndexTest.validateIndex(Seq.empty, table1, table3)(_) + CreateIndexTest.validateIndex(Nil, table1, table3)(_) } executeQ(s"select * from $table1 /*+ ${QueryHint.Index}($index1) */, $table3 " + @@ -405,7 +405,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { } executeQ(s"select * from $table1 tab1 join $table2 tab2 on tab1.col2 = tab2.col2") { - CreateIndexTest.validateIndex(Seq.empty, table1, table2)(_) + CreateIndexTest.validateIndex(Nil, table1, table2)(_) } try { @@ -603,7 +603,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { s"$table2 t2 where xx.col2 = t2.col2 and xx.col3 = t2.col3 " + s"and t1.col4 = xx.col5 ") { // t1 -> t4, t2 -> t4 - CreateIndexTest.validateIndex(Seq.empty, table1, table2, table4)(_) + CreateIndexTest.validateIndex(Nil, table1, table2, table4)(_) } executeQ(s"select t1.col2, t2.col3 from $table1 t1, $table4 t4, $rtable5 t5, $table2 t2 " + @@ -702,7 +702,7 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { val executeQ = CreateIndexTest.QueryExecutor(snContext, false, false) val selDF = executeQ(s"select * from $table1") { - CreateIndexTest.validateIndex(Seq.empty, s"$table1")(_) + CreateIndexTest.validateIndex(Nil, s"$table1")(_) } val baseRows = selDF.collect().toSet 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 b9fe20d503..1cc1756c98 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 @@ -421,7 +421,7 @@ abstract class CatalogTestUtils { } def newFunc(name: String, database: Option[String] = None): CatalogFunction = { - CatalogFunction(FunctionIdentifier(name, database), funcClass, Seq.empty[FunctionResource]) + CatalogFunction(FunctionIdentifier(name, database), funcClass, Nil) } /** diff --git a/docs/connectors/jdbc_streaming_connector.md b/docs/connectors/jdbc_streaming_connector.md index fcc95c2255..4641ecfb18 100644 --- a/docs/connectors/jdbc_streaming_connector.md +++ b/docs/connectors/jdbc_streaming_connector.md @@ -95,7 +95,7 @@ return reader.writeStream() .format("snappystore") .option("sink", Mysink.class.getName()) .option("checkpointLocation", - Utils.createTempDir("/data/wrk/w/snappydata/tmg-temp", "tmg-spark") + Utils.createTempDir("/data/wrk/w/snappydata/temp", "snappy-sink") .getCanonicalPath()) .option("tableName", tableName) .start(); diff --git a/docs/security/row_level_security.md b/docs/security/row_level_security.md index 3040e2dadb..8fb54f01b5 100644 --- a/docs/security/row_level_security.md +++ b/docs/security/row_level_security.md @@ -54,9 +54,9 @@ Initially all the users can view all the records in the table. You can restrict $ SELECT * FROM clients; id | account_name | account_manager ----+--------------+----------------- - 1 | jnj | tom - 2 | tmg | harris - 3 | tibco | greg + 1 | ibm | tom + 2 | apple | harris + 3 | msft | greg (3 rows) ``` @@ -114,7 +114,7 @@ Now the users are permitted to view the records of only those rows that are perm $ SELECT * FROM clients; id | account_name | account_manager ----+--------------+----------------- - 2 | tmg | tom + 2 | ibm | tom (1 row) ``` @@ -144,7 +144,7 @@ Here in the following example, multiple policies are created for the table named ``` CREATE POLICY mypolicy1 on mytable using user_col = current_user(); CREATE POLICY mypolicy2 on mytable using id < 4; -CREATE POLICY mypolicy3 on mytable using account_name = ‘tibco’; +CREATE POLICY mypolicy3 on mytable using account_name = ‘msft’; ALTER TABLE mytable ENABLE ROW LEVEL SECURITY; @@ -155,12 +155,12 @@ These policies are combined as shown in this example: SELECT * FROM mytable WHERE user_col = current_user() # current_user is AND id<4 -AND account_name = ‘tibco’; +AND account_name = ‘ibm’; $ select * from mytable; id | account_name | account_manager ----+--------------+----------------- - 3 | tibco | tom + 3 | ibm | tom (1 row) @@ -216,4 +216,4 @@ For example, DROP POLICY just_own_clients ``` !!!Caution - If you drop a table, all the policies associated with the table will also get dropped. \ No newline at end of file + If you drop a table, all the policies associated with the table will also get dropped. diff --git a/dtests/src/resources/scripts/cdcConnector/bulkDelete.sql b/dtests/src/resources/scripts/cdcConnector/bulkDelete.sql index bb752d95ab..43d56486ff 100644 --- a/dtests/src/resources/scripts/cdcConnector/bulkDelete.sql +++ b/dtests/src/resources/scripts/cdcConnector/bulkDelete.sql @@ -115,15 +115,15 @@ DELETE FROM [testdatabase].[dbo].[REFERENCE_DATA ] WHE DELETE FROM [testdatabase].[dbo].[RETURNED_MAIL ] WHERE RTRN_MAIL_ID > ? ; DELETE FROM [testdatabase].[dbo].[REVENUE_CODE ] WHERE REV_CD_ID > ? ; DELETE FROM [testdatabase].[dbo].[SERVICE_CODE ] WHERE SERV_CD_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_ACCOUNTING_CODES ] WHERE ACCOUNTING_CODE_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_CLIENT_MASTER ] WHERE CLIENT_REF_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_DENIAL_REASON_CONFIG ] WHERE DENIAL_REASON_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_DNIS_CONFIG ] WHERE DNIS_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_GROUP_CODE_CONFIG ] WHERE GROUP_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_KEY_GENERATOR ] WHERE KEY_GEN_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_PLAN_CODE_CONFIG ] WHERE PLAN_CODE_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS ] WHERE VARIABLE_ID > ? ; -DELETE FROM [testdatabase].[dbo].[TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS] WHERE SUBJ_CAT_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_ACCOUNTING_CODES ] WHERE ACCOUNTING_CODE_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_CLIENT_MASTER ] WHERE CLIENT_REF_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_DENIAL_REASON_CONFIG ] WHERE DENIAL_REASON_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_DNIS_CONFIG ] WHERE DNIS_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_GROUP_CODE_CONFIG ] WHERE GROUP_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_KEY_GENERATOR ] WHERE KEY_GEN_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_PLAN_CODE_CONFIG ] WHERE PLAN_CODE_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_PTMR_VARIABLE_TRANSLATIONS ] WHERE VARIABLE_ID > ? ; +DELETE FROM [testdatabase].[dbo].[SERVICES_SUBJECT_CATEGORY_TRANSLATIONS ] WHERE SUBJ_CAT_ID > ? ; DELETE FROM [testdatabase].[dbo].[TOPIC ] WHERE TPC_ID > ? ; DELETE FROM [testdatabase].[dbo].[TOPIC_COMMUNICATION ] WHERE TPC_INQ_ID > ? ; DELETE FROM [testdatabase].[dbo].[UM_ACTIVITY ] WHERE UM_ACTY_ID > ? ; diff --git a/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCColTables.sql b/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCColTables.sql index 07d7278733..528b0084f6 100644 --- a/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCColTables.sql +++ b/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCColTables.sql @@ -1015,75 +1015,75 @@ CREATE EXTERNAL TABLE staging_RECEIPT CREATE TABLE RECEIPT USING column OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,RCPT_ID ' ) AS (SELECT * FROM staging_RECEIPT); -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_PLAN_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_PLAN_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_PLAN_CODE_CONFIG USING column OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',key_columns 'PLAN_CODE_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_PLAN_CODE_CONFIG); +CREATE EXTERNAL TABLE staging_SERVICES_PLAN_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_PLAN_CODE_CONFIG USING column OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',key_columns 'PLAN_CODE_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_PLAN_CODE_CONFIG); -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS staging_TMGSERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS staging_SERVICES_KEY_GENERATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_KEY_GENERATOR - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_KEY_GENERATOR USING column OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',key_columns 'KEY_GEN_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_KEY_GENERATOR); +CREATE EXTERNAL TABLE staging_SERVICES_KEY_GENERATOR + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_KEY_GENERATOR USING column OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',key_columns 'KEY_GEN_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_KEY_GENERATOR); -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_GROUP_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_GROUP_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_GROUP_CODE_CONFIG USING column OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',key_columns 'GROUP_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_GROUP_CODE_CONFIG); +CREATE EXTERNAL TABLE staging_SERVICES_GROUP_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_GROUP_CODE_CONFIG USING column OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',key_columns 'GROUP_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_GROUP_CODE_CONFIG); -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DNIS_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_DNIS_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_DNIS_CONFIG USING column OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',key_columns ' DNIS_ID,CLIENT_ID' ) AS (SELECT * FROM staging_TMGSERVICES_DNIS_CONFIG); +CREATE EXTERNAL TABLE staging_SERVICES_DNIS_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_DNIS_CONFIG USING column OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',key_columns ' DNIS_ID,CLIENT_ID' ) AS (SELECT * FROM staging_SERVICES_DNIS_CONFIG); -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DENIAL_REASON_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_DENIAL_REASON_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_DENIAL_REASON_CONFIG USING column OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',key_columns 'DENIAL_REASON_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_DENIAL_REASON_CONFIG); +CREATE EXTERNAL TABLE staging_SERVICES_DENIAL_REASON_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_DENIAL_REASON_CONFIG USING column OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',key_columns 'DENIAL_REASON_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_DENIAL_REASON_CONFIG); -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS staging_TMGSERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS staging_SERVICES_CLIENT_MASTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_CLIENT_MASTER - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_CLIENT_MASTER USING column OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',key_columns 'CLIENT_REF_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_CLIENT_MASTER); +CREATE EXTERNAL TABLE staging_SERVICES_CLIENT_MASTER + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_CLIENT_MASTER USING column OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',key_columns 'CLIENT_REF_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_CLIENT_MASTER); -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS USING column OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',key_columns 'SUBJ_CAT_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS); +CREATE EXTERNAL TABLE staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_SUBJECT_CATEGORY_TRANSLATIONS USING column OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',key_columns 'SUBJ_CAT_ID,CLIENT_ID ' ) AS (SELECT * FROM staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS); -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS USING column OPTIONS(partition_by 'VARIABLE_ID', buckets '32',key_columns 'VARIABLE_ID ' ) AS (SELECT * FROM staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS); +CREATE EXTERNAL TABLE staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_PTMR_VARIABLE_TRANSLATIONS USING column OPTIONS(partition_by 'VARIABLE_ID', buckets '32',key_columns 'VARIABLE_ID ' ) AS (SELECT * FROM staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS); -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; -DROP TABLE IF EXISTS staging_TMGSERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS staging_SERVICES_ACCOUNTING_CODES; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE staging_TMGSERVICES_ACCOUNTING_CODES - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE TMGSERVICES_ACCOUNTING_CODES USING column OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',key_columns ' ACCOUNTING_CODE_ID,CLIENT_ID' ) AS (SELECT * FROM staging_TMGSERVICES_ACCOUNTING_CODES); +CREATE EXTERNAL TABLE staging_SERVICES_ACCOUNTING_CODES + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE SERVICES_ACCOUNTING_CODES USING column OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',key_columns ' ACCOUNTING_CODE_ID,CLIENT_ID' ) AS (SELECT * FROM staging_SERVICES_ACCOUNTING_CODES); DROP TABLE IF EXISTS UNAPPLIED_CASH; diff --git a/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCMixedTables.sql b/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCMixedTables.sql index a895d9d28c..07ef630d9c 100644 --- a/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCMixedTables.sql +++ b/dtests/src/resources/scripts/cdcConnector/createAndLoadCDCMixedTables.sql @@ -2520,12 +2520,12 @@ CREATE TABLE IF NOT EXISTS RECEIPT( ) USING row OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',redundancy '1'); INSERT INTO RECEIPT SELECT * FROM staging_RECEIPT; -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_PLAN_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PLAN_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PLAN_CODE_CONFIG ( PLAN_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2555,14 +2555,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (PLAN_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_PLAN_CODE_CONFIG; + INSERT INTO SERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_SERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS staging_TMGSERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS staging_SERVICES_KEY_GENERATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_KEY_GENERATOR - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_KEY_GENERATOR + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_KEY_GENERATOR ( KEY_GEN_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2580,14 +2580,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (KEY_GEN_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',redundancy '1' ); - INSERT INTO TMGSERVICES_KEY_GENERATOR SELECT * FROM staging_TMGSERVICES_KEY_GENERATOR; + INSERT INTO SERVICES_KEY_GENERATOR SELECT * FROM staging_SERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_GROUP_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); - CREATE TABLE IF NOT EXISTS TMGSERVICES_GROUP_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_GROUP_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + CREATE TABLE IF NOT EXISTS SERVICES_GROUP_CODE_CONFIG ( GROUP_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2606,14 +2606,14 @@ CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG PRIMARY KEY (GROUP_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_GROUP_CODE_CONFIG; + INSERT INTO SERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_SERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DNIS_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DNIS_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DNIS_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DNIS_CONFIG ( DNIS_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2632,14 +2632,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DNIS_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_DNIS_CONFIG SELECT * FROM staging_TMGSERVICES_DNIS_CONFIG; + INSERT INTO SERVICES_DNIS_CONFIG SELECT * FROM staging_SERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DENIAL_REASON_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DENIAL_REASON_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DENIAL_REASON_CONFIG ( DENIAL_REASON_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2656,14 +2656,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DENIAL_REASON_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_TMGSERVICES_DENIAL_REASON_CONFIG; + INSERT INTO SERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_SERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS staging_TMGSERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS staging_SERVICES_CLIENT_MASTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_CLIENT_MASTER - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_CLIENT_MASTER + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_CLIENT_MASTER ( CLIENT_REF_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2671,7 +2671,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER CLIENT_LEGACY_CD VARCHAR(5) NULL, CLIENT_NAME VARCHAR(10) NULL, MEMBER_ID_FORMAT VARCHAR(15) NULL, - TMG_CALL_CLIENT_CODE VARCHAR(10) NULL, + CALL_CLIENT_CODE VARCHAR(10) NULL, CREATE_date date NULL, UPDATED_date date NULL, USER_NAME VARCHAR(15) NULL, @@ -2681,14 +2681,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (CLIENT_REF_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_CLIENT_MASTER SELECT * FROM staging_TMGSERVICES_CLIENT_MASTER; + INSERT INTO SERVICES_CLIENT_MASTER SELECT * FROM staging_SERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS ( SUBJ_CAT_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2710,14 +2710,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (SUBJ_CAT_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; + INSERT INTO SERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS ( VARIABLE_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2735,14 +2735,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS PRIMARY KEY (VARIABLE_ID) ) USING row OPTIONS(partition_by 'VARIABLE_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; + INSERT INTO SERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; -DROP TABLE IF EXISTS staging_TMGSERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS staging_SERVICES_ACCOUNTING_CODES; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_ACCOUNTING_CODES - USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/TMGSERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_ACCOUNTING_CODES + USING com.databricks.spark.csv OPTIONS (path ':dataFilesLocationRow/SERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_ACCOUNTING_CODES ( ACCOUNTING_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2765,7 +2765,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (ACCOUNTING_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_ACCOUNTING_CODES SELECT * FROM staging_TMGSERVICES_ACCOUNTING_CODES; + INSERT INTO SERVICES_ACCOUNTING_CODES SELECT * FROM staging_SERVICES_ACCOUNTING_CODES; DROP TABLE IF EXISTS UNAPPLIED_CASH; DROP TABLE IF EXISTS staging_UNAPPLIED_CASHE; diff --git a/dtests/src/resources/scripts/cdcConnector/createAndLoadCdcCol50Tables.sql b/dtests/src/resources/scripts/cdcConnector/createAndLoadCdcCol50Tables.sql index 500ef1fc00..201caa4a77 100644 --- a/dtests/src/resources/scripts/cdcConnector/createAndLoadCdcCol50Tables.sql +++ b/dtests/src/resources/scripts/cdcConnector/createAndLoadCdcCol50Tables.sql @@ -1,206 +1,206 @@ CREATE EXTERNAL TABLE staging_PERSON_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/PERSON_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/PERSON_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE PERSON_EVENT USING column OPTIONS(partition_by 'PRSN_EVNT_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_EVNT_ID ') AS (SELECT * FROM staging_PERSON_EVENT); CREATE EXTERNAL TABLE staging_PERSON_EVENT_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/PERSON_EVENT_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/PERSON_EVENT_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE PERSON_EVENT_ATTRIBUTE USING column OPTIONS(partition_by 'PRSN_EVNT_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_EVNT_ID,PRSN_EVNT_ATTR_ID ') AS (SELECT * FROM staging_PERSON_EVENT_ATTRIBUTE); CREATE EXTERNAL TABLE staging_CLAIM_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_STATUS USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_STAT_ID ') AS (SELECT * FROM staging_CLAIM_STATUS); CREATE EXTERNAL TABLE staging_CLAIM_ADDITIONAL_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_ADDITIONAL_DIAGNOSIS USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_ADD_DIAG_ID ') AS (SELECT * FROM staging_CLAIM_ADDITIONAL_DIAGNOSIS); CREATE EXTERNAL TABLE staging_CLAIM_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_DETAIL USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_DTL_ID ') AS (SELECT * FROM staging_CLAIM_DETAIL); CREATE EXTERNAL TABLE staging_CLAIM_PAYMENT_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_PAYMENT_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_PAYMENT_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_PAYMENT_DETAIL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_PAY_ID,CLM_PAY_DTL_ID ') AS (SELECT * FROM staging_CLAIM_PAYMENT_DETAIL); CREATE EXTERNAL TABLE staging_CLAIM_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_ATTRIBUTE USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_ATTR_ID ') AS (SELECT * FROM staging_CLAIM_ATTRIBUTE); CREATE EXTERNAL TABLE staging_CLAIM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID ') AS (SELECT * FROM staging_CLAIM); CREATE EXTERNAL TABLE staging_PERSON_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/PERSON_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/PERSON_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE PERSON_CONTACT USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CNTC_ID,PRSN_CNTC_ID ') AS (SELECT * FROM staging_PERSON_CONTACT); CREATE EXTERNAL TABLE staging_ORGANIZATION_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/ORGANIZATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/ORGANIZATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE ORGANIZATION_CODE USING column OPTIONS(partition_by 'ORG_ID', buckets '32',key_columns 'CLIENT_ID,ORG_ID,CD_VAL_ID,ORG_CD_ID ') AS (SELECT * FROM staging_ORGANIZATION_CODE); CREATE EXTERNAL TABLE staging_COMPLAINT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/COMPLAINT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/COMPLAINT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE COMPLAINT_STATUS USING column OPTIONS(partition_by 'INQ_ID',buckets '32',key_columns 'CLIENT_ID,INQ_ID,COMPLAINT_ID,COMPLAINT_STAT_ID ' ) AS (SELECT * FROM staging_COMPLAINT_STATUS); CREATE EXTERNAL TABLE staging_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CONTACT USING column OPTIONS(partition_by 'CNTC_ID', buckets '32',key_columns 'CLIENT_ID,CNTC_ID' ) AS (SELECT * FROM staging_CONTACT); CREATE EXTERNAL TABLE staging_CLAIM_PAYMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CLAIM_PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CLAIM_PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_PAYMENT USING column OPTIONS(partition_by 'CLM_PAY_ID', buckets '32',key_columns 'CLIENT_ID,CLM_PAY_ID ' ) AS (SELECT * FROM staging_CLAIM_PAYMENT); CREATE EXTERNAL TABLE staging_TOPIC_COMMUNICATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/TOPIC_COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/TOPIC_COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE TOPIC_COMMUNICATION USING column OPTIONS(partition_by 'CMCN_INQ_ID', buckets '32',key_columns ' CLIENT_ID,CMCN_INQ_ID,TPC_INQ_ID,CMCN_ID,TPC_ID' ) AS (SELECT * FROM staging_TOPIC_COMMUNICATION); CREATE EXTERNAL TABLE staging_CONTACT_TYPE_CONTACT -USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CONTACT_TYPE_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CONTACT_TYPE_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CONTACT_TYPE_CONTACT USING column OPTIONS(partition_by 'CNTC_ID', buckets '32',key_columns 'CLIENT_ID,CNTC_ID,ORG_CNTC_TYP_ID,CNTC_TYP_CNTC_ID ' ) AS (SELECT * FROM staging_CONTACT_TYPE_CONTACT); CREATE EXTERNAL TABLE staging_TOPIC - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/TOPIC.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/TOPIC.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE TOPIC USING column OPTIONS(partition_by 'INQ_ID',buckets '32',key_columns 'CLIENT_ID,INQ_ID,TPC_ID ' ) AS (SELECT * FROM staging_TOPIC); ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE staging_LINE_ADDITIONAL_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/LINE_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/LINE_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE LINE_ADDITIONAL_DIAGNOSIS USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_ADD_DIAG_ID,LN_ADD_DIAG_ID ' ) AS (SELECT * FROM staging_LINE_ADDITIONAL_DIAGNOSIS); CREATE EXTERNAL TABLE staging_PROCEDURE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/PROCEDURE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/PROCEDURE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE PROCEDURE_CODE USING column OPTIONS(partition_by 'PR_CD_ID', buckets '32',key_columns 'CLIENT_ID,PR_CD_ID ' ) AS (SELECT * FROM staging_PROCEDURE_CODE); CREATE EXTERNAL TABLE staging_CODE_VALUE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/CODE_VALUE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/CODE_VALUE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CODE_VALUE USING column OPTIONS(partition_by 'CD_VAL_ID', buckets '32',key_columns 'CLIENT_ID,CD_VAL_ID ' ) AS (SELECT * FROM staging_CODE_VALUE); CREATE EXTERNAL TABLE staging_POSTAL_ADDRESS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/DataGenerator/POSTAL_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/DataGenerator/POSTAL_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE POSTAL_ADDRESS USING column OPTIONS(partition_by 'CNTC_ID',key_columns 'CLIENT_ID,CNTC_ID,PSTL_ADDR_ID') AS (SELECT * FROM staging_POSTAL_ADDRESS); ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE staging_ADJUSTMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ADJUSTMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ADJUSTMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE ADJUSTMENT USING column OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,ADJ_ID ' ) AS (SELECT * FROM staging_ADJUSTMENT); CREATE EXTERNAL TABLE staging_AGREEMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE AGREEMENT USING column OPTIONS(partition_by 'AGREE_ID', buckets '32',key_columns 'CLIENT_ID,AGREE_ID ' ) AS (SELECT * FROM staging_AGREEMENT); CREATE EXTERNAL TABLE staging_BANK_ACCOUNT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BANK_ACCOUNT USING column OPTIONS(partition_by 'BNK_ORG_ID',buckets '32',key_columns 'CLIENT_ID,BNK_ORG_ID,BNK_ID,BNK_ACCT_ID ' ) AS (SELECT * FROM staging_BANK_ACCOUNT); CREATE EXTERNAL TABLE staging_BANK - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BANK.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BANK.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BANK USING column OPTIONS(partition_by 'BNK_ORG_ID', buckets '32',key_columns 'CLIENT_ID,BNK_ORG_ID,BNK_ID ' ) AS (SELECT * FROM staging_BANK); CREATE EXTERNAL TABLE staging_BENEFIT_GROUP_NAME - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BENEFIT_GROUP_NAME USING column OPTIONS(partition_by 'GRP_ID', buckets '32',key_columns 'CLIENT_ID,GRP_ID,BENE_GRP_ID,BENE_GRP_NM_ID ' ) AS (SELECT * FROM staging_BENEFIT_GROUP_NAME); CREATE EXTERNAL TABLE staging_BENEFIT_GROUPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BENEFIT_GROUPS USING column OPTIONS(partition_by 'GRP_ID', buckets '32',key_columns 'CLIENT_ID,GRP_ID,BENE_PKG_ID,BENE_GRP_ID ' ) AS (SELECT * FROM staging_BENEFIT_GROUPS); CREATE EXTERNAL TABLE staging_BENEFIT_PACKAGE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BENEFIT_PACKAGE_ATTRIBUTE USING column OPTIONS(partition_by 'BENE_PKG_ID', buckets '32',key_columns 'CLIENT_ID,BENE_PKG_ID,BENE_PKG_ATTR_ID ' ) AS (SELECT * FROM staging_BENEFIT_PACKAGE_ATTRIBUTE); CREATE EXTERNAL TABLE staging_BENEFIT_PACKAGE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BENEFIT_PACKAGE USING column OPTIONS(partition_by 'BENE_PKG_ID', buckets '32',key_columns 'CLIENT_ID,BENE_PKG_ID' ) AS (SELECT * FROM staging_BENEFIT_PACKAGE); CREATE EXTERNAL TABLE staging_BENEFIT_PACKAGE_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BENEFIT_PACKAGE_RELATION USING column OPTIONS(partition_by 'BENE_PKG_ID', buckets '32',key_columns 'CLIENT_ID,BENE_PKG_ID,PKG_RELN_ID ' ) AS (SELECT * FROM staging_BENEFIT_PACKAGE_RELATION); CREATE EXTERNAL TABLE staging_BILLING_ENTITY_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_ENTITY_CONTACT USING column OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,CNTC_ID,BILL_ENT_CNTC_ID ' ) AS (SELECT * FROM staging_BILLING_ENTITY_CONTACT); CREATE EXTERNAL TABLE staging_BILLING_ENTITY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_ENTITY USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID') AS (SELECT * FROM staging_BILLING_ENTITY); CREATE EXTERNAL TABLE staging_BILLING_ENTITY_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_ENTITY_DETAIL USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID ' ) AS (SELECT * FROM staging_BILLING_ENTITY_DETAIL); CREATE EXTERNAL TABLE staging_BILLING_ENTITY_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_ENTITY_SCHEDULE USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,BILL_SCHD_ID,BILL_ENT_SCHD_ID ' ) AS (SELECT * FROM staging_BILLING_ENTITY_SCHEDULE); CREATE EXTERNAL TABLE staging_BILLING_RECONCILIATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_RECONCILIATION USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,BILL_RECON_ID ' ) AS (SELECT * FROM staging_BILLING_RECONCILIATION); CREATE EXTERNAL TABLE staging_BILLING_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_SCHEDULE USING column OPTIONS(partition_by 'BILL_SCHD_ID', buckets '32',key_columns 'CLIENT_ID,BILL_SCHD_ID ' ) AS (SELECT * FROM staging_BILLING_SCHEDULE); CREATE EXTERNAL TABLE staging_BILLING_SOURCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE BILLING_SOURCE USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,SRC_TYP_REF_ID,BILL_SRC_ID ' ) AS (SELECT * FROM staging_BILLING_SOURCE); CREATE EXTERNAL TABLE staging_CHARGE_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CHARGE_ITEM USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',key_columns 'CLIENT_ID,BILL_ENT_ID,BILL_ENT_SCHD_ID,CHRG_ITM_ID ' ) AS (SELECT * FROM staging_CHARGE_ITEME); CREATE EXTERNAL TABLE staging_CHECKS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CHECKS USING column OPTIONS(partition_by 'CLM_PAY_ID', buckets '32',key_columns 'CLIENT_ID,CLM_PAY_ID,CHK_ID ' ) AS (SELECT * FROM staging_CHECKS); CREATE EXTERNAL TABLE staging_CHECK_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CHECK_STATUS USING column OPTIONS(partition_by 'CLM_PAY_ID',buckets '32',key_columns 'CLIENT_ID,CLM_PAY_ID,CHK_ID,CHK_STAT_ID ' ) AS (SELECT * FROM staging_CHECK_STATUS); CREATE EXTERNAL TABLE staging_CLAIM_COB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_COB USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_COB_ID ' ) AS (SELECT * FROM staging_CLAIM_COB); CREATE EXTERNAL TABLE staging_CLAIM_COSHARE_TRACKING - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_COSHARE_TRACKING USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLAIM_ID,LINE_NO ' ) AS (SELECT * FROM staging_CLAIM_COSHARE_TRACKING); CREATE EXTERNAL TABLE staging_CLAIM_HOSPITAL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_HOSPITAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_HOSPITAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_HOSPITAL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_HOSP_ID ' ) AS (SELECT * FROM staging_CLAIM_HOSPITAL); CREATE EXTERNAL TABLE staging_CLAIM_LINE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_LINE_ATTRIBUTE USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_DTL_ID,CLM_LN_ATTR_ID ' ) AS (SELECT * FROM staging_CLAIM_LINE_ATTRIBUTE); CREATE EXTERNAL TABLE staging_CLAIM_PAYMENT_REDUCTION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_PAYMENT_REDUCTION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_PAYMENT_REDUCTION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_PAYMENT_REDUCTION USING column OPTIONS(partition_by 'CLM_PAY_RDCTN_ID', buckets '32',key_columns 'CLIENT_ID,CLM_PAY_RDCTN_ID ' ) AS (SELECT * FROM staging_CLAIM_PAYMENT_REDUCTION); CREATE EXTERNAL TABLE staging_CLAIM_REDUCTION_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_REDUCTION_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_REDUCTION_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_REDUCTION_DETAIL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_PAY_RDCTN_ID,CLM_RDCTN_DTL_ID ' ) AS (SELECT * FROM staging_CLAIM_REDUCTION_DETAIL); CREATE EXTERNAL TABLE staging_CLAIM_REDUCTION_HISTORY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_REDUCTION_HISTORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_REDUCTION_HISTORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLAIM_REDUCTION_HISTORY USING column OPTIONS(partition_by 'CLM_PAY_RDCTN_ID', buckets '32',key_columns 'CLIENT_ID,CLM_PAY_RDCTN_ID,CLM_RDCTN_HIST_ID ' ) AS (SELECT * FROM staging_CLAIM_REDUCTION_HISTORY); CREATE EXTERNAL TABLE staging_CLIENT_REFERENCE_DATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLIENT_REFERENCE_DATA USING column OPTIONS(partition_by 'CLIENT_REF_DTA_ID', buckets '32',key_columns 'CLIENT_ID,CLIENT_REF_DTA_ID ' ) AS (SELECT * FROM staging_CLIENT_REFERENCE_DATA); CREATE EXTERNAL TABLE staging_CLIENTS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLIENTS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLIENTS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE CLIENTS USING column OPTIONS(partition_by 'CLIENT_ID', buckets '32',key_columns 'CLIENT_ID ' ) AS (SELECT * FROM staging_CLIENTS); CREATE EXTERNAL TABLE staging_COB_CLAIM_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE COB_CLAIM_DIAGNOSIS USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,PRSN_COB_ID,REC_ORD ' ) AS (SELECT * FROM staging_COB_CLAIM_DIAGNOSIS); CREATE EXTERNAL TABLE staging_COB_ORGANIZATION_PERSON - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE COB_ORGANIZATION_PERSON USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',key_columns 'CLIENT_ID,PRSN_ID,PRSN_COB_ID,ORG_PRSN_TYP_REF_ID ' ) AS (SELECT * FROM staging_COB_ORGANIZATION_PERSON); diff --git a/dtests/src/resources/scripts/cdcConnector/createCDCRowTables.sql b/dtests/src/resources/scripts/cdcConnector/createCDCRowTables.sql index 584d1f2381..189bec01f9 100644 --- a/dtests/src/resources/scripts/cdcConnector/createCDCRowTables.sql +++ b/dtests/src/resources/scripts/cdcConnector/createCDCRowTables.sql @@ -1,5 +1,5 @@ CREATE EXTERNAL TABLE IF NOT EXISTS staging_INQUIRY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/INQUIRY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/INQUIRY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INQUIRY( INQ_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -15,7 +15,7 @@ USING row OPTIONS(partition_by 'INQ_ID', buckets '32',redundancy '1'); INSERT INTO INQUIRY SELECT * FROM staging_INQUIRY; CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ORGANIZATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ORGANIZATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATIONS(ORG_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -35,7 +35,7 @@ USING row OPTIONS(partition_by 'ORG_ID', buckets '32',redundancy '1'); INSERT INTO ORGANIZATIONS SELECT * FROM staging_ORGANIZATIONS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_REVIEW - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_REVIEW(UM_RVW_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -82,7 +82,7 @@ INSERT INTO UM_REVIEW SELECT * FROM staging_UM_REVIEW; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/INVOICE_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/INVOICE_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE_DETAIL( INV_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -109,7 +109,7 @@ INSERT INTO INVOICE_DETAIL SELECT * FROM staging_INVOICE_DETAIL; ---- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_GROUP_NAME - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_GROUP_NAME( GRP_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -131,7 +131,7 @@ CREATE TABLE IF NOT EXISTS BENEFIT_GROUP_NAME( ---- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_GROUPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_GROUPS( GRP_ID BIGINT NOT NULL, BENE_PKG_ID BIGINT NOT NULL, @@ -152,7 +152,7 @@ CREATE TABLE IF NOT EXISTS BENEFIT_GROUPS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_RECONCILIATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_RECONCILIATION( BILL_ENT_ID BIGINT NOT NULL, BILL_RECON_ID BIGINT NOT NULL, @@ -175,7 +175,7 @@ CREATE TABLE IF NOT EXISTS BILLING_RECONCILIATION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHECKS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHECKS( CLM_PAY_ID BIGINT NOT NULL, CHK_ID BIGINT NOT NULL, @@ -199,7 +199,7 @@ CREATE TABLE IF NOT EXISTS CHECKS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLIENT_REFERENCE_DATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLIENT_REFERENCE_DATA( CLIENT_REF_DTA_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -220,7 +220,7 @@ CREATE TABLE IF NOT EXISTS CLIENT_REFERENCE_DATA( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_COB_CLAIM_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COB_CLAIM_DIAGNOSIS( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -240,7 +240,7 @@ CREATE TABLE IF NOT EXISTS COB_CLAIM_DIAGNOSIS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_COB_ORGANIZATION_PERSON - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COB_ORGANIZATION_PERSON( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -271,7 +271,7 @@ INSERT INTO COB_ORGANIZATION_PERSON SELECT * FROM staging_COB_ORGANIZATION_PERSO ---- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_DIAGNOSIS_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/DIAGNOSIS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/DIAGNOSIS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS DIAGNOSIS_CODE( DIAG_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -295,7 +295,7 @@ CREATE TABLE IF NOT EXISTS DIAGNOSIS_CODE( ---- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ELECTRONIC_ADDRESS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ELECTRONIC_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ELECTRONIC_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ELECTRONIC_ADDRESS( CNTC_ID BIGINT NOT NULL, ELEC_ADDR_ID BIGINT NOT NULL, @@ -315,7 +315,7 @@ CREATE TABLE IF NOT EXISTS ELECTRONIC_ADDRESS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ENTITY_BANK_ACCOUNT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ENTITY_BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ENTITY_BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ENTITY_BANK_ACCOUNT( bill_ent_id bigint NOT NULL, bnk_acct_id bigint NOT NULL, @@ -334,7 +334,7 @@ CREATE TABLE IF NOT EXISTS ENTITY_BANK_ACCOUNT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ETL_METADATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ETL_METADATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ETL_METADATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ETL_METADATA( MAIN_TBL VARCHAR(15) NOT NULL, TGT_TBL VARCHAR(15) NOT NULL, @@ -357,7 +357,7 @@ CREATE TABLE IF NOT EXISTS ETL_METADATA( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT( EXHIBIT_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -373,7 +373,7 @@ CREATE TABLE IF NOT EXISTS EXHIBIT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXPLANATION_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/EXPLANATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/EXPLANATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXPLANATION_CODE( EXPLAIN_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -400,7 +400,7 @@ CREATE TABLE IF NOT EXISTS EXPLANATION_CODE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FILE_TRANSFER_RUN - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/FILE_TRANSFER_RUN.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/FILE_TRANSFER_RUN.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FILE_TRANSFER_RUN( FL_TRANS_RUN_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -432,7 +432,7 @@ CREATE TABLE IF NOT EXISTS FILE_TRANSFER_RUN( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FILE_TRANSFER_STAT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/FILE_TRANSFER_STAT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/FILE_TRANSFER_STAT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FILE_TRANSFER_STAT( FL_TRANS_STAT_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -469,7 +469,7 @@ CREATE TABLE IF NOT EXISTS FILE_TRANSFER_STAT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GENERAL_LEDGER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/GENERAL_LEDGER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/GENERAL_LEDGER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GENERAL_LEDGER( BILL_ENT_ID BIGINT NOT NULL, GL_ID BIGINT NOT NULL, @@ -494,7 +494,7 @@ CREATE TABLE IF NOT EXISTS GENERAL_LEDGER( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GROUP_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/GROUP_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/GROUP_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GROUP_RELATION( GRP_ID BIGINT NOT NULL, GRP_RELN_ID BIGINT NOT NULL, @@ -512,7 +512,7 @@ CREATE TABLE IF NOT EXISTS GROUP_RELATION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LEP_APPEAL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LEP_APPEAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LEP_APPEAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LEP_APPEAL( PRSN_ID BIGINT NOT NULL, LEP_APL_ID BIGINT NOT NULL, @@ -532,7 +532,7 @@ CREATE TABLE IF NOT EXISTS LEP_APPEAL( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -548,7 +548,7 @@ CREATE TABLE IF NOT EXISTS LETTER( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LINE_DISALLOW_EXPLANATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LINE_DISALLOW_EXPLANATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LINE_DISALLOW_EXPLANATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LINE_DISALLOW_EXPLANATION( PRSN_ID BIGINT NOT NULL, CLM_ID BIGINT NOT NULL, @@ -569,7 +569,7 @@ CREATE TABLE IF NOT EXISTS LINE_DISALLOW_EXPLANATION( ---- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LINE_PROCEDURE_MODIFIER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LINE_PROCEDURE_MODIFIER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LINE_PROCEDURE_MODIFIER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LINE_PROCEDURE_MODIFIER( PRSN_ID BIGINT NOT NULL, CLM_ID BIGINT NOT NULL, @@ -589,7 +589,7 @@ CREATE TABLE IF NOT EXISTS LINE_PROCEDURE_MODIFIER( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MARX_CALENDAR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/MARX_CALENDAR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/MARX_CALENDAR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MARX_CALENDAR( MARX_CAL_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -609,7 +609,7 @@ CREATE TABLE IF NOT EXISTS MARX_CALENDAR( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_NOTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/NOTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/NOTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS NOTE( INQ_ID BIGINT NOT NULL, SRC_ID BIGINT NOT NULL, @@ -632,7 +632,7 @@ CREATE TABLE IF NOT EXISTS NOTE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATION_CONTACT_TYPE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ORGANIZATION_CONTACT_TYPE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ORGANIZATION_CONTACT_TYPE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATION_CONTACT_TYPE( ORG_ID BIGINT NOT NULL, CNTC_TYP_REF_ID BIGINT NOT NULL, @@ -649,7 +649,7 @@ CREATE TABLE IF NOT EXISTS ORGANIZATION_CONTACT_TYPE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PAYMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PAYMENT( BILL_ENT_ID BIGINT NOT NULL, PAY_ID BIGINT NOT NULL, @@ -675,7 +675,7 @@ CREATE TABLE IF NOT EXISTS PAYMENT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_COB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_COB( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -725,7 +725,7 @@ CREATE TABLE IF NOT EXISTS PERSON_COB( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_CODE( PRSN_ID BIGINT NOT NULL, CD_VAL_ID BIGINT NOT NULL, @@ -745,7 +745,7 @@ CREATE TABLE IF NOT EXISTS PERSON_CODE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_LEP_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_LEP_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_EVENT( PRSN_ID BIGINT NOT NULL, PRSN_LEP_EVNT_ID BIGINT NOT NULL, @@ -766,7 +766,7 @@ CREATE TABLE IF NOT EXISTS PERSON_LEP_EVENT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_PROFILE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_LEP_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_LEP_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE( PRSN_ID BIGINT NOT NULL, PRSN_LEP_PRFL_ID BIGINT NOT NULL, @@ -785,7 +785,7 @@ CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ORGANIZATION_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_ORGANIZATION_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_ORGANIZATION_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ORGANIZATION_RELATION( PRSN_ID BIGINT NOT NULL, ORG_ID BIGINT NOT NULL, @@ -806,7 +806,7 @@ CREATE TABLE IF NOT EXISTS PERSON_ORGANIZATION_RELATION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_PAYMENT_OPTIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_PAYMENT_OPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_PAYMENT_OPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_PAYMENT_OPTIONS( PRSN_ID BIGINT NOT NULL, PRSN_PAY_OPTN_ID BIGINT NOT NULL, @@ -827,7 +827,7 @@ CREATE TABLE IF NOT EXISTS PERSON_PAYMENT_OPTIONS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_RELATION( PRSN_ID BIGINT NOT NULL, RLTD_PRSN_ID BIGINT NOT NULL, @@ -848,7 +848,7 @@ CREATE TABLE IF NOT EXISTS PERSON_RELATION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ROLE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_ROLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_ROLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ROLE( PRSN_ID BIGINT NOT NULL, PRSN_TYP_ID BIGINT NOT NULL, @@ -868,7 +868,7 @@ CREATE TABLE IF NOT EXISTS PERSON_ROLE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_SUBSIDY_PROFILE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_SUBSIDY_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_SUBSIDY_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_SUBSIDY_PROFILE( PRSN_ID BIGINT NOT NULL, PRSN_SBSDY_PRFL_ID BIGINT NOT NULL, @@ -890,7 +890,7 @@ CREATE TABLE IF NOT EXISTS PERSON_SUBSIDY_PROFILE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_WORK_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_WORK_ITEM( PRSN_ID BIGINT NOT NULL, PRSN_WRK_ITM_ID BIGINT NOT NULL, @@ -912,7 +912,7 @@ CREATE TABLE IF NOT EXISTS PERSON_WORK_ITEM( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PHONE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PHONE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PHONE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PHONE( CNTC_ID BIGINT NOT NULL, PHN_ID BIGINT NOT NULL, @@ -933,7 +933,7 @@ CREATE TABLE IF NOT EXISTS PHONE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PLAN_MEMBERSHIPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PLAN_MEMBERSHIPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PLAN_MEMBERSHIPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PLAN_MEMBERSHIPS( PRSN_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -959,7 +959,7 @@ CREATE TABLE IF NOT EXISTS PLAN_MEMBERSHIPS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_POS_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/POS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/POS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS POS_CODE( POS_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -980,7 +980,7 @@ CREATE TABLE IF NOT EXISTS POS_CODE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PREMIUM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PREMIUM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM( GRP_ID BIGINT NOT NULL, PREM_CAT_ID BIGINT NOT NULL, @@ -1003,7 +1003,7 @@ CREATE TABLE IF NOT EXISTS PREMIUM( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_REFERENCE_DATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS REFERENCE_DATA( REF_DTA_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1019,7 +1019,7 @@ CREATE TABLE IF NOT EXISTS REFERENCE_DATA( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_RETURNED_MAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/RETURNED_MAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/RETURNED_MAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS RETURNED_MAIL( RTRN_MAIL_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1040,7 +1040,7 @@ CREATE TABLE IF NOT EXISTS RETURNED_MAIL( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_ACTIVITY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_ACTIVITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_ACTIVITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_ACTIVITY( UM_RVW_ID BIGINT NOT NULL, UM_ACTY_ID BIGINT NOT NULL, @@ -1068,7 +1068,7 @@ CREATE TABLE IF NOT EXISTS UM_ACTIVITY( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_DIAGNOSIS_LINE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_DIAGNOSIS_LINE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_DIAGNOSIS_LINE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_LINE( UM_RVW_ID BIGINT NOT NULL, UM_DIAG_SET_ID BIGINT NOT NULL, @@ -1090,7 +1090,7 @@ CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_LINE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_DIAGNOSIS_SET - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_DIAGNOSIS_SET.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_DIAGNOSIS_SET.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_SET( UM_RVW_ID BIGINT NOT NULL, UM_DIAG_SET_ID BIGINT NOT NULL, @@ -1110,7 +1110,7 @@ CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_SET( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_PACKAGE_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_RELATION( BENE_PKG_ID BIGINT NOT NULL, PKG_RELN_ID BIGINT NOT NULL, @@ -1128,7 +1128,7 @@ CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_RELATION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_CONTACT( BILL_ENT_ID BIGINT NOT NULL, CNTC_ID BIGINT NOT NULL, @@ -1149,7 +1149,7 @@ CREATE TABLE IF NOT EXISTS BILLING_ENTITY_CONTACT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_DETAIL( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1167,7 +1167,7 @@ CREATE TABLE IF NOT EXISTS BILLING_ENTITY_DETAIL( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHECK_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHECK_STATUS( CLM_PAY_ID BIGINT NOT NULL, CHK_ID BIGINT NOT NULL, @@ -1189,7 +1189,7 @@ CREATE TABLE IF NOT EXISTS CHECK_STATUS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT_GROUP - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/EXHIBIT_GROUP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/EXHIBIT_GROUP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP( EXHIBIT_GRP_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1211,7 +1211,7 @@ CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT_GROUP_EXHIBIT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/EXHIBIT_GROUP_EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/EXHIBIT_GROUP_EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP_EXHIBIT( EXHIBIT_ID BIGINT NOT NULL, EXHIBIT_GRP_ID BIGINT NOT NULL, @@ -1231,7 +1231,7 @@ CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP_EXHIBIT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GENERAL_LEDGER_MAP - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/GENERAL_LEDGER_MAP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/GENERAL_LEDGER_MAP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GENERAL_LEDGER_MAP( GL_MAP_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1251,7 +1251,7 @@ CREATE TABLE IF NOT EXISTS GENERAL_LEDGER_MAP( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LEP_APPEAL_DECISION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LEP_APPEAL_DECISION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LEP_APPEAL_DECISION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LEP_APPEAL_DECISION( PRSN_ID BIGINT NOT NULL, LEP_APL_ID BIGINT NOT NULL, @@ -1273,7 +1273,7 @@ CREATE TABLE IF NOT EXISTS LEP_APPEAL_DECISION( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LETTER_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LETTER_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_DETAIL( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -1290,7 +1290,7 @@ CREATE TABLE IF NOT EXISTS LETTER_DETAIL( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_JOB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LETTER_JOB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LETTER_JOB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_JOB( PRSN_ID BIGINT NOT NULL, LTR_DTL_ID BIGINT NOT NULL, @@ -1314,7 +1314,7 @@ CREATE TABLE IF NOT EXISTS LETTER_JOB( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_REGISTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LETTER_REGISTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LETTER_REGISTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_REGISTER( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -1337,7 +1337,7 @@ CREATE TABLE IF NOT EXISTS LETTER_REGISTER( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_WORK_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/LETTER_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/LETTER_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_WORK_ITEM( PRSN_ID BIGINT NOT NULL, LTR_RGSTR_ID BIGINT NOT NULL, @@ -1356,7 +1356,7 @@ CREATE TABLE IF NOT EXISTS LETTER_WORK_ITEM( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_EVENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_EVENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS( PRSN_EVNT_ID BIGINT NOT NULL, PRSN_EVNT_STAT_ID BIGINT NOT NULL, @@ -1375,7 +1375,7 @@ CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT_STATUS_REASON - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_EVENT_STATUS_REASON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_EVENT_STATUS_REASON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS_REASON( PRSN_EVNT_STAT_ID BIGINT NOT NULL, PRSN_EVNT_STAT_RSN_ID BIGINT NOT NULL, @@ -1393,7 +1393,7 @@ CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS_REASON( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_PROFILE_RECORD - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_LEP_PROFILE_RECORD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_LEP_PROFILE_RECORD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE_RECORD( PRSN_ID BIGINT NOT NULL, PRSN_LEP_PRFL_ID BIGINT NOT NULL, @@ -1422,7 +1422,7 @@ CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE_RECORD( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_CATEGORY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PREMIUM_CATEGORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PREMIUM_CATEGORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_CATEGORY( GRP_ID BIGINT NOT NULL, PREM_CAT_ID BIGINT NOT NULL, @@ -1442,7 +1442,7 @@ CREATE TABLE IF NOT EXISTS PREMIUM_CATEGORY( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_PART - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PREMIUM_PART.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PREMIUM_PART.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_PART( GRP_ID BIGINT NOT NULL, PREM_PART_ID BIGINT NOT NULL, @@ -1464,7 +1464,7 @@ CREATE TABLE IF NOT EXISTS PREMIUM_PART( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_TABLE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PREMIUM_TABLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PREMIUM_TABLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_TABLE( GRP_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -1485,7 +1485,7 @@ CREATE TABLE IF NOT EXISTS PREMIUM_TABLE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_REVIEW - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_INPATIENT_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_INPATIENT_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_REVIEW( UM_RVW_ID BIGINT NOT NULL, UM_INPT_RVW_ID BIGINT NOT NULL, @@ -1513,7 +1513,7 @@ CREATE TABLE IF NOT EXISTS UM_INPATIENT_REVIEW( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_INPATIENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_INPATIENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_STATUS( UM_RVW_ID BIGINT NOT NULL, UM_INPT_STAT BIGINT NOT NULL, @@ -1536,7 +1536,7 @@ CREATE TABLE IF NOT EXISTS UM_INPATIENT_STATUS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_SERVICE_OVERRIDE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_SERVICE_OVERRIDE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_SERVICE_OVERRIDE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_SERVICE_OVERRIDE( UM_RVW_ID BIGINT NOT NULL, UM_SERV_ID BIGINT NOT NULL, @@ -1562,7 +1562,7 @@ INSERT INTO UM_SERVICE_OVERRIDE SELECT * FROM staging_UM_SERVICE_OVERRIDE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BANK_ACCOUNT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BANK_ACCOUNT( BNK_ORG_ID BIGINT NOT NULL, BNK_ID BIGINT NOT NULL, @@ -1583,7 +1583,7 @@ CREATE TABLE IF NOT EXISTS BANK_ACCOUNT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_STAY_LENGTH - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UM_INPATIENT_STAY_LENGTH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UM_INPATIENT_STAY_LENGTH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_STAY_LENGTH( UM_RVW_ID BIGINT NOT NULL, UM_INPT_RVW_ID BIGINT NOT NULL, @@ -1617,7 +1617,7 @@ CREATE TABLE IF NOT EXISTS UM_INPATIENT_STAY_LENGTH( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_REVENUE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/REVENUE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/REVENUE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS REVENUE_CODE( REV_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1639,7 +1639,7 @@ CREATE TABLE IF NOT EXISTS REVENUE_CODE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/SERVICE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS SERVICE_CODE( SERV_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1658,7 +1658,7 @@ CREATE TABLE IF NOT EXISTS SERVICE_CODE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_AGREEMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS AGREEMENT( AGREE_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1677,7 +1677,7 @@ CREATE TABLE IF NOT EXISTS AGREEMENT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATION_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/ORGANIZATION_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/ORGANIZATION_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATION_EVENT( ORG_ID BIGINT NOT NULL, ORG_EVNT_ID BIGINT NOT NULL, @@ -1698,7 +1698,7 @@ CREATE TABLE IF NOT EXISTS ORGANIZATION_EVENT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_TX_IDCARD - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/FDI_TX_IDCARD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/FDI_TX_IDCARD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_TX_IDCARD ( ACN_ID BIGINT NOT NULL, @@ -1736,7 +1736,7 @@ CREATE TABLE IF NOT EXISTS FDI_TX_IDCARD ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_TX_LETTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/FDI_TX_LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/FDI_TX_LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_TX_LETTER ( ACN_ID BIGINT NOT NULL, @@ -1770,7 +1770,7 @@ CREATE TABLE IF NOT EXISTS FDI_TX_LETTER ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_PACKAGE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_ATTRIBUTE( BENE_PKG_ID BIGINT NOT NULL, BENE_PKG_ATTR_ID BIGINT NOT NULL, @@ -1789,7 +1789,7 @@ CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_ATTRIBUTE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_SCHEDULE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1811,7 +1811,7 @@ CREATE TABLE IF NOT EXISTS BILLING_ENTITY_SCHEDULE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_SCHEDULE( BILL_SCHD_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1831,7 +1831,7 @@ CREATE TABLE IF NOT EXISTS BILLING_SCHEDULE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_SOURCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_SOURCE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1849,7 +1849,7 @@ CREATE TABLE IF NOT EXISTS BILLING_SOURCE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHARGE_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHARGE_ITEM( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1873,7 +1873,7 @@ CREATE TABLE IF NOT EXISTS CHARGE_ITEM( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_COSHARE_TRACKING - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_COSHARE_TRACKING( PRSN_ID BIGINT NOT NULL, VER BIGINT, @@ -1901,7 +1901,7 @@ CREATE TABLE IF NOT EXISTS CLAIM_COSHARE_TRACKING( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_LINE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_LINE_ATTRIBUTE( PRSN_ID BIGINT NOT NULL, CLM_DTL_ID BIGINT NOT NULL, @@ -1925,7 +1925,7 @@ CREATE TABLE IF NOT EXISTS CLAIM_LINE_ATTRIBUTE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_CORRESPONDENCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/FDI_CORRESPONDENCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/FDI_CORRESPONDENCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_CORRESPONDENCE ( CLIENT_ID BIGINT NOT NULL, @@ -1946,7 +1946,7 @@ CREATE TABLE IF NOT EXISTS FDI_CORRESPONDENCE ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/INVOICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/INVOICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1981,7 +1981,7 @@ CREATE TABLE IF NOT EXISTS INVOICE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/INVOICE_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/INVOICE_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE_STATUS( INV_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2001,7 +2001,7 @@ CREATE TABLE IF NOT EXISTS INVOICE_STATUS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_BALANCE_EXCEPTIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/MOOP_BALANCE_EXCEPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/MOOP_BALANCE_EXCEPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_BALANCE_EXCEPTIONS( CLIENT_ID INTEGER NOT NULL, VLD_FRM_DT DATE NOT NULL, @@ -2023,7 +2023,7 @@ CREATE TABLE IF NOT EXISTS MOOP_BALANCE_EXCEPTIONS( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_BALANCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/MOOP_BALANCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/MOOP_BALANCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_BALANCE( PRSN_ID BIGINT NOT NULL, CLIENT_ID INTEGER NOT NULL, @@ -2047,7 +2047,7 @@ CREATE TABLE IF NOT EXISTS MOOP_BALANCE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_ACCUMULATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/MOOP_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/MOOP_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_ACCUMULATOR ( SUPPL_ID VARCHAR(15) NOT NULL, CLIENT_ID INTEGER NOT NULL, @@ -2068,7 +2068,7 @@ SUPPL_ID VARCHAR(15) NOT NULL, ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ACCUMULATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PERSON_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PERSON_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ACCUMULATOR( PRSN_ID BIGINT NOT NULL, PRSN_ACCUM_ID BIGINT NOT NULL, @@ -2098,7 +2098,7 @@ CREATE TABLE IF NOT EXISTS PERSON_ACCUMULATOR( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PROCEDURE_PRICE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/PROCEDURE_PRICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/PROCEDURE_PRICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PROCEDURE_PRICE( PR_CD_ID BIGINT NOT NULL, PR_PRC_ID BIGINT NOT NULL, @@ -2120,7 +2120,7 @@ CREATE TABLE IF NOT EXISTS PROCEDURE_PRICE( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_RECEIPT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/RECEIPT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/RECEIPT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS RECEIPT( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2163,9 +2163,9 @@ CREATE TABLE IF NOT EXISTS RECEIPT( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PLAN_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PLAN_CODE_CONFIG ( PLAN_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2195,13 +2195,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (PLAN_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_PLAN_CODE_CONFIG; + INSERT INTO SERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_SERVICES_PLAN_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_KEY_GENERATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_KEY_GENERATOR + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_KEY_GENERATOR ( KEY_GEN_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2219,13 +2219,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (KEY_GEN_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',redundancy '1' ); - INSERT INTO TMGSERVICES_KEY_GENERATOR SELECT * FROM staging_TMGSERVICES_KEY_GENERATOR; + INSERT INTO SERVICES_KEY_GENERATOR SELECT * FROM staging_SERVICES_KEY_GENERATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); - CREATE TABLE IF NOT EXISTS TMGSERVICES_GROUP_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_GROUP_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + CREATE TABLE IF NOT EXISTS SERVICES_GROUP_CODE_CONFIG ( GROUP_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2244,12 +2244,12 @@ CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG PRIMARY KEY (GROUP_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_GROUP_CODE_CONFIG; + INSERT INTO SERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_SERVICES_GROUP_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DNIS_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DNIS_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DNIS_CONFIG ( DNIS_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2268,12 +2268,12 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DNIS_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_DNIS_CONFIG SELECT * FROM staging_TMGSERVICES_DNIS_CONFIG; + INSERT INTO SERVICES_DNIS_CONFIG SELECT * FROM staging_SERVICES_DNIS_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DENIAL_REASON_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DENIAL_REASON_CONFIG ( DENIAL_REASON_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2290,13 +2290,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DENIAL_REASON_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_TMGSERVICES_DENIAL_REASON_CONFIG; + INSERT INTO SERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_SERVICES_DENIAL_REASON_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_CLIENT_MASTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_CLIENT_MASTER + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_CLIENT_MASTER ( CLIENT_REF_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2304,7 +2304,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER CLIENT_LEGACY_CD VARCHAR(5) NULL, CLIENT_NAME VARCHAR(10) NULL, MEMBER_ID_FORMAT VARCHAR(15) NULL, - TMG_CALL_CLIENT_CODE VARCHAR(10) NULL, + CALL_CLIENT_CODE VARCHAR(10) NULL, CREATE_date date NULL, UPDATED_date date NULL, USER_NAME VARCHAR(15) NULL, @@ -2314,12 +2314,12 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (CLIENT_REF_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_CLIENT_MASTER SELECT * FROM staging_TMGSERVICES_CLIENT_MASTER; + INSERT INTO SERVICES_CLIENT_MASTER SELECT * FROM staging_SERVICES_CLIENT_MASTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS ( SUBJ_CAT_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2341,13 +2341,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (SUBJ_CAT_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; + INSERT INTO SERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS ( VARIABLE_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2365,13 +2365,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS PRIMARY KEY (VARIABLE_ID) ) USING row OPTIONS(partition_by 'VARIABLE_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; + INSERT INTO SERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_ACCOUNTING_CODES - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/TMGSERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_ACCOUNTING_CODES + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/SERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_ACCOUNTING_CODES ( ACCOUNTING_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2394,12 +2394,12 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (ACCOUNTING_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',redundancy '1'); - INSERT INTO TMGSERVICES_ACCOUNTING_CODES SELECT * FROM staging_TMGSERVICES_ACCOUNTING_CODES; + INSERT INTO SERVICES_ACCOUNTING_CODES SELECT * FROM staging_SERVICES_ACCOUNTING_CODES; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UNAPPLIED_CASH - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/UNAPPLIED_CASH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/UNAPPLIED_CASH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UNAPPLIED_CASH( UNAPP_CSH_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -2420,7 +2420,7 @@ CREATE TABLE IF NOT EXISTS UNAPPLIED_CASH( ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_WORK_GENERATED_KEYS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data/WORK_GENERATED_KEYS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data/RowTable_Data/WORK_GENERATED_KEYS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS WORK_GENERATED_KEYS( CLIENT_ID BIGINT NOT NULL, GEN_KEY_ID BIGINT NOT NULL, diff --git a/dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql b/dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql index 34aab327d2..07f9c851d1 100644 --- a/dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql +++ b/dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql @@ -115,15 +115,15 @@ SELECT * FROM REFERENCE_DATA WHERE REF_DTA_ID = 42 SELECT * FROM RETURNED_MAIL WHERE RTRN_MAIL_ID = 10000 AND PRSN_ID = 5790; SELECT * FROM REVENUE_CODE WHERE REV_CD_ID = 52011; SELECT * FROM SERVICE_CODE WHERE SERV_CD_ID = 654292; -SELECT * FROM TMGSERVICES_ACCOUNTING_CODES WHERE ACCOUNTING_CODE_ID = 726283 AND CLIENT_ID = 84375; -SELECT * FROM TMGSERVICES_CLIENT_MASTER WHERE CLIENT_REF_ID = 82501 AND CLIENT_ID = 27982; -SELECT * FROM TMGSERVICES_DENIAL_REASON_CONFIG WHERE DENIAL_REASON_ID = 60072 AND CLIENT_REF_ID = 2575092; -SELECT * FROM TMGSERVICES_DNIS_CONFIG WHERE DNIS_ID = 3263 AND CLIENT_REF_ID = 1703380; -SELECT * FROM TMGSERVICES_GROUP_CODE_CONFIG WHERE GROUP_ID = 520494 AND CLIENT_REF_ID = 2020245; -SELECT * FROM TMGSERVICES_KEY_GENERATOR WHERE KEY_GEN_ID = 9218 AND CLIENT_ID = 1464; -SELECT * FROM TMGSERVICES_PLAN_CODE_CONFIG WHERE PLAN_CODE_ID = 13234 AND CLIENT_REF_ID = 3585543; -SELECT * FROM TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS WHERE VARIABLE_ID = 84362 AND CLIENT_ID = 92141; -SELECT * FROM TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS WHERE SUBJ_CAT_ID = 33 AND CLIENT_REF_ID = 3764443; +SELECT * FROM SERVICES_ACCOUNTING_CODES WHERE ACCOUNTING_CODE_ID = 726283 AND CLIENT_ID = 84375; +SELECT * FROM SERVICES_CLIENT_MASTER WHERE CLIENT_REF_ID = 82501 AND CLIENT_ID = 27982; +SELECT * FROM SERVICES_DENIAL_REASON_CONFIG WHERE DENIAL_REASON_ID = 60072 AND CLIENT_REF_ID = 2575092; +SELECT * FROM SERVICES_DNIS_CONFIG WHERE DNIS_ID = 3263 AND CLIENT_REF_ID = 1703380; +SELECT * FROM SERVICES_GROUP_CODE_CONFIG WHERE GROUP_ID = 520494 AND CLIENT_REF_ID = 2020245; +SELECT * FROM SERVICES_KEY_GENERATOR WHERE KEY_GEN_ID = 9218 AND CLIENT_ID = 1464; +SELECT * FROM SERVICES_PLAN_CODE_CONFIG WHERE PLAN_CODE_ID = 13234 AND CLIENT_REF_ID = 3585543; +SELECT * FROM SERVICES_PTMR_VARIABLE_TRANSLATIONS WHERE VARIABLE_ID = 84362 AND CLIENT_ID = 92141; +SELECT * FROM SERVICES_SUBJECT_CATEGORY_TRANSLATIONS WHERE SUBJ_CAT_ID = 33 AND CLIENT_REF_ID = 3764443; SELECT * FROM TOPIC WHERE TPC_ID = 2916442 AND INQ_ID = 2916442; SELECT * FROM TOPIC_COMMUNICATION WHERE TPC_INQ_ID = 10 AND CMCN_INQ_ID = 10; SELECT * FROM UM_ACTIVITY WHERE UM_RVW_ID = 7277 AND UM_ACTY_ID = 24221; diff --git a/dtests/src/resources/scripts/cdcConnector/scriptsForApp1/insert4.sql b/dtests/src/resources/scripts/cdcConnector/scriptsForApp1/insert4.sql index 55a7900abe..17c877707e 100644 --- a/dtests/src/resources/scripts/cdcConnector/scriptsForApp1/insert4.sql +++ b/dtests/src/resources/scripts/cdcConnector/scriptsForApp1/insert4.sql @@ -14,10 +14,10 @@ INSERT INTO [testdatabase].[dbo].[FILE_TRANSFER_STAT] VALUES (?, 4187111, 1, N'I INSERT INTO [testdatabase].[dbo].[RETURNED_MAIL] VALUES (?, 2534, 1, 53370, N'2016-06-13', N'd', N'DnYuOz', N'DVYS', N'2016-07-23', N'2016-06-23', N'6349', N'2342'); INSERT INTO [testdatabase].[dbo].[REVENUE_CODE] VALUES (?, 9197, 1, N'JwTGI', N'2016-07-28', N'2016-06-18', N'vg', N'NSoCMDhm', 1759734, N'2016-06-05', N'2016-05-28', N'5805', N'5614'); INSERT INTO [testdatabase].[dbo].[SERVICE_CODE] VALUES (?, 461, 1, N'xUscj', N'soVG', N'ChaKw', N'2016-07-19', N'2016-05-01', N'1886', N'6019'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_ACCOUNTING_CODES] VALUES (?, 3893554, 8433, 1, N'uGn', N'k', N'CTXhWJ', 'G', N'WCaOs', N'jjU', N'Di', N'dxCmNYg', N'2016-06-25', N'2016-06-08', N'qhqpcexY', N'2016-05-31', N'2016-08-07', N'5603', N'3404'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_CLIENT_MASTER] VALUES (?, 3364, 1, N'jC', N'yTFExSF', N'JiTvgi', N'uYdAL', N'2016-06-16', N'2016-07-25', N'ayXBKO', N'2016-05-24', N'2016-07-14', N'8505', N'594'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_DNIS_CONFIG] VALUES (?, 3552757, 8946, 1, N'ABEqEQlm', N'PuIlxh', N'PiPzxFpxhA', N'yH', N'2016-07-12', N'2016-07-22', N'kCPJlyGVW', N'2016-07-08', N'2016-05-29', N'5652', N'5592'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_GROUP_CODE_CONFIG] VALUES (?, 3521714, 7543, 1, N'QZVT', N'uD', 'U', N'2016-05-15', N'2016-06-25', N'Uilk', N'2016-05-26', N'2016-07-27', N'9676', N'8465'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_KEY_GENERATOR] VALUES (?, 4424588, 1449, 1, N'oKkh', N'Q', N'zHx', N'2016-06-19', N'2016-05-07', N'cpgklMW', N'2016-05-11', N'2016-06-29', N'8219', N'5628'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_PLAN_CODE_CONFIG] VALUES (?, 1218154, 4308, 1, N'eX', N'QY', N'Bgeh', 'E', 'l', 'f', 'K', 'L', 1905702, 2153849, 'S', 'R', 'H', 'O', 'm', N'2016-06-22', N'2016-05-12', N'QIuRtZWFL', N'2016-06-15', N'2016-05-15', N'3589', N'1312'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS] VALUES (?, 7044, 1, N'Z', N'M', N'kCYzL', N'2016-05-13', N'2016-06-16', N'mKhu', N'2016-05-12', N'2016-06-07', N'3396', N'2794'); +INSERT INTO [testdatabase].[dbo].[SERVICES_ACCOUNTING_CODES] VALUES (?, 3893554, 8433, 1, N'uGn', N'k', N'CTXhWJ', 'G', N'WCaOs', N'jjU', N'Di', N'dxCmNYg', N'2016-06-25', N'2016-06-08', N'qhqpcexY', N'2016-05-31', N'2016-08-07', N'5603', N'3404'); +INSERT INTO [testdatabase].[dbo].[SERVICES_CLIENT_MASTER] VALUES (?, 3364, 1, N'jC', N'yTFExSF', N'JiTvgi', N'uYdAL', N'2016-06-16', N'2016-07-25', N'ayXBKO', N'2016-05-24', N'2016-07-14', N'8505', N'594'); +INSERT INTO [testdatabase].[dbo].[SERVICES_DNIS_CONFIG] VALUES (?, 3552757, 8946, 1, N'ABEqEQlm', N'PuIlxh', N'PiPzxFpxhA', N'yH', N'2016-07-12', N'2016-07-22', N'kCPJlyGVW', N'2016-07-08', N'2016-05-29', N'5652', N'5592'); +INSERT INTO [testdatabase].[dbo].[SERVICES_GROUP_CODE_CONFIG] VALUES (?, 3521714, 7543, 1, N'QZVT', N'uD', 'U', N'2016-05-15', N'2016-06-25', N'Uilk', N'2016-05-26', N'2016-07-27', N'9676', N'8465'); +INSERT INTO [testdatabase].[dbo].[SERVICES_KEY_GENERATOR] VALUES (?, 4424588, 1449, 1, N'oKkh', N'Q', N'zHx', N'2016-06-19', N'2016-05-07', N'cpgklMW', N'2016-05-11', N'2016-06-29', N'8219', N'5628'); +INSERT INTO [testdatabase].[dbo].[SERVICES_PLAN_CODE_CONFIG] VALUES (?, 1218154, 4308, 1, N'eX', N'QY', N'Bgeh', 'E', 'l', 'f', 'K', 'L', 1905702, 2153849, 'S', 'R', 'H', 'O', 'm', N'2016-06-22', N'2016-05-12', N'QIuRtZWFL', N'2016-06-15', N'2016-05-15', N'3589', N'1312'); +INSERT INTO [testdatabase].[dbo].[SERVICES_PTMR_VARIABLE_TRANSLATIONS] VALUES (?, 7044, 1, N'Z', N'M', N'kCYzL', N'2016-05-13', N'2016-06-16', N'mKhu', N'2016-05-12', N'2016-06-07', N'3396', N'2794'); diff --git a/dtests/src/resources/scripts/cdcConnector/scriptsForApp2/insert4.sql b/dtests/src/resources/scripts/cdcConnector/scriptsForApp2/insert4.sql index 74d2342497..16119594a9 100644 --- a/dtests/src/resources/scripts/cdcConnector/scriptsForApp2/insert4.sql +++ b/dtests/src/resources/scripts/cdcConnector/scriptsForApp2/insert4.sql @@ -14,10 +14,10 @@ INSERT INTO [testdatabase].[dbo].[FILE_TRANSFER_STAT] VALUES (?, 4187112, 2, N'I INSERT INTO [testdatabase].[dbo].[RETURNED_MAIL] VALUES (?, 2534, 2, 53370, N'2016-06-13', N'd', N'DnYuOz', N'DVYS', N'2016-07-23', N'2016-06-23', N'6349', N'2342'); INSERT INTO [testdatabase].[dbo].[REVENUE_CODE] VALUES (?, 9197, 2, N'JwTGI', N'2016-07-28', N'2016-06-18', N'vg', N'NSoCMDhm', 1759734, N'2016-06-05', N'2016-05-28', N'5805', N'5614'); INSERT INTO [testdatabase].[dbo].[SERVICE_CODE] VALUES (?, 462, 2, N'xUscj', N'soVG', N'ChaKw', N'2016-07-19', N'2016-05-01', N'1886', N'6019'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_ACCOUNTING_CODES] VALUES (?, 3893554, 8433, 2, N'uGn', N'k', N'CTXhWJ', 'G', N'WCaOs', N'jjU', N'Di', N'dxCmNYg', N'2016-06-25', N'2016-06-08', N'qhqpcexY', N'2016-05-31', N'2016-08-07', N'5603', N'3404'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_CLIENT_MASTER] VALUES (?, 3364, 2, N'jC', N'yTFExSF', N'JiTvgi', N'uYdAL', N'2016-06-16', N'2016-07-25', N'ayXBKO', N'2016-05-24', N'2016-07-14', N'8505', N'594'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_DNIS_CONFIG] VALUES (?, 3552757, 8946, 2, N'ABEqEQlm', N'PuIlxh', N'PiPzxFpxhA', N'yH', N'2016-07-12', N'2016-07-22', N'kCPJlyGVW', N'2016-07-08', N'2016-05-29', N'5652', N'5592'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_GROUP_CODE_CONFIG] VALUES (?, 3521714, 7543, 2, N'QZVT', N'uD', 'U', N'2016-05-15', N'2016-06-25', N'Uilk', N'2016-05-26', N'2016-07-27', N'9676', N'8465'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_KEY_GENERATOR] VALUES (?, 4424588, 1449, 2, N'oKkh', N'Q', N'zHx', N'2016-06-19', N'2016-05-07', N'cpgklMW', N'2016-05-11', N'2016-06-29', N'8219', N'5628'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_PLAN_CODE_CONFIG] VALUES (?, 1218154, 4308, 2, N'eX', N'QY', N'Bgeh', 'E', 'l', 'f', 'K', 'L', 1905702, 2153849, 'S', 'R', 'H', 'O', 'm', N'2016-06-22', N'2016-05-12', N'QIuRtZWFL', N'2016-06-15', N'2016-05-15', N'3589', N'1312'); -INSERT INTO [testdatabase].[dbo].[TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS] VALUES (?, 7044, 2, N'Z', N'M', N'kCYzL', N'2016-05-13', N'2016-06-16', N'mKhu', N'2016-05-12', N'2016-06-07', N'3396', N'2794'); +INSERT INTO [testdatabase].[dbo].[SERVICES_ACCOUNTING_CODES] VALUES (?, 3893554, 8433, 2, N'uGn', N'k', N'CTXhWJ', 'G', N'WCaOs', N'jjU', N'Di', N'dxCmNYg', N'2016-06-25', N'2016-06-08', N'qhqpcexY', N'2016-05-31', N'2016-08-07', N'5603', N'3404'); +INSERT INTO [testdatabase].[dbo].[SERVICES_CLIENT_MASTER] VALUES (?, 3364, 2, N'jC', N'yTFExSF', N'JiTvgi', N'uYdAL', N'2016-06-16', N'2016-07-25', N'ayXBKO', N'2016-05-24', N'2016-07-14', N'8505', N'594'); +INSERT INTO [testdatabase].[dbo].[SERVICES_DNIS_CONFIG] VALUES (?, 3552757, 8946, 2, N'ABEqEQlm', N'PuIlxh', N'PiPzxFpxhA', N'yH', N'2016-07-12', N'2016-07-22', N'kCPJlyGVW', N'2016-07-08', N'2016-05-29', N'5652', N'5592'); +INSERT INTO [testdatabase].[dbo].[SERVICES_GROUP_CODE_CONFIG] VALUES (?, 3521714, 7543, 2, N'QZVT', N'uD', 'U', N'2016-05-15', N'2016-06-25', N'Uilk', N'2016-05-26', N'2016-07-27', N'9676', N'8465'); +INSERT INTO [testdatabase].[dbo].[SERVICES_KEY_GENERATOR] VALUES (?, 4424588, 1449, 2, N'oKkh', N'Q', N'zHx', N'2016-06-19', N'2016-05-07', N'cpgklMW', N'2016-05-11', N'2016-06-29', N'8219', N'5628'); +INSERT INTO [testdatabase].[dbo].[SERVICES_PLAN_CODE_CONFIG] VALUES (?, 1218154, 4308, 2, N'eX', N'QY', N'Bgeh', 'E', 'l', 'f', 'K', 'L', 1905702, 2153849, 'S', 'R', 'H', 'O', 'm', N'2016-06-22', N'2016-05-12', N'QIuRtZWFL', N'2016-06-15', N'2016-05-15', N'3589', N'1312'); +INSERT INTO [testdatabase].[dbo].[SERVICES_PTMR_VARIABLE_TRANSLATIONS] VALUES (?, 7044, 2, N'Z', N'M', N'kCYzL', N'2016-05-13', N'2016-06-16', N'mKhu', N'2016-05-12', N'2016-06-07', N'3396', N'2794'); diff --git a/dtests/src/resources/scripts/cdcConnector/source_destination_tables.properties b/dtests/src/resources/scripts/cdcConnector/source_destination_tables.properties index 44a075c493..83f6870acf 100644 --- a/dtests/src/resources/scripts/cdcConnector/source_destination_tables.properties +++ b/dtests/src/resources/scripts/cdcConnector/source_destination_tables.properties @@ -115,15 +115,15 @@ testdatabase.cdc.dbo_REFERENCE_DATA_CT=REFERENCE_DATA testdatabase.cdc.dbo_RETURNED_MAIL_CT=RETURNED_MAIL testdatabase.cdc.dbo_REVENUE_CODE_CT=REVENUE_CODE testdatabase.cdc.dbo_SERVICE_CODE_CT=SERVICE_CODE -testdatabase.cdc.dbo_TMGSERVICES_ACCOUNTING_CODES_CT=TMGSERVICES_ACCOUNTING_CODES -testdatabase.cdc.dbo_TMGSERVICES_CLIENT_MASTER_CT=TMGSERVICES_CLIENT_MASTER -testdatabase.cdc.dbo_TMGSERVICES_DENIAL_REASON_CONFIG_CT=TMGSERVICES_DENIAL_REASON_CONFIG -testdatabase.cdc.dbo_TMGSERVICES_DNIS_CONFIG_CT=TMGSERVICES_DNIS_CONFIG -testdatabase.cdc.dbo_TMGSERVICES_GROUP_CODE_CONFIG_CT=TMGSERVICES_GROUP_CODE_CONFIG -testdatabase.cdc.dbo_TMGSERVICES_KEY_GENERATOR_CT=TMGSERVICES_KEY_GENERATOR -testdatabase.cdc.dbo_TMGSERVICES_PLAN_CODE_CONFIG_CT=TMGSERVICES_PLAN_CODE_CONFIG -testdatabase.cdc.dbo_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS_CT=TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS -testdatabase.cdc.dbo_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS_CT=TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS +testdatabase.cdc.dbo_SERVICES_ACCOUNTING_CODES_CT=SERVICES_ACCOUNTING_CODES +testdatabase.cdc.dbo_SERVICES_CLIENT_MASTER_CT=SERVICES_CLIENT_MASTER +testdatabase.cdc.dbo_SERVICES_DENIAL_REASON_CONFIG_CT=SERVICES_DENIAL_REASON_CONFIG +testdatabase.cdc.dbo_SERVICES_DNIS_CONFIG_CT=SERVICES_DNIS_CONFIG +testdatabase.cdc.dbo_SERVICES_GROUP_CODE_CONFIG_CT=SERVICES_GROUP_CODE_CONFIG +testdatabase.cdc.dbo_SERVICES_KEY_GENERATOR_CT=SERVICES_KEY_GENERATOR +testdatabase.cdc.dbo_SERVICES_PLAN_CODE_CONFIG_CT=SERVICES_PLAN_CODE_CONFIG +testdatabase.cdc.dbo_SERVICES_PTMR_VARIABLE_TRANSLATIONS_CT=SERVICES_PTMR_VARIABLE_TRANSLATIONS +testdatabase.cdc.dbo_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS_CT=SERVICES_SUBJECT_CATEGORY_TRANSLATIONS testdatabase.cdc.dbo_TOPIC_CT=TOPIC testdatabase.cdc.dbo_TOPIC_COMMUNICATION_CT=TOPIC_COMMUNICATION testdatabase.cdc.dbo_UM_ACTIVITY_CT=UM_ACTIVITY diff --git a/dtests/src/resources/scripts/cdcConnector/tableNameList.txt b/dtests/src/resources/scripts/cdcConnector/tableNameList.txt index 381c329c7b..5d71faa9f6 100644 --- a/dtests/src/resources/scripts/cdcConnector/tableNameList.txt +++ b/dtests/src/resources/scripts/cdcConnector/tableNameList.txt @@ -114,14 +114,14 @@ REFERENCE_DATA=CLIENT_REF_DTA_ID; RETURNED_MAIL=RTRN_MAIL_ID; REVENUE_CODE=REV_CD_ID; SERVICE_CODE=SERV_CD_ID; -TMGSERVICES_ACCOUNTING_CODES=ACCOUNTING_CODE_ID; -TMGSERVICES_CLIENT_MASTER=CLIENT_REF_ID; -TMGSERVICES_DENIAL_REASON_CONFIG=DENIAL_REASON_ID; -TMGSERVICES_DNIS_CONFIG=DNIS_ID; -TMGSERVICES_GROUP_CODE_CONFIG=GROUP_ID; -TMGSERVICES_KEY_GENERATOR=KEY_GEN_ID; -TMGSERVICES_PLAN_CODE_CONFIG=PLAN_CODE_ID; -TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS=VARIABLE_ID; +SERVICES_ACCOUNTING_CODES=ACCOUNTING_CODE_ID; +SERVICES_CLIENT_MASTER=CLIENT_REF_ID; +SERVICES_DENIAL_REASON_CONFIG=DENIAL_REASON_ID; +SERVICES_DNIS_CONFIG=DNIS_ID; +SERVICES_GROUP_CODE_CONFIG=GROUP_ID; +SERVICES_KEY_GENERATOR=KEY_GEN_ID; +SERVICES_PLAN_CODE_CONFIG=PLAN_CODE_ID; +SERVICES_PTMR_VARIABLE_TRANSLATIONS=VARIABLE_ID; TOPIC=TPC_ID; TOPIC_COMMUNICATION=TPC_INQ_ID; UM_ACTIVITY=UM_ACTY_ID; diff --git a/dtests/src/resources/scripts/clusterRecovery/createAndLoadColocatedTables.sql b/dtests/src/resources/scripts/clusterRecovery/createAndLoadColocatedTables.sql index 4ce887d7fe..ec7bb01304 100644 --- a/dtests/src/resources/scripts/clusterRecovery/createAndLoadColocatedTables.sql +++ b/dtests/src/resources/scripts/clusterRecovery/createAndLoadColocatedTables.sql @@ -2526,12 +2526,12 @@ CREATE TABLE IF NOT EXISTS RECEIPT( ) USING row OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',redundancy '1',colocate_with 'BILLING_ENTITY',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); INSERT INTO RECEIPT SELECT * FROM staging_RECEIPT; -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_PLAN_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PLAN_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PLAN_CODE_CONFIG ( PLAN_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2561,14 +2561,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (PLAN_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_PLAN_CODE_CONFIG; + INSERT INTO SERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_SERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS staging_TMGSERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS staging_SERVICES_KEY_GENERATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_KEY_GENERATOR - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_KEY_GENERATOR + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_KEY_GENERATOR ( KEY_GEN_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2586,14 +2586,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (KEY_GEN_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true' ); - INSERT INTO TMGSERVICES_KEY_GENERATOR SELECT * FROM staging_TMGSERVICES_KEY_GENERATOR; + INSERT INTO SERVICES_KEY_GENERATOR SELECT * FROM staging_SERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_GROUP_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); - CREATE TABLE IF NOT EXISTS TMGSERVICES_GROUP_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_GROUP_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + CREATE TABLE IF NOT EXISTS SERVICES_GROUP_CODE_CONFIG ( GROUP_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2612,14 +2612,14 @@ CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG PRIMARY KEY (GROUP_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_GROUP_CODE_CONFIG; + INSERT INTO SERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_SERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DNIS_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DNIS_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DNIS_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DNIS_CONFIG ( DNIS_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2638,14 +2638,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DNIS_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_DNIS_CONFIG SELECT * FROM staging_TMGSERVICES_DNIS_CONFIG; + INSERT INTO SERVICES_DNIS_CONFIG SELECT * FROM staging_SERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DENIAL_REASON_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DENIAL_REASON_CONFIG + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DENIAL_REASON_CONFIG ( DENIAL_REASON_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2662,14 +2662,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DENIAL_REASON_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_TMGSERVICES_DENIAL_REASON_CONFIG; + INSERT INTO SERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_SERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS staging_TMGSERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS staging_SERVICES_CLIENT_MASTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_CLIENT_MASTER - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_CLIENT_MASTER + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_CLIENT_MASTER ( CLIENT_REF_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2677,7 +2677,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER CLIENT_LEGACY_CD VARCHAR(50) NULL, CLIENT_NAME VARCHAR(10) NULL, MEMBER_ID_FORMAT VARCHAR(15) NULL, - TMG_CALL_CLIENT_CODE VARCHAR(10) NULL, + CALL_CLIENT_CODE VARCHAR(10) NULL, CREATE_date date NULL, UPDATED_date date NULL, USER_NAME VARCHAR(15) NULL, @@ -2687,14 +2687,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (CLIENT_REF_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_CLIENT_MASTER SELECT * FROM staging_TMGSERVICES_CLIENT_MASTER; + INSERT INTO SERVICES_CLIENT_MASTER SELECT * FROM staging_SERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS ( SUBJ_CAT_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2716,14 +2716,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (SUBJ_CAT_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; + INSERT INTO SERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS ( VARIABLE_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2741,14 +2741,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS PRIMARY KEY (VARIABLE_ID) ) USING row OPTIONS(partition_by 'VARIABLE_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; + INSERT INTO SERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; -DROP TABLE IF EXISTS staging_TMGSERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS staging_SERVICES_ACCOUNTING_CODES; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_ACCOUNTING_CODES - USING com.databricks.spark.csv OPTIONS (path ':dataLocation/TMGSERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_ACCOUNTING_CODES + USING com.databricks.spark.csv OPTIONS (path ':dataLocation/SERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_ACCOUNTING_CODES ( ACCOUNTING_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2771,7 +2771,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (ACCOUNTING_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_ACCOUNTING_CODES SELECT * FROM staging_TMGSERVICES_ACCOUNTING_CODES; + INSERT INTO SERVICES_ACCOUNTING_CODES SELECT * FROM staging_SERVICES_ACCOUNTING_CODES; DROP TABLE IF EXISTS UNAPPLIED_CASH; DROP TABLE IF EXISTS staging_UNAPPLIED_CASHE; diff --git a/dtests/src/resources/scripts/clusterRecovery/createAndLoadMixedTables.sql b/dtests/src/resources/scripts/clusterRecovery/createAndLoadMixedTables.sql index f181c5c5ae..d049364aeb 100644 --- a/dtests/src/resources/scripts/clusterRecovery/createAndLoadMixedTables.sql +++ b/dtests/src/resources/scripts/clusterRecovery/createAndLoadMixedTables.sql @@ -1,226 +1,226 @@ DROP TABLE IF EXISTS PERSON_EVENT; DROP TABLE IF EXISTS staging_PERSON_EVENT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT USING column OPTIONS(partition_by 'PRSN_EVNT_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_EVNT_ID ' ) AS (SELECT * FROM staging_PERSON_EVENT); DROP TABLE IF EXISTS PERSON_EVENT_ATTRIBUTE; DROP TABLE IF EXISTS staging_PERSON_EVENT_ATTRIBUTE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_EVENT_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_EVENT_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT_ATTRIBUTE USING column OPTIONS(partition_by 'PRSN_EVNT_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_EVNT_ID,PRSN_EVNT_ATTR_ID ' ) AS (SELECT * FROM staging_PERSON_EVENT_ATTRIBUTE); DROP TABLE IF EXISTS CLAIM_STATUS; DROP TABLE IF EXISTS staging_CLAIM_STATUS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_STATUS USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_STAT_ID ' ) AS (SELECT * FROM staging_CLAIM_STATUS); DROP TABLE IF EXISTS CLAIM_ADDITIONAL_DIAGNOSIS; DROP TABLE IF EXISTS staging_CLAIM_ADDITIONAL_DIAGNOSIS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_ADDITIONAL_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_ADDITIONAL_DIAGNOSIS USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_ADD_DIAG_ID ' ) AS (SELECT * FROM staging_CLAIM_ADDITIONAL_DIAGNOSIS); DROP TABLE IF EXISTS CLAIM_DETAIL; DROP TABLE IF EXISTS staging_CLAIM_DETAIL; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_DETAIL USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_DTL_ID ' ) AS (SELECT * FROM staging_CLAIM_DETAIL); DROP TABLE IF EXISTS CLAIM_PAYMENT_DETAIL; DROP TABLE IF EXISTS staging_CLAIM_PAYMENT_DETAIL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_PAYMENT_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_PAYMENT_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_PAYMENT_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_PAYMENT_DETAIL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_PAY_ID,CLM_PAY_DTL_ID ' ) AS (SELECT * FROM staging_CLAIM_PAYMENT_DETAIL); DROP TABLE IF EXISTS CLAIM_ATTRIBUTE; DROP TABLE IF EXISTS staging_CLAIM_ATTRIBUTE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_ATTRIBUTE USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_ATTR_ID ' ) AS (SELECT * FROM staging_CLAIM_ATTRIBUTE); DROP TABLE IF EXISTS CLAIM; DROP TABLE IF EXISTS staging_CLAIM; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID ' ) AS (SELECT * FROM staging_CLAIM); DROP TABLE IF EXISTS PERSON_CONTACT; DROP TABLE IF EXISTS staging_PERSON_CONTACT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_CONTACT USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CNTC_ID,PRSN_CNTC_ID ' ) AS (SELECT * FROM staging_PERSON_CONTACT); DROP TABLE IF EXISTS ORGANIZATION_CODE; DROP TABLE IF EXISTS staging_ORGANIZATION_CODE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATION_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ORGANIZATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ORGANIZATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATION_CODE USING column OPTIONS(partition_by 'ORG_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,ORG_ID,CD_VAL_ID,ORG_CD_ID ' ) AS (SELECT * FROM staging_ORGANIZATION_CODE); DROP TABLE IF EXISTS COMPLAINT_STATUS; DROP TABLE IF EXISTS staging_COMPLAINT_STATUS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_COMPLAINT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COMPLAINT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COMPLAINT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COMPLAINT_STATUS USING column OPTIONS(partition_by 'INQ_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,INQ_ID,COMPLAINT_ID,COMPLAINT_STAT_ID ' ) AS (SELECT * FROM staging_COMPLAINT_STATUS); DROP TABLE IF EXISTS CONTACT; DROP TABLE IF EXISTS staging_CONTACT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CONTACT USING column OPTIONS(partition_by 'CNTC_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CNTC_ID' ) AS (SELECT * FROM staging_CONTACT); DROP TABLE IF EXISTS CLAIM_PAYMENT; DROP TABLE IF EXISTS staging_CLAIM_PAYMENT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_PAYMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_PAYMENT USING column OPTIONS(partition_by 'CLM_PAY_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CLM_PAY_ID ' ) AS (SELECT * FROM staging_CLAIM_PAYMENT); DROP TABLE IF EXISTS TOPIC_COMMUNICATION; DROP TABLE IF EXISTS staging_TOPIC_COMMUNICATION; CREATE EXTERNAL TABLE IF NOT EXISTS staging_TOPIC_COMMUNICATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TOPIC_COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/TOPIC_COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS TOPIC_COMMUNICATION USING column OPTIONS(partition_by 'CMCN_INQ_ID', buckets '32',redundancy '1',key_columns ' CLIENT_ID,CMCN_INQ_ID,TPC_INQ_ID,CMCN_ID,TPC_ID' ) AS (SELECT * FROM staging_TOPIC_COMMUNICATION); DROP TABLE IF EXISTS CONTACT_TYPE_CONTACT; DROP TABLE IF EXISTS staging_CONTACT_TYPE_CONTACT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CONTACT_TYPE_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CONTACT_TYPE_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CONTACT_TYPE_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CONTACT_TYPE_CONTACT USING column OPTIONS(partition_by 'CNTC_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CNTC_ID,ORG_CNTC_TYP_ID,CNTC_TYP_CNTC_ID ' ) AS (SELECT * FROM staging_CONTACT_TYPE_CONTACT); DROP TABLE IF EXISTS TOPIC; DROP TABLE IF EXISTS staging_TOPIC; CREATE EXTERNAL TABLE IF NOT EXISTS staging_TOPIC - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TOPIC.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/TOPIC.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS TOPIC USING column OPTIONS(partition_by 'INQ_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,INQ_ID,TPC_ID ' ) AS (SELECT * FROM staging_TOPIC); DROP TABLE IF EXISTS LINE_ADDITIONAL_DIAGNOSIS; DROP TABLE IF EXISTS staging_LINE_ADDITIONAL_DIAGNOSIS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_LINE_ADDITIONAL_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LINE_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LINE_ADDITIONAL_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LINE_ADDITIONAL_DIAGNOSIS USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,SEQ_NUM,CLM_ADD_DIAG_ID,LN_ADD_DIAG_ID ' ) AS (SELECT * FROM staging_LINE_ADDITIONAL_DIAGNOSIS); DROP TABLE IF EXISTS PROCEDURE_CODE; DROP TABLE IF EXISTS staging_PROCEDURE_CODE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_PROCEDURE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PROCEDURE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PROCEDURE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PROCEDURE_CODE USING column OPTIONS(partition_by 'PR_CD_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PR_CD_ID ' ) AS (SELECT * FROM staging_PROCEDURE_CODE); DROP TABLE IF EXISTS CODE_VALUE; DROP TABLE IF EXISTS staging_CODE_VALUE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CODE_VALUE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CODE_VALUE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CODE_VALUE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CODE_VALUE USING column OPTIONS(partition_by 'CD_VAL_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CD_VAL_ID') AS (SELECT * FROM staging_CODE_VALUE); DROP TABLE IF EXISTS POSTAL_ADDRESS; DROP TABLE IF EXISTS staging_POSTAL_ADDRESS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_POSTAL_ADDRESS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/POSTAL_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/POSTAL_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS POSTAL_ADDRESS USING column OPTIONS(partition_by 'CNTC_ID',redundancy '1',key_columns 'CLIENT_ID,CNTC_ID,PSTL_ADDR_ID') AS (SELECT * FROM staging_POSTAL_ADDRESS); DROP TABLE IF EXISTS CLIENTS; DROP TABLE IF EXISTS staging_CLIENTS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLIENTS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLIENTS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLIENTS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLIENTS USING column OPTIONS(partition_by 'CLIENT_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID ' ) AS (SELECT * FROM staging_CLIENTS); DROP TABLE IF EXISTS PERSONS; DROP TABLE IF EXISTS staging_PERSONS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSONS USING column OPTIONS(partition_by 'PRSN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID ' ) AS (SELECT * FROM staging_PERSONS); DROP TABLE IF EXISTS BANK; DROP TABLE IF EXISTS staging_BANK; CREATE EXTERNAL TABLE IF NOT EXISTS staging_BANK - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BANK.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BANK.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BANK USING column OPTIONS(partition_by 'BNK_ORG_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,BNK_ORG_ID,BNK_ID ' ) AS (SELECT * FROM staging_BANK); DROP TABLE IF EXISTS BILLING_ENTITY; DROP TABLE IF EXISTS staging_BILLING_ENTITY; CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY USING column OPTIONS(partition_by 'BILL_ENT_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,BILL_ENT_ID') AS (SELECT * FROM staging_BILLING_ENTITY); DROP TABLE IF EXISTS BENEFIT_PACKAGE; DROP TABLE IF EXISTS staging_BENEFIT_PACKAGE; CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_PACKAGE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE USING column OPTIONS(partition_by 'BENE_PKG_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,BENE_PKG_ID' ) AS (SELECT * FROM staging_BENEFIT_PACKAGE); DROP TABLE IF EXISTS GROUPS; DROP TABLE IF EXISTS staging_GROUPS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_GROUPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GROUPS USING column OPTIONS(partition_by 'GRP_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,GRP_ID' ) AS (SELECT * FROM staging_GROUPS); DROP TABLE IF EXISTS COMMUNICATION; DROP TABLE IF EXISTS staging_COMMUNICATION; CREATE EXTERNAL TABLE IF NOT EXISTS staging_COMMUNICATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COMMUNICATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COMMUNICATION USING column OPTIONS(partition_by 'INQ_ID', buckets '32',redundancy '1',key_columns ' CLIENT_ID,INQ_ID,CMCN_ID' ) AS (SELECT * FROM staging_COMMUNICATION); DROP TABLE IF EXISTS COMPLAINT; DROP TABLE IF EXISTS staging_COMPLAINT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_COMPLAINT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COMPLAINT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COMPLAINT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COMPLAINT USING column OPTIONS(partition_by 'INQ_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,INQ_ID,COMPLAINT_ID ' ) AS (SELECT * FROM staging_COMPLAINT); DROP TABLE IF EXISTS ADJUSTMENT; DROP TABLE IF EXISTS staging_ADJUSTMENT; CREATE EXTERNAL TABLE IF NOT EXISTS staging_ADJUSTMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ADJUSTMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ADJUSTMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ADJUSTMENT USING column OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,BILL_ENT_ID,ADJ_ID ' ) AS (SELECT * FROM staging_ADJUSTMENT); DROP TABLE IF EXISTS CLAIM_PAYMENT_REDUCTION; DROP TABLE IF EXISTS staging_CLAIM_PAYMENT_REDUCTION; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_PAYMENT_REDUCTION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_PAYMENT_REDUCTION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_PAYMENT_REDUCTION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_PAYMENT_REDUCTION USING column OPTIONS(partition_by 'CLM_PAY_RDCTN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CLM_PAY_RDCTN_ID ' ) AS (SELECT * FROM staging_CLAIM_PAYMENT_REDUCTION); DROP TABLE IF EXISTS CLAIM_REDUCTION_DETAIL; DROP TABLE IF EXISTS staging_CLAIM_REDUCTION_DETAIL; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_REDUCTION_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_REDUCTION_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_REDUCTION_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_REDUCTION_DETAIL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_PAY_RDCTN_ID,CLM_RDCTN_DTL_ID ' ) AS (SELECT * FROM staging_CLAIM_REDUCTION_DETAIL); DROP TABLE IF EXISTS CLAIM_REDUCTION_HISTORY; DROP TABLE IF EXISTS staging_CLAIM_REDUCTION_HISTORY; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_REDUCTION_HISTORY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_REDUCTION_HISTORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_REDUCTION_HISTORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_REDUCTION_HISTORY USING column OPTIONS(partition_by 'CLM_PAY_RDCTN_ID', buckets '32',redundancy '1',key_columns 'CLIENT_ID,CLM_PAY_RDCTN_ID,CLM_RDCTN_HIST_ID ' ) AS (SELECT * FROM staging_CLAIM_REDUCTION_HISTORY); DROP TABLE IF EXISTS CLAIM_COB; DROP TABLE IF EXISTS staging_CLAIM_COB; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_COB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_COB USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_COB_ID ' ) AS (SELECT * FROM staging_CLAIM_COB); DROP TABLE IF EXISTS CLAIM_HOSPITAL; DROP TABLE IF EXISTS staging_CLAIM_HOSPITAL; CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_HOSPITAL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_HOSPITAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_HOSPITAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_HOSPITAL USING column OPTIONS(partition_by 'PRSN_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,PRSN_ID,CLM_ID,CLM_HOSP_ID ' ) AS (SELECT * FROM staging_CLAIM_HOSPITAL); DROP TABLE IF EXISTS UM_INPATIENT; DROP TABLE IF EXISTS staging_UM_INPATIENT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_INPATIENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_INPATIENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT USING column OPTIONS(partition_by 'UM_RVW_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,UM_RVW_ID,UM_INPT_ID ' ) AS (SELECT * FROM staging_UM_INPATIENT); DROP TABLE IF EXISTS UM_SERVICE; DROP TABLE IF EXISTS staging_UM_SERVICE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_SERVICE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_SERVICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_SERVICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_SERVICE USING column OPTIONS(partition_by 'UM_RVW_ID',buckets '32',redundancy '1',key_columns 'CLIENT_ID,UM_RVW_ID,UM_SERV_ID ' ) AS (SELECT * FROM staging_UM_SERVICE); DROP TABLE IF EXISTS INQUIRY; DROP TABLE IF EXISTS staging_INQUIRY; CREATE EXTERNAL TABLE IF NOT EXISTS staging_INQUIRY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/INQUIRY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/INQUIRY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INQUIRY( INQ_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -238,7 +238,7 @@ INSERT INTO INQUIRY SELECT * FROM staging_INQUIRY; DROP TABLE IF EXISTS ORGANIZATIONS; DROP TABLE IF EXISTS staging_ORGANIZATIONS; CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ORGANIZATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ORGANIZATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATIONS(ORG_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -260,7 +260,7 @@ INSERT INTO ORGANIZATIONS SELECT * FROM staging_ORGANIZATIONS; DROP TABLE IF EXISTS UM_REVIEW; DROP TABLE IF EXISTS staging_UM_REVIEW; CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_REVIEW - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_REVIEW(UM_RVW_ID BIGINT NOT NULL, VER BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -309,7 +309,7 @@ DROP TABLE IF EXISTS INVOICE_DETAIL; DROP TABLE IF EXISTS staging_INVOICE_DETAIL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/INVOICE_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/INVOICE_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE_DETAIL( INV_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -338,7 +338,7 @@ DROP TABLE IF EXISTS BENEFIT_GROUP_NAME; DROP TABLE IF EXISTS staging_BENEFIT_GROUP_NAME; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_GROUP_NAME - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_GROUP_NAME.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_GROUP_NAME( GRP_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -362,7 +362,7 @@ DROP TABLE IF EXISTS BENEFIT_GROUPS; DROP TABLE IF EXISTS staging_BENEFIT_GROUPS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_GROUPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_GROUPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_GROUPS( GRP_ID BIGINT NOT NULL, BENE_PKG_ID BIGINT NOT NULL, @@ -385,7 +385,7 @@ DROP TABLE IF EXISTS BILLING_RECONCILIATION; DROP TABLE IF EXISTS staging_BILLING_RECONCILIATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_RECONCILIATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_RECONCILIATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_RECONCILIATION( BILL_ENT_ID BIGINT NOT NULL, BILL_RECON_ID BIGINT NOT NULL, @@ -410,7 +410,7 @@ DROP TABLE IF EXISTS CHECKS; DROP TABLE IF EXISTS staging_CHECKS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHECKS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHECKS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHECKS( CLM_PAY_ID BIGINT NOT NULL, CHK_ID BIGINT NOT NULL, @@ -436,7 +436,7 @@ DROP TABLE IF EXISTS CLIENT_REFERENCE_DATA; DROP TABLE IF EXISTS staging_CLIENT_REFERENCE_DATA; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLIENT_REFERENCE_DATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLIENT_REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLIENT_REFERENCE_DATA( CLIENT_REF_DTA_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -459,7 +459,7 @@ DROP TABLE IF EXISTS COB_CLAIM_DIAGNOSIS; DROP TABLE IF EXISTS staging_COB_CLAIM_DIAGNOSIS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_COB_CLAIM_DIAGNOSIS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COB_CLAIM_DIAGNOSIS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COB_CLAIM_DIAGNOSIS( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -481,7 +481,7 @@ DROP TABLE IF EXISTS COB_ORGANIZATION_PERSON; DROP TABLE IF EXISTS staging_COB_ORGANIZATION_PERSON; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_COB_ORGANIZATION_PERSON - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/COB_ORGANIZATION_PERSON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS COB_ORGANIZATION_PERSON( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -514,7 +514,7 @@ DROP TABLE IF EXISTS DIAGNOSIS_CODE; DROP TABLE IF EXISTS staging_DIAGNOSIS_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_DIAGNOSIS_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/DIAGNOSIS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/DIAGNOSIS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS DIAGNOSIS_CODE( DIAG_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -540,7 +540,7 @@ DROP TABLE IF EXISTS ELECTRONIC_ADDRESS; DROP TABLE IF EXISTS staging_ELECTRONIC_ADDRESS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ELECTRONIC_ADDRESS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ELECTRONIC_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ELECTRONIC_ADDRESS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ELECTRONIC_ADDRESS( CNTC_ID BIGINT NOT NULL, ELEC_ADDR_ID BIGINT NOT NULL, @@ -561,7 +561,7 @@ DROP TABLE IF EXISTS ENTITY_BANK_ACCOUNT; DROP TABLE IF EXISTS staging_ENTITY_BANK_ACCOUNT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ENTITY_BANK_ACCOUNT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ENTITY_BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ENTITY_BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ENTITY_BANK_ACCOUNT( bill_ent_id bigint NOT NULL, bnk_acct_id bigint NOT NULL, @@ -582,7 +582,7 @@ DROP TABLE IF EXISTS ETL_METADATA; DROP TABLE IF EXISTS staging_ETL_METADATA; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ETL_METADATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ETL_METADATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ETL_METADATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ETL_METADATA( MAIN_TBL VARCHAR(15) NOT NULL, TGT_TBL VARCHAR(15) NOT NULL, @@ -607,7 +607,7 @@ DROP TABLE IF EXISTS EXHIBIT; DROP TABLE IF EXISTS staging_EXHIBIT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT( EXHIBIT_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -625,7 +625,7 @@ DROP TABLE IF EXISTS EXPLANATION_CODE; DROP TABLE IF EXISTS staging_EXPLANATION_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXPLANATION_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/EXPLANATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/EXPLANATION_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXPLANATION_CODE( EXPLAIN_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -654,7 +654,7 @@ DROP TABLE IF EXISTS FILE_TRANSFER_RUN; DROP TABLE IF EXISTS staging_FILE_TRANSFER_RUN; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FILE_TRANSFER_RUN - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/FILE_TRANSFER_RUN.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/FILE_TRANSFER_RUN.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FILE_TRANSFER_RUN( FL_TRANS_RUN_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -688,7 +688,7 @@ DROP TABLE IF EXISTS FILE_TRANSFER_STAT; DROP TABLE IF EXISTS staging_FILE_TRANSFER_STAT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FILE_TRANSFER_STAT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/FILE_TRANSFER_STAT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/FILE_TRANSFER_STAT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FILE_TRANSFER_STAT( FL_TRANS_STAT_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -727,7 +727,7 @@ DROP TABLE IF EXISTS GENERAL_LEDGER; DROP TABLE IF EXISTS staging_GENERAL_LEDGER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GENERAL_LEDGER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/GENERAL_LEDGER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/GENERAL_LEDGER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GENERAL_LEDGER( BILL_ENT_ID BIGINT NOT NULL, GL_ID BIGINT NOT NULL, @@ -754,7 +754,7 @@ DROP TABLE IF EXISTS GROUP_RELATION; DROP TABLE IF EXISTS staging_GROUP_RELATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GROUP_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/GROUP_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/GROUP_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GROUP_RELATION( GRP_ID BIGINT NOT NULL, GRP_RELN_ID BIGINT NOT NULL, @@ -774,7 +774,7 @@ DROP TABLE IF EXISTS LEP_APPEAL; DROP TABLE IF EXISTS staging_LEP_APPEAL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LEP_APPEAL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LEP_APPEAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LEP_APPEAL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LEP_APPEAL( PRSN_ID BIGINT NOT NULL, LEP_APL_ID BIGINT NOT NULL, @@ -796,7 +796,7 @@ DROP TABLE IF EXISTS LETTER; DROP TABLE IF EXISTS staging_LETTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -814,7 +814,7 @@ DROP TABLE IF EXISTS LINE_DISALLOW_EXPLANATION; DROP TABLE IF EXISTS staging_BLINE_DISALLOW_EXPLANATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LINE_DISALLOW_EXPLANATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LINE_DISALLOW_EXPLANATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LINE_DISALLOW_EXPLANATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LINE_DISALLOW_EXPLANATION( PRSN_ID BIGINT NOT NULL, CLM_ID BIGINT NOT NULL, @@ -837,7 +837,7 @@ DROP TABLE IF EXISTS LINE_PROCEDURE_MODIFIER; DROP TABLE IF EXISTS staging_LINE_PROCEDURE_MODIFIER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LINE_PROCEDURE_MODIFIER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LINE_PROCEDURE_MODIFIER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LINE_PROCEDURE_MODIFIER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LINE_PROCEDURE_MODIFIER( PRSN_ID BIGINT NOT NULL, CLM_ID BIGINT NOT NULL, @@ -859,7 +859,7 @@ DROP TABLE IF EXISTS MARX_CALENDAR; DROP TABLE IF EXISTS staging_MARX_CALENDAR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MARX_CALENDAR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/MARX_CALENDAR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/MARX_CALENDAR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MARX_CALENDAR( MARX_CAL_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -881,7 +881,7 @@ DROP TABLE IF EXISTS NOTE; DROP TABLE IF EXISTS staging_NOTE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_NOTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/NOTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/NOTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS NOTE( INQ_ID BIGINT NOT NULL, SRC_ID BIGINT NOT NULL, @@ -906,7 +906,7 @@ DROP TABLE IF EXISTS ORGANIZATION_CONTACT_TYPE; DROP TABLE IF EXISTS staging_ORGANIZATION_CONTACT_TYPE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATION_CONTACT_TYPE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ORGANIZATION_CONTACT_TYPE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ORGANIZATION_CONTACT_TYPE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATION_CONTACT_TYPE( ORG_ID BIGINT NOT NULL, CNTC_TYP_REF_ID BIGINT NOT NULL, @@ -925,7 +925,7 @@ DROP TABLE IF EXISTS PAYMENT; DROP TABLE IF EXISTS staging_PAYMENT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PAYMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PAYMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PAYMENT( BILL_ENT_ID BIGINT NOT NULL, PAY_ID BIGINT NOT NULL, @@ -953,7 +953,7 @@ DROP TABLE IF EXISTS PERSON_COB; DROP TABLE IF EXISTS staging_PERSON_COB; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_COB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_COB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_COB( PRSN_ID BIGINT NOT NULL, PRSN_COB_ID BIGINT NOT NULL, @@ -1004,7 +1004,7 @@ DROP TABLE IF EXISTS PERSON_CODE; DROP TABLE IF EXISTS staging_PERSON_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_CODE( PRSN_ID BIGINT NOT NULL, CD_VAL_ID BIGINT NOT NULL, @@ -1025,7 +1025,7 @@ DROP TABLE IF EXISTS PERSON_LEP_EVENT; DROP TABLE IF EXISTS staging_PERSON_LEP_EVENT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_LEP_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_LEP_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_EVENT( PRSN_ID BIGINT NOT NULL, PRSN_LEP_EVNT_ID BIGINT NOT NULL, @@ -1047,7 +1047,7 @@ DROP TABLE IF EXISTS PERSON_LEP_PROFILE; DROP TABLE IF EXISTS staging_PERSON_LEP_PROFILE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_PROFILE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_LEP_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_LEP_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE( PRSN_ID BIGINT NOT NULL, PRSN_LEP_PRFL_ID BIGINT NOT NULL, @@ -1067,7 +1067,7 @@ DROP TABLE IF EXISTS PERSON_ORGANIZATION_RELATION; DROP TABLE IF EXISTS staging_PERSON_ORGANIZATION_RELATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ORGANIZATION_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_ORGANIZATION_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_ORGANIZATION_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ORGANIZATION_RELATION( PRSN_ID BIGINT NOT NULL, ORG_ID BIGINT NOT NULL, @@ -1089,7 +1089,7 @@ DROP TABLE IF EXISTS PERSON_PAYMENT_OPTIONS; DROP TABLE IF EXISTS staging_PERSON_PAYMENT_OPTIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_PAYMENT_OPTIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_PAYMENT_OPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_PAYMENT_OPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_PAYMENT_OPTIONS( PRSN_ID BIGINT NOT NULL, PRSN_PAY_OPTN_ID BIGINT NOT NULL, @@ -1111,7 +1111,7 @@ DROP TABLE IF EXISTS PERSON_RELATION; DROP TABLE IF EXISTS staging_PERSON_RELATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_RELATION( PRSN_ID BIGINT NOT NULL, RLTD_PRSN_ID BIGINT NOT NULL, @@ -1133,7 +1133,7 @@ DROP TABLE IF EXISTS PERSON_ROLE; DROP TABLE IF EXISTS staging_PERSON_ROLE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ROLE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_ROLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_ROLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ROLE( PRSN_ID BIGINT NOT NULL, PRSN_TYP_ID BIGINT NOT NULL, @@ -1154,7 +1154,7 @@ DROP TABLE IF EXISTS PERSON_SUBSIDY_PROFILE; DROP TABLE IF EXISTS staging_PERSON_SUBSIDY_PROFILE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_SUBSIDY_PROFILE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_SUBSIDY_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_SUBSIDY_PROFILE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_SUBSIDY_PROFILE( PRSN_ID BIGINT NOT NULL, PRSN_SBSDY_PRFL_ID BIGINT NOT NULL, @@ -1177,7 +1177,7 @@ DROP TABLE IF EXISTS PERSON_WORK_ITEM; DROP TABLE IF EXISTS staging_PERSON_WORK_ITEM; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_WORK_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_WORK_ITEM( PRSN_ID BIGINT NOT NULL, PRSN_WRK_ITM_ID BIGINT NOT NULL, @@ -1200,7 +1200,7 @@ DROP TABLE IF EXISTS PHONE; DROP TABLE IF EXISTS staging_PHONE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PHONE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PHONE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PHONE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PHONE( CNTC_ID BIGINT NOT NULL, PHN_ID BIGINT NOT NULL, @@ -1222,7 +1222,7 @@ DROP TABLE IF EXISTS PLAN_MEMBERSHIPS; DROP TABLE IF EXISTS staging_PLAN_MEMBERSHIPS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PLAN_MEMBERSHIPS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PLAN_MEMBERSHIPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PLAN_MEMBERSHIPS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PLAN_MEMBERSHIPS( PRSN_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -1249,7 +1249,7 @@ DROP TABLE IF EXISTS POS_CODE; DROP TABLE IF EXISTS staging_POS_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_POS_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/POS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/POS_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS POS_CODE( POS_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1271,7 +1271,7 @@ DROP TABLE IF EXISTS PREMIUM; DROP TABLE IF EXISTS staging_PREMIUM; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PREMIUM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PREMIUM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM( GRP_ID BIGINT NOT NULL, PREM_CAT_ID BIGINT NOT NULL, @@ -1295,7 +1295,7 @@ DROP TABLE IF EXISTS REFERENCE_DATA; DROP TABLE IF EXISTS staging_REFERENCE_DATAY; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_REFERENCE_DATA - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/REFERENCE_DATA.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS REFERENCE_DATA( REF_DTA_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1312,7 +1312,7 @@ DROP TABLE IF EXISTS RETURNED_MAIL; DROP TABLE IF EXISTS staging_RETURNED_MAIL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_RETURNED_MAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/RETURNED_MAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/RETURNED_MAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS RETURNED_MAIL( RTRN_MAIL_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1334,7 +1334,7 @@ DROP TABLE IF EXISTS UM_ACTIVITY; DROP TABLE IF EXISTS staging_UM_ACTIVITY; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_ACTIVITY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_ACTIVITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_ACTIVITY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_ACTIVITY( UM_RVW_ID BIGINT NOT NULL, UM_ACTY_ID BIGINT NOT NULL, @@ -1363,7 +1363,7 @@ DROP TABLE IF EXISTS UM_DIAGNOSIS_LINE; DROP TABLE IF EXISTS staging_UM_DIAGNOSIS_LINE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_DIAGNOSIS_LINE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_DIAGNOSIS_LINE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_DIAGNOSIS_LINE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_LINE( UM_RVW_ID BIGINT NOT NULL, UM_DIAG_SET_ID BIGINT NOT NULL, @@ -1386,7 +1386,7 @@ DROP TABLE IF EXISTS UM_DIAGNOSIS_SET; DROP TABLE IF EXISTS staging_UM_DIAGNOSIS_SET; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_DIAGNOSIS_SET - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_DIAGNOSIS_SET.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_DIAGNOSIS_SET.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_DIAGNOSIS_SET( UM_RVW_ID BIGINT NOT NULL, UM_DIAG_SET_ID BIGINT NOT NULL, @@ -1407,7 +1407,7 @@ DROP TABLE IF EXISTS BENEFIT_PACKAGE_RELATION; DROP TABLE IF EXISTS staging_BENEFIT_PACKAGE_RELATION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_PACKAGE_RELATION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE_RELATION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_RELATION( BENE_PKG_ID BIGINT NOT NULL, PKG_RELN_ID BIGINT NOT NULL, @@ -1427,7 +1427,7 @@ DROP TABLE IF EXISTS BILLING_ENTITY_CONTACT; DROP TABLE IF EXISTS staging_BILLING_ENTITY_CONTACT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_CONTACT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_CONTACT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_CONTACT( BILL_ENT_ID BIGINT NOT NULL, CNTC_ID BIGINT NOT NULL, @@ -1449,7 +1449,7 @@ DROP TABLE IF EXISTS BILLING_ENTITY_DETAIL; DROP TABLE IF EXISTS staging_BILLING_ENTITY_DETAIL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_DETAIL( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -1469,7 +1469,7 @@ DROP TABLE IF EXISTS CHECK_STATUS; DROP TABLE IF EXISTS staging_CHECK_STATUS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHECK_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHECK_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHECK_STATUS( CLM_PAY_ID BIGINT NOT NULL, CHK_ID BIGINT NOT NULL, @@ -1494,7 +1494,7 @@ DROP TABLE IF EXISTS EXHIBIT_GROUP; DROP TABLE IF EXISTS staging_EXHIBIT_GROUP; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT_GROUP - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/EXHIBIT_GROUP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/EXHIBIT_GROUP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP( EXHIBIT_GRP_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1517,7 +1517,7 @@ DROP TABLE IF EXISTS EXHIBIT_GROUP_EXHIBIT; DROP TABLE IF EXISTS staging_EXHIBIT_GROUP_EXHIBIT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_EXHIBIT_GROUP_EXHIBIT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/EXHIBIT_GROUP_EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/EXHIBIT_GROUP_EXHIBIT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS EXHIBIT_GROUP_EXHIBIT( EXHIBIT_ID BIGINT NOT NULL, EXHIBIT_GRP_ID BIGINT NOT NULL, @@ -1539,7 +1539,7 @@ DROP TABLE IF EXISTS GENERAL_LEDGER_MAP; DROP TABLE IF EXISTS staging_GENERAL_LEDGER_MAP; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_GENERAL_LEDGER_MAP - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/GENERAL_LEDGER_MAP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/GENERAL_LEDGER_MAP.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS GENERAL_LEDGER_MAP( GL_MAP_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1560,7 +1560,7 @@ DROP TABLE IF EXISTS LEP_APPEAL_DECISION; DROP TABLE IF EXISTS staging_LEP_APPEAL_DECISION; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LEP_APPEAL_DECISION - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LEP_APPEAL_DECISION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LEP_APPEAL_DECISION.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LEP_APPEAL_DECISION( PRSN_ID BIGINT NOT NULL, LEP_APL_ID BIGINT NOT NULL, @@ -1583,7 +1583,7 @@ DROP TABLE IF EXISTS LETTER_DETAIL; DROP TABLE IF EXISTS staging_LETTER_DETAIL; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_DETAIL - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LETTER_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LETTER_DETAIL.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_DETAIL( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -1602,7 +1602,7 @@ DROP TABLE IF EXISTS LETTER_JOB; DROP TABLE IF EXISTS staging_LETTER_JOB; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_JOB - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LETTER_JOB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LETTER_JOB.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_JOB( PRSN_ID BIGINT NOT NULL, LTR_DTL_ID BIGINT NOT NULL, @@ -1627,7 +1627,7 @@ DROP TABLE IF EXISTS LETTER_REGISTER; DROP TABLE IF EXISTS staging_LETTER_REGISTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_REGISTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LETTER_REGISTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LETTER_REGISTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_REGISTER( PRSN_ID BIGINT NOT NULL, LTR_ID BIGINT NOT NULL, @@ -1651,7 +1651,7 @@ DROP TABLE IF EXISTS LETTER_WORK_ITEM; DROP TABLE IF EXISTS staging_LETTER_WORK_ITEM; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_LETTER_WORK_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/LETTER_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/LETTER_WORK_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS LETTER_WORK_ITEM( PRSN_ID BIGINT NOT NULL, LTR_RGSTR_ID BIGINT NOT NULL, @@ -1672,7 +1672,7 @@ DROP TABLE IF EXISTS PERSON_EVENT_STATUS; DROP TABLE IF EXISTS staging_PERSON_EVENT_STATUS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_EVENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_EVENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS( PRSN_EVNT_ID BIGINT NOT NULL, PRSN_EVNT_STAT_ID BIGINT NOT NULL, @@ -1692,7 +1692,7 @@ DROP TABLE IF EXISTS PERSON_EVENT_STATUS_REASON; DROP TABLE IF EXISTS staging_PERSON_EVENT_STATUS_REASON; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_EVENT_STATUS_REASON - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_EVENT_STATUS_REASON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_EVENT_STATUS_REASON.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_EVENT_STATUS_REASON( PRSN_EVNT_STAT_ID BIGINT NOT NULL, PRSN_EVNT_STAT_RSN_ID BIGINT NOT NULL, @@ -1711,7 +1711,7 @@ DROP TABLE IF EXISTS PERSON_LEP_PROFILE_RECORD; DROP TABLE IF EXISTS staging_PERSON_LEP_PROFILE_RECORD; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_LEP_PROFILE_RECORD - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_LEP_PROFILE_RECORD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_LEP_PROFILE_RECORD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_LEP_PROFILE_RECORD( PRSN_ID BIGINT NOT NULL, PRSN_LEP_PRFL_ID BIGINT NOT NULL, @@ -1742,7 +1742,7 @@ DROP TABLE IF EXISTS PREMIUM_CATEGORY; DROP TABLE IF EXISTS staging_PREMIUM_CATEGORY; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_CATEGORY - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PREMIUM_CATEGORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PREMIUM_CATEGORY.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_CATEGORY( GRP_ID BIGINT NOT NULL, PREM_CAT_ID BIGINT NOT NULL, @@ -1763,7 +1763,7 @@ DROP TABLE IF EXISTS PREMIUM_PART; DROP TABLE IF EXISTS staging_PREMIUM_PART; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_PART - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PREMIUM_PART.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PREMIUM_PART.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_PART( GRP_ID BIGINT NOT NULL, PREM_PART_ID BIGINT NOT NULL, @@ -1786,7 +1786,7 @@ DROP TABLE IF EXISTS PREMIUM_TABLE; DROP TABLE IF EXISTS staging_PREMIUM_TABLE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PREMIUM_TABLE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PREMIUM_TABLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PREMIUM_TABLE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PREMIUM_TABLE( GRP_ID BIGINT NOT NULL, BENE_GRP_ID BIGINT NOT NULL, @@ -1808,7 +1808,7 @@ DROP TABLE IF EXISTS UM_INPATIENT_REVIEW; DROP TABLE IF EXISTS staging_UM_INPATIENT_REVIEW; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_REVIEW - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_INPATIENT_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_INPATIENT_REVIEW.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_REVIEW( UM_RVW_ID BIGINT NOT NULL, UM_INPT_RVW_ID BIGINT NOT NULL, @@ -1837,7 +1837,7 @@ DROP TABLE IF EXISTS UM_INPATIENT_STATUS; DROP TABLE IF EXISTS staging_UM_INPATIENT_STATUS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_INPATIENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_INPATIENT_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_STATUS( UM_RVW_ID BIGINT NOT NULL, UM_INPT_STAT BIGINT NOT NULL, @@ -1861,7 +1861,7 @@ DROP TABLE IF EXISTS UM_SERVICE_OVERRIDE; DROP TABLE IF EXISTS staging_UM_SERVICE_OVERRIDE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_SERVICE_OVERRIDE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_SERVICE_OVERRIDE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_SERVICE_OVERRIDE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_SERVICE_OVERRIDE( UM_RVW_ID BIGINT NOT NULL, UM_SERV_ID BIGINT NOT NULL, @@ -1888,7 +1888,7 @@ DROP TABLE IF EXISTS BANK_ACCOUNT; DROP TABLE IF EXISTS staging_BANK_ACCOUNT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BANK_ACCOUNT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BANK_ACCOUNT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BANK_ACCOUNT( BNK_ORG_ID BIGINT NOT NULL, BNK_ID BIGINT NOT NULL, @@ -1910,7 +1910,7 @@ DROP TABLE IF EXISTS UM_INPATIENT_STAY_LENGTH; DROP TABLE IF EXISTS staging_UM_INPATIENT_STAY_LENGTH; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UM_INPATIENT_STAY_LENGTH - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UM_INPATIENT_STAY_LENGTH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UM_INPATIENT_STAY_LENGTH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UM_INPATIENT_STAY_LENGTH( UM_RVW_ID BIGINT NOT NULL, UM_INPT_RVW_ID BIGINT NOT NULL, @@ -1945,7 +1945,7 @@ DROP TABLE IF EXISTS REVENUE_CODE; DROP TABLE IF EXISTS staging_REVENUE_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_REVENUE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/REVENUE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/REVENUE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS REVENUE_CODE( REV_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1968,7 +1968,7 @@ DROP TABLE IF EXISTS SERVICE_CODE; DROP TABLE IF EXISTS staging_SERVICE_CODE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICE_CODE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/SERVICE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICE_CODE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS SERVICE_CODE( SERV_CD_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -1989,7 +1989,7 @@ DROP TABLE IF EXISTS AGREEMENT; DROP TABLE IF EXISTS staging_AGREEMENT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_AGREEMENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/AGREEMENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS AGREEMENT( AGREE_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -2010,7 +2010,7 @@ DROP TABLE IF EXISTS ORGANIZATION_EVENT; DROP TABLE IF EXISTS staging_ORGANIZATION_EVENT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_ORGANIZATION_EVENT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/ORGANIZATION_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/ORGANIZATION_EVENT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS ORGANIZATION_EVENT( ORG_ID BIGINT NOT NULL, ORG_EVNT_ID BIGINT NOT NULL, @@ -2032,7 +2032,7 @@ DROP TABLE IF EXISTS FDI_TX_IDCARD; DROP TABLE IF EXISTS staging_FDI_TX_IDCARD; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_TX_IDCARD - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/FDI_TX_IDCARD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/FDI_TX_IDCARD.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_TX_IDCARD ( ACN_ID BIGINT NOT NULL, @@ -2071,7 +2071,7 @@ DROP TABLE IF EXISTS FDI_TX_LETTER; DROP TABLE IF EXISTS staging_FDI_TX_LETTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_TX_LETTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/FDI_TX_LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/FDI_TX_LETTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_TX_LETTER ( ACN_ID BIGINT NOT NULL, @@ -2107,7 +2107,7 @@ DROP TABLE IF EXISTS BENEFIT_PACKAGE_ATTRIBUTE; DROP TABLE IF EXISTS staging_BENEFIT_PACKAGE_ATTRIBUTE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BENEFIT_PACKAGE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BENEFIT_PACKAGE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BENEFIT_PACKAGE_ATTRIBUTE( BENE_PKG_ID BIGINT NOT NULL, BENE_PKG_ATTR_ID BIGINT NOT NULL, @@ -2128,7 +2128,7 @@ DROP TABLE IF EXISTS BILLING_ENTITY_SCHEDULE; DROP TABLE IF EXISTS staging_BILLING_ENTITY_SCHEDULE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_ENTITY_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_ENTITY_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_ENTITY_SCHEDULE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2151,7 +2151,7 @@ DROP TABLE IF EXISTS BILLING_SCHEDULE; DROP TABLE IF EXISTS staging_BILLING_SCHEDULE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_SCHEDULE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_SCHEDULE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_SCHEDULE( BILL_SCHD_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2172,7 +2172,7 @@ DROP TABLE IF EXISTS BILLING_SOURCE; DROP TABLE IF EXISTS staging_BILLING_SOURCE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_BILLING_SOURCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/BILLING_SOURCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS BILLING_SOURCE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2191,7 +2191,7 @@ DROP TABLE IF EXISTS CHARGE_ITEM; DROP TABLE IF EXISTS staging_CHARGE_ITEM; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CHARGE_ITEM - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CHARGE_ITEM.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CHARGE_ITEM( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2217,7 +2217,7 @@ DROP TABLE IF EXISTS CLAIM_COSHARE_TRACKING; DROP TABLE IF EXISTS staging_CLAIM_COSHARE_TRACKING; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_COSHARE_TRACKING - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_COSHARE_TRACKING.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_COSHARE_TRACKING( PRSN_ID BIGINT NOT NULL, VER BIGINT, @@ -2246,7 +2246,7 @@ DROP TABLE IF EXISTS CLAIM_LINE_ATTRIBUTE; DROP TABLE IF EXISTS staging_CLAIM_LINE_ATTRIBUTE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_CLAIM_LINE_ATTRIBUTE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/CLAIM_LINE_ATTRIBUTE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS CLAIM_LINE_ATTRIBUTE( PRSN_ID BIGINT NOT NULL, CLM_DTL_ID BIGINT NOT NULL, @@ -2271,7 +2271,7 @@ DROP TABLE IF EXISTS FDI_CORRESPONDENCE; DROP TABLE IF EXISTS staging_FDI_CORRESPONDENCE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_FDI_CORRESPONDENCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/FDI_CORRESPONDENCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/FDI_CORRESPONDENCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS FDI_CORRESPONDENCE ( CLIENT_ID BIGINT NOT NULL, @@ -2294,7 +2294,7 @@ DROP TABLE IF EXISTS INVOICE; DROP TABLE IF EXISTS staging_INVOICE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/INVOICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/INVOICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2331,7 +2331,7 @@ DROP TABLE IF EXISTS INVOICE_STATUS; DROP TABLE IF EXISTS staging_INVOICE_STATUS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_INVOICE_STATUS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/INVOICE_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/INVOICE_STATUS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS INVOICE_STATUS( INV_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2352,7 +2352,7 @@ DROP TABLE IF EXISTS MOOP_BALANCE_EXCEPTIONS; DROP TABLE IF EXISTS staging_MOOP_BALANCE_EXCEPTIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_BALANCE_EXCEPTIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/MOOP_BALANCE_EXCEPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/MOOP_BALANCE_EXCEPTIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_BALANCE_EXCEPTIONS( CLIENT_ID INTEGER NOT NULL, VLD_FRM_DT DATE NOT NULL, @@ -2376,7 +2376,7 @@ DROP TABLE IF EXISTS MOOP_BALANCE; DROP TABLE IF EXISTS staging_MOOP_BALANCE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_BALANCE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/MOOP_BALANCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/MOOP_BALANCE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_BALANCE( PRSN_ID BIGINT NOT NULL, CLIENT_ID INTEGER NOT NULL, @@ -2402,7 +2402,7 @@ DROP TABLE IF EXISTS MOOP_ACCUMULATOR; DROP TABLE IF EXISTS staging_MOOP_ACCUMULATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_MOOP_ACCUMULATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/MOOP_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/MOOP_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS MOOP_ACCUMULATOR ( SUPPL_ID VARCHAR(15) NOT NULL, CLIENT_ID INTEGER NOT NULL, @@ -2424,7 +2424,7 @@ DROP TABLE IF EXISTS PERSON_ACCUMULATOR; DROP TABLE IF EXISTS staging_PERSON_ACCUMULATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PERSON_ACCUMULATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PERSON_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PERSON_ACCUMULATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PERSON_ACCUMULATOR( PRSN_ID BIGINT NOT NULL, PRSN_ACCUM_ID BIGINT NOT NULL, @@ -2455,7 +2455,7 @@ DROP TABLE IF EXISTS PROCEDURE_PRICE; DROP TABLE IF EXISTS staging_PROCEDURE_PRICE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_PROCEDURE_PRICE - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/PROCEDURE_PRICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/PROCEDURE_PRICE.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS PROCEDURE_PRICE( PR_CD_ID BIGINT NOT NULL, PR_PRC_ID BIGINT NOT NULL, @@ -2479,7 +2479,7 @@ DROP TABLE IF EXISTS RECEIPT; DROP TABLE IF EXISTS staging_RECEIPT; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_RECEIPT - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/RECEIPT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/RECEIPT.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS RECEIPT( BILL_ENT_ID BIGINT NOT NULL, CLIENT_ID BIGINT NOT NULL, @@ -2520,12 +2520,12 @@ CREATE TABLE IF NOT EXISTS RECEIPT( ) USING row OPTIONS(partition_by 'BILL_ENT_ID',buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); INSERT INTO RECEIPT SELECT * FROM staging_RECEIPT; -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_PLAN_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PLAN_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PLAN_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_PLAN_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PLAN_CODE_CONFIG ( PLAN_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2555,14 +2555,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PLAN_CODE_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (PLAN_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'PLAN_CODE_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_PLAN_CODE_CONFIG; + INSERT INTO SERVICES_PLAN_CODE_CONFIG SELECT * FROM staging_SERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS staging_TMGSERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS staging_SERVICES_KEY_GENERATOR; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_KEY_GENERATOR - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_KEY_GENERATOR + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_KEY_GENERATOR.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_KEY_GENERATOR ( KEY_GEN_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2580,14 +2580,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_KEY_GENERATOR SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (KEY_GEN_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'KEY_GEN_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true' ); - INSERT INTO TMGSERVICES_KEY_GENERATOR SELECT * FROM staging_TMGSERVICES_KEY_GENERATOR; + INSERT INTO SERVICES_KEY_GENERATOR SELECT * FROM staging_SERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_GROUP_CODE_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); - CREATE TABLE IF NOT EXISTS TMGSERVICES_GROUP_CODE_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_GROUP_CODE_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_GROUP_CODE_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + CREATE TABLE IF NOT EXISTS SERVICES_GROUP_CODE_CONFIG ( GROUP_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2606,14 +2606,14 @@ CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_GROUP_CODE_CONFIG PRIMARY KEY (GROUP_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'GROUP_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_TMGSERVICES_GROUP_CODE_CONFIG; + INSERT INTO SERVICES_GROUP_CODE_CONFIG SELECT * FROM staging_SERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DNIS_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DNIS_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DNIS_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_DNIS_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DNIS_CONFIG ( DNIS_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2632,14 +2632,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DNIS_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DNIS_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DNIS_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_DNIS_CONFIG SELECT * FROM staging_TMGSERVICES_DNIS_CONFIG; + INSERT INTO SERVICES_DNIS_CONFIG SELECT * FROM staging_SERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS staging_SERVICES_DENIAL_REASON_CONFIG; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_DENIAL_REASON_CONFIG - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_DENIAL_REASON_CONFIG + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_DENIAL_REASON_CONFIG.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_DENIAL_REASON_CONFIG ( DENIAL_REASON_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2656,14 +2656,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_DENIAL_REASON_CONFIG SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (DENIAL_REASON_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'DENIAL_REASON_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_TMGSERVICES_DENIAL_REASON_CONFIG; + INSERT INTO SERVICES_DENIAL_REASON_CONFIG SELECT * FROM staging_SERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS staging_TMGSERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS staging_SERVICES_CLIENT_MASTER; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_CLIENT_MASTER - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_CLIENT_MASTER + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_CLIENT_MASTER.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_CLIENT_MASTER ( CLIENT_REF_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2671,7 +2671,7 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER CLIENT_LEGACY_CD VARCHAR(10) NULL, CLIENT_NAME VARCHAR(10) NULL, MEMBER_ID_FORMAT VARCHAR(15) NULL, - TMG_CALL_CLIENT_CODE VARCHAR(10) NULL, + CALL_CLIENT_CODE VARCHAR(10) NULL, CREATE_date date NULL, UPDATED_date date NULL, USER_NAME VARCHAR(15) NULL, @@ -2681,14 +2681,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_CLIENT_MASTER SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (CLIENT_REF_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'CLIENT_REF_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_CLIENT_MASTER SELECT * FROM staging_TMGSERVICES_CLIENT_MASTER; + INSERT INTO SERVICES_CLIENT_MASTER SELECT * FROM staging_SERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_SUBJECT_CATEGORY_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS ( SUBJ_CAT_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2710,14 +2710,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (SUBJ_CAT_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'SUBJ_CAT_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; + INSERT INTO SERVICES_SUBJECT_CATEGORY_TRANSLATIONS SELECT * FROM staging_SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_PTMR_VARIABLE_TRANSLATIONS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS ( VARIABLE_ID INT NOT NULL, VER BIGINT NOT NULL, @@ -2735,14 +2735,14 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS PRIMARY KEY (VARIABLE_ID) ) USING row OPTIONS(partition_by 'VARIABLE_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; + INSERT INTO SERVICES_PTMR_VARIABLE_TRANSLATIONS SELECT * FROM staging_SERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; -DROP TABLE IF EXISTS staging_TMGSERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS staging_SERVICES_ACCOUNTING_CODES; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- -CREATE EXTERNAL TABLE IF NOT EXISTS staging_TMGSERVICES_ACCOUNTING_CODES - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/TMGSERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); -CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES +CREATE EXTERNAL TABLE IF NOT EXISTS staging_SERVICES_ACCOUNTING_CODES + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/SERVICES_ACCOUNTING_CODES.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); +CREATE TABLE IF NOT EXISTS SERVICES_ACCOUNTING_CODES ( ACCOUNTING_CODE_ID INT NOT NULL, CLIENT_REF_ID INT NOT NULL, @@ -2765,13 +2765,13 @@ CREATE TABLE IF NOT EXISTS TMGSERVICES_ACCOUNTING_CODES SRC_SYS_REC_ID VARCHAR(15) NULL, PRIMARY KEY (ACCOUNTING_CODE_ID,CLIENT_ID) ) USING row OPTIONS(partition_by 'ACCOUNTING_CODE_ID,CLIENT_ID', buckets '32',redundancy '1',EVICTION_BY 'LRUHEAPPERCENT', overflow 'true'); - INSERT INTO TMGSERVICES_ACCOUNTING_CODES SELECT * FROM staging_TMGSERVICES_ACCOUNTING_CODES; + INSERT INTO SERVICES_ACCOUNTING_CODES SELECT * FROM staging_SERVICES_ACCOUNTING_CODES; DROP TABLE IF EXISTS UNAPPLIED_CASH; DROP TABLE IF EXISTS staging_UNAPPLIED_CASHE; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_UNAPPLIED_CASH - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/UNAPPLIED_CASH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/UNAPPLIED_CASH.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS UNAPPLIED_CASH( UNAPP_CSH_ID BIGINT NOT NULL, VER BIGINT NOT NULL, @@ -2794,7 +2794,7 @@ DROP TABLE IF EXISTS WORK_GENERATED_KEYS; DROP TABLE IF EXISTS staging_WORK_GENERATED_KEYS; ----- CREATE TEMPORARY STAGING TABLE TO LOAD CSV FORMATTED DATA ----- CREATE EXTERNAL TABLE IF NOT EXISTS staging_WORK_GENERATED_KEYS - USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/TMG_Data_20G/WORK_GENERATED_KEYS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); + USING com.databricks.spark.csv OPTIONS (path '/export/shared/QA_DATA/USER2_Data_20G/WORK_GENERATED_KEYS.dat', header 'true', inferSchema 'true', nullValue 'NULL', maxCharsPerColumn '4096'); CREATE TABLE IF NOT EXISTS WORK_GENERATED_KEYS( CLIENT_ID BIGINT NOT NULL, GEN_KEY_ID BIGINT NOT NULL, diff --git a/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInRightOrder.sql b/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInRightOrder.sql index 28ea11fc54..af18ec2b22 100644 --- a/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInRightOrder.sql +++ b/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInRightOrder.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS WORK_GENERATED_KEYS; DROP TABLE IF EXISTS UNAPPLIED_CASH; -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; DROP TABLE IF EXISTS RECEIPT; DROP TABLE IF EXISTS PROCEDURE_PRICE; DROP TABLE IF EXISTS PERSON_ACCUMULATOR; diff --git a/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInWrongOrder.sql b/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInWrongOrder.sql index af736bd372..4aeced3165 100644 --- a/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInWrongOrder.sql +++ b/dtests/src/resources/scripts/clusterRecovery/dropColocatedTablesInWrongOrder.sql @@ -127,14 +127,14 @@ DROP TABLE IF EXISTS MOOP_ACCUMULATOR; DROP TABLE IF EXISTS PERSON_ACCUMULATOR; DROP TABLE IF EXISTS PROCEDURE_PRICE; DROP TABLE IF EXISTS RECEIPT; -DROP TABLE IF EXISTS TMGSERVICES_PLAN_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_KEY_GENERATOR; -DROP TABLE IF EXISTS TMGSERVICES_GROUP_CODE_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DNIS_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_DENIAL_REASON_CONFIG; -DROP TABLE IF EXISTS TMGSERVICES_CLIENT_MASTER; -DROP TABLE IF EXISTS TMGSERVICES_SUBJECT_CATEGORY_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_PTMR_VARIABLE_TRANSLATIONS; -DROP TABLE IF EXISTS TMGSERVICES_ACCOUNTING_CODES; +DROP TABLE IF EXISTS SERVICES_PLAN_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_KEY_GENERATOR; +DROP TABLE IF EXISTS SERVICES_GROUP_CODE_CONFIG; +DROP TABLE IF EXISTS SERVICES_DNIS_CONFIG; +DROP TABLE IF EXISTS SERVICES_DENIAL_REASON_CONFIG; +DROP TABLE IF EXISTS SERVICES_CLIENT_MASTER; +DROP TABLE IF EXISTS SERVICES_SUBJECT_CATEGORY_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_PTMR_VARIABLE_TRANSLATIONS; +DROP TABLE IF EXISTS SERVICES_ACCOUNTING_CODES; DROP TABLE IF EXISTS UNAPPLIED_CASH; DROP TABLE IF EXISTS WORK_GENERATED_KEYS; diff --git a/dtests/src/test/java/io/snappydata/hydra/cdcConnector/cdcConnector.bt b/dtests/src/test/java/io/snappydata/hydra/cdcConnector/cdcConnector.bt index c54a3ef962..5675c0daaf 100644 --- a/dtests/src/test/java/io/snappydata/hydra/cdcConnector/cdcConnector.bt +++ b/dtests/src/test/java/io/snappydata/hydra/cdcConnector/cdcConnector.bt @@ -4,7 +4,7 @@ io/snappydata/hydra/cdcConnector/cdcBasicValidationTest.conf C=locator locatorHosts=1 locatorVMsPerHost=1 locatorThreadsPerVM=1 D=worker workerHosts=1 workerVMsPerHost=1 workerThreadsPerVM=1 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" queryFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/scriptsForApp1/selectOps.sql" deleteQFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/bulkDeleteTemp.sql" insertQueryPath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector" @@ -30,7 +30,7 @@ io/snappydata/hydra/cdcConnector/cdcBasicValidationTest.conf C=locator locatorHosts=1 locatorVMsPerHost=1 locatorThreadsPerVM=1 D=worker workerHosts=1 workerVMsPerHost=1 workerThreadsPerVM=1 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" queryFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/pointqueries.sql" deleteQFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/bulkDeleteTemp.sql" insertQueryPath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector" @@ -53,8 +53,8 @@ io/snappydata/hydra/cdcConnector/cdcBasicValidationTest.conf io/snappydata/hydra/cdcConnector/cdcConnectorMultiSqlServerInstanceTest.conf A=snappy snappyHosts=1 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocationCol="/export/shared/QA_DATA/TMG_Data/TMG_Cluster_Data" - dataFilesLocationRow="/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data" + dataFilesLocationCol="/export/shared/QA_DATA/USER2_Data/Cluster_Data" + dataFilesLocationRow="/export/shared/QA_DATA/USER2_Data/RowTable_Data" queryFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql" deleteQFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/bulkDelete.sql" insertQueryPath1="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/scriptsForApp1" @@ -76,8 +76,8 @@ io/snappydata/hydra/cdcConnector/cdcBasicValidationTest.conf io/snappydata/hydra/cdcConnector/cdcConnectorLongRunningHAWithMultiSqlServer.conf A=snappy snappyHosts=1 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocationCol="/export/shared/QA_DATA/TMG_Data/TMG_Cluster_Data" - dataFilesLocationRow="/export/shared/QA_DATA/TMG_Data/TMG_RowTable_Data" + dataFilesLocationCol="/export/shared/QA_DATA/USER2_Data/Cluster_Data" + dataFilesLocationRow="/export/shared/QA_DATA/USER2_Data/RowTable_Data" queryFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/pointLookUpQueries.sql" deleteQFilePath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/bulkDelete.sql" updateQueryPath="$GEMFIRE/../../../dtests/src/resources/scripts/cdcConnector/update.sql" diff --git a/dtests/src/test/java/io/snappydata/hydra/clusterRecovery/clusterRecovery.bt b/dtests/src/test/java/io/snappydata/hydra/clusterRecovery/clusterRecovery.bt index 91aced2c3a..dbb1f98e7d 100644 --- a/dtests/src/test/java/io/snappydata/hydra/clusterRecovery/clusterRecovery.bt +++ b/dtests/src/test/java/io/snappydata/hydra/clusterRecovery/clusterRecovery.bt @@ -1,7 +1,7 @@ io/snappydata/hydra/clusterRecovery/newNodeStartingFirstTest.conf A=snappy snappyHosts=1 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" logPath="/nfs/users/spillai" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" snappyFileLoc="$GEMFIRE/../snappy" @@ -11,7 +11,7 @@ io/snappydata/hydra/clusterRecovery/newNodeStartingFirstTest.conf io/snappydata/hydra/clusterRecovery/allLocatorsDownRebalanceTest.conf A=snappy snappyHosts=3 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" logPath="/nfs/users/spillai" snappyPath="$GEMFIRE/../snappy" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" @@ -21,7 +21,7 @@ io/snappydata/hydra/clusterRecovery/allLocatorsDownRebalanceTest.conf io/snappydata/hydra/clusterRecovery/newNodeRebalanceMeanKill.conf A=snappy snappyHosts=3 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" insertQueryPath1="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery" logPath="/nfs/users/spillai" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" @@ -31,7 +31,7 @@ io/snappydata/hydra/clusterRecovery/newNodeRebalanceMeanKill.conf io/snappydata/hydra/clusterRecovery/clusterRestartWithMinMem.conf A=snappy snappyHosts=3 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" logPath="/nfs/users/spillai" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" newNode="dev1" @@ -40,7 +40,7 @@ io/snappydata/hydra/clusterRecovery/clusterRestartWithMinMem.conf io/snappydata/hydra/clusterRecovery/colocationChainDropTest.conf A=snappy snappyHosts=1 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" logPath="/nfs/users/spillai" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" snappyPath="$GEMFIRE/../snappy" @@ -48,7 +48,7 @@ io/snappydata/hydra/clusterRecovery/colocationChainDropTest.conf io/snappydata/hydra/clusterRecovery/missingDiskStoreRecoveryTest.conf A=snappy snappyHosts=1 snappyVMsPerHost=1 snappyThreadsPerVM=10 testJar="$GEMFIRE/../../../dtests/build-artifacts/scala-2.11/libs/snappydata-store-scala-tests-0.1.0-SNAPSHOT-tests.jar" - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" logPath="/nfs/users/spillai" queryPath="$GEMFIRE/../../../dtests/src/resources/scripts/clusterRecovery/selectQuery.sql" snappyPath="$GEMFIRE/../snappy" diff --git a/dtests/src/test/java/io/snappydata/hydra/putInto/concPutInto.bt b/dtests/src/test/java/io/snappydata/hydra/putInto/concPutInto.bt index 650b9e6a2a..555b14992b 100644 --- a/dtests/src/test/java/io/snappydata/hydra/putInto/concPutInto.bt +++ b/dtests/src/test/java/io/snappydata/hydra/putInto/concPutInto.bt @@ -17,7 +17,7 @@ io/snappydata/hydra/putInto/putIntoMemLeak.conf A=snappyStore snappyStoreHosts=1 snappyStoreVMsPerHost=1 snappyStoreThreadsPerVM=1 B=lead leadHosts=1 leadVMsPerHost=1 leadThreadsPerVM=2 C=locator locatorHosts=1 locatorVMsPerHost=1 locatorThreadsPerVM=1 - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" startRange=1000001 endRange=2000001 threadCnt=5 @@ -28,7 +28,7 @@ io/snappydata/hydra/putInto/putIntoWithOverlappingKeys.conf A=snappyStore snappyStoreHosts=1 snappyStoreVMsPerHost=1 snappyStoreThreadsPerVM=1 B=lead leadHosts=1 leadVMsPerHost=1 leadThreadsPerVM=2 C=locator locatorHosts=1 locatorVMsPerHost=1 locatorThreadsPerVM=1 - dataFilesLocation="/export/shared/QA_DATA/TMG_Data_20G" + dataFilesLocation="/export/shared/QA_DATA/USER2_Data_20G" startRange=1000001 endRange=1001000 threadCnt=4 From ca906c64c64f77d844191f0f43d759bd9acd7349 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sun, 23 Dec 2018 03:40:00 +0530 Subject: [PATCH 2/5] Fix a deadlock between store catalog and spark one in local mode --- .../apache/spark/sql/hive/SnappyHiveExternalCatalog.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) 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 4e8b92d270..c7c1c4bc6d 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 @@ -21,6 +21,7 @@ import java.lang.reflect.InvocationTargetException import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.ExecutionException import com.gemstone.gemfire.cache.CacheClosedException import com.gemstone.gemfire.internal.cache.{LocalRegion, PartitionedRegion} @@ -145,6 +146,7 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, } function case e: InvocationTargetException => throw e.getCause + case e: ExecutionException => throw e.getCause } finally { skipFlags.skipDDLocks = oldSkipLocks skipFlags.skipHiveCatalogCalls = oldSkipCatalogCalls @@ -479,7 +481,11 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, if (nonExistentTables.getIfPresent(name) eq java.lang.Boolean.TRUE) { throw new TableNotFoundException(schema, table) } - cachedCatalogTables(name) + // need to do the load under a sync block to avoid deadlock due to lock inversion + // (sync block and map loader future) so do a get separately first + val catalogTable = cachedCatalogTables.getIfPresent(name) + if (catalogTable ne null) catalogTable + else withHiveExceptionHandling(cachedCatalogTables.get(name)) } override def getTableOption(schema: String, table: String): Option[CatalogTable] = { From ba9f5c56625e485c68f67018716551a73f307526 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 28 Dec 2018 02:44:41 +0530 Subject: [PATCH 3/5] Update gradle to version 5.0 - updated build files for gradle 5.0 - updated dependency versions to compatible recent ones where possible - changed all tests to avoid using println/show as far as possible and use logInfo instead; this is to avoid polluting the standard output with all kinds of information - updated NOTICE file with the current dependency jar versions --- NOTICE | 574 +++++++++--------- build.gradle | 475 ++++++++------- cluster/build.gradle | 5 +- .../snappydata/gemxd/SnappySystemAdmin.java | 28 +- cluster/src/test/resources/log4j.properties | 30 +- .../test/scala/io/snappydata/QueryTest.scala | 43 +- .../test/scala/io/snappydata/Snap_213.scala | 4 +- .../snappydata/benchmark/TPCH_Queries.scala | 13 +- .../PreparedQueryRoutingSingleNodeSuite.scala | 193 ++---- .../cluster/QueryRoutingSingleNodeSuite.scala | 110 +--- .../cluster/StringAsClobTestSuite.scala | 6 +- .../memory/SnappyMemoryAccountingSuite.scala | 8 +- .../org/apache/spark/sql/IndexTest.scala | 16 +- .../scala/org/apache/spark/sql/MiscTest.scala | 2 +- .../SnappyTableMutableAPISuite.scala | 4 +- .../benchmark/ColumnCacheBenchmark.scala | 7 +- .../sql/execution/benchmark/TAQTest.scala | 3 - .../kafka010/SnappyStructuredKafkaSuite.scala | 7 +- core/build.gradle | 20 +- .../SplitClusterDUnitSecurityTest.scala | 2 +- .../apache/spark/sql/CachedDataFrame.scala | 2 - .../org/apache/spark/sql/SnappySession.scala | 6 +- .../apache/spark/sql/SnappyStrategies.scala | 2 +- .../sql/execution/CodegenSparkFallback.scala | 14 +- .../spark/sql/execution/ExistingPlans.scala | 2 +- .../encoding/ColumnDeltaEncoder.scala | 2 +- .../sql/internal/SnappySessionState.scala | 8 +- .../spark/sql/store/CodeGeneration.scala | 8 +- core/src/test/resources/log4j.properties | 30 +- .../snappydata/ColumnUpdateDeleteTests.scala | 14 +- .../scala/io/snappydata/SnappyFunSuite.scala | 10 +- .../io/snappydata/SnappyTestRunner.scala | 79 ++- .../io/snappydata/app/SampleTableQuery.scala | 2 +- .../spark/jdbc/ConnectionConfTest.scala | 2 +- .../store/ColumnTableBatchInsertTest.scala | 92 ++- .../spark/sql/store/ColumnTableTest.scala | 18 +- .../spark/sql/store/CreateIndexTest.scala | 9 +- .../store/CubeRollupGroupingSetsTest.scala | 60 +- .../apache/spark/sql/store/RowTableTest.scala | 60 +- .../spark/sql/store/TokenizationTest.scala | 14 +- ...tication_connecting_to_a_secure_cluster.md | 2 +- dtests/build.gradle | 2 +- .../regression_docs/HowToRunRowStoreTests.md | 4 +- .../hydra/rowStoreRegressionScript.sh | 2 +- .../hydra/snappyRegressionScript.sh | 2 +- dunit/build.gradle | 2 +- gradle.properties | 3 +- gradle/wrapper/gradle-wrapper.jar | Bin 54711 -> 55741 bytes gradle/wrapper/gradle-wrapper.properties | 3 +- gradlew | 8 +- gradlew.bat | 2 +- jdbc/build.gradle | 4 - settings.gradle | 2 - spark | 2 +- spark-jobserver | 2 +- store | 2 +- 56 files changed, 978 insertions(+), 1048 deletions(-) diff --git a/NOTICE b/NOTICE index 262c7d21d9..d6cba4b41e 100644 --- a/NOTICE +++ b/NOTICE @@ -5,305 +5,301 @@ Copyright 2018 and onwards SnappyData Inc. This is a comprehensive list of software libraries used by SnappyData in version 1.0. More details on license types, license versions, and contributors can be found further down in this file -HikariCP-2.6.0.jar : Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -JavaEWAH-0.3.2.jar : Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -RoaringBitmap-0.5.11.jar: Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -ST4-4.0.4.jar: BSD License http://antlr.org/license.html -Vis.js Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -activation-1.1.1.jar: CDDL 1.0 https://glassfish.dev.java.net/public/CDDLv1.0.html -akka-actor_2.11-2.3.16.jar: Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -akka-cluster_2.11-2.3.16.jar: Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -akka-remote_2.11-2.3.16.jar: Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -akka-slf4j_2.11-2.3.16.jar: Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -ant-1.9.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -antlr-2.7.7.jar BSD License http://antlr.org/license.html -antlr-runtime-3.4.jar BSD License http://antlr.org/license.html -antlr4-runtime-4.5.3.jar BSD License http://antlr.org/license.html +HikariCP-2.7.9.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +JavaEWAH-0.3.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +RoaringBitmap-0.6.66.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +ST4-4.0.4.jar BSD License: http://antlr.org/license.html +Vis.js Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +activation-1.1.1.jar CDDL 1.0: https://glassfish.dev.java.net/public/CDDLv1.0.html +akka-actor_2.11-2.3.16.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +akka-cluster_2.11-2.3.16.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +akka-remote_2.11-2.3.16.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +akka-slf4j_2.11-2.3.16.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +ant-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +antlr-2.7.7.jar BSD License: http://antlr.org/license.html +antlr-runtime-3.4.jar BSD License: http://antlr.org/license.html +antlr4-runtime-4.5.3.jar BSD License: http://antlr.org/license.html aopalliance-1.0.jar Public Domain -aopalliance-repackaged-2.4.0-b34.jar CDDL/GPLv2+CE https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html -apache-log4j-extras-1.2.17.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -apacheds-i18n-2.0.0-M15.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -apacheds-kerberos-codec-2.0.0-M15.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -api-asn1-api-1.0.0-M20.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -api-util-1.0.0-M20.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -arpack_combined_all-0.1.jar BSD License http://www.opensource.org/licenses/bsd-license.php -avro-1.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -avro-ipc-1.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -avro-mapred-1.7.7-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +aopalliance-repackaged-2.5.0-b42.jar CDDL/GPLv2+CE: https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html +apache-log4j-extras-1.2.17.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +arpack_combined_all-0.1.jar BSD License: http://www.opensource.org/licenses/bsd-license.php +audience-annotations-0.5.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +avro-1.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +avro-ipc-1.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +avro-mapred-1.7.7-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 aws-java-sdk-1.7.4.jar Apache V2: https://aws.amazon.com/apache2.0 base64-2.3.8.jar Public Domain -bcprov-jdk15on-1.51.jar Bouncy Castle License http://www.bouncycastle.org/licence.html -bonecp-0.8.0.RELEASE.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -breeze-macros_2.11-0.12.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -breeze_2.11-0.12.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -calcite-avatica-1.2.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -calcite-core-1.2.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -calcite-linq4j-1.2.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -cglib-2.2.1-v20090111.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -chill-java-0.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -chill_2.11-0.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-beanutils-1.9.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-cli-1.3.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-codec-1.10.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-collections-3.2.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-compiler-3.0.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-compress-1.4.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-configuration-1.10.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-crypto-1.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-dbcp-1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-httpclient-3.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-io-2.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-lang-2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-lang3-3.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-logging-1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-math3-3.4.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-modeler-2.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-net-3.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-pool-1.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -commons-pool2-2.4.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -compress-lzf-1.0.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -concurrentlinkedhashmap-lru-1.4.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -config-1.3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -core-1.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -curator-client-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -curator-framework-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -curator-recipes-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -d3.js BSD-3 https://github.com/d3/d3/blob/master/LICENSE -datanucleus-api-jdo-3.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -datanucleus-core-3.2.10.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -datanucleus-rdbms-3.2.9.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -derby-10.12.1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -eclipse-collections-9.2.0.jar EPL-1.0: https://www.eclipse.org/legal/epl-v10.html -eigenbase-properties-1.1.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -flyway-core-3.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -gemfire-core-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -gemfire-jgroups-1.6.2.jar LGPL2.1 http://www.opensource.org/licenses/lgpl-2.1.php -gemfire-joptsimple-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -gemfire-json-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -gemfire-shared-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -gemfire-trove-1.6.2.jar LGPL2.1 http://www.opensource.org/licenses/lgpl-2.1.php -gson-2.2.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -guava-14.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -guice-3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -guice-servlet-3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +bcprov-jdk15on-1.52.jar Bouncy Castle License: http://www.bouncycastle.org/licence.html +bonecp-0.8.0.RELEASE.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +breeze-macros_2.11-0.13.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +breeze_2.11-0.13.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +calcite-avatica-1.4.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +calcite-core-1.4.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +calcite-linq4j-1.4.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +cglib-2.2.1-v20090111.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +chill-java-0.8.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +chill_2.11-0.8.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-beanutils-1.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-cli-1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-codec-1.11.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-collections-3.2.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-compiler-3.0.11.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-compress-1.4.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-configuration-1.10.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-crypto-1.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-dbcp-1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-httpclient-3.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-io-2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-lang-2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-lang3-3.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-logging-1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-math3-3.6.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-modeler-2.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-net-3.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-pool-1.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +commons-pool2-2.6.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +compress-lzf-1.0.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +concurrentlinkedhashmap-lru-1.4.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +config-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +core-1.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +curator-client-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +curator-framework-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +curator-recipes-2.7.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +d3.js BSD-3: https://github.com/d3/d3/blob/master/LICENSE +datanucleus-api-jdo-3.2.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +datanucleus-core-3.2.15.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +datanucleus-rdbms-3.2.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +derby-10.14.2.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +eclipse-collections-api-9.2.0.jar EPL-1.0: https://www.eclipse.org/legal/epl-v10.html +eclipse-collections-9.2.0.jar EPL-1.0: https://www.eclipse.org/legal/epl-v10.html +eigenbase-properties-1.1.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +flyway-core-3.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +gemfire-core-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +gemfire-jgroups-1.6.2.1.jar LGPL2.1: http://www.opensource.org/licenses/lgpl-2.1.php +gemfire-shared-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +gemfire-trove-1.6.2.1.jar LGPL2.1: http://www.opensource.org/licenses/lgpl-2.1.php +gemfire-util-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +gson-2.2.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +guava-14.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +guice-3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +guice-servlet-3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 h2-1.3.176.jar H2 License V1: http://h2database.com/html/license.html -hadoop-annotations-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-auth-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-aws-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-client-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-common-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-hdfs-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-mapreduce-client-app-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-mapreduce-client-common-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-mapreduce-client-core-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-mapreduce-client-jobclient-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-mapreduce-client-shuffle-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-api-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-client-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-common-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-server-common-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-server-nodemanager-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hadoop-yarn-server-web-proxy-2.7.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hammer.js MIT http://www.opensource.org/licenses/mit-license.php -hbase-client-0.98.17-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hbase-common-0.98.17-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hbase-protocol-0.98.17-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hbase-server-0.98.17-hadoop2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hive-beeline-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hive-cli-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hive-exec-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hive-jdbc-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hive-metastore-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -hk2-api-2.4.0-b34.jar CDDL/GPLv2+CE https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html -hk2-locator-2.4.0-b34.jar CDDL/GPLv2+CE https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html -hk2-utils-2.4.0-b34.jar CDDL/GPLv2+CE https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html -htrace-core-2.05.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -htrace-core-3.2.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -httpclient-4.5.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -httpcore-4.4.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -ivy-2.4.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-annotations-2.6.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-core-2.6.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-core-asl-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-databind-2.6.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-jaxrs-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-mapper-asl-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-module-paranamer-2.6.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-module-scala_2.11-2.6.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jackson-xc-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -janino-3.0.7.jar BSD https://raw.githubusercontent.com/janino-compiler/janino/master/LICENSE -java-xmlbuilder-1.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -javassist-3.18.1-GA.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -javax.annotation-api-1.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.inject-1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -javax.inject-2.4.0-b34.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.mail-api-1.5.5.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.resource-api-1.7.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.servlet-api-3.1.0.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.transaction-api-1.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javax.ws.rs-api-2.0.1.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -javolution-5.5.1.jar BSD http://javolution.org/LICENSE.txt -jaxb-api-2.2.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jcl-over-slf4j-1.7.21.jar MIT http://www.opensource.org/licenses/mit-license.php -jdo-api-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jersey-client-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-common-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-container-servlet-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-container-servlet-core-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-guava-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-media-jaxb-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jersey-server-2.22.2.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jets3t-0.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jettison-1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-6.1.26.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-client-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-continuation-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-http-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-io-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-jndi-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-plus-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-proxy-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-security-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-server-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-servlet-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-servlets-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-util-6.1.26.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-util-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-webapp-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jetty-xml-9.2.22.v20170606.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jline-2.14.2.jar BSD http://www.opensource.org/licenses/bsd-license.php -jna-4.2.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -joda-convert-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -joda-time-2.9.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jodd-core-3.5.2.jar BSD http://jodd.org/license.html +hadoop-annotations-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-auth-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-aws-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-client-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-common-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-hdfs-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-mapreduce-client-app-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-mapreduce-client-common-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-mapreduce-client-core-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-mapreduce-client-jobclient-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-mapreduce-client-shuffle-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-api-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-client-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-common-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-server-common-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-server-nodemanager-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hadoop-yarn-server-web-proxy-2.7.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hammer.js MIT: http://www.opensource.org/licenses/mit-license.php +hive-beeline-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hive-cli-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hive-exec-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hive-jdbc-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hive-metastore-1.2.1.spark2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +hk2-api-2.5.0-b42.jar CDDL/GPLv2+CE: https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html +hk2-locator-2.5.0-b42.jar CDDL/GPLv2+CE: https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html +hk2-utils-2.5.0-b42.jar CDDL/GPLv2+CE: https://glassfish.java.net/nonav/public/CDDL+GPL_1_1.html +htrace-core-3.2.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +httpclient-4.5.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +httpcore-4.4.10.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +ivy-2.4.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-annotations-2.6.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-core-2.6.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-core-asl-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-databind-2.6.7.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-jaxrs-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-mapper-asl-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-module-paranamer-2.6.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-module-scala_2.11-2.6.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jackson-xc-1.9.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +janino-3.0.11.jar BSD: https://raw.githubusercontent.com/janino-compiler/janino/master/LICENSE +java-xmlbuilder-1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +javassist-3.22.0-CR2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +javax.annotation-api-1.2.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javax.inject-1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +javax.inject-2.5.0-b42.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javax.resource-api-1.7.1.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javax.servlet-api-4.0.1.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javax.transaction-api-1.3.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javax.ws.rs-api-2.1.1.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +javolution-5.5.1.jar BSD: http://javolution.org/LICENSE.txt +jaxb-api-2.2.2.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jcl-over-slf4j-1.7.25.jar MIT: http://www.opensource.org/licenses/mit-license.php +jdo-api-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jersey-client-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-common-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-container-servlet-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-container-servlet-core-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-hk2-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-media-jaxb-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jersey-server-2.27.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jets3t-0.9.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jettison-1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-6.1.26.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-client-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-continuation-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-http-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-io-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-jndi-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-plus-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-proxy-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-security-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-server-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-servlet-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-servlets-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-sslengine-6.1.26.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-util-6.1.26.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-util-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-webapp-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jetty-xml-9.2.26.v20180806.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jline-2.14.6.jar BSD: http://www.opensource.org/licenses/bsd-license.php +jna-4.5.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +joda-convert-2.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +joda-time-2.10.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jodd-core-5.0.6.jar BSD: http://jodd.org/license.html jpam-1.1.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt json4s-ast_2.11-3.2.11.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt json4s-core_2.11-3.2.11.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt json4s-jackson_2.11-3.2.11.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt -jsp-api-2.1.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -jsr305-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +jsp-api-2.1.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +jsr305-3.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 jta-1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -jtransforms-2.4.0.jar MPL http://www.mozilla.org/MPL/2.0/index.txt -jul-to-slf4j-1.7.21.jar MIT http://www.opensource.org/licenses/mit-license.php -kafka-clients-0.8.2.1.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt -kafka_2.11-0.8.2.1.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt -kryo-shaded-4.0.0.jar BSD http://www.opensource.org/licenses/bsd-license.php -leveldbjni-all-1.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libfb303-0.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libgemfirexd.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libgemfirexd64.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libgemfirexd64_g.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libgemfirexd_g.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -libthrift-0.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -liquidFillGauge.js BSD http://choosealicense.com/licenses/bsd-2-clause -log4j-1.2.17.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -lz4-1.3.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -macro-compat_2.11-1.1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -mail-1.4.7.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -metrics-core-2.2.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -metrics-core-3.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -metrics-graphite-3.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -metrics-json-3.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -metrics-jvm-3.1.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -mimepull-1.9.5.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -minlog-1.3.0.jar BSD http://www.opensource.org/licenses/bsd-license.php -moment.js MIT http://www.opensource.org/licenses/mit-license.php -mx4j-3.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -mx4j-remote-3.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -mx4j-tools-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -netty-3.9.9.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -netty-all-4.0.43.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -objenesis-2.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -opencsv-2.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -org.osgi.core-6.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -oro-2.0.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -osgi-resource-locator-1.0.1.jar CDDL/GPLv2+CE https://glassfish.java.net/public/CDDL+GPL_1_1.html -paranamer-2.6.jar BSD http://www.opensource.org/licenses/bsd-license.php -parboiled-core-1.1.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parboiled-scala_2.11-1.1.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parboiled_2.11-2.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-column-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-common-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-encoding-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-format-2.3.0-incubating.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-hadoop-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-hadoop-bundle-1.6.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -parquet-jackson-1.8.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -pmml-model-1.2.15.jar BSD-3 http://opensource.org/licenses/BSD-3-Clause -pmml-schema-1.2.15.jar BSD-3 http://opensource.org/licenses/BSD-3-Clause -protobuf-java-2.6.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -py4j-0.10.4.jar BSD http://www.opensource.org/licenses/bsd-license.php -pyrolite-4.13.jar MIT https://raw.githubusercontent.com/irmen/Pyrolite/master/LICENSE -scala-compiler-2.11.8.jar BSD-3 http://www.scala-lang.org/license.html -scala-library-2.11.8.jar BSD-3 http://www.scala-lang.org/license.html -scala-parser-combinators_2.11-1.0.4.jar BSD-3 http://www.scala-lang.org/license.html -scala-reflect-2.11.8.jar BSD-3 http://www.scala-lang.org/license.html -scala-xml_2.11-1.0.4.jar BSD-3 http://www.scala-lang.org/license.html -scalap-2.11.8.jar BSD-3 http://www.scala-lang.org/license.html -shapeless_2.11-2.3.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -shiro-core-1.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -slf4j-api-1.7.21.jar MIT http://www.opensource.org/licenses/mit-license.php -slf4j-log4j12-1.7.21.jar MIT http://www.opensource.org/licenses/mit-license.php -slick_2.11-2.1.0.jar BSD http://github.com/slick/slick/blob/master/LICENSE.txt -snappy-0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-java-1.1.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-catalyst_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-core_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-graphx_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-hive-thriftserver_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-hive_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-launcher_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-mllib-local_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-mllib_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-network-common_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-network-shuffle_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-repl_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-sketch_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-sql_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-streaming-kafka-0.10_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-streaming_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-tags_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-unsafe_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappy-spark-yarn_2.11-2.1.1.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-cluster_2.11-1.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-core_2.11-1.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-store-client-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-store-core-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-store-hibernate-1.6.2.jar LGPL2.1 http://www.gnu.org/licenses/lgpl-2.1.html -snappydata-store-shared-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -snappydata-store-tools-1.6.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spark-jobserver_2.11-0.6.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spire-macros_2.11-0.7.4.jar MIT http://opensource.org/licenses/MIT -spire_2.11-0.7.4.jar MIT http://opensource.org/licenses/MIT -spray-caching_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-can_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-client_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-http_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-httpx_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-io_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-json_2.11-1.3.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-routing-shapeless2_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -spray-util_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -stax-api-1.0-2.jar CDDL 1.0 https://opensource.org/licenses/CDDL-1.0 -stax-api-1.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -stream-2.7.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -stringtemplate-3.2.1.jar BSD http://antlr.org/license.html -super-csv-2.2.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -tomcat-jdbc-8.5.9.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -tomcat-juli-8.5.9.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -twitter4j-core-4.0.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -twitter4j-stream-4.0.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -uncommons-maths-1.2.2a.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -univocity-parsers-2.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -validation-api-1.1.0.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -xbean-asm5-shaded-4.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -xercesImpl-2.9.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -xml-apis-2.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -xmlenc-0.52.jar BSD http://www.opensource.org/licenses/bsd-license.php -xom-1.2.10.jar LGPL2.1 http://www.gnu.org/licenses/lgpl-2.1.html +jtransforms-2.4.0.jar MPL: http://www.mozilla.org/MPL/2.0/index.txt +jul-to-slf4j-1.7.25.jar MIT: http://www.opensource.org/licenses/mit-license.php +kafka-clients-0.10.0.1.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt +kafka_2.11-0.10.0.1.jar Apache V2: http://jpam.sourceforge.net/LICENSE.txt +kryo-shaded-4.0.2.jar BSD: http://www.opensource.org/licenses/bsd-license.php +leveldbjni-all-1.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libfb303-0.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libgemfirexd.dylib Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libgemfirexd.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libgemfirexd64.dylib Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libgemfirexd64.so Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +libthrift-0.9.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +liquidFillGauge.js BSD: http://choosealicense.com/licenses/bsd-2-clause +log4j-1.2.17.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +lz4-java-1.5.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +machinist_2.11-0.6.1.jar MIT: http://opensource.org/licenses/MIT +macro-compat_2.11-1.1.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +metrics-core-2.2.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +metrics-core-3.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +metrics-graphite-3.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +metrics-json-3.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +metrics-jvm-3.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +mimepull-1.9.5.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +minlog-1.3.0.jar BSD: http://www.opensource.org/licenses/bsd-license.php +moment.js MIT: http://www.opensource.org/licenses/mit-license.php +mx4j-3.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +mx4j-remote-3.0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +mx4j-tools-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +netty-3.10.6.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +netty-all-4.0.56.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +objenesis-3.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +opencsv-2.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +org.osgi.core-6.0.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +oro-2.0.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +osgi-resource-locator-1.0.1.jar CDDL/GPLv2+CE: https://glassfish.java.net/public/CDDL+GPL_1_1.html +paranamer-2.6.jar BSD: http://www.opensource.org/licenses/bsd-license.php +parboiled-core-1.1.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parboiled-scala_2.11-1.1.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parboiled_2.11-2.1.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-column-1.8.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-common-1.8.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-encoding-1.8.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-format-2.3.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-hadoop-1.8.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-hadoop-bundle-1.6.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +parquet-jackson-1.8.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +pmml-model-1.2.17.jar BSD-3: http://opensource.org/licenses/BSD-3-Clause +pmml-schema-1.2.17.jar BSD-3: http://opensource.org/licenses/BSD-3-Clause +protobuf-java-3.6.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +py4j-0.10.8.1.jar BSD: http://www.opensource.org/licenses/bsd-license.php +pyrolite-4.22.jar MIT: https://raw.githubusercontent.com/irmen/Pyrolite/master/LICENSE +scala-compiler-2.11.8.jar BSD-3: http://www.scala-lang.org/license.html +scala-library-2.11.8.jar BSD-3: http://www.scala-lang.org/license.html +scala-parser-combinators_2.11-1.0.4.jar BSD-3: http://www.scala-lang.org/license.html +scala-reflect-2.11.8.jar BSD-3: http://www.scala-lang.org/license.html +scala-xml_2.11-1.0.4.jar BSD-3: http://www.scala-lang.org/license.html +scalap-2.11.8.jar BSD-3: http://www.scala-lang.org/license.html +servlet-api-2.5-20081211.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +shapeless_2.11-2.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +shiro-core-1.2.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +slf4j-api-1.7.25.jar MIT: http://www.opensource.org/licenses/mit-license.php +slf4j-log4j12-1.7.25.jar MIT: http://www.opensource.org/licenses/mit-license.php +slick_2.11-2.1.0.jar BSD: http://github.com/slick/slick/blob/master/LICENSE.txt +snappy-0.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-java-1.1.7.2.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-catalyst_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-core_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-graphx_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-hive-thriftserver_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-hive_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-launcher_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-mllib-local_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-mllib_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-network-common_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-network-shuffle_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-repl_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-sketch_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-sql_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-sql-kafka-0.10_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-streaming-kafka-0.10_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-streaming_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-tags_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-unsafe_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappy-spark-yarn_2.11-2.1.1.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-cluster_2.11-1.0.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-core_2.11-1.0.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-jdbc_2.11-1.0.2.1-only.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-launcher-1.0.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-store-client-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-store-core-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-store-hibernate-1.6.2.1.jar LGPL2.1: http://www.gnu.org/licenses/lgpl-2.1.html +snappydata-store-shared-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +snappydata-store-tools-1.6.2.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spark-jobserver_2.11-0.6.2.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spire-macros_2.11-0.13.0.jar MIT: http://opensource.org/licenses/MIT +spire_2.11-0.13.0.jar MIT: http://opensource.org/licenses/MIT +spray-caching_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-can_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-client_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-http_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-httpx_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-io_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-json_2.11-1.3.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-routing-shapeless2_2.11-1.3.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +spray-util_2.11-1.3.4.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +stax-api-1.0-2.jar CDDL 1.0: https://opensource.org/licenses/CDDL-1.0 +stax-api-1.0.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +stream-2.9.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +stringtemplate-3.2.1.jar BSD: http://antlr.org/license.html +super-csv-2.2.0.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +tomcat-jdbc-8.5.37.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +tomcat-juli-8.5.37.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +twitter4j-core-4.0.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +twitter4j-stream-4.0.7.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +uncommons-maths-1.2.2a.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +univocity-parsers-2.7.6.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +validation-api-1.1.0.Final.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +xbean-asm5-shaded-4.5.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +xercesImpl-2.9.1.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +xml-apis-1.4.01.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +xmlenc-0.52.jar BSD: http://www.opensource.org/licenses/bsd-license.php +xom-1.2.10.jar LGPL2.1: http://www.gnu.org/licenses/lgpl-2.1.html xz-1.0.jar Public Domain -zkclient-0.3.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 -zookeeper-3.4.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +zkclient-0.8.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 +zookeeper-3.4.13.jar Apache V2: http://www.apache.org/licenses/LICENSE-2.0 diff --git a/build.gradle b/build.gradle index 9326b6280b..a5ca9e17fc 100644 --- a/build.gradle +++ b/build.gradle @@ -15,43 +15,43 @@ * LICENSE file. */ -import groovy.json.JsonSlurper import org.gradle.api.tasks.testing.logging.* import org.gradle.internal.logging.* -apply plugin: 'wrapper' -apply plugin: 'distribution' - -if (JavaVersion.current().isJava8Compatible()) { - allprojects { - tasks.withType(Javadoc) { - options.addStringOption('Xdoclint:none', '-quiet') - /* - if (javax.tools.ToolProvider.getSystemDocumentationTool().isSupportedOption("--allow-script-in-comments") == 0) { - options.addBooleanOption("-allow-script-in-comments", true) - } - */ - } - } -} buildscript { repositories { maven { url 'https://plugins.gradle.org/m2' } mavenCentral() } dependencies { - classpath 'io.snappydata:gradle-scalatest:0.16' + classpath 'io.snappydata:gradle-scalatest:0.23' classpath 'org.github.ngbinh.scalastyle:gradle-scalastyle-plugin_2.11:0.9.0' - classpath "net.rdrei.android.buildtimetracker:gradle-plugin:0.8.+" - classpath 'com.netflix.nebula:gradle-ospackage-plugin:4.4.+' + classpath 'com.github.jengelman.gradle.plugins:shadow:4.0.3' + classpath 'de.undercouch:gradle-download-task:3.4.3' + classpath 'net.rdrei.android.buildtimetracker:gradle-plugin:0.11.+' + classpath 'com.netflix.nebula:gradle-ospackage-plugin:5.2.+' } } +apply plugin: 'wrapper' +apply plugin: 'distribution' +apply plugin: 'nebula.ospackage-base' +apply plugin: "nebula.ospackage" + allprojects { // We want to see all test results. This is equivalent to setting --continue // on the command line. gradle.startParameter.continueOnFailure = true + tasks.withType(Javadoc) { + options.addStringOption('Xdoclint:none', '-quiet') + /* + if (javax.tools.ToolProvider.getSystemDocumentationTool().isSupportedOption("--allow-script-in-comments") == 0) { + options.addBooleanOption("-allow-script-in-comments", true) + } + */ + } + repositories { mavenCentral() maven { url 'https://dl.bintray.com/big-data/maven' } @@ -65,10 +65,9 @@ allprojects { apply plugin: 'java' apply plugin: 'maven' apply plugin: 'scalaStyle' + apply plugin: 'com.github.johnrengelman.shadow' apply plugin: 'idea' apply plugin: "build-time-tracker" - apply plugin: "nebula.ospackage" - apply plugin: 'nebula.ospackage-base' group = 'io.snappydata' version = '1.0.2.1' @@ -84,9 +83,10 @@ allprojects { options.forkOptions.jvmArgs = [ '-J-Xmx2g', '-J-Xms2g', '-J-XX:ReservedCodeCacheSize=512m', '-J-Djava.net.preferIPv4Stack=true' ] } tasks.withType(ScalaCompile) { + options.fork = true + options.forkOptions.jvmArgs = [ '-Xmx2g', '-Xms2g', '-XX:ReservedCodeCacheSize=512m', '-Djava.net.preferIPv4Stack=true' ] // scalaCompileOptions.optimize = true // scalaCompileOptions.useAnt = false - // scalaCompileOptions.fork = false scalaCompileOptions.deprecation = false scalaCompileOptions.additionalParameters = [ '-feature' ] options.encoding = 'UTF-8' @@ -96,13 +96,6 @@ allprojects { javadoc.options.charSet = 'UTF-8' - gradle.taskGraph.whenReady( { graph -> - tasks.withType(Tar).each { tar -> - tar.compression = Compression.GZIP - tar.extension = 'tar.gz' - } - }) - ext { productName = 'SnappyData' vendorName = 'SnappyData, Inc.' @@ -116,20 +109,47 @@ allprojects { log4jVersion = '1.2.17' slf4jVersion = '1.7.25' junitVersion = '4.12' - hadoopVersion = '2.7.3' + mockitoVersion = '1.10.19' + hadoopVersion = '2.7.7' scalatestVersion = '2.2.6' - jettyVersion = '9.2.22.v20170606' + jettyVersion = '9.2.26.v20180806' guavaVersion = '14.0.1' kryoVersion = '4.0.1' thriftVersion = '0.9.3' - metricsVersion = '3.2.5' - janinoVersion = '3.0.8' - derbyVersion = '10.12.1.1' + metricsVersion = '4.0.3' + metrics2Version = '2.2.0' + janinoVersion = '3.0.11' + derbyVersion = '10.14.2.0' + parboiledVersion = '2.1.5' + tomcatJdbcVersion = '8.5.37' + hikariCPVersion = '2.7.9' + twitter4jVersion = '4.0.7' + objenesisVersion = '3.0.1' + rabbitMqVersion = '4.9.1' + akkaVersion = '2.3.16' + sprayVersion = '1.3.4' + sprayJsonVersion = '1.3.5' + sprayShapelessVersion = '1.3.3' + sprayTestkitVersion = '1.3.4' + jodaVersion = '2.1.2' + jodaTimeVersion = '2.10.1' + slickVersion = '2.1.0' + h2Version = '1.3.176' + commonsIoVersion = '2.6' + commonsPoolVersion = '1.6' + dbcpVersion = '1.4' + shiroVersion = '1.2.6' + flywayVersion = '3.2.1' + typesafeConfigVersion = '1.3.3' + mssqlVersion = '7.0.0.jre8' + antlr2Version = '2.7.7' + pegdownVersion = '1.6.0' snappyStoreVersion = '1.6.2.1' snappydataVersion = version pulseVersion = '1.5.1' zeppelinInterpreterVersion = '0.7.3.4' + buildFlags = '' createdBy = System.getProperty('user.name') osArch = System.getProperty('os.arch') @@ -140,6 +160,7 @@ allprojects { jdkVersion = System.getProperty('java.version') sparkJobServerVersion = '0.6.2.8' eclipseCollectionsVersion = '9.2.0' + fastutilVersion = '8.2.2' gitCmd = "git --git-dir=${rootDir}/.git --work-tree=${rootDir}" gitBranch = "${gitCmd} rev-parse --abbrev-ref HEAD".execute().text.trim() @@ -175,8 +196,14 @@ allprojects { // force same output directory for IDEA and gradle idea { module { - outputDir file("${project.buildDir}/classes/main") - testOutputDir file("${project.buildDir}/classes/test") + def projOutDir = file("${projectDir}/src/main/scala").exists() + ? "${project.sourceSets.main.java.outputDir}/../../scala/main" + : project.sourceSets.main.java.outputDir + def projTestOutDir = file("${projectDir}/src/test/scala").exists() + ? "${project.sourceSets.test.java.outputDir}/../../scala/test" + : project.sourceSets.test.java.outputDir + outputDir file(projOutDir) + testOutputDir file(projTestOutDir) } } } @@ -294,7 +321,8 @@ subprojects { } task scalaTest(type: Test) { - actions = [ new com.github.maiflai.ScalaTestAction() ] + def factory = new com.github.maiflai.BackwardsCompatibleJavaExecActionFactory(gradle.gradleVersion) + actions = [ new com.github.maiflai.ScalaTestAction(factory) ] // top-level default is single process run since scalatest does not // spawn separate JVMs maxParallelForks = 1 @@ -311,6 +339,7 @@ subprojects { testLogging.exceptionFormat = TestExceptionFormat.FULL testLogging.events = TestLogEvent.values() as Set + extensions.add(com.github.maiflai.ScalaTestAction.TAGS, new org.gradle.api.tasks.util.PatternSet()) List suites = [] extensions.add(com.github.maiflai.ScalaTestAction.SUITES, suites) extensions.add('suite', { String name -> suites.add(name) } ) @@ -322,11 +351,11 @@ subprojects { def output = new StringBuilder() extensions.add(com.github.maiflai.ScalaTestAction.TESTOUTPUT, output) - extensions.add('testOutput', { String name -> output.setLength(0); output.append(name) } ) + extensions.add('testOutput', { String name -> output.setLength(0); output.append(name) }) def errorOutput = new StringBuilder() extensions.add(com.github.maiflai.ScalaTestAction.TESTERROR, errorOutput) - extensions.add('testError', { String name -> errorOutput.setLength(0); errorOutput.append(name) } ) + extensions.add('testError', { String name -> errorOutput.setLength(0); errorOutput.append(name) }) // running a single scala suite if (rootProject.hasProperty('singleSuite')) { @@ -334,7 +363,7 @@ subprojects { } workingDir = "${testResultsBase}/scalatest" - testResult '/dev/tty' + // testResult '/dev/tty' testOutput "${workingDir}/output.txt" testError "${workingDir}/error.txt" binResultsDir = file("${workingDir}/binary/${project.name}") @@ -397,35 +426,6 @@ subprojects { "-Dio.netty.allocator.numHeapArenas=${numArenas}", "-Dio.netty.allocator.numDirectArenas=${numArenas}"] - String single = System.getProperty('dunit.single') - if (single == null || single.length() == 0) { - single = rootProject.hasProperty('dunit.single') ? - rootProject.property('dunit.single') : null - } - if (single == null || single.length() == 0) { - def dunitTests = fileTree(dir: testClassesDir, - includes: ['**/*DUnitTest.class', '**/*DUnit.class'], - excludes: ['**/*DUnitSecurityTest.class', '**/NCJ*DUnit.class', '**/pivotal/gemfirexd/wan/**/*DUnit.class']) - FileTree includeTestFiles = dunitTests - int dunitFrom = rootProject.hasProperty('dunit.from') ? - getLast(includeTestFiles, rootProject.property('dunit.from')) : 0 - int dunitTo = rootProject.hasProperty('dunit.to') ? - getLast(includeTestFiles, rootProject.property('dunit.to')) : includeTestFiles.size() - - int begin = dunitFrom != -1 ? dunitFrom : 0 - int end = dunitTo != -1 ? dunitTo : includeTestFiles.size() - def filteredSet = includeTestFiles.drop(begin).take(end-begin+1).collect {f -> "**/" + f.name} - if (begin != 0 || end != includeTestFiles.size()) { - println("Picking tests :") - filteredSet.each { a -> println(a) } - } - include filteredSet - } else { - include single - } - exclude '**/*Suite.class' - exclude '**/*DUnitSecurityTest.class' - workingDir = "${testResultsBase}/dunit" binResultsDir = file("${workingDir}/binary/${project.name}") @@ -481,36 +481,6 @@ subprojects { "-Dio.netty.allocator.numHeapArenas=${numArenas}", "-Dio.netty.allocator.numDirectArenas=${numArenas}"] - def single = rootProject.hasProperty('dunitSecurity.single') ? - rootProject.property('dunitSecurity.single') : null - if (single == null || single.length() == 0) { - def dunitSecurityTests = fileTree(dir: testClassesDir, - includes: ['**/*DUnitSecurityTest.class'], - excludes: ['**/*DUnitTest.class', '**/NCJ*DUnit.class', '**/pivotal/gemfirexd/wan/**/*DUnit.class']) - FileTree includeTestFiles = dunitSecurityTests - int dunitFrom = rootProject.hasProperty('dunitSecurity.from') ? - getLast(includeTestFiles, rootProject.property('dunitSecurity.from')) : 0 - int dunitTo = rootProject.hasProperty('dunitSecurity.to') ? - getLast(includeTestFiles, rootProject.property('dunitSecurity.to')) : includeTestFiles.size() - - int begin = dunitFrom != -1 ? dunitFrom : 0 - int end = dunitTo != -1 ? dunitTo : includeTestFiles.size() - def filteredSet = includeTestFiles.drop(begin).take(end-begin+1).collect {f -> "**/" + f.name} - if (begin != 0 || end != includeTestFiles.size()) { - println("Picking tests :") - filteredSet.each { a -> println(a) } - } - include filteredSet - } else { - include single - } - exclude '**/*Suite.class' - exclude '**/*DUnitTest.class' - exclude '**/*DUnit.class' - if (!rootProject.hasProperty('snappydata.enterprise')) { - exclude '**/*DUnitSecurityTest.class' - } - workingDir = "${testResultsBase}/dunit-security" binResultsDir = file("${workingDir}/binary/${project.name}") @@ -528,87 +498,6 @@ subprojects { } } - gradle.taskGraph.whenReady({ graph -> - tasks.withType(Jar).each { pack -> - if (pack.name == 'packageTests') { - pack.from(pack.project.sourceSets.test.output.classesDir, sourceSets.test.resources.srcDirs) - } - } - tasks.withType(Test).each { test -> - test.configure { - - String logLevel = System.getProperty('logLevel') - if (logLevel != null && logLevel.length() > 0) { - systemProperties 'gemfire.log-level' : logLevel, - 'logLevel' : logLevel - } - logLevel = System.getProperty('securityLogLevel') - if (logLevel != null && logLevel.length() > 0) { - systemProperties 'gemfire.security-log-level' : logLevel, - 'securityLogLevel' : logLevel - } - - environment 'SNAPPY_HOME': snappyProductDir, - 'APACHE_SPARK_HOME': sparkProductDir, - 'APACHE_SPARK_CURRENT_HOME': sparkCurrentProductDir, - 'SNAPPY_DIST_CLASSPATH': "${sourceSets.test.runtimeClasspath.asPath}" - - def failureCount = new java.util.concurrent.atomic.AtomicInteger(0) - def progress = new File(workingDir, 'progress.txt') - def output = new File(workingDir, 'output.txt') - - String eol = System.getProperty('line.separator') - beforeTest { desc -> - String now = now() - progress << "${now} Starting test ${desc.className} ${desc.name}${eol}" - output << "${now} STARTING TEST ${desc.className} ${desc.name}${eol}${eol}" - } - onOutput { desc, event -> - String msg = event.message - if (event.destination.toString() == 'StdErr') { - msg = msg.replace(eol, "${eol}[error] ") - } - output << msg - } - afterTest { desc, result -> - String now = now() - progress << "${now} Completed test ${desc.className} ${desc.name} with result: ${result.resultType}${eol}" - output << "${eol}${now} COMPLETED TEST ${desc.className} ${desc.name} with result: ${result.resultType}${eol}${eol}" - def exceptions = result.exceptions - if (exceptions.size() > 0) { - exceptions.each { t -> - progress << " EXCEPTION: ${getStackTrace(t)}${eol}" - output << "${getStackTrace(t)}${eol}" - } - failureCount.incrementAndGet() - } - } - doLast { - def report = "${test.reports.html.destination}/index.html" - boolean hasProgress = progress.exists() - if (failureCount.get() > 0) { - println() - def failureMsg = "FAILED: There were ${failureCount.get()} failures.${eol}" - if (hasProgress) { - failureMsg += " See the progress report in: file://$progress${eol}" - } - failureMsg += " HTML report in: file://$report" - throw new GradleException(failureMsg) - } else if (hasProgress) { - println() - println("SUCCESS: See the progress report in: file://$progress") - println(" HTML report in: file://$report") - println() - } else { - println() - println("SUCCESS: See the HTML report in: file://$report") - println() - } - } - } - } - }) - // apply default manifest if (rootProject.hasProperty('enablePublish')) { createdBy = 'SnappyData Build Team' @@ -662,6 +551,10 @@ subprojects { "org.apache.hadoop:hadoop-yarn-server-nodemanager:${hadoopVersion}", "org.apache.hadoop:hadoop-yarn-server-web-proxy:${hadoopVersion}" } + configurations.testRuntime { + // below is included indirectly by hadoop deps and conflicts with embedded 1.5.7 apacheds + exclude(group: 'org.apache.directory.server', module: 'apacheds-kerberos-codec') + } task packageTests(type: Jar, dependsOn: testClasses) { description 'Assembles a jar archive of test classes.' @@ -744,6 +637,165 @@ subprojects { } } +// apply common test and misc configuration +gradle.taskGraph.whenReady { graph -> + + String dunitSingle = System.getProperty('dunit.single') + if (dunitSingle == null || dunitSingle.length() == 0) { + dunitSingle = rootProject.hasProperty('dunit.single') ? + rootProject.property('dunit.single') : null + } + String dunitSecSingle = System.getProperty('dunitSecurity.single') + if (dunitSecSingle == null || dunitSecSingle.length() == 0) { + dunitSecSingle = rootProject.hasProperty('dunitSecurity.single') ? + rootProject.property('dunitSecurity.single') : null + } + + def allTasks = subprojects.collect { it.tasks }.flatten() + allTasks.each { task -> + if (task instanceof Tar) { + def tar = (Tar)task + tar.compression = Compression.GZIP + tar.extension = 'tar.gz' + } else if (task instanceof Jar) { + def pack = (Jar)task + if (pack.name == 'packageTests') { + pack.from(pack.project.sourceSets.test.output.classesDirs, pack.project.sourceSets.test.resources.srcDirs) + } + } else if (task instanceof Test) { + def test = (Test)task + test.configure { + + if (test.name == 'dunitTest') { + includes.clear() + excludes.clear() + if (dunitSingle == null || dunitSingle.length() == 0) { + def dunitTests = testClassesDirs.asFileTree.matching { + includes = [ '**/*DUnitTest.class', '**/*DUnit.class' ] + excludes = [ '**/*Suite.class', '**/*DUnitSecurityTest.class', '**/NCJ*DUnit.class', + '**/BackwardCompatabilityPart*DUnit.class', '**/*Perf*DUnit.class', '**/ListAggDUnit.class', + '**/SingleHop*TransactionDUnit.class', '**/*Parallel*AsyncEvent*DUnit.class', '**/pivotal/gemfirexd/wan/**/*DUnit.class' ] + } + FileTree includeTestFiles = dunitTests + int dunitFrom = rootProject.hasProperty('dunit.from') ? + getLast(includeTestFiles, rootProject.property('dunit.from')) : 0 + int dunitTo = rootProject.hasProperty('dunit.to') ? + getLast(includeTestFiles, rootProject.property('dunit.to')) : includeTestFiles.size() + + int begin = dunitFrom != -1 ? dunitFrom : 0 + int end = dunitTo != -1 ? dunitTo : includeTestFiles.size() + def filteredSet = includeTestFiles.drop(begin).take(end-begin+1).collect {f -> "**/" + f.name} + if (begin != 0 || end != includeTestFiles.size()) { + println("Picking tests :") + filteredSet.each { a -> println(a) } + } + include filteredSet + } else { + include dunitSingle + } + } else if (test.name == 'dunitSecurityTest') { + includes.clear() + excludes.clear() + if (!rootProject.hasProperty('snappydata.enterprise')) { + excludes = [ '**/*Suite.class', '**/*DUnitSecurityTest.class', '**/*DUnitTest.class', '**/*DUnit.class' ] + } else if (dunitSecSingle == null || dunitSecSingle.length() == 0) { + def dunitSecurityTests = testClassesDirs.asFileTree.matching { + includes = [ '**/*DUnitSecurityTest.class' ] + excludes = [ '**/*Suite.class', '**/*DUnitTest.class', '**/*DUnit.class' ] + } + FileTree includeTestFiles = dunitSecurityTests + int dunitFrom = rootProject.hasProperty('dunitSecurity.from') ? + getLast(includeTestFiles, rootProject.property('dunitSecurity.from')) : 0 + int dunitTo = rootProject.hasProperty('dunitSecurity.to') ? + getLast(includeTestFiles, rootProject.property('dunitSecurity.to')) : includeTestFiles.size() + + int begin = dunitFrom != -1 ? dunitFrom : 0 + int end = dunitTo != -1 ? dunitTo : includeTestFiles.size() + def filteredSet = includeTestFiles.drop(begin).take(end-begin+1).collect {f -> "**/" + f.name} + if (begin != 0 || end != includeTestFiles.size()) { + println("Picking tests :") + filteredSet.each { a -> println(a) } + } + include filteredSet + } else { + include dunitSecSingle + } + } + + String logLevel = System.getProperty('logLevel') + if (logLevel != null && logLevel.length() > 0) { + systemProperties 'gemfire.log-level' : logLevel, + 'logLevel' : logLevel + } + logLevel = System.getProperty('securityLogLevel') + if (logLevel != null && logLevel.length() > 0) { + systemProperties 'gemfire.security-log-level' : logLevel, + 'securityLogLevel' : logLevel + } + + environment 'SNAPPY_HOME': snappyProductDir, + 'APACHE_SPARK_HOME': sparkProductDir, + 'APACHE_SPARK_CURRENT_HOME': sparkCurrentProductDir, + 'SNAPPY_DIST_CLASSPATH': test.classpath.asPath + + def failureCount = new java.util.concurrent.atomic.AtomicInteger(0) + def progress = new File(workingDir, 'progress.txt') + def output = new File(workingDir, 'output.txt') + + String eol = System.getProperty('line.separator') + beforeTest { desc -> + String now = now() + progress << "${now} Starting test ${desc.className} ${desc.name}${eol}" + output << "${now} STARTING TEST ${desc.className} ${desc.name}${eol}${eol}" + } + onOutput { desc, event -> + String msg = event.message + if (event.destination.toString() == 'StdErr') { + msg = msg.replace(eol, "${eol}[error] ") + } + output << msg + } + afterTest { desc, result -> + String now = now() + progress << "${now} Completed test ${desc.className} ${desc.name} with result: ${result.resultType}${eol}" + output << "${eol}${now} COMPLETED TEST ${desc.className} ${desc.name} with result: ${result.resultType}${eol}${eol}" + def exceptions = result.exceptions + if (exceptions.size() > 0) { + exceptions.each { t -> + progress << " EXCEPTION: ${getStackTrace(t)}${eol}" + output << "${getStackTrace(t)}${eol}" + } + failureCount.incrementAndGet() + } + } + doLast { + def report = "${test.reports.html.destination}/index.html" + boolean hasProgress = progress.exists() + if (failureCount.get() > 0) { + println() + def failureMsg = "FAILED: There were ${failureCount.get()} failures.${eol}" + if (hasProgress) { + failureMsg += " See the progress report in: file://$progress${eol}" + } + failureMsg += " HTML report in: file://$report" + throw new GradleException(failureMsg) + } else if (hasProgress) { + println() + println("SUCCESS: See the progress report in: file://$progress") + println(" HTML report in: file://$report") + println() + } else { + println() + println("SUCCESS: See the HTML report in: file://$report") + println() + } + } + } + } + } +} + + task publishLocal { dependsOn subprojects.findAll { p -> p.name != 'snappydata-native' && p.name != 'snappydata-store-prebuild' && p.name != 'snappydata-store' }.collect { @@ -762,7 +814,7 @@ task publishMaven { task product(type: Zip) { dependsOn ":snappy-cluster_${scalaBinaryVersion}:jar" dependsOn ":snappy-examples_${scalaBinaryVersion}:jar" - dependsOn ":snappy-spark:snappy-spark-assembly_${scalaBinaryVersion}:product" + dependsOn ":snappy-spark:snappy-spark-assembly_${scalaBinaryVersion}:sparkProduct" dependsOn ':snappy-launcher:jar' dependsOn ':jdbcJar' @@ -1029,12 +1081,12 @@ ospackage { } buildRpm { + dependsOn ':packageVSD' + dependsOn ':packageZeppelinInterpreter' requires('glibc') requires('bash') requires('perl') requires('curl') - dependsOn ':packageVSD' - dependsOn ':packageZeppelinInterpreter' if (rootProject.hasProperty('hadoop-provided')) { classifier 'without_hadoop' } @@ -1043,13 +1095,13 @@ buildRpm { } buildDeb { + dependsOn ':packageVSD' + dependsOn ':packageZeppelinInterpreter' requires('libc6') requires('bash') requires('perl') requires('curl') recommends('java8-sdk') - dependsOn ':packageVSD' - dependsOn ':packageZeppelinInterpreter' if (rootProject.hasProperty('hadoop-provided')) { classifier 'without-hadoop' } @@ -1141,19 +1193,27 @@ task generateSources { dependsOn ':snappy-spark:generateSources', ':snappy-store:generateSources' // copy all resource files into build classes path because new versions of IDEA // do not include separate resources path in CLASSPATH if output path has been customized - getSubprojects().collect { proj -> - String resourcesDir = "${proj.projectDir}/src/main/resources" - if (file(resourcesDir).exists()) { - copy { - from resourcesDir - into "${proj.buildDir}/classes/main" + doLast { + subprojects.collect { proj -> + String resourcesDir = proj.sourceSets.main.output.resourcesDir + if (file(resourcesDir).exists()) { + def projOutDir = file("${proj.projectDir}/src/main/scala").exists() + ? "${proj.sourceSets.main.java.outputDir}/../../scala/main" + : proj.sourceSets.main.java.outputDir + copy { + from resourcesDir + into projOutDir + } } - } - resourcesDir = "${proj.projectDir}/src/test/resources" - if (file(resourcesDir).exists()) { - copy { - from resourcesDir - into "${proj.buildDir}/classes/test" + resourcesDir = proj.sourceSets.test.output.resourcesDir + if (file(resourcesDir).exists()) { + def projOutDir = file("${proj.projectDir}/src/test/scala").exists() + ? "${proj.sourceSets.test.java.outputDir}/../../scala/test" + : proj.sourceSets.test.java.outputDir + copy { + from resourcesDir + into projOutDir + } } } } @@ -1202,9 +1262,9 @@ task allReports(type: TestReport) { destinationDir = file("${testResultsBase}/combined-reports") mustRunAfter checkAll } -gradle.taskGraph.whenReady({ graph -> +gradle.taskGraph.whenReady { graph -> tasks.getByName('allReports').reportOn rootProject.subprojects.collect{ it.tasks.withType(Test) }.flatten() -}) +} def writeProperties(def parent, def name, def comment, def propsMap) { parent.exists() || parent.mkdirs() @@ -1407,26 +1467,21 @@ if (rootProject.hasProperty('trackBuildTime') ) { } } -// log build output to buildOutput.log +// log build output to buildOutput.log in addition to console output def buildOutput = new File("${rootDir}/buildOutput.log") - // delete build output file if it has become large if (buildOutput.length() > 1000000) { delete buildOutput } - -gradle.services.get(LoggingOutputInternal).addStandardOutputListener (new StandardOutputListener () { +def gradleLogger = new org.gradle.api.logging.StandardOutputListener() { void onOutput(CharSequence output) { buildOutput << output } -}) - -gradle.services.get(LoggingOutputInternal).addStandardErrorListener (new StandardOutputListener () { - void onOutput(CharSequence output) { - buildOutput << output - } -}) +} +def loggerService = gradle.services.get(LoggingOutputInternal) +loggerService.addStandardOutputListener(gradleLogger) +loggerService.addStandardErrorListener(gradleLogger) println() println('-------------------------------------------------') diff --git a/cluster/build.gradle b/cluster/build.gradle index 081ac93272..8775faa4a8 100644 --- a/cluster/build.gradle +++ b/cluster/build.gradle @@ -128,10 +128,11 @@ dependencies { exclude(group: 'com.sun.jersey.contribs') exclude(group: 'com.google.protobuf', module: 'protobuf-java') exclude(group: 'com.jcraft', module: 'jsch') + exclude(group: 'org.apache.directory.server', module: 'apacheds-kerberos-codec') } testCompile project(':dunit') - testCompile 'it.unimi.dsi:fastutil:8.2.2' + testCompile "it.unimi.dsi:fastutil:${fastutilVersion}" testCompile "org.scalatest:scalatest_${scalaBinaryVersion}:${scalatestVersion}" if (new File(rootDir, 'aqp/build.gradle').exists() && rootProject.hasProperty('snappydata.enterprise')) { @@ -143,7 +144,7 @@ dependencies { // Creates the version properties file and writes it to the resources dir task createVersionPropertiesFile(dependsOn: 'processResources') { - def propertiesDir = file("${buildDir}/classes/main/io/snappydata") + def propertiesDir = file("${sourceSets.main.scala.outputDir}/io/snappydata") outputs.file "${propertiesDir}/SnappyDataVersion.properties" inputs.file "${rootProject.projectDir}/build.gradle" diff --git a/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java b/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java index c20d1f6a51..be595f1541 100644 --- a/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java +++ b/cluster/src/main/java/io/snappydata/gemxd/SnappySystemAdmin.java @@ -32,7 +32,7 @@ public class SnappySystemAdmin extends GfxdSystemAdmin { - SnappySystemAdmin() { + private SnappySystemAdmin() { super(); UTIL_Tools_DSProps = "UTIL_Snappy_Tools_DSProps"; UTIL_DSProps_HelpPost = "UTIL_Snappy_Tools_DSProps_HelpPost"; @@ -91,31 +91,28 @@ public void invoke(String[] args) { super.invoke(args); } finally { - // remove zero-sized log-file - if (this.defaultLogFileName != null) { - try { - File logFile = new File(this.defaultLogFileName); - if (logFile.exists() && logFile.isFile() && logFile.length() == 0) { - logFile.delete(); - } - } catch (Throwable t) { - // ignore at this point + // remove zero-sized generatedcode.log file + try { + File codeLogFile = new File("generatedcode.log"); + if (codeLogFile.exists() && codeLogFile.isFile() && codeLogFile.length() == 0) { + codeLogFile.delete(); } + } catch (Throwable t) { + // ignore at this point } } } public boolean handleVersion(String[] args) { - String cmd = null; - final ArrayList cmdLine = new ArrayList(Arrays.asList(args)); + String cmd; + final ArrayList cmdLine = new ArrayList<>(Arrays.asList(args)); try { Iterator it = cmdLine.iterator(); while (it.hasNext()) { String arg = it.next(); if (arg.startsWith("-")) { checkDashArg(null, arg, it); - } - else { + } else { break; } } @@ -159,9 +156,8 @@ public boolean handleVersion(String[] args) { } if (cmd.equalsIgnoreCase("version")) { - Boolean optionOK = (cmdLine.size() == 0); + boolean optionOK = (cmdLine.size() == 0); if (cmdLine.size() == 1) { - optionOK = false; String option = cmdLine.get(0); if ("CREATE".equals(option) || "FULL".equalsIgnoreCase(option)) { optionOK = true; diff --git a/cluster/src/test/resources/log4j.properties b/cluster/src/test/resources/log4j.properties index 222d083979..85513aca15 100644 --- a/cluster/src/test/resources/log4j.properties +++ b/cluster/src/test/resources/log4j.properties @@ -38,11 +38,20 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.RollingFileAppender log4j.appender.file.append=true log4j.appender.file.file=snappydata.log -log4j.appender.file.MaxFileSize=100MB +log4j.appender.file.MaxFileSize=1GB log4j.appender.file.MaxBackupIndex=10000 log4j.appender.file.layout=io.snappydata.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS zzz} %t %p %c{1}: %m%n +# Appender for code dumps of WholeStageCodegenExec, CodeGenerator etc +log4j.appender.code=org.apache.log4j.RollingFileAppender +log4j.appender.code.append=true +log4j.appender.code.file=generatedcode.log +log4j.appender.code.MaxFileSize=1GB +log4j.appender.code.MaxBackupIndex=10000 +log4j.appender.code.layout=io.snappydata.log4j.PatternLayout +log4j.appender.code.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS zzz} %t %p %c{1}: %m%n + # Console appender log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.target=System.out @@ -92,8 +101,6 @@ log4j.logger.org.apache.spark.scheduler.FairSchedulableBuilder=WARN log4j.logger.org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint=WARN log4j.logger.org.apache.spark.storage.BlockManagerInfo=WARN log4j.logger.org.apache.hadoop.hive=WARN -# for all Spark generated code (including ad-hoc UnsafeProjection calls etc) -log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=WARN log4j.logger.org.apache.spark.sql.execution.datasources=WARN log4j.logger.org.apache.spark.scheduler.SnappyTaskSchedulerImpl=WARN log4j.logger.org.apache.spark.MapOutputTrackerMasterEndpoint=WARN @@ -110,7 +117,20 @@ log4j.logger.org.datanucleus=ERROR log4j.logger.org.apache.spark.Task=WARN log4j.logger.org.apache.spark.sql.catalyst.parser.CatalystSqlParser=WARN +# Keep log-level of some classes as INFO even if root level is higher +log4j.logger.io.snappydata.impl.LeadImpl=INFO +log4j.logger.io.snappydata.impl.ServerImpl=INFO +log4j.logger.io.snappydata.impl.LocatorImpl=INFO +log4j.logger.spray.can.server.HttpListener=INFO + # for generated code of plans -# log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=DEBUG +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=DEBUG, code +log4j.additivity.org.apache.spark.sql.execution.WholeStageCodegenExec=false +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenRDD=INFO, code +log4j.additivity.org.apache.spark.sql.execution.WholeStageCodegenRDD=false +# for all Spark generated code (including ad-hoc UnsafeProjection calls etc) +log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=WARN, code +log4j.additivity.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=false # for SnappyData generated code used on store (ComplexTypeSerializer, JDBC inserts ...) -# log4j.logger.org.apache.spark.sql.store.CodeGeneration=DEBUG +log4j.logger.org.apache.spark.sql.store.CodeGeneration=INFO, code +log4j.additivity.org.apache.spark.sql.store.CodeGeneration=false diff --git a/cluster/src/test/scala/io/snappydata/QueryTest.scala b/cluster/src/test/scala/io/snappydata/QueryTest.scala index 3f63ffe3a1..838b414d40 100644 --- a/cluster/src/test/scala/io/snappydata/QueryTest.scala +++ b/cluster/src/test/scala/io/snappydata/QueryTest.scala @@ -67,8 +67,7 @@ class QueryTest extends SnappyFunSuite { val df = snContext.sql("SELECT title, price FROM titles WHERE EXISTS (" + "SELECT * FROM sales WHERE sales.title_id = titles.title_id AND qty >30)") - - df.show() + df.collect() } test("SNAP-1159_1482") { @@ -131,49 +130,49 @@ class QueryTest extends SnappyFunSuite { "PARTITION_BY 'col2'," + "BUCKETS '1')") snc.sql("insert into ColumnTable(\"a/b\",col2,col3) values(1,2,3)") - snc.sql("select col2,col3 from columnTable").show() - snc.sql("select col2, col3, `a/b` from columnTable").show() - snc.sql("select col2, col3, \"a/b\" from columnTable").show() - snc.sql("select col2, col3, \"A/B\" from columnTable").show() - snc.sql("select col2, col3, `A/B` from columnTable").show() - - snc.sql("select col2,col3 from columnTable").show() - snc.table("columnTable").select("col3", "col2", "a/b").show() - snc.table("columnTable").select("col3", "Col2", "A/b").show() - snc.table("columnTable").select("COL3", "Col2", "A/B").show() - snc.table("columnTable").select("COL3", "Col2", "`A/B`").show() - snc.table("columnTable").select("COL3", "Col2", "`a/b`").show() + snc.sql("select col2,col3 from columnTable").collect() + snc.sql("select col2, col3, `a/b` from columnTable").collect() + snc.sql("select col2, col3, \"a/b\" from columnTable").collect() + snc.sql("select col2, col3, \"A/B\" from columnTable").collect() + snc.sql("select col2, col3, `A/B` from columnTable").collect() + + snc.sql("select col2,col3 from columnTable").collect() + snc.table("columnTable").select("col3", "col2", "a/b").collect() + snc.table("columnTable").select("col3", "Col2", "A/b").collect() + snc.table("columnTable").select("COL3", "Col2", "A/B").collect() + snc.table("columnTable").select("COL3", "Col2", "`A/B`").collect() + snc.table("columnTable").select("COL3", "Col2", "`a/b`").collect() snc.conf.set("spark.sql.caseSensitive", "true") try { - snc.table("columnTable").select("col3", "col2", "a/b").show() + snc.table("columnTable").select("col3", "col2", "a/b").collect() fail("expected to fail for case-sensitive=true") } catch { case _: AnalysisException => // expected } try { - snc.table("columnTable").select("COL3", "COL2", "A/B").show() + snc.table("columnTable").select("COL3", "COL2", "A/B").collect() fail("expected to fail for case-sensitive=true") } catch { case _: AnalysisException => // expected } try { - snc.sql("select col2, col3, \"A/B\" from columnTable").show() + snc.sql("select col2, col3, \"A/B\" from columnTable").collect() fail("expected to fail for case-sensitive=true") } catch { case _: AnalysisException => // expected } try { - snc.sql("select COL2, COL3, `A/B` from columnTable").show() + snc.sql("select COL2, COL3, `A/B` from columnTable").collect() fail("expected to fail for case-sensitive=true") } catch { case _: AnalysisException => // expected } // hive meta-store is case-insensitive so column table names are not - snc.sql("select COL2, COL3, \"a/b\" from columnTable").show() - snc.sql("select COL2, COL3, `a/b` from ColumnTable").show() - snc.table("columnTable").select("COL3", "COL2", "a/b").show() - snc.table("COLUMNTABLE").select("COL3", "COL2", "a/b").show() + snc.sql("select COL2, COL3, \"a/b\" from columnTable").collect() + snc.sql("select COL2, COL3, `a/b` from ColumnTable").collect() + snc.table("columnTable").select("COL3", "COL2", "a/b").collect() + snc.table("COLUMNTABLE").select("COL3", "COL2", "a/b").collect() } private def setupTestData(session: SnappySession): Unit = { diff --git a/cluster/src/test/scala/io/snappydata/Snap_213.scala b/cluster/src/test/scala/io/snappydata/Snap_213.scala index 93d4d7d835..6a7e18fd7f 100644 --- a/cluster/src/test/scala/io/snappydata/Snap_213.scala +++ b/cluster/src/test/scala/io/snappydata/Snap_213.scala @@ -30,7 +30,7 @@ class Snap_213 override def beforeAll(): Unit = { super.beforeAll() // force boot GemFireXD if not booted; just getting SnappyContext should do - println(s"Using SnappyContext $snc") + logInfo(s"Using SnappyContext $snc") } override def afterAll(): Unit = { @@ -42,7 +42,7 @@ class Snap_213 DriverRegistry.register(Constant.JDBC_CLIENT_DRIVER) val hostPort = TestUtil.startNetServer() - println("server started ") + logInfo("server started") val conn: Connection = DriverManager.getConnection( "jdbc:snappydata://" + hostPort) 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 aaecd957b5..4c34944d2b 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCH_Queries.scala @@ -17,13 +17,13 @@ package io.snappydata.benchmark -import scala.util.Random +import org.apache.spark.Logging -object TPCH_Queries { +object TPCH_Queries extends Logging { private var random = new scala.util.Random(42) - def setRandomSeed(randomSeed : Integer = 42): Unit ={ + def setRandomSeed(randomSeed: Integer = 42): Unit = { this.random = new scala.util.Random(randomSeed) } @@ -112,13 +112,12 @@ object TPCH_Queries { } def createQuery(query: String, paramters: Array[String]): String = { - // scalastyle:off println var generatedQuery = query for (s <- paramters) { - println(s"KBKBKB : createQuery : $s") + logInfo(s"KBKBKB : createQuery : $s") generatedQuery = generatedQuery.replaceFirst("\\?", s) } - println(s"KBKBKB : My query : $generatedQuery") + logInfo(s"KBKBKB : My query : $generatedQuery") generatedQuery } @@ -298,7 +297,7 @@ object TPCH_Queries { val fromDate = java.time.LocalDate.of(1995, 3, 1) val toDate = java.time.LocalDate.of(1995, 3, 31) val diff = java.time.temporal.ChronoUnit.DAYS.between(fromDate, toDate) - //val random = new random(System.nanoTime) + // val random = new random(System.nanoTime) // You may want a different seed val selectedDate = fromDate.plusDays(random.nextInt(diff.toInt)) Array(segment, selectedDate.toString, selectedDate.toString) diff --git a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala index 85e89e1132..799d72bae1 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/PreparedQueryRoutingSingleNodeSuite.scala @@ -23,15 +23,14 @@ import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils import io.snappydata.{SnappyFunSuite, SnappyTableStatsProviderService} import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkConf -import org.apache.spark.sql.{ParseException, SnappyContext, SnappySession} +import org.apache.spark.sql.{SnappyContext, SnappySession} +import org.apache.spark.{Logging, SparkConf} class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll { - // Logger.getLogger("org").setLevel(Level.DEBUG) + private val default_chunk_size = GemFireXDUtils.DML_MAX_CHUNK_SIZE - val default_chunk_size = GemFireXDUtils.DML_MAX_CHUNK_SIZE - protected override def newSparkConf(addOn: (SparkConf) => SparkConf): SparkConf = { + protected override def newSparkConf(addOn: SparkConf => SparkConf): SparkConf = { /** * Setting local[n] here actually supposed to affect number of reservoir created * while sampling. @@ -237,7 +236,7 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val serverHostPort = TestUtil.startNetServer() - // println("network server started") + // logInfo("network server started") PreparedQueryRoutingSingleNodeSuite.insertRows(tableName, 1000, serverHostPort) query0(tableName, serverHostPort) } finally { @@ -257,7 +256,7 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA "options( partition_by 'ol_int_id, ol_int2_id', buckets '2')") val serverHostPort = TestUtil.startNetServer() - // println("network server started") + // logInfo("network server started") PreparedQueryRoutingSingleNodeSuite.insertRows(tableName, 100, serverHostPort) query6(tableName, serverHostPort) query7(tableName, serverHostPort) @@ -482,16 +481,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 20) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -525,16 +520,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 100) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -565,16 +556,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 100) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -606,16 +593,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 20) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -649,16 +632,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 20) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -690,16 +669,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 100) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -731,16 +706,12 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val i = rs.getInt(1) // val j = rs.getInt(2) // val s = rs.getString(3) - // scalastyle:off println - // println(s"row($index) $i $j $s ") - // scalastyle:on println + // logInfo(s"row($index) $i $j $s ") index += 1 } assert(index == 20) - // scalastyle:off println - // println(s"$qryName Number of rows read " + index) - // scalastyle:on println + // logInfo(s"$qryName Number of rows read " + index) rs.close() // Thread.sleep(1000000) } finally { @@ -766,7 +737,7 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA val serverHostPort = TestUtil.startNetServer() - // println("network server started") + // logInfo("network server started") PreparedQueryRoutingSingleNodeSuite.insertRows(tableName1, 1000, serverHostPort) PreparedQueryRoutingSingleNodeSuite.insertRows(tableName2, 1000, serverHostPort) query1(tableName1, tableName2, serverHostPort) @@ -782,14 +753,14 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA test("update delete on column table") { val snc = this.snc val serverHostPort = TestUtil.startNetServer() - // println("network server started") + // logInfo("network server started") PreparedQueryRoutingSingleNodeSuite.updateDeleteOnColumnTable(snc, serverHostPort) } test("SNAP-1981: Equality on string columns") { val snc = this.snc val serverHostPort = TestUtil.startNetServer() - // println("network server started") + // logInfo("network server started") PreparedQueryRoutingSingleNodeSuite.equalityOnStringColumn(snc, serverHostPort) } @@ -830,15 +801,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"1-row($index) $i $j") - // scalastyle:on println + logInfo(s"1-row($index) $i $j") index += 1 assert(i == 1 || i == 2) } - // scalastyle:off println - println(s"1-Number of rows read " + index) - // scalastyle:on println + logInfo(s"1-Number of rows read " + index) assert(index == 2) assert(cacheMap.size() == 1) @@ -848,15 +815,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"2-row($index) $i $j") - // scalastyle:on println + logInfo(s"2-row($index) $i $j") index += 1 assert(i == 1 || i == 2 || i == 3) } - // scalastyle:off println - println(s"2-Number of rows read " + index) - // scalastyle:on println + logInfo(s"2-Number of rows read " + index) assert(index == 3) assert(cacheMap.size() == 1) close(prepStatement0) @@ -869,15 +832,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"3-row($index) $i $j") - // scalastyle:on println + logInfo(s"3-row($index) $i $j") index += 1 assert(i > 2 && i < 6) } - // scalastyle:off println - println(s"3-Number of rows read " + index) - // scalastyle:on println + logInfo(s"3-Number of rows read " + index) assert(index == 3) assert(cacheMap.size() == 2) @@ -887,15 +846,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"4-row($index) $i $j") - // scalastyle:on println + logInfo(s"4-row($index) $i $j") index += 1 assert(i > 3 && i < 7) } - // scalastyle:off println - println(s"4-Number of rows read " + index) - // scalastyle:on println + logInfo(s"4-Number of rows read " + index) assert(index == 3) assert(cacheMap.size() == 2) close(prepStatement1) @@ -909,15 +864,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getString(2) - // scalastyle:off println - println(s"5-row($index) $i $j") - // scalastyle:on println + logInfo(s"5-row($index) $i $j") index += 1 assert(i == 1) } - // scalastyle:off println - println(s"5-Number of rows read " + index) - // scalastyle:on println + logInfo(s"5-Number of rows read " + index) assert(index == 1) assert(cacheMap.size() == 3) @@ -927,15 +878,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getString(2) - // scalastyle:off println - println(s"6-row($index) $i $j") - // scalastyle:on println + logInfo(s"6-row($index) $i $j") index += 1 assert(i == 2) } - // scalastyle:off println - println(s"6-Number of rows read " + index) - // scalastyle:on println + logInfo(s"6-Number of rows read " + index) assert(index == 1) assert(cacheMap.size() == 3) close(prepStatement2) @@ -953,15 +900,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA index = 0 while (update.next()) { val i = update.getInt(1) - // scalastyle:off println - println(s"7-row($index) $i") - // scalastyle:on println + logInfo(s"7-row($index) $i") index += 1 assert(i == 1 || i == 2) } - // scalastyle:off println - println(s"7-Number of rows read " + index) - // scalastyle:on println + logInfo(s"7-Number of rows read " + index) assert(index == 2) assert(cacheMap.size() == 0) @@ -971,15 +914,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA index = 0 while (update.next()) { val i = update.getInt(1) - // scalastyle:off println - println(s"8-row($index) $i") - // scalastyle:on println + logInfo(s"8-row($index) $i") index += 1 assert(i == 2 || i == 3) } - // scalastyle:off println - println(s"8-Number of rows read " + index) - // scalastyle:on println + logInfo(s"8-Number of rows read " + index) assert(index == 2) assert(cacheMap.size() == 0) close(prepStatement3) @@ -993,15 +932,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"9-row($index) $i $j") - // scalastyle:on println + logInfo(s"9-row($index) $i $j") index += 1 assert(i == 1 || i == 2) } - // scalastyle:off println - println(s"9-Number of rows read " + index) - // scalastyle:on println + logInfo(s"9-Number of rows read " + index) assert(index == 2) assert(cacheMap.size() == 1) close(prepStatement4) @@ -1016,15 +951,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA while (update.next()) { val i = update.getInt(1) val j = update.getBigDecimal(2) - // scalastyle:off println - println(s"10-row($index) $i $j") - // scalastyle:on println + logInfo(s"10-row($index) $i $j") index += 1 assert(i == 1) } - // scalastyle:off println - println(s"10-Number of rows read " + index) - // scalastyle:on println + logInfo(s"10-Number of rows read " + index) assert(index == 1) assert(cacheMap.size() == 2) close(prepStatement5) @@ -1139,15 +1070,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA "MCI", "STL", "MSY", "SAT", "SNA", "DAL", "PDX", "SMF", "HOU", "SAN", "OAK", "SJC") while (update.next()) { val s = update.getString(3) - // scalastyle:off println - // println(s"1-row($index) $s ") - // scalastyle:on println - result1.contains(s) + // logInfo(s"1-row($index) $s ") + assert(result1.contains(s)) index += 1 } - // scalastyle:off println - println(s"1-Number of rows read " + index) - // scalastyle:on println + logInfo(s"1-Number of rows read " + index) assert(index == 46) assert(cacheMap.size() == 0) @@ -1164,15 +1091,11 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA "SMF", "ONT", "SJC", "OAK", "HOU", "DAL", "BUR") while (update.next()) { val s = update.getString(3) - // scalastyle:off println - // println(s"2-row($index) $s ") - // scalastyle:on println - result2.contains(s) + // logInfo(s"2-row($index) $s ") + assert(result2.contains(s)) index += 1 } - // scalastyle:off println - println(s"2-Number of rows read " + index) - // scalastyle:on println + logInfo(s"2-Number of rows read " + index) assert(index == 65) assert(cacheMap.size() == 0) close(prepStatement1) @@ -1186,7 +1109,7 @@ class PreparedQueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndA } } -object PreparedQueryRoutingSingleNodeSuite{ +object PreparedQueryRoutingSingleNodeSuite extends Logging { def insertRows(tableName: String, numRows: Int, serverHostPort: String): Unit = { @@ -1211,9 +1134,7 @@ object PreparedQueryRoutingSingleNodeSuite{ }) val ret = prepareStatement.executeBatch() ret.foreach(r => assert(r == 1)) - // scalastyle:off println - println(s"committed $numRows rows") - // scalastyle:on println + logInfo(s"committed $numRows rows") } finally { prepareStatement.close() conn.close() @@ -1229,22 +1150,16 @@ object PreparedQueryRoutingSingleNodeSuite{ val i = rs.getInt(1) val j = rs.getInt(2) val s = rs.getString(3) - // scalastyle:off println - println(s"$qry row($index) $i $j $s ") - // scalastyle:on println + logInfo(s"$qry row($index) $i $j $s ") index += 1 assert(results.contains(i)) } - // scalastyle:off println - println(s"$qry Number of rows read " + index) - // scalastyle:on println + logInfo(s"$qry Number of rows read " + index) assert(index == results.length) rs.close() - // scalastyle:off println - println(s"cachemapsize = ${cacheMapSize} and .size = ${cacheMap.size()}") - // scalastyle:on println + logInfo(s"cachemapsize = $cacheMapSize and .size = ${cacheMap.size()}") assert( cacheMap.size() == cacheMapSize || -1 == cacheMapSize) } diff --git a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala index 64f196b3a8..39194fa11a 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala @@ -39,17 +39,13 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll // results in multiple batches setDMLMaxChunkSize(50L) serverHostPort = TestUtil.startNetServer() - // scalastyle:off println - println("network server started") - // scalastyle:on println + logInfo("network server started") } override def afterAll(): Unit = { setDMLMaxChunkSize(default_chunk_size) TestUtil.stopNetServer() - // scalastyle:off println - println("network server stopped") - // scalastyle:on println + logInfo("network server stopped") super.afterAll() } @@ -75,9 +71,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll } }) stmt.executeBatch() - // scalastyle:off println - println(s"committed $numRows rows") - // scalastyle:on println + logInfo(s"committed $numRows rows") } finally { stmt.close() conn.close() @@ -97,9 +91,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll rs.getInt(1) index += 1 } - // scalastyle:off println - println("Number of rows read " + index) - // scalastyle:on println + logInfo("Number of rows read " + index) rs.close() } finally { stmt.close() @@ -135,9 +127,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll } }) stmt.executeBatch() - // scalastyle:off println - println(s"committed $numRows rows") - // scalastyle:on println + logInfo(s"committed $numRows rows") } finally { stmt.close() conn.close() @@ -154,17 +144,13 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll val i = rs.getInt(1) val j = rs.getInt(2) val s = rs.getString(3) - // scalastyle:off println - println(s"$qry row($index) $i $j $s ") - // scalastyle:on println + logInfo(s"$qry row($index) $i $j $s") index += 1 assert(results.contains(i)) } - // scalastyle:off println - println(s"$qry Number of rows read " + index) - // scalastyle:on println + logInfo(s"$qry Number of rows read " + index) assert(index == results.length) rs.close() } @@ -241,16 +227,12 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll s") " + s" limit 20" + s"" - // scalastyle:off println - println(s"Iter ${iter} QUERY = ${qry}") - // scalastyle:on println + logInfo(s"Iter $iter QUERY = $qry") val df1 = snc.sql(qry) val res1 = df1.collect() - // scalastyle:off println - println(s"Iter ${iter} with query = ${qry}") - res1.foreach(println) - println(s"Iter ${iter} query end and res1 size = ${res1.length}") - // scalastyle:on println + logInfo(s"Iter $iter with query = $qry") + logInfo(res1.mkString("\n")) + logInfo(s"Iter $iter query end and res1 size = ${res1.length}") assert(res1.length == 3) val qry2 = s"select ol_1_int_id, ol_1_int2_id, ol_1_str_id " + @@ -265,12 +247,10 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll s"" val df2 = snc.sql(qry2) val res2 = df2.collect() - // scalastyle:off println - println(s"Iter ${iter} with query2 = ${qry2}") - res2.foreach(println) - println(s"Iter ${iter} query2 end with res size = ${res2.length}") - // scalastyle:on println - assert(!(res1.sameElements(res2))) + logInfo(s"Iter $iter with query2 = $qry2") + logInfo(res2.mkString("\n")) + logInfo(s"Iter $iter query2 end with res size = ${res2.length}") + assert(!res1.sameElements(res2)) assert(res2.length == 3) } @@ -314,9 +294,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll stmt.addBatch(s"insert into $tName values(5,null,'ddd')") stmt.addBatch(s"insert into $tName values(6,10.6,'ddd')") stmt.executeBatch() - // scalastyle:off println - println(s"inserted rows") - // scalastyle:on println + logInfo(s"inserted rows") } finally { stmt.close() conn.close() @@ -337,9 +315,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll assert(rs.getString(2) != null) index += 1 } - // scalastyle:off println - println(s"Number of rows read $index sum=$sum") - // scalastyle:on println + logInfo(s"Number of rows read $index sum=$sum") assert(index == 5, index) assert(sum - 18138.2 == 0, sum) rs.close() @@ -367,9 +343,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll } }) stmt.executeBatch() - // scalastyle:off println - println(s"committed $numRows rows") - // scalastyle:on println + logInfo(s"committed $numRows rows") } finally { stmt.close() conn.close() @@ -383,13 +357,9 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll val stmt = conn.createStatement() try { val query = s"select distinct ol_w_id from order_line_row_bool" - - snc.sql(query).show() - val count = snc.sql(query).count() + val count = snc.sql(query).collect().length assert(count == 2) - // scalastyle:off println - println("snc: Number of rows read " + count) - // scalastyle:on println + logInfo("snc: Number of rows read " + count) val rs = stmt.executeQuery(query) var index = 0 @@ -397,9 +367,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll rs.getInt(1) index += 1 } - // scalastyle:off println - println("jdbc: Number of rows read " + index) - // scalastyle:on println + logInfo("jdbc: Number of rows read " + index) assert(index == 2) rs.close() } finally { @@ -459,25 +427,18 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll s" (p.ds_class_id = 'C' AND p.property = 'DOUBLE_PROP' AND p.double_value > 0.2) OR " + s" (p.ds_class_id = 'C' AND p.property = 'DOUBLE_PROP' AND p.double_value < 0.2)" - snc.sql(query).show() - val count = snc.sql(query).count() + val count = snc.sql(query).collect().length assert(count == 2) - // scalastyle:off println - println("snc: Number of rows read " + count) - // scalastyle:on println + logInfo("snc: Number of rows read " + count) val rs = stmt.executeQuery(query) var index = 0 while (rs.next()) { index += 1 - // scalastyle:off println - println(s"$index: ${rs.getString(1)} ${rs.getString(2)} ${rs.getString(3)} " + + logInfo(s"$index: ${rs.getString(1)} ${rs.getString(2)} ${rs.getString(3)} " + s"${rs.getString(4)} ${rs.getString(5)} ${rs.getLong(6)} ${rs.getBigDecimal(7)}") - // scalastyle:on println } - // scalastyle:off println - println("jdbc: Number of rows read " + index) - // scalastyle:on println + logInfo("jdbc: Number of rows read " + index) assert(index == 2) rs.close() } finally { @@ -504,9 +465,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll } }) stmt.executeBatch() - // scalastyle:off println - println(s"committed $numRows rows") - // scalastyle:on println + logInfo(s"committed $numRows rows") } finally { stmt.close() conn.close() @@ -651,9 +610,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll } }) stmt.executeBatch() - // scalastyle:off println - println(s"insertRows2: committed $numRows rows") - // scalastyle:on println + logInfo(s"insertRows2: committed $numRows rows") } finally { stmt.close() conn.close() @@ -665,9 +622,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll val stmt = conn.createStatement() try { val numRows = stmt.executeUpdate(s"insert into $tableName1 select * from $tableName2") - // scalastyle:off println - println(s"insertInto $numRows rows") - // scalastyle:on println + logInfo(s"insertInto $numRows rows") assert(numRows == rowsExpected) } finally { stmt.close() @@ -680,9 +635,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll val stmt = conn.createStatement() try { val numRows = stmt.executeUpdate(s"put into $tableName1 select * from $tableName2") - // scalastyle:off println - println(s"putInto $numRows rows") - // scalastyle:on println + logInfo(s"putInto $numRows rows") assert(numRows == rowsExpected) } finally { stmt.close() @@ -723,11 +676,6 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll df.foreach(r => { val col1 = r.getInt(0) val col2 = r.getInt(1) - - // scalastyle:off println - println(s"select row $r") - // scalastyle:on println - if (col1 < 6) { assertionNotFailed = assertionNotFailed && (col1 + 1 == col2) } else { diff --git a/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala index 68d32f65c0..93e30f8311 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/StringAsClobTestSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterAll class StringAsClobTestSuite extends SnappyFunSuite with BeforeAndAfterAll { - val default_chunk_size = GemFireXDUtils.DML_MAX_CHUNK_SIZE + private val default_chunk_size = GemFireXDUtils.DML_MAX_CHUNK_SIZE var serverHostPort = "" val tableName = "order_line_col" @@ -48,7 +48,7 @@ class StringAsClobTestSuite extends SnappyFunSuite with BeforeAndAfterAll { test("Test char") { snc val serverHostPort2 = TestUtil.startNetServer() - println("network server started") + logInfo("network server started") val conn = DriverManager.getConnection(s"jdbc:snappydata://$serverHostPort2") val s = conn.createStatement() s.executeUpdate(s"create table $tableName (id int not null primary key, name String, address " + @@ -60,7 +60,7 @@ class StringAsClobTestSuite extends SnappyFunSuite with BeforeAndAfterAll { s.executeUpdate(s"update $tableName set name='abc1' where id=111") val rs = s.executeQuery(s"select id, name, address from $tableName") while (rs.next()) { - println(s"${rs.getInt(1)} ${rs.getString(2)} ${rs.getString(3)}") + logInfo(s"${rs.getInt(1)} ${rs.getString(2)} ${rs.getString(3)}") } val rs2 = s.executeQuery(s"select id from $tableName where name='abc1'") if (rs2.next()) { 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 7bcc559f9f..df997aa904 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/SnappyMemoryAccountingSuite.scala @@ -235,21 +235,19 @@ class SnappyMemoryAccountingSuite extends MemoryFunSuite { // 208 *10. 208 is the row size + memory overhead var rows = 0 - // scalastyle:off try { for (i <- 1 to 100) { val row = Row(100000000, 10000000, 10000000) - println(s"RowCount1 = $rows") + logInfo(s"RowCount1 = $rows") snSession.insert("t1", row) rows += 1 - println(s"RowCount2 = $rows") + logInfo(s"RowCount2 = $rows") } } catch { case sqle: SQLException if sqle.getSQLState == "XCL54" => - println(s"RowCount3 in exception = $rows") + logInfo(s"RowCount3 in exception = $rows") assert(totalEvictedBytes > 0) } - // scalastyle:on SparkEnv.get.memoryManager. asInstanceOf[SnappyUnifiedMemoryManager].dropAllObjects(memoryMode) val count = snSession.sql("select * from t1").count() 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 58ce43b926..4501ede1c9 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/IndexTest.scala @@ -53,7 +53,6 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { snc.sql("create table checko (col1 Integer primary key, col2 Integer) using row options " + "(partition_by 'col1') ") - // scalastyle:off println val data = sc.parallelize(Seq(Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), Row(5, 5), Row(6, 6))) @@ -74,7 +73,6 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { df.filter("b < 2").selectExpr("i as col1").write.deleteFrom("APP.CHECKO") assert(snc.sql("select * from checko").count() == 3) - // scalastyle:on println } test("check varchar index") { @@ -115,12 +113,11 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { snc.sql("update ods.organizations set descr = 'EL " + " " + " ' where client_id = 8006") - snc.sql("select * from ods.organizations").show() - snc.sql("select client_id, descr from ods.organizations where client_id = 8006").show() + snc.sql("select * from ods.organizations").collect() + snc.sql("select client_id, descr from ods.organizations where client_id = 8006").collect() } test("tpch queries") { - // scalastyle:off println val qryProvider = new TPCH with SnappyAdapter val queries = Array("1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", @@ -151,7 +148,7 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { |$results """.stripMargin } - println(s"Done $qNum") + logInfo(s"Done $qNum") } snc.setConf(io.snappydata.Property.EnableExperimentalFeatures.name, existing) @@ -185,7 +182,7 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { (tableName, snc.table(tableName).count()) }.toMap - tableSizes.foreach(println) + logInfo(tableSizes.mkString("\n")) runBenchmark("select o_orderkey from orders where o_orderkey = 1", tableSizes, 2) runBenchmark("select o_orderkey from orders where o_orderkey = 32", tableSizes) runBenchmark("select o_orderkey from orders where o_orderkey = 801", tableSizes) @@ -247,7 +244,7 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { def executor(str: String) = snc.sql(str) val size = qryProvider.estimateSizes(query, tableSizes, executor) - println(s"$qNum size $size") + logInfo(s"$qNum size $size") val b = new Benchmark(s"JoinOrder optimization", size, minNumIters = 10) def case1(): Unit = snc.setConf(io.snappydata.Property.EnableExperimentalFeatures.name, @@ -287,12 +284,11 @@ class IndexTest extends SnappyFunSuite with PlanTest with BeforeAndAfterEach { } test("northwind queries") { - println("") // val sctx = sc(c => c.set("spark.sql.inMemoryColumnarStorage.batchSize", "40000")) // val snc = getOrCreate(sctx) // NorthWindDUnitTest.createAndLoadColumnTables(snc) // val s = "select distinct shipcountry from orders" - // snc.sql(s).show() + // snc.sql(s).collect() // NWQueries.assertJoin(snc, NWQueries.Q42, "Q42", 22, 1, classOf[LocalJoin]) /* Thread.sleep(1000 * 60 * 60) 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 2654af2230..aaeaf0c3ac 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/MiscTest.scala @@ -35,7 +35,7 @@ class MiscTest extends SnappyFunSuite with Logging { snc.sql("WITH temp_table AS ( SELECT ol_1_int2_id as col1," + " sum(ol_1_int_id) AS col2 FROM table1 GROUP BY ol_1_int2_id)" + " SELECT ol_1_int2_id FROM temp_table ," + - " table1 WHERE ol_1_int2_id = col1 LIMIT 100 ").show + " table1 WHERE ol_1_int2_id = col1 LIMIT 100 ").collect() } test("Pool test") { 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 42d6b4a308..31242ad34a 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 @@ -308,7 +308,7 @@ class SnappyTableMutableAPISuite extends SnappyFunSuite with Logging with Before snc.insert("row_table", Row(4, "4", "3", 3)) val df = snc.sql("update row_table set col3 = '5' where col2 in (select col2 from col_table)") - df.show + df.collect() val resultdf = snc.table("row_table").collect() assert(resultdf.length == 4) @@ -1151,7 +1151,7 @@ class SnappyTableMutableAPISuite extends SnappyFunSuite with Logging with Before SnappyContext.globalSparkContext.stop() snc = new SnappySession(sc) - snc.sql("select count(1) from t1").show + snc.sql("select count(1) from t1").collect() } test("Bug-2348 : Invalid stats bitmap") { 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 ace93d2a51..8a01a308ba 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 @@ -438,12 +438,12 @@ class ColumnCacheBenchmark extends SnappyFunSuite { testDF.write.insertInto("wide_table1") val uniqDf = snappySession.table("wide_table").dropDuplicates(Array("C1")) - uniqDf.count() + logInfo("Number of unique rows in wide_table = " + uniqDf.count()) // check fallback plans being invoked via API - uniqDf.show() + logInfo(uniqDf.collect().mkString("\n")) // and also via SQL val s = (2 to num_col).map(i => s"last(C$i)").mkString(",") - snappySession.sql(s"select C1, $s from wide_table group by C1").show() + snappySession.sql(s"select C1, $s from wide_table group by C1").collect() val df = snappySession.sql("select *" + " from wide_table a , wide_table1 b where a.c1 = b.c1 and a.c1 = '1'") @@ -451,7 +451,6 @@ class ColumnCacheBenchmark extends SnappyFunSuite { val df0 = snappySession.sql(s"select * from wide_table") df0.collect() - df0.show() val avgProjections = (1 to num_col).map(i => s"AVG(C$i)").mkString(",") val df1 = snappySession.sql(s"select $avgProjections from wide_table") 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 dfe97273ba..e0f4d4c550 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 @@ -258,9 +258,6 @@ object TAQTest extends Logging with Assertions { private def collect(df: Dataset[Row], expectedNumResults: Int): Unit = { val result = df.collect() assert(result.length === expectedNumResults) - // scalastyle:off - println(s"Count = ${result.length}") - // scalastyle:on } private def doGC(): Unit = { 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 c54fc80da7..27754afb50 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 @@ -21,14 +21,13 @@ import java.util.concurrent.atomic.AtomicInteger import io.snappydata.SnappyFunSuite import org.apache.kafka.common.TopicPartition - -import org.apache.spark.sql.functions.{count, window} -import org.apache.spark.sql.streaming.ProcessingTime import org.scalatest.concurrent.Eventually import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.streaming.ProcessingTime case class Account(accountName: String) @@ -236,7 +235,7 @@ class SnappyStructuredKafkaSuite extends SnappyFunSuite with Eventually .start() streamingQuery.processAllAvailable() - session.sql("select * from snappyWindowAggrTable").show(200) + logInfo(session.sql("select * from snappyWindowAggrTable").limit(200).collect().mkString("\n")) streamingQuery.stop() } diff --git a/core/build.gradle b/core/build.gradle index e7647f35d9..518b3a2b4d 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -15,12 +15,8 @@ * LICENSE file. */ -plugins { - id 'com.github.johnrengelman.shadow' version '2.0.4' - id 'de.undercouch.download' version '3.4.3' -} - apply plugin: 'scala' +apply plugin: 'de.undercouch.download' compileScala.options.encoding = 'UTF-8' // fix scala+java mix to all use compileScala which uses correct dependency order @@ -94,21 +90,21 @@ dependencies { } compile project(":snappy-jdbc_${scalaBinaryVersion}") - compile("org.parboiled:parboiled_${scalaBinaryVersion}:2.1.4") { + compile("org.parboiled:parboiled_${scalaBinaryVersion}:${parboiledVersion}") { exclude(group: 'org.scala-lang', module: 'scala-library') exclude(group: 'org.scala-lang', module: 'scala-reflect') exclude(group: 'org.scala-lang', module: 'scala-compiler') } - compile 'org.apache.tomcat:tomcat-jdbc:8.5.23' - compile 'com.zaxxer:HikariCP:2.7.1' - // compile 'org.spark-project:dstream-twitter_2.11:0.1.0' - compile 'org.twitter4j:twitter4j-stream:4.0.6' - compile 'org.objenesis:objenesis:2.6' + compile "org.apache.tomcat:tomcat-juli:${tomcatJdbcVersion}" + compile "org.apache.tomcat:tomcat-jdbc:${tomcatJdbcVersion}" + compile "com.zaxxer:HikariCP:${hikariCPVersion}" + compile "org.twitter4j:twitter4j-stream:${twitter4jVersion}" + compile "org.objenesis:objenesis:${objenesisVersion}" compile "com.esotericsoftware:kryo-shaded:${kryoVersion}" compile "org.eclipse.collections:eclipse-collections-api:${eclipseCollectionsVersion}" compile "org.eclipse.collections:eclipse-collections:${eclipseCollectionsVersion}" - compileOnly 'com.rabbitmq:amqp-client:3.5.7' + compileOnly "com.rabbitmq:amqp-client:${rabbitMqVersion}" testCompile project(':dunit') testCompile 'org.scala-lang:scala-actors:' + scalaVersion diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala index 67098d0da5..946179e55e 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala @@ -804,7 +804,7 @@ class SplitClusterDUnitSecurityTest(s: String) def getJobJar(className: String, packageStr: String = ""): String = { val dir = new File(s"$snappyProductDir/../../../cluster/build-artifacts/scala-2.11/classes/" - + s"test/$packageStr") + + s"scala/test/$packageStr") assert(dir.exists() && dir.isDirectory, s"snappy-cluster scala tests not compiled. Directory " + s"not found: $dir") val jar = TestPackageUtils.createJarFile(dir.listFiles(new FileFilter { 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 63a33b34de..61cac1d3e9 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -354,7 +354,6 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti val (executedPlan, withFallback) = SnappySession.getExecutedPlan(queryExecution.executedPlan) def execute(): (Iterator[R], Long) = withNewExecutionIdTiming { - snappySession.addContextObject(SnappySession.ExecutionKey, () => queryExecution) def executeCollect(): Array[InternalRow] = { if (withFallback ne null) withFallback.executeCollect() @@ -412,7 +411,6 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti try { withCallback("collect")(_ => execute()) } finally { - snappySession.removeContextObject(SnappySession.ExecutionKey) if (!hasLocalCallSite) { sc.clearCallSite() } 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 331a802f78..b49f812901 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -295,7 +295,7 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { /** * Remove a context object registered using [[addContextObject]]. */ - private[sql] def removeContextObject(key: Any): Unit = { + private[sql] def removeContextObject(key: Any): Any = { contextObjects.remove(key) } @@ -1815,8 +1815,8 @@ object SnappySession extends Logging { } def getExecutedPlan(plan: SparkPlan): (SparkPlan, CodegenSparkFallback) = plan match { - case cg@CodegenSparkFallback(WholeStageCodegenExec(p)) => (p, cg) - case cg@CodegenSparkFallback(p) => (p, cg) + case cg@CodegenSparkFallback(WholeStageCodegenExec(p), _) => (p, cg) + case cg@CodegenSparkFallback(p, _) => (p, cg) case WholeStageCodegenExec(p) => (p, null) case _ => (plan, null) } 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 856517acaf..6c81b914d3 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyStrategies.scala @@ -802,7 +802,7 @@ 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) + case _ => CodegenSparkFallback(plan, session) } } 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 a62e0258da..71cfaf0e7b 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 @@ -35,7 +35,8 @@ import org.apache.spark.sql.{SnappyContext, SnappySession, ThinClientConnectorMo * Catch exceptions in code generation of SnappyData plans and fallback * to Spark plans as last resort (including non-code generated paths). */ -case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { +case class CodegenSparkFallback(var child: SparkPlan, + @transient session: SnappySession) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -43,6 +44,9 @@ case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { override def outputOrdering: Seq[SortOrder] = child.outputOrdering + @transient private[this] val execution = + session.getContextObject[() => QueryExecution](SnappySession.ExecutionKey) + protected[sql] def isCodeGenerationException(t: Throwable): Boolean = { // search for any janino or code generation exception var cause = t @@ -116,8 +120,6 @@ case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { // is still usable: SystemFailure.checkFailure() - val session = sqlContext.sparkSession.asInstanceOf[SnappySession] - val isCatalogStale = isConnectorCatalogStaleException(t, session) if (isCatalogStale) { session.externalCatalog.invalidateAll() @@ -131,7 +133,7 @@ case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { } if (isCatalogStale || isCodeGenerationException(t)) { // fallback to Spark plan for code-generation exception - session.getContextObject[() => QueryExecution](SnappySession.ExecutionKey) match { + execution match { case Some(exec) => if (!isCatalogStale) { val msg = new StringBuilder @@ -147,7 +149,7 @@ case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { } try { val plan = exec().executedPlan.transform { - case CodegenSparkFallback(p) => p + case CodegenSparkFallback(p, _) => p } val result = f(plan) // update child for future executions @@ -161,7 +163,7 @@ case class CodegenSparkFallback(var child: SparkPlan) extends UnaryExecNode { } finally if (!isCatalogStale) { session.sessionState.disableStoreOptimizations = false } - case None => throw t + case _ => throw t } } else { throw t 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 31dd21ed10..34c1f8155f 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 @@ -212,7 +212,7 @@ private[sql] object PartitionedPhysicalScan { def getSparkPlanInfo(fullPlan: SparkPlan, paramLiterals: Array[ParamLiteral] = EMPTY_PARAMS, paramsId: Int = -1): SparkPlanInfo = { val plan = fullPlan match { - case CodegenSparkFallback(child) => child + case CodegenSparkFallback(child, _) => child case _ => fullPlan } val children = plan match { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala index d6491c71d4..2cd3679b5d 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/ColumnDeltaEncoder.scala @@ -675,7 +675,7 @@ object DeltaWriter { val evaluator = new CompilerFactory().newScriptEvaluator() evaluator.setClassName("io.snappydata.execute.GeneratedDeltaWriterFactory") evaluator.setParentClassLoader(getClass.getClassLoader) - evaluator.setDefaultImports(defaultImports) + evaluator.setDefaultImports(defaultImports: _*) val (name, complexType) = dataType match { case BooleanType => ("Boolean", "") diff --git a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala index 0a98815521..1e0d1080ef 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/SnappySessionState.scala @@ -683,11 +683,11 @@ class SnappySessionState(val snappySession: SnappySession) protected def newQueryExecution(plan: LogicalPlan): QueryExecution = { new QueryExecution(snappySession, plan) { - snappySession.addContextObject(SnappySession.ExecutionKey, - () => newQueryExecution(plan)) - - override protected def preparations: Seq[Rule[SparkPlan]] = + override protected def preparations: Seq[Rule[SparkPlan]] = { + snappySession.addContextObject(SnappySession.ExecutionKey, + () => newQueryExecution(plan)) queryPreparations(topLevel = true) + } } } 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 418eaf3b78..4ddebc3ec7 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 @@ -281,7 +281,7 @@ object CodeGeneration extends Logging { val evaluator = new CompilerFactory().newScriptEvaluator() evaluator.setClassName("io.snappydata.execute.GeneratedEvaluation") evaluator.setParentClassLoader(getClass.getClassLoader) - evaluator.setDefaultImports(defaultImports) + evaluator.setDefaultImports(defaultImports: _*) val separator = "\n " val varDeclarations = ctx.mutableStates.map { case (javaType, name, init) => s"$javaType $name;$separator${init.replace("this.", "")}" @@ -329,7 +329,7 @@ object CodeGeneration extends Logging { val evaluator = new CompilerFactory().newScriptEvaluator() evaluator.setClassName("io.snappydata.execute.GeneratedIndexEvaluation") evaluator.setParentClassLoader(getClass.getClassLoader) - evaluator.setDefaultImports(defaultImports) + evaluator.setDefaultImports(defaultImports: _*) val separator = "\n " val varDeclarations = ctx.mutableStates.map { case (javaType, name, init) => s"$javaType $name;$separator${init.replace("this.", "")}" @@ -428,7 +428,7 @@ object CodeGeneration extends Logging { classOf[CalendarInterval].getName, classOf[ArrayData].getName, classOf[MapData].getName, - classOf[InternalDataSerializer].getName)) + classOf[InternalDataSerializer].getName): _*) val separator = "\n " val varDeclarations = ctx.mutableStates.map { case (javaType, name, init) => s"$javaType $name;$separator${init.replace("this.", "")}" @@ -453,7 +453,7 @@ object CodeGeneration extends Logging { def executeUpdate(name: String, stmt: PreparedStatement, rows: Seq[Row], multipleRows: Boolean, batchSize: Int, schema: Array[StructField], dialect: JdbcDialect): Int = { - val iterator = new java.util.Iterator[InternalRow] { + val iterator: java.util.Iterator[InternalRow] = new java.util.Iterator[InternalRow] { private val baseIterator = rows.iterator private val encoder = RowEncoder(StructType(schema)) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 222d083979..0df4d6bfa0 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -38,11 +38,20 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.RollingFileAppender log4j.appender.file.append=true log4j.appender.file.file=snappydata.log -log4j.appender.file.MaxFileSize=100MB +log4j.appender.file.MaxFileSize=1GB log4j.appender.file.MaxBackupIndex=10000 log4j.appender.file.layout=io.snappydata.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS zzz} %t %p %c{1}: %m%n +# Appender for code dumps of WholeStageCodegenExec, CodeGenerator etc +log4j.appender.code=org.apache.log4j.RollingFileAppender +log4j.appender.code.append=true +log4j.appender.code.file=generatedcode.log +log4j.appender.code.MaxFileSize=1GB +log4j.appender.code.MaxBackupIndex=10000 +log4j.appender.code.layout=io.snappydata.log4j.PatternLayout +log4j.appender.code.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS zzz} %t %p %c{1}: %m%n + # Console appender log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.target=System.out @@ -92,8 +101,6 @@ log4j.logger.org.apache.spark.scheduler.FairSchedulableBuilder=WARN log4j.logger.org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint=WARN log4j.logger.org.apache.spark.storage.BlockManagerInfo=WARN log4j.logger.org.apache.hadoop.hive=WARN -# for all Spark generated code (including ad-hoc UnsafeProjection calls etc) -log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=WARN log4j.logger.org.apache.spark.sql.execution.datasources=WARN log4j.logger.org.apache.spark.scheduler.SnappyTaskSchedulerImpl=WARN log4j.logger.org.apache.spark.MapOutputTrackerMasterEndpoint=WARN @@ -110,7 +117,20 @@ log4j.logger.org.datanucleus=ERROR log4j.logger.org.apache.spark.Task=WARN log4j.logger.org.apache.spark.sql.catalyst.parser.CatalystSqlParser=WARN +# Keep log-level of some classes as INFO even if root level is higher +log4j.logger.io.snappydata.impl.LeadImpl=INFO +log4j.logger.io.snappydata.impl.ServerImpl=INFO +log4j.logger.io.snappydata.impl.LocatorImpl=INFO +log4j.logger.spray.can.server.HttpListener=INFO + # for generated code of plans -# log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=DEBUG +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=INFO, code +log4j.additivity.org.apache.spark.sql.execution.WholeStageCodegenExec=false +log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenRDD=INFO, code +log4j.additivity.org.apache.spark.sql.execution.WholeStageCodegenRDD=false +# for all Spark generated code (including ad-hoc UnsafeProjection calls etc) +log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=WARN, code +log4j.additivity.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=false # for SnappyData generated code used on store (ComplexTypeSerializer, JDBC inserts ...) -# log4j.logger.org.apache.spark.sql.store.CodeGeneration=DEBUG +log4j.logger.org.apache.spark.sql.store.CodeGeneration=INFO, code +log4j.additivity.org.apache.spark.sql.store.CodeGeneration=false diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index ad5e4d5f0c..2e44e4a15d 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -73,10 +73,10 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.sql(s"update updateTable set id = id + ($numElements / 2) where id <> 73") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) session.sql(s"update updateTable set id = id + ($numElements / 2) where id <> 73") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) assert(session.table("updateTable").count() === numElements) assert(session.table("checkTable1").count() === numElements) @@ -106,7 +106,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { "case when (id % 2) = 0 then true else false end").write.insertInto("checkTable2") session.sql(s"update updateTable set addr = concat(addr, '_update') where id <> 32") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) assert(session.table("updateTable").count() === numElements) assert(session.table("checkTable2").count() === numElements) @@ -134,7 +134,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { "case when (id % 2) = 1 then true else false end").write.insertInto("checkTable3") session.sql(s"update updateTable set status = not status where id <> 87") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) assert(session.table("updateTable").count() === numElements) assert(session.table("checkTable3").count() === numElements) @@ -314,10 +314,10 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { "case when (id % 2) = 0 then true else false end").write.insertInto("checkTable2") session.sql(s"update updateTable set id = id + ($numElements / 2) where id <> 73") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) session.sql(s"update updateTable set id = id + ($numElements / 2) where id <> 73") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) assert(session.table("updateTable").count() === (numElements * 9) / 10) assert(session.table("updateTable").collect().length === (numElements * 9) / 10) @@ -542,7 +542,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { assert(exceptions.isEmpty, s"Failed with exceptions: $exceptions") - session.table("updateTable").show() + logInfo(session.table("updateTable").limit(20).collect().mkString("\n")) var res = session.sql( "select * from updateTable EXCEPT select * from checkTable1").collect() diff --git a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala index bc0b9c1c1d..b57b0fede1 100644 --- a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala +++ b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala @@ -67,7 +67,7 @@ abstract class SnappyFunSuite } } - protected def sc(addOn: (SparkConf) => SparkConf): SparkContext = { + protected def sc(addOn: SparkConf => SparkConf): SparkContext = { val ctx = SnappyContext.globalSparkContext if (ctx != null && !ctx.isStopped) { ctx @@ -78,7 +78,7 @@ abstract class SnappyFunSuite } } - protected def scWithConf(addOn: (SparkConf) => SparkConf): SparkContext = { + protected def scWithConf(addOn: SparkConf => SparkConf): SparkContext = { new SparkContext(newSparkConf(addOn)) } @@ -194,12 +194,6 @@ abstract class SnappyFunSuite fileName } - protected def logStdOut(msg: String): Unit = { - // scalastyle:off - println(msg) - // scalastyle:on - } - def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = SnappyFunSuite.checkAnswer(df, expectedAnswer) } diff --git a/core/src/test/scala/io/snappydata/SnappyTestRunner.scala b/core/src/test/scala/io/snappydata/SnappyTestRunner.scala index abe1194e33..82b716c377 100644 --- a/core/src/test/scala/io/snappydata/SnappyTestRunner.scala +++ b/core/src/test/scala/io/snappydata/SnappyTestRunner.scala @@ -49,6 +49,7 @@ with Logging with Retries { var snappyHome = "" var localHostName = "" var currWorkingDir = "" + private val commandOutput = "command-output.txt" // One can ovveride this method to pass other parameters like heap size. def servers: String = s"$localHostName\n$localHostName" @@ -88,10 +89,10 @@ with Logging with Retries { } def stopCluster(): Unit = { - executeProcess("snappyCluster", s"$snappyHome/sbin/snappy-stop-all.sh") + executeProcess("snappyCluster", s"$snappyHome/sbin/snappy-stop-all.sh", Some(commandOutput)) new File(s"$snappyHome/conf/servers").delete() new File(s"$snappyHome/conf/leads").delete() - executeProcess("sparkCluster", s"$snappyHome/sbin/stop-all.sh") + executeProcess("sparkCluster", s"$snappyHome/sbin/stop-all.sh", Some(commandOutput)) } def startupCluster(): Unit = { @@ -108,21 +109,29 @@ with Logging with Retries { } leadFile.deleteOnExit() - val (out, _) = executeProcess("snappyCluster", s"$snappyHome/sbin/snappy-start-all.sh") + val (out, _) = executeProcess("snappyCluster", s"$snappyHome/sbin/snappy-start-all.sh", + Some(commandOutput)) if (!out.contains(clusterSuccessString)) { throw new Exception(s"Failed to start Snappy cluster") } - executeProcess("sparkCluster", s"$snappyHome/sbin/start-all.sh") + executeProcess("sparkCluster", s"$snappyHome/sbin/start-all.sh", Some(commandOutput)) } // scalastyle:off println - def executeProcess(name: String , command: String): (String, String) = { + def executeProcess(name: String , command: String, + outFile: Option[String] = None): (String, String) = { val stdoutStream = new ByteArrayOutputStream val stderrStream = new ByteArrayOutputStream - val teeOut = new TeeOutputStream(stdout, new BufferedOutputStream(stdoutStream)) - val teeErr = new TeeOutputStream(stderr, new BufferedOutputStream(stderrStream)) + val (out, err) = outFile match { + case None => stdout -> stderr + case Some(f) => + val writer = new BufferedOutputStream(new FileOutputStream(f, true)) + writer -> writer + } + val teeOut = new TeeOutputStream(out, new BufferedOutputStream(stdoutStream)) + val teeErr = new TeeOutputStream(err, new BufferedOutputStream(stderrStream)) val stdoutWriter = new PrintStream(teeOut, true) val stderrWriter = new PrintStream(teeErr, true) @@ -136,6 +145,9 @@ with Logging with Retries { "PYTHONPATH" -> s"$snappyHome/python/lib/py4j-0.10.4-src.zip:$snappyHome/python") ! ProcessLogger(stdoutWriter.println, stderrWriter.println) var stdoutStr = stdoutStream.toString + if (out ne stdout) { + out.close() + } if (code != 0) { // add an exception to the output to force failure stdoutStr += s"\n***** Exit with Exception code = $code\n" @@ -145,12 +157,18 @@ with Logging with Retries { def SnappyShell(name: String, sqlCommand: Seq[String]): Unit = { - sqlCommand pipe snappyShell foreach (s => { - println(s) - if (s.toString.contains("ERROR") || s.toString.contains("Failed")) { - throw new Exception(s"Failed to run Query: $s") - } - }) + val writer = new PrintWriter(new BufferedWriter(new OutputStreamWriter( + new FileOutputStream(commandOutput, true)))) + try { + sqlCommand pipe snappyShell foreach (s => { + writer.println(s) + if (s.toString.contains("ERROR") || s.toString.contains("Failed")) { + throw new Exception(s"Failed to run Query: $s") + } + }) + } finally { + writer.close() + } } def Job(jobClass: String, lead: String, jarPath: String, @@ -165,7 +183,7 @@ with Logging with Retries { val jobCommand: String = s"$jobSubmit --app-name " + s"${jobClass}_${System.currentTimeMillis()} --class $jobClass $confStr" - val (out, err) = executeProcess("snappyCluster", jobCommand) + val (out, err) = executeProcess("snappyCluster", jobCommand, Some(commandOutput)) val jobSubmitStr = out @@ -178,14 +196,13 @@ with Logging with Retries { map.asInstanceOf[Map[String, Map[String, Any]]]("result")("jobId") case other => throw new Exception(s"bad result : $jsonStr") } - println("jobID " + jobID) - + logInfo("jobID " + jobID) var status = "RUNNING" while (status == "RUNNING") { Thread.sleep(3000) val statusCommand = s"$jobStatus $jobID" - val (out, err) = executeProcess("snappyCluster", statusCommand) + val (out, err) = executeProcess("snappyCluster", statusCommand, Some(commandOutput)) val jobSubmitStatus = out @@ -193,7 +210,7 @@ with Logging with Retries { statusjson match { case Some(map: Map[_, _]) => val v = map.asInstanceOf[Map[String, Any]]("status") - println("Current status of job: " + v) + logInfo("Current status of job: " + v) status = v.toString case other => "bad Result" } @@ -238,7 +255,7 @@ with Logging with Retries { val confStr = if (confs.size > 0) confs.foldLeft("")((r, c) => s"$r --conf $c") else "" val classStr = if (appClass.isEmpty) "" else s"--class $appClass" val sparkSubmit = s"$snappyHome/bin/spark-submit $classStr --master $masterStr $confStr $appJar" - val (out, err) = executeProcess(name, sparkSubmit) + val (out, err) = executeProcess(name, sparkSubmit, Some(commandOutput)) if (checkException(out) || checkException(err)) { throw new Exception(s"Failed to submit $appClass") @@ -249,7 +266,7 @@ with Logging with Retries { args: Seq[String] = Nil): Unit = { val argsStr = args.mkString(" ") val runExample = s"$snappyHome/bin/run-example $exampleClas $argsStr" - val (out, err) = executeProcess(name, runExample) + val (out, err) = executeProcess(name, runExample, Some(commandOutput)) if (checkException(out) || checkException(err)) { throw new Exception(s"Failed to run $exampleClas") @@ -259,7 +276,7 @@ with Logging with Retries { def SparkShell(confs: Seq[String], options: String, scriptFile : String): Unit = { val confStr = if (confs.size > 0) confs.foldLeft("")((r, c) => s"$r --conf $c") else "" val shell = s"$sparkShell $confStr $options -i $scriptFile" - val (out, err) = executeProcess("snappyCluster", shell) + val (out, err) = executeProcess("snappyCluster", shell, Some(commandOutput)) if (checkException(out) || checkException(err)) { throw new Exception(s"Failed to run $shell") } @@ -267,13 +284,19 @@ with Logging with Retries { def SparkShell(confs: Seq[String], options: String, scalaStatements: Seq[String]): Unit = { - val confStr = if (confs.size > 0) confs.foldLeft("")((r, c) => s"$r --conf $c") else "" - scalaStatements pipe s"$snappyShell $confStr $options" foreach (s => { - println(s) - if (s.toString.contains("ERROR") || s.toString.contains("Failed")) { - throw new Exception(s"Failed to run Scala statement") - } - }) + val writer = new PrintWriter(new BufferedWriter(new OutputStreamWriter( + new FileOutputStream(commandOutput, true)))) + try { + val confStr = if (confs.size > 0) confs.foldLeft("")((r, c) => s"$r --conf $c") else "" + scalaStatements pipe s"$sparkShell $confStr $options" foreach (s => { + writer.println(s) + if (s.toString.contains("ERROR") || s.toString.contains("Failed")) { + throw new Exception(s"Failed to run Scala statement") + } + }) + } finally { + writer.close() + } } /* diff --git a/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala b/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala index 2d51475c3f..5c1e2f3bd0 100644 --- a/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala +++ b/core/src/test/scala/io/snappydata/app/SampleTableQuery.scala @@ -41,7 +41,7 @@ object SampleTableQuery extends Serializable { conf.set("spark.sql.hive.metastore.sharedPrefixes","com.mysql.jdbc,org.postgresql,com.microsoft.sqlserver,oracle.jdbc,com.mapr.fs.shim.LibraryLoader,com.mapr.security.JNISecurity,com.mapr.fs.jni,org.apache.commons") conf.set("spark.sql.unsafe.enabled", "false") val sc = new SparkContext(conf) - sc.addJar("/Users/ashahid/workspace/snappy/snappy-commons/snappy-core/build-artifacts/scala-2.10/classes/test/app.jar") + sc.addJar("/Users/ashahid/workspace/snappy/snappy-commons/snappy-core/build-artifacts/scala-2.10/classes/scala/test/app.jar") val spc = SnappyContext(sc) // val hiveContext = new HiveContext(spc) 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 3e47acb199..73687362c3 100644 --- a/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala +++ b/core/src/test/scala/org/apache/spark/jdbc/ConnectionConfTest.scala @@ -206,7 +206,7 @@ class ConnectionConfTest extends SnappyFunSuite with Logging with BeforeAndAfter }) val result = snc.sql("SELECT col1 from TEST_JDBC_TABLE_1") - result.show() + result.collect() // result.collect().foreach(v => assert(v(0) == 9)) snc.sql("drop table TEST_JDBC_TABLE_1") 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 86c55b43d8..18eb1fa629 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 @@ -16,15 +16,14 @@ */ package org.apache.spark.sql.store - import scala.collection.mutable import io.snappydata.SnappyFunSuite import io.snappydata.core.{Data, TestData} import org.scalatest.{Assertions, BeforeAndAfter} +import org.apache.spark.sql.{Dataset, Row, SaveMode} import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.sql.{Dataset, Row, SaveMode, SnappySession} class ColumnTableBatchInsertTest extends SnappyFunSuite with Logging @@ -71,8 +70,8 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite "PARTITION_BY 'Key1'," + "BUCKETS '1')") - //val r = result.collect - //assert(r.length == 0) + // val r = result.collect + // assert(r.length == 0) var rdd = sc.parallelize( (1 to 10).map(i => TestData(i, i.toString))) @@ -99,12 +98,12 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite val r2 = result.collect assert(r2.length == 20) - println("Successful") + logInfo("Successful") } test("test the shadow table creation without partition by clause") { - //snc.sql(s"DROP TABLE IF EXISTS $tableName") + // snc.sql(s"DROP TABLE IF EXISTS $tableName") val df = snc.sql(s"CREATE TABLE $tableName(Key1 INT ,Value STRING) " + "USING column " + @@ -124,11 +123,11 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite dataDF.write.insertInto(tableName) val r2 = result.collect assert(r2.length == 19999) - println("Successful") + logInfo("Successful") } test("test the shadow table with persistence") { - //snc.sql(s"DROP TABLE IF EXISTS $tableName") + // snc.sql(s"DROP TABLE IF EXISTS $tableName") val df = snc.sql(s"CREATE TABLE $tableName(Key1 INT ,Value STRING)" + "USING column " + @@ -150,11 +149,11 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite val r2 = result.collect assert(r2.length == 19999) - println("Successful") + logInfo("Successful") } test("test the shadow table with eviction") { - //snc.sql(s"DROP TABLE IF EXISTS $tableName") + // snc.sql(s"DROP TABLE IF EXISTS $tableName") val df = snc.sql(s"CREATE TABLE $tableName(Key1 INT ,Value STRING)" + "USING column " + @@ -174,7 +173,7 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite dataDF.write.insertInto(tableName) val r2 = result.collect assert(r2.length == 19999) - println("Successful") + logInfo("Successful") } test("test the shadow table with options on compressed table") { @@ -200,18 +199,17 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite val r2 = result.collect val r3 = mutable.HashSet[Int]() - r2.map( i => { + r2.map(i => { r3.add(i.getInt(0)) }) (1 to 19999).map(i => { - if(!r3.contains(i)) - println (s"Does not contain ${i}") + if (!r3.contains(i)) logInfo(s"Does not contain $i") }) assert(r2.length == 19999) - println("Successful") + logInfo("Successful") } test("test the shadow table with eviction options on compressed table") { @@ -236,7 +234,7 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite dataDF.write.insertInto(tableName) val r2 = result.collect assert(r2.length == 19999) - println("Successful") + logInfo("Successful") } test("test create table as select with alias") { @@ -255,38 +253,38 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite val tempRowTableName = "testRowTable" val tempColTableName = "testcolTable" - snc.sql("DROP TABLE IF EXISTS "+tempRowTableName) - snc.sql("CREATE TABLE " + tempRowTableName + " AS (SELECT col1 as field1,col2 as field2 FROM " + rowTable + ")" - ) + snc.sql("DROP TABLE IF EXISTS " + tempRowTableName) + snc.sql("CREATE TABLE " + tempRowTableName + + " AS (SELECT col1 as field1,col2 as field2 FROM " + rowTable + ")") var testResults1 = snc.sql("SELECT * FROM " + tempRowTableName).collect assert(testResults1.length == 5) - snc.sql("DROP TABLE IF EXISTS "+tempRowTableName) + snc.sql("DROP TABLE IF EXISTS " + tempRowTableName) - snc.sql("DROP TABLE IF EXISTS "+tempRowTableName) - snc.sql("CREATE TABLE " + tempRowTableName + " AS (SELECT col1 as field1,col2 as field2 FROM " + colTable + ")" - ) + snc.sql("DROP TABLE IF EXISTS " + tempRowTableName) + snc.sql("CREATE TABLE " + tempRowTableName + + " AS (SELECT col1 as field1,col2 as field2 FROM " + colTable + ")") var testResults2 = snc.sql("SELECT * FROM " + tempRowTableName).collect assert(testResults2.length == 5) - snc.sql("DROP TABLE IF EXISTS "+tempRowTableName) + snc.sql("DROP TABLE IF EXISTS " + tempRowTableName) - snc.sql("DROP TABLE IF EXISTS "+tempColTableName) - snc.sql("CREATE TABLE " + tempColTableName + " USING COLUMN OPTIONS() AS (SELECT col1 as field1,col2 as field2 FROM " + rowTable + ")" - ) + snc.sql("DROP TABLE IF EXISTS " + tempColTableName) + snc.sql("CREATE TABLE " + tempColTableName + + " USING COLUMN OPTIONS() AS (SELECT col1 as field1,col2 as field2 FROM " + rowTable + ")") var testResults3 = snc.sql("SELECT * FROM " + tempColTableName).collect assert(testResults3.length == 5) - snc.sql("DROP TABLE IF EXISTS "+tempColTableName) + snc.sql("DROP TABLE IF EXISTS " + tempColTableName) - snc.sql("DROP TABLE IF EXISTS "+tempColTableName) - snc.sql("CREATE TABLE " + tempColTableName + " USING COLUMN OPTIONS() AS (SELECT col1 as field1,col2 as field2 FROM " + colTable + ")" - ) + snc.sql("DROP TABLE IF EXISTS " + tempColTableName) + snc.sql("CREATE TABLE " + tempColTableName + + " USING COLUMN OPTIONS() AS (SELECT col1 as field1,col2 as field2 FROM " + colTable + ")") var testResults4 = snc.sql("SELECT * FROM " + tempColTableName).collect assert(testResults4.length == 5) - snc.sql("DROP TABLE IF EXISTS "+tempColTableName) + snc.sql("DROP TABLE IF EXISTS " + tempColTableName) - snc.sql("DROP TABLE IF EXISTS "+rowTable) - snc.sql("DROP TABLE IF EXISTS "+colTable) + snc.sql("DROP TABLE IF EXISTS " + rowTable) + snc.sql("DROP TABLE IF EXISTS " + colTable) } @@ -294,32 +292,28 @@ class ColumnTableBatchInsertTest extends SnappyFunSuite test("test table with column name having slash") { snc.sql(s"DROP TABLE IF EXISTS $tableName") val df = snc.sql("CREATE TABLE ColumnTable(\"a/b\" INT ,Col2 INT, Col3 INT) " + - "USING column " + - "options " + - "(" + - "PARTITION_BY 'col2'," + - "BUCKETS '1')") + "USING column " + + "options " + + "(" + + "PARTITION_BY 'col2'," + + "BUCKETS '1')") snc.sql("CREATE TABLE rowTable(\"a/b\" INT ,Col2 INT, Col3 INT) " + - "USING row " + - "options " + - "()") + "USING row " + + "options " + + "()") snc.sql("insert into ColumnTable(\"a/b\",col2,col3) values(1,2,3)") snc.sql("insert into rowTable(\"a/b\",col2,col3)values(1,2,3)") val result = snc.sql("SELECT col2+1 FROM " + tableName) - val r = result.collect - result.show() + val r = result.collect() assert(r.length == 1) - val result1 = snc.sql("SELECT \"a/b\"/1 FROM " + tableName) - val r1 = result1.collect - result1.show() - snc.sql("SELECT \"a/b\"/1 FROM rowTable").show + val r1 = result1.collect() + snc.sql("SELECT \"a/b\"/1 FROM rowTable").collect() assert(r1.length == 1) - snc.sql("drop table if exists columntable") snc.sql("drop table if exists rowtable") logInfo("Successful") 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 5abfc7c2f3..b7b8576ef8 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 @@ -489,7 +489,7 @@ class ColumnTableTest val rdd = sc.parallelize(data, data.length).map(s => new Data(s(0), s(1), s(2))) val dataDF = snc.createDataFrame(rdd) snc.registerDataFrameAsTable(dataDF, "tempTable") - snc.sql("select * from tempTable").show + snc.sql("select * from tempTable").collect() intercept[AnalysisException] { // not supported snc.sql("alter table tempTable add column age int") @@ -1235,7 +1235,7 @@ class ColumnTableTest snc.sql("create table t1(a int,b int) using column options()") snc.sql("insert into t1 values(1,2)") - snc.sql("select * from t1").show + snc.sql("select * from t1").collect() snc.sql("create table t2(c int,d int) using column options() as (select * from t1)") snc.sql("create table t3 using column options() as (select * from t1)") @@ -1262,21 +1262,19 @@ class ColumnTableTest }', header 'false', inferschema 'true')") snc.sql("create table test2 using column options() as (select * from test1)") val df2 = snc.sql("select * from test2") - df2.show() + df2.collect() snc.sql("drop table test2") snc.sql("create table test2(_col1 integer,__col2 integer) using column options()") snc.sql("insert into test2 values(1,2)") snc.sql("insert into test2 values(2,3)") val df3 = snc.sql("select _col1,__col2 from test2") - df3.show() + df3.collect() val struct = (new StructType()) .add(StructField("_COL1", IntegerType, true)) .add(StructField("__COL2", IntegerType, true)) - df3.printSchema() assert(struct == df3.schema) - } test("Test loading json data to column table") { @@ -1305,7 +1303,7 @@ class ColumnTableTest "address.district, " + "address.lane " + "FROM people") - nameAndAddress.toJSON.show(truncate = false) + logInfo(nameAndAddress.toJSON.collect().mkString("\n")) assert(nameAndAddress.count() == 2) val rows: Array[String] = nameAndAddress.toJSON.collect() @@ -1368,7 +1366,7 @@ class ColumnTableTest snc.sql(s"insert into t1 values(2,'test2')") snc.sql(s"insert into t1 values(3,'test3')") val df = snc.sql("select * from t1") - df.show + df.collect() val tempPath = "/tmp/" + System.currentTimeMillis() assert(df.count() == 3) @@ -1377,7 +1375,7 @@ class ColumnTableTest Map("path" -> tempPath, "header" -> "true", "inferSchema" -> "true")) val dataDF = snc.sql("select * from TEST_EXTERNAL order by c1") - snc.sql("select * from TEST_EXTERNAL").show + snc.sql("select * from TEST_EXTERNAL").collect() assert(dataDF.count == 3) @@ -1497,7 +1495,7 @@ class ColumnTableTest snc.sql(s"insert into t1 values(2,'test2')") snc.sql(s"insert into t1 values(3,'test3')") val df = snc.sql("select * from t1") - df.show + df.collect() val tempPath = System.getProperty("user.dir") + System.currentTimeMillis() assert(df.count() == 3) 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 5b1b08e1ea..fb649da488 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 @@ -771,7 +771,6 @@ class CreateIndexTest extends SnappyFunSuite with BeforeAndAfterEach { val result = snContext.sql("select COL1 from " + tableName + " where COL2 like '%a%'") - result.explain(true) doPrint("") doPrint("=============== RESULTS START ===============") result.collect.foreach(doPrint) @@ -808,17 +807,15 @@ object CreateIndexTest extends SnappyFunSuite { val selectRes = snContext.sql(sqlText) if (withExplain || explainQ) { - selectRes.explain(true) + // selectRes.explain(true) } validate(selectRes) if (showResults) { - selectRes.show + logInfo(selectRes.collect().take(20).mkString("\n")) } else { - // scalastyle:off println - selectRes.collect().take(10).foreach(println) - // scalastyle:on println + logInfo(selectRes.collect().take(10).mkString("\n")) } selectRes 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 2aa255799a..a6939284f9 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 @@ -20,8 +20,8 @@ import io.snappydata.SnappyFunSuite import io.snappydata.core.Data import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.{SaveMode, Row, DataFrame} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{DataFrame, SaveMode} class CubeRollupGroupingSetsTest extends SnappyFunSuite with BeforeAndAfterAll { private var testData: DataFrame = _ @@ -29,7 +29,7 @@ class CubeRollupGroupingSetsTest extends SnappyFunSuite with BeforeAndAfterAll { override def beforeAll() { val data = Seq(Seq(1, 2005, 12000), Seq(1, 2006, 18000), Seq(1, 2007, 25000), Seq(2, 2005, 15000), Seq(2, 2006, 6000), Seq(2, 2007, 25000)) - val testRDD = sc.parallelize(data, data.length).map(s => new Data(s(0), s(1), s(2))) + val testRDD = sc.parallelize(data, data.length).map(s => Data(s.head, s(1), s(2))) testData = snc.createDataFrame(testRDD) snc.sql("Create table mytable (col1 INT, col2 INT, col3 INT)") testData.write.format("row").mode(SaveMode.Append).saveAsTable("mytable") @@ -40,47 +40,51 @@ class CubeRollupGroupingSetsTest extends SnappyFunSuite with BeforeAndAfterAll { } test("snappy cube_rollup query") { - //group by - val dfGroupByResult = testData.groupBy("col1", "col2").agg(sum("col2")).orderBy("col1", "col2").collect() + // group by + val dfGroupByResult = testData.groupBy("col1", "col2") + .agg(sum("col2")).orderBy("col1", "col2").collect() val snappyGroupByResult = snc.sql("select col1, col2, sum(col2) from mytable " + "group by col1, col2 order by col1, col2").collect() - println("DataFrame group by result") - dfGroupByResult.foreach(println) - println("SnappySQL group by result") - snappyGroupByResult.foreach(println) + logInfo("DataFrame group by result") + logInfo(dfGroupByResult.mkString("\n")) + logInfo("SnappySQL group by result") + logInfo(snappyGroupByResult.mkString("\n")) assert(dfGroupByResult.sameElements(snappyGroupByResult)) - //roll up - val dfRollupResult = testData.rollup("col1", "col2").agg(sum("col3")).orderBy("col1", "col2").collect() - val snappyRollupResult = snc.sql("select col1, col2, sum(col3) from mytable group by col1, col2 " + - "with rollup order by col1, col2").collect() + // roll up + val dfRollupResult = testData.rollup("col1", "col2") + .agg(sum("col3")).orderBy("col1", "col2").collect() + val snappyRollupResult = snc.sql( + "select col1, col2, sum(col3) from mytable group by col1, col2 " + + "with rollup order by col1, col2").collect() - println("DataFrame rollup result") - dfRollupResult.foreach(println) - println("SnappySQL rollup result") - snappyRollupResult.foreach(println) + logInfo("DataFrame rollup result") + logInfo(dfRollupResult.mkString("\n")) + logInfo("SnappySQL rollup result") + logInfo(snappyRollupResult.mkString("\n")) assert(dfRollupResult.sameElements(snappyRollupResult)) // cube - val dfCubeResult = testData.cube("col1", "col2").agg(sum("col3")).orderBy("col1", "col2").collect() - val snappyCubeResult = snc.sql("select col1, col2, sum(col3) from mytable group by col1, col2 " + - "with cube order by col1, col2").collect() + val dfCubeResult = testData.cube("col1", "col2") + .agg(sum("col3")).orderBy("col1", "col2").collect() + val snappyCubeResult = snc.sql( + "select col1, col2, sum(col3) from mytable group by col1, col2 " + + "with cube order by col1, col2").collect() - println("DataFrame cube result") - dfCubeResult.foreach(println) - println("SnappySQL cube result") - snappyCubeResult.foreach(println) + logInfo("DataFrame cube result") + logInfo(dfCubeResult.mkString("\n")) + logInfo("SnappySQL cube result") + logInfo(snappyCubeResult.mkString("\n")) assert(dfCubeResult.sameElements(snappyCubeResult)) // grouping sets query equivalent to above cube query val snappyGoupingSetResult = snc.sql("select col1, col2, sum(col3) from mytable group by col1, col2 " + "grouping sets ((col1, col2), (col1), (col2), ()) order by col1, col2").collect() - println("DataFrame cube result") - dfCubeResult.foreach(println) - println("SnappySQL gouping sets result") - snappyGoupingSetResult.foreach(println) + logInfo("DataFrame cube result") + logInfo(dfCubeResult.mkString("\n")) + logInfo("SnappySQL gouping sets result") + logInfo(snappyGoupingSetResult.mkString("\n")) assert(dfCubeResult.sameElements(snappyGoupingSetResult)) - } } 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 364895a9da..b3c43197dc 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 @@ -54,12 +54,12 @@ class RowTableTest dataDF.write.format("row").saveAsTable("MY_SCHEMA.MY_TABLE") var result = snc.sql("SELECT * FROM MY_SCHEMA.MY_TABLE" ) var r = result.collect - println(r.length) + logInfo(r.length.toString) snc.sql("drop table MY_SCHEMA.MY_TABLE" ) snc.sql("drop schema MY_SCHEMA") - println("Successful") + logInfo("Successful") } @@ -72,7 +72,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 0) - println("Successful") + logInfo("Successful") } test("Test the fetch first n row only test. with and without n parameter") { @@ -88,7 +88,7 @@ class RowTableTest result = snc.sql("SELECT * FROM " + tableName + " fetch first row only") r = result.collect assert(r.length == 0) - println("Successful") + logInfo("Successful") } test("Test the creation of table using DataSource API") { @@ -102,7 +102,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 5) - println("Successful") + logInfo("Successful") } test("Test the creation of table using DataSource API(PUT)") { @@ -115,14 +115,14 @@ class RowTableTest } dataDF.write.format("row").options(props).saveAsTable(tableName) - //Again do putInto, as there is no primary key, all will be appended + // Again do putInto, as there is no primary key, all will be appended dataDF.write.format("row").mode(SaveMode.Overwrite).options(props).putInto(tableName) val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect // no primary key assert(r.length == 10) - println("Successful") + logInfo("Successful") } @@ -174,7 +174,7 @@ class RowTableTest r = result.collect assert(r.length == 6) - println("Successful") + logInfo("Successful") } val options = "OPTIONS (PARTITION_BY 'Col1')" @@ -188,7 +188,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 0) - println("Successful") + logInfo("Successful") } test("Test the creation/dropping of table using SQ with explicit URL") { @@ -199,7 +199,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 0) - println("Successful") + logInfo("Successful") } test("Test the creation using SQL and insert a DF in append/overwrite/errorifexists mode") { @@ -220,7 +220,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 5) - println("Successful") + logInfo("Successful") } test("Test the creation using SQL and put a DF in append/overwrite/errorifexists mode") { @@ -237,7 +237,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 5) - println("Successful") + logInfo("Successful") } test("Test the creation using SQL and put a seq of rows in append/overwrite/errorifexists mode") { @@ -252,7 +252,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 5) - println("Successful") + logInfo("Successful") } // should throw exception if primary key is getting updated? @@ -270,14 +270,14 @@ class RowTableTest val r = result.collect assert(r.length == 5) - //check if the row against primary key 1 is 1, 200, 300 + // check if the row against primary key 1 is 1, 200, 300 - val row1 = snc.sql(s"SELECT * FROM $tableName WHERE Col1='1'") - assert(row1.collect.length == 1) + val row1 = snc.sql(s"SELECT * FROM $tableName WHERE Col1='1'").collect() + assert(row1.length == 1) - println(row1.show) + logInfo(row1.mkString("\n")) - println("Successful") + logInfo("Successful") } test("Test Creation using SQL with Primary Key and PUT INTO SELECT AS ") { @@ -309,15 +309,15 @@ class RowTableTest val r = result.collect assert(r.length == 5) - //check if the row against primary key 1 is 1, 200, 300 + // check if the row against primary key 1 is 1, 200, 300 - val row1 = snc.sql(s"SELECT * FROM $tableName WHERE Col1='1'") - assert(row1.collect.length == 1) + val row1 = snc.sql(s"SELECT * FROM $tableName WHERE Col1='1'").collect() + assert(row1.length == 1) - println(row1.show) + logInfo(row1.mkString("\n")) snc.dropTable("tempTable") - println("Successful") + logInfo("Successful") } test("PUT INTO TABLE USING SQL"){ @@ -368,7 +368,7 @@ class RowTableTest val result = snc.sql("SELECT * FROM " + tableName) val r = result.collect assert(r.length == 5) - println("Successful") + logInfo("Successful") } test("Test the creation of table using CREATE TABLE AS STATEMENT ") { @@ -393,7 +393,7 @@ class RowTableTest assert(r.length == 10) snc.dropTable(tableName2) - println("Successful") + logInfo("Successful") } test("Test alter table SQL syntax") { @@ -479,7 +479,7 @@ class RowTableTest r = result.collect assert(r.length == 0) - println("Successful") + logInfo("Successful") } test("Test the drop syntax SnappyContext and SQL ") { @@ -501,7 +501,7 @@ class RowTableTest snc.sql("DROP TABLE IF EXISTS " + tableName) - println("Successful") + logInfo("Successful") } test("Test the drop syntax SQL and SnappyContext ") { @@ -523,7 +523,7 @@ class RowTableTest snc.dropTable(tableName, true) - println("Successful") + logInfo("Successful") } test("Test the update table ") { @@ -556,7 +556,7 @@ class RowTableTest snc.dropTable("RowTableUpdate") snc.dropTable("RowTableUpdate2") - println("Successful") + logInfo("Successful") } @@ -684,7 +684,7 @@ class RowTableTest assert(r.length == 10) snc.dropTable(tableName2) - println("Successful") + logInfo("Successful") } test("Test create table from CSV without header- SNAP-1442") { 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 d21c840aa1..1e26c5a5dc 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 @@ -278,7 +278,6 @@ class TokenizationTest res = snc.sql(s"select quartile, avg(c) as avgC, max(c) as maxC" + s" from (select c, ntile(4) over (order by c) as quartile from $table ) x " + s"group by quartile order by quartile").collect() - // res.foreach(println) // Unix timestamp val df = snc.sql(s"select * from $table where UNIX_TIMESTAMP('2015-01-01 12:00:00') > a") @@ -480,20 +479,18 @@ class TokenizationTest s"select * from $table where a = $x" } val start = System.currentTimeMillis() - // scalastyle:off println q.zipWithIndex.foreach { case (x, i) => var result = snc.sql(x).collect() assert(result.length === 1) result.foreach( r => { - println(s"${r.get(0)}, ${r.get(1)}, ${r.get(2)}, ${i}") + logInfo(s"${r.get(0)}, ${r.get(1)}, ${r.get(2)}, $i") assert(r.get(0) == r.get(1) && r.get(2) == i) }) } val end = System.currentTimeMillis() // snc.sql(s"select * from $table where a = 1200").collect() - println("Time taken = " + (end - start)) - // scalastyle:on println + logInfo("Time taken = " + (end - start)) val cacheMap = SnappySession.getPlanCache.asMap() assert( cacheMap.size() == 1) @@ -672,9 +669,8 @@ class TokenizationTest var query = s"select * from $table t1, $table2 t2 where t1.a = t2.a and t1.b = 5 limit 2" // snc.sql("set spark.sql.autoBroadcastJoinThreshold=-1") val result1 = snc.sql(query).collect() - // scalastyle:off println result1.foreach( r => { - println(r.get(0) + ", " + r.get(1) + r.get(2) + ", " + r.get(3) + r.get(4) + + logInfo(r.get(0) + ", " + r.get(1) + r.get(2) + ", " + r.get(3) + r.get(4) + ", " + r.get(5)) }) val cacheMap = SnappySession.getPlanCache.asMap() @@ -684,10 +680,9 @@ class TokenizationTest query = s"select * from $table t1, $table2 t2 where t1.a = t2.a and t1.b = 7 limit 2" val result2 = snc.sql(query).collect() result2.foreach( r => { - println(r.get(0) + ", " + r.get(1) + r.get(2) + ", " + r.get(3) + r.get(4) + + logInfo(r.get(0) + ", " + r.get(1) + r.get(2) + ", " + r.get(3) + r.get(4) + ", " + r.get(5)) }) - // scalastyle:on println assert( cacheMap.size() == 1) assert(!result1.sameElements(result2)) assert(result1.length > 0) @@ -854,7 +849,6 @@ class TokenizationTest val rows1 = rs1.collect() assert(rows0.sameElements(rows1)) - // rows1.foreach(println) val cacheMap = SnappySession.getPlanCache.asMap() assert(cacheMap.size() == 0) diff --git a/docs/security/authentication_connecting_to_a_secure_cluster.md b/docs/security/authentication_connecting_to_a_secure_cluster.md index 8fc11e8a1c..e9c687097c 100644 --- a/docs/security/authentication_connecting_to_a_secure_cluster.md +++ b/docs/security/authentication_connecting_to_a_secure_cluster.md @@ -48,7 +48,7 @@ val conf = new SparkConf() .setMaster(s"spark://$hostName:7077") .set("spark.executor.cores", TestUtils.defaultCores.toString) .set("spark.executor.extraClassPath", - getEnvironmentVariable("SNAPPY_DIST_CLASSPATH")) + getEnvironmentVariable("SNAPPY_HOME") + "/jars/*" ) .set("snappydata.connection", snappydataLocatorURL) .set("spark.snappydata.store.user", username) .set("spark.snappydata.store.password", password) diff --git a/dtests/build.gradle b/dtests/build.gradle index f99cc4fd27..a4ca06e625 100644 --- a/dtests/build.gradle +++ b/dtests/build.gradle @@ -153,7 +153,7 @@ testClasses.doLast { include '**/*.inc' include '**/*.sql' } - into project(':snappy-store:snappydata-store-tests').sourceSets.main.output.classesDir + into project(':snappy-store:snappydata-store-tests').sourceSets.main.java.outputDir } } } diff --git a/dtests/src/resources/regression_docs/HowToRunRowStoreTests.md b/dtests/src/resources/regression_docs/HowToRunRowStoreTests.md index 3306af654e..6cba8901ac 100755 --- a/dtests/src/resources/regression_docs/HowToRunRowStoreTests.md +++ b/dtests/src/resources/regression_docs/HowToRunRowStoreTests.md @@ -16,12 +16,12 @@ ``` export SNAPPYDATA_SOURCE_DIR= -export JTESTS=$SNAPPYDATA_SOURCE_DIR/store/tests/sql/build-artifacts/linux/classes/main +export JTESTS=$SNAPPYDATA_SOURCE_DIR/store/tests/sql/build-artifacts/linux/classes/java/main $SNAPPYDATA_SOURCE_DIR/sample-runbt.sh $SNAPPYDATA_SOURCE_DIR [-l -r -m ] E.g. For running sql.bt export SNAPPYDATA_SOURCE_DIR= -export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/main +export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/java/main $SNAPPYDATA_SOURCE_DIR/store/tests/core/src/main/java/bin/sample-runbt.sh $SNAPPYDATA_SOURCE_DIR -l $JTESTS/sql/snappy.local.conf sql/sql.bt ``` diff --git a/dtests/src/test/java/io/snappydata/hydra/rowStoreRegressionScript.sh b/dtests/src/test/java/io/snappydata/hydra/rowStoreRegressionScript.sh index c9bdae9643..2594996e33 100644 --- a/dtests/src/test/java/io/snappydata/hydra/rowStoreRegressionScript.sh +++ b/dtests/src/test/java/io/snappydata/hydra/rowStoreRegressionScript.sh @@ -1,4 +1,4 @@ -export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/main +export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/java/main $SNAPPY_HOME/store/tests/core/src/main/java/bin/sample-runbt.sh $OUTPUT_DIR/masterLogs $SNAPPY_HOME -l $JTESTS/sql/snappy.local.conf sql/sqlTx/thinClient/thinClientTx.bt sleep 30; diff --git a/dtests/src/test/java/io/snappydata/hydra/snappyRegressionScript.sh b/dtests/src/test/java/io/snappydata/hydra/snappyRegressionScript.sh index 6d372eae87..86978e38da 100644 --- a/dtests/src/test/java/io/snappydata/hydra/snappyRegressionScript.sh +++ b/dtests/src/test/java/io/snappydata/hydra/snappyRegressionScript.sh @@ -1,4 +1,4 @@ -export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/main +export JTESTS=$SNAPPY_HOME/store/tests/sql/build-artifacts/linux/classes/java/main $SNAPPY_HOME/store/tests/core/src/main/java/bin/sample-runbt.sh $OUTPUT_DIR/snappyHydraLogs $SNAPPY_HOME -l $JTESTS/io/snappydata/hydra/local.smartConnectorMode.conf -d false io/snappydata/hydra/northwind/northWind.bt sleep 30; diff --git a/dunit/build.gradle b/dunit/build.gradle index b4ebc7e445..02dbdae414 100644 --- a/dunit/build.gradle +++ b/dunit/build.gradle @@ -23,7 +23,7 @@ version = '1.0.3.6' compileJava.options.encoding = 'UTF-8' dependencies { - compile 'commons-io:commons-io:2.5' + compile "commons-io:commons-io:${commonsIoVersion}" compile "junit:junit:${junitVersion}" if (new File(rootDir, 'store/build.gradle').exists()) { compile project(':snappy-store:snappydata-store-core') diff --git a/gradle.properties b/gradle.properties index 19ccf6b25a..905f38f193 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,5 @@ org.gradle.daemon=false +org.gradle.warning.mode=none #org.gradle.parallel=true # Set this on the command line with -P or in ~/.gradle/gradle.properties @@ -8,5 +9,3 @@ buildRoot= # Empty credentials for maven publish on Sonatype ossrhUsername= ossrhPassword= -archivaUsername= -archivaPassword= diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index f808147c25e097ea8fc879b4336725042a14bfe3..457aad0d98108420a977756b7145c93c8910b076 100644 GIT binary patch delta 49886 zcmZ6wV{9f~xW(J))V6KAeLJ;n+qT`lQ``2`wr$(CZR`GX?oDpaN%n{RaVPu9de-`_ zmWtn94Zq_6&^!%ybG#rRAS&?!ijWxrjrUH!K;m%Xo84f_*UVObgM)yaLnr1DVkgS~ z!UJlnqpG8Q$E2GW;DH+p1V#h`P#D6Smepz-rMifI(f_g$*+3X0j*>H*9~+du!Sw!u z?0zni*RAk{ZmeEkTKSIrj^wj^NgoqH387oPUkA-^Gx$sant%O(+B z-4II`yc5=|lTMnrHZu;9`*|$CR(ST@aC-agKEFtt994zM34PEVJhb)LuT#7yft5;S zEk9UWLflM=k~j1Q=dOMQDy$zEFaT7S$#Gx!#yYr3!4yt{p()GDg8ghefEu{&k>B5okL|9SMm1KevPxC)DnKW`J6c-mpa@iTM0}w( zJx@>4&ri}ENoc?9iquiz=@lN=zLg2nE>*fC<}*6oM$^~s7H+I~#mZF}0)0#@*4+yb zN6+22h_lbeIQLx8z-aR=-goxN+n4qcwfHI&3oH~<7H$bc8UKlosJVk*@@Bo%S#pX- z*`Y!j7?SL;!eE!hN({2)7Gnio=h&(Km5Otu|pE7l*Bj)Zcl@L<2Cr%d6AZ^3#c@a#EbY zuTW^603Wsc_eu|e7+CMYUt+_LdwN-<@DxrYV( zrTY41ucH=ejsw5kZ23D3mPkO(Vigt%J&W1zLdwJ_#OO-HkxSGdr+LCSr@T3wes+=i zi#Y334dIitMg($(rCL@7(Ip(W=-`9BQ@i==@!3{a{fK zk~iI?pD<#egxq}sx4)d$2i5DZen+H%$2tCJKb?L_#yDDL20xjwyKg_7Jm#r`2+NMF6EMbe(&UIJyJ>jRc9EA39Ps| z`ht}Y#Ie(aGvEs>)3NCklU)31rn32$`8bWx%o*kB6ukILl_3!RE)3r;&fx3O{t^-`82P+U$aK8FT_0D6dql z{QNguABKxXVL=`)Dwu|atUVIz)m$HsLI<)}NLmyCi-h}@@obL$nd{&as383t3=ISX z3hF=Y6a=wQ{PxZN?>3)k55W#>qH$vkfL(#;3gF)r((wpb4OvA=NKo+xU=s8Shg0t_ zV;{pKcE9M|jG*@72ty1r5s{gDJEw1AYM~5kZ(!MqcGC^%B2jQBT3)~w#MF1MXAu)m zRx0cKiMJlZr7V}?`ktUy3!3Z0I^)ptVB{sXt!Tv%yIUwq(v(g6wv=E!A#%CRw$lK^ z%)|5x@|)$1p$!(0|8-6R2;%coQ8%<)EUMGCM)a8qY9$& zHqhJXHfyNciXI9JZ05T~>d!5v#o?eNT1hCN3@psbt+P+av1V(%N6O4X`Xc9t^Y;>( zb}GYg%%W|DW~Y14{CU}IOV{t|`2lkvjK!BRSoV+9#C4{g=}_q4P886$0*pp%84M?kW}!j@6)iI_%W5? z8b&(dN?p6IthW;z;j6W!MD+P$j|e4`Zd*b3qtPlY1-9=uNr2%GWyuU<~tHz7GYX_!ofV2?qvxC>jL2d zZ=8o?YIXv-ptYqlIpyDm^&2EldwT@Abx2+&s~>$DtD&-TY9o~Bd`JkzdKl*YKq|9u zbpSb;2+tB{GsrdJpL=7KyhM~Zi7A9m8&=VOr>;`l!Nk4hqV)zl{b1?1NV}f1d%R_B zou{=`yztZs3ABbtk^K7gzsCsmT$a5M0R+Sz9|VN>{~IF|Ali!Ox1VJbt5$T7#`Mv> zQ6RkwPE;QT2M0>3092kR3GDDQQFLMOQVb6_ES2m&Cw*0BP^0Am8AVL?pj_5ACkHKF z!m6v*W^t7_z&4S#f`cSU*#g5uT(4TsX3*pBagycsXv~s&a%oDStICVplv7Tqh_=Jl2if00WruH!OFq)AB5BN)+c_#1G;Wxodf3cMfBCwRu`{jJ*lC1p5 z1yITqyNIU9#GrVMGL&%pA5d6|rjev&mon!K#z}r-!dt?P89`^(2bGK~38!G!ZziD^ zSF(`-jmFE7LQI;dtz|STrgg<74yohjUpd}>R_eoq?n-_}tIxXfv$OmT9y`4`8khdpUxB`m;Pa%VH*U(7J92Nr zZY&2Xw6HMe?GP{5TdcPNl_#$`R{z44`wF>%9LKjwOI_7~W#;2GQpu8oEOU7|EO*R-QwTv!F~AHaV~Bw3EVxuJFd2!o@w? zWQ3io9BFc0??NELpBdQgWC`-m?11WG@WBkmvCni_;Q)aaZlN*ECkiDweY5?f1N;KDe6L3`-m(*8Hk>6SLO+8wb8DsT)lG6U#nz?}8^>15 zSNdxOIDk#TT>6uBN6E?;eZ7jRQ!VB2QNM2`Z{;=@XC|x#XUSeT73U_*MP(0MCvb)6 z3NqNj)3(pBmd7D89IZN(&2(Qo8@$y4+LfPHze!Pz1cyW#x9eaxQL>+-@X4)Nnr|%g zmKkRM0svIQDy7~wIgel4WVeY&2;D6wRLNa(STo34Vm5Kk=*3YnEuuGPX~iQ4O5~gq z@7psvzG_=Wo)e%&n<|6Ku_S=t{a55cf>)YuYDt%~r>hm1r^!i`n28t!0p`S2ID|lK*qhz~ww=}+ z`DS^VT7?AM`k{o4TT8(Y(A!4CZ**b4`lYxH`ZbCT`ZnR~#zJK`4G*-N`dc(UvoT#K zz_0Q+N?oq-?D%;bBb*c?v9km&V2Rar-J#x^>7FHz@g9v{A#~EN?fP+wi)0IVhbbas z1vpx*NzG-0NzWseQ!g+faNc^1nEc9kUObl!O~)f$atE9Fu$KJGNu;(gZ7($Pjxr1_ zCMu_4VJ;#1N=m~{*^=KtQYHq6#bPWDui=m$1+zEdz-cf6bsS#Ja#z!dP_OZ@(CS*< ziSM5mV)?MyfU~@&$1im-_Hl1Z>5yP%kG<7u|L&Y_-?=`v-}J!tpVPVTQn6@@KQ|$bbPjBrSncQ!}OA)mH|=g8*fOLc3JXq9zF$FOU^S7M7<6BB_9rdZ)lb`h`0yi6_mxKI5gE``{{kYq=x5N3Buh=BgN0MArP7q3pE$;{!|in+`^9!ww5qf6(CT)H!Ue z4Ya$Mx9XVjZP(Mb^L}h;&a|B5rctG7Ogv_;jp!TPh`}~=M%#Fnk*a97ZdAJNleHos z%TVs;MXrJLurUP(Io=HF4pj_O&#NlqEiEx9YeAYk&_b2M^@5R>>=94>NZL4A*4DmUY(~GK-0Ue5GkYGWToA!>R@{buAEPsv+9zkir`a7>?3-M2 zKVtjIA4J2Xz(>skdRe8 z-lM6hCn@{{VXMa}7BQOZoeywjJfubUZIDmcExnLk{l-J5(I^jq@uaC7!Fd$iB?<$o zO6;K6V2j5>j=s^U!a4v8BE^mLIwh_WDlGDPi#eg+YKvi20&%hVq`YbX**M5m};sg znP#gsNWBou*O%rb(DZ8ZrQIvxFDrpc4;-s5>WL4q2ko;&w^!6A(Jg7EKy>vab65b6L4%*|j?8RXVz= z-MoDZDU8hO7`m&Pf}hTwskB7`ILGX}n|!*q7-A@Fy%!^vu~mZ}^%8lN#Uw?t7;5nLKml zU+&gz$%qb%?+$DtZfuK}*}0c_>Wz-$+r7|x#sXFjJsP!5 zA3NfbIJYc*csIquExSPI&QL+?xC8J`OiXMVM3#=75uFr-E9FMa%Ejt9bU)Mv=8`(frzP&fLJ}JM#~= z>sFn~wSU_V?O?x`4-VMQ=pyCg?kypNVulzG^IXlHC)lnpt-aCxPx`az50i&tq-$M3 zc7{Og;CFoCxetH~t&~3Y7#$hV1G=APb5oO9L-}|N#Z;lv%Chru1ZqfY`EVm0;VlhV zT@7Kx+>u!IsJVrKmSiE{C2S1c$_K7XxTOWLDHzGC0wmoB&P>#jEbvZ8BvWCCgU(6kY zz(Vzhdrhx-%H0rBZlb^nQC!V;aYFX@jA1&r?@>hA(!s<}7k@>c)pbT^SYj&}C;sYA zplL#-yfKl&H&lXWiXKjvKx7RkFnp)--V%qcBpaI9HTM+aw$IKrX6#K*mN7f(b6n>H zyg+q*N4^5dpCO=-QrDC{eCsSOBG;?w0BL$Nd#p){nyuiftS0(Z#))OOoiJysa5sqp zjuEI{DcX9y$)Dh^;iN~CQ+u3@rNVm$HXm9w2N3z2=RTdcUTaIM=|kboI-NRZE?e$x zSj-bo^lw=BuY%*>#Km)=(2l86B&N=a?^9eUn$qz7uLSLPIULUYqwXtTbBO5~im_Ny7_ zq!oGkFyq(C|B#iq&O&=IG)V43yJ#(L?#hy2lIA1$vz5m*N9UkkToR-8bWaqsu8^vs z3UmG{xbQApP2GMfsGW=I4Du24@tt}Fj7|IZ|z9Ff3Xp4>u#8# z9e=V`KJLJ3`+EaV^5oM7%Nd-=4`!vb)2C!Hphtr4=td89BLb>8bDynua`mAG6jZtP zbu(BE+rP@ht5Bwf!OH4%YW12a4D(z_Q0Nb`M5|0S>oI=N`cE?_GCMhSSYs+_rHNc- zWAKD9D0PWa&CTM7qjD(pFY9OI=!#p1er{K2^K!Uvnf^yTRF_ zo9KWNtqs29P>zVE_^+6DM^}~GbTtnt{U&rj(3$*VV>oqJx&tk^^OA$bF)+1TP59h} zTTlP#@P>hdR54go($wOa*cc2wq|=;MBRzKK<})DOVX^Ow69xp>dZ=#6d1eg1^*THC z?6J1#+4ARD`Clri%ktJ{T?d~tU3Od1C`7H8qgE=m#Z?htjuFcV=eBBBy^`lbLi991k zwF+Qju`z>0Y|QY~Um~U?=!4qLtbJT4dG?)pXVDevWgdAhKfBdNTT@4(4&w8M2X&J< z_3^`|h@&c|LFRUojv;dYF5&{=t|8F{$3k6rIHm~lWU96~ZVIyoOd6I3DR#*3Sbor| zalO*QiZsvEs|&u8f~I zvRr2kO9+BV+k80q*$h%qVu~#-U6V`#PxTmanY)n&|lbYCpkWJvqS7XR8-*o8fO?ex&UY-P>mWpfjad)ul(~Uoc=d z#ZZL5B9F;y`Oc!ttUA9?M1i%CstaVSBFgBN*|7W#5EkCyaKKOojpc_*0Y$c7>vg}? z{k=2r0?!lbjH3s*h43SaOM^C-S?|hH^Z$u^Ad*Flh^$Z2Q3TrW%C!UsYQQ!tug z@Vfs|k*u5!fy#k#!l$TUP2A#=WvkC(p6J}n9=m31J%GWN3r`!f>DgwO1IfhU@5W64 zL%3QE%tV{p@bC-|gTlQUGH}yU%^e9+-C}CyI=!9sS!%NG&SLoz^%r+{ih6f(sy?YL z!QK)sPC53bHB@@xkLzGA;sydtbu%XAE3Kq84Qot>lb*WL`v>Y#k9-}nzQdg+XmxY>4}yO$&SR3XARS|4~KT6cizxCN5&YyEYh3PC&| z-AffR5Ch{jrRx6@zhicqlQ~2T%fm;KuOZ=bZ;e@{*>`TqbKKAp*LDw(#vitA%so4> z<;#dzuDig_UGDcg9 z*1v;@MWwB3dV&usB!JUTuKaGeu_wl58u(>CJm@tC;eQ>kYRX>mFC~Xs6+|bZxmQ1) z*Kd*|x}%F<=89PqN4~FrZxU`F4##&$z#cxKLG$_xd7+;c2EM=-KI)&KazBt=A4S7Q z4f7uV+<3^lFoojhOYXs zs_N|+8-xIw1ptSvhU-VNlaB z(F^OnP+foM6R5&?u~0vl84&LHCO!(%U}~NGN6U5@s(9d|Q!W~lDWg3s_Cq^`~V%n+ZrCGYg`2iBJaw&HwDtBx{k?Kd<9fkMniA z2o_y|fgO&epA}CO5s;4$f-F&SCCIJS+lKIOkv?JpdWxT<>)to}zd`Lux$Y3g6j_=V zY^47&glo*Ue=_<%Ms~LKcKcWcz9S6t50s1tg^U)8Y3gMm@??Aw{_iCmcNs+I`CnNP zlK7911Gpi#A%MgSOXe2kznCNqgC<2y);I{hJ`XFJZy-RTlO#~cBTAtO9#}^@l9#W) z6bV);;d?3EQ&!G(5hP87cxY($e0Vwe=VFWh$L|}IDG(f7F-p>d!dRydY##*$Mkp=m zPNz~|WHx{rS`r*)*yNO?CyANTQ;F?`3kyOx=;Qx5Yin*74gKWp<%!lN$oy|F?it<-Triqo-`5adKMl zCiD#~qABa)Y^CkuIb4e9LhaRJpIcjuuh(==9bk*I*?FZaFAOq!>{YQ)s8?vu$Y-h;~3-Wv3YkJC!Or;(y5tuW}-_3LcpvSOm_F%STgPE2C` zv=E-%1}nid*e&Pit-CCP8PSpvel%WlBTSC+mSly)Qfkoh*9(LwdKDgle;}TU1?FDq z=j?OKBHYuI??7;bcdlZe%dFN!ubD-vf$J3>h%B@hW9tU!53drwH7#~m__qN&AWQA+ zLX_yagv^bptvy$5CbRs8Sg|`@oZ)8fWhL10c0y_Ejwfj_juvw9hUa?p7jnU@J6%a1 zOxq?LIkN*{^SD-`GI+SE%r43ld`tAdww;aXk^2YDi&o{sUa3X%D52F~BQO4S<2oR5cp!(89mXMQ6J12P7_n85%5u#Y7O|e$truG^ zq!d**8;CmR;1}G^5_Bv0H|6wBE3633cxxghcfRl7`M-dHqbhCMVmw_hq({;x@fk; z-KjlVB0LdZ`Ovv<3C5oN+B(%4ZvOl0?V8=+Fnp&&*w9`IqVxnxx5VfZw}$A_x5(hs z=(OaikW6_7^O3RTr6U70#kXOfPo+8pi4&TP1TXu>G@Y-ln?Rt@5ak^;dhReP4PW3) zxd%$F@~tPhzCsskWYvcAtA8 zammrWE~8!Q&W-1tFS`3KB#htbP%PP9tm-EifAYkP$|qir!kzS|R9}ti;{gB^*6q33 zHHZrl;nDS(U=z61Q1W0fv;AG2WA{t{G|W%OKK#5cK825#AUxUzC2h-Y7$t?E0piJG z9tjF9-reQ7#Wk3VyWYoYp6Ualo$I4vcF1#FXPYQJO<+N(d3cENs5bHE8>Ds!v zd_IlGioTD!qh;#(7N{F*cH!j{@?&Qh0kz6_NcPhCH{TbQM$b*67+8{g;FuVyHYgzx z_7?>pTR|$cS?|c3!}MybS)H?IQCPHvhr3`rFu&TxfqF?jwfF> zKieztestu{ME^|jnNNU>_UJd`0gyKTqE&lF*Q$9?sVkP0yuP^m&W|kwKVk|#2 zqXsGru(zrWK-n!jpxP}ufC1`dbxZr%K7RoV2VkQ;jo}CuV)p4Nl8g%l4*3?< zvE_;|C)v_`Ayk~-jAqL0is2ipA=QRzgX&N!hY4Xciu7m4RQ)C4ILzwE4{Xf(bU=kc zA=%d<5>cJP@F64UF2@|BBGQtFA^>&+iLuzT!hOxs;E)pRj3G@;_+Jg0+9KW9AsJdG zo7qw35CBrldH5jb7dofS@NdgpFppd?gpsK_fi(3UVqQbAb2WvzJT(qfIDr1o5an0Wvp)@2L!pXzR5)-E~;&sx6rWsvTbJ(86GLY3WQf zK{@(;6t%Qg6=Z9+I*b--qimVC%rk%IW@O1OzC^XNTC>G5`v^~|=h)#XRN%CW;s6b^ zJltT+um&1$I<1AM?|@A89Sf`NxoP#lu&2hO$#v+%V|@LYM9~Cb{#Bhs_xxtUSSD|{}pSi zu2XBRyqy)*>O?%9Wz!hJDX@ZZu&8c=fhSEtbNTkW|4-kU?nMMd6t*naTMD{P#E_ml zq2USIIK^fA1B+V+3Cqgg3roF`7I3N%TX0$a-4w!Vk}U1fr^&07+9Nu5{6`pFBo{+X zkBe)ga*?A$l>xlYTM^AHLAUF3$K`C)hxABkP&iy;s5RwXE>y znJ$v*g_6UTTOIRD(?hgzxy3cl4a==O*|#d5!4#8pH)+P+q=lD$9dlFe_A7kslJ475 zLw?;TV@X~q$f=1Tci{E9EWA3ZKRaJBt3zss$IoBbXzjz}w%K)2WAiBX^TCo9i@){` z1>7SFH;!h_!B5pEN?JoMHM~QZ4{e?&wE8$P8TB`iGr4|md<>nf5jboEy^*xe;cZ-D z+dG%#hru4^HEpV_1Bcu()sjE@2rcQZ=G%b)lMM01gGZ0AB_MpDMud~iuU&t_z*R*5 ziq=f<tKq7@cgjmV2OD+Ev% zy!{qDLpapP!F_^=8luZP$w?S?xqK5uO!*fysZa`q&|%Wq$>%i$uDws(f^G>6h>|_$W=Maz-COC4Rk|iD6(JfCwe5~*u}6pyeL9< zH(-@JfZ2?!@(9{#uxRk*ynYRT4ZR}b>pwZ9`qGMl7I~G1I49<#6IT@Sd^w=XA6$n- zH4dA5S$W`3)EOV$1BoF<2xi&f%!uiiF1vWekxtmKJa@;V#7a$U#`EX>oI26s+BCe+ z7_!Ib1#$(y=l!cO1nrY#n@_h?Fh;OgXHHguI~9QEI?KRsR(!x$YxS4v{dOJXkVEv?k*0%{Gjb~R?$ zLNpOm^TO5$VfR2F#OmkqA!Qx(-EV@Z@duOTNhs?%KLIz2f=87x=078*YXqv z{MuK$yTRbe?Z6qn2KONTLocR`^GxzRB_>%DFp(qH7NGqbjy-oY2HyyKKwB#5mwsCf zFEBV}tB;atW4UQT(E+l-+q^Vh>)PtkwaDQpX9+uM3&q$K5Gy2Wu4Nh@{F&3xan)ew z7Le0mqBr5QE%s}pcI!I;zBKZBYgp9}Ut2pJXP1$&YJQdUn03S|y=Wr+Ot8Hx{nzSf zkcFw3{hm8+kM86dZTieSqM#H^!nxbKIS`|*agW@i>}o))Gs=@8OjB^Dljka4%p+&m z2i%<}un5MD>Xwi4w-ksw5{%v*UYo#oLRoHa_E{_9+L6GpU~WeLb_XkX?^HLMmnjkD z9A!fp8RY>z1Vy+gUD;V>d6V>LxO-nBP9|s&;Z|nc&7M%+n3A>Q$fZ45UI^b zMYB^WCf>8l#gO!pwDGc92^4vGREb+>W*K9XT-+PK5qWOa&={dgC`Co7S8bHb|K^v5 z!Hm~@RQ8j}OB`zs_>ZT8W z1O7uJ?GccwKEgu)JzoBS6z)AzLK2p%C5m)P~BNG?v&B+@8z3^_t zBlA~l4oh}Y)(w(H3X5jL?o)boZ*p?FwHzAnLY=ryMY##Co|Q>yt`u#d#&BJ1Mq7S` z^dJA}#)D&}Kk74DV&_2>Cl4W+II!>AT8if*7h7=h=b6nm_zZHejo{OCThm1$)CMP| zK%^&%ss43%&}oX~|LU?&u z+5esRqJziMc*yh}fa&rI3ihFbs;gG_#SZz08;gkvX>iya+icOcyR^913BZHg1G$u4 zbO)Z5x;iz^s(WERK!L;BE=*LtEeyHY_r`HMz<}4T zRUP11>m=M@e>4aU$zCloMC_K=ZmMSc1o;A0UrD@J&bKs$S06yUBZZgWV;k7GlCkvA znts_G!SM^owek?-;7_#XA8CM@23mL^%)pAYKeS(1z6NjdMQVCgfn^MH7Y-!^IO7~U z=ncTO{@YE(@e9Ue?+i+1|HSE;yM^(q*mIxn|8ECxp0d&nB%U8)*DN~9&&}T4L4pGp zZ*jg0_gp_U2lQ{%VE{x#=?fQVRW_kl9{8dkdaOHw$1WZS+;jB*5U{=afOXy$Pw<}k zfI9#@F!kF$(Y?{Dj4to$Jr1@~ljS&BB8m1_#ja2cOXvyaJ8z(CuOltx!`EWgWkzHm zX);T7p0}SKw2i$+D^5z5=ox|3XvRVUYiD3PkF6NxDpF*cEqB!!0k@9 zq)dINaTUZGf3hrN8f3JJa<};tH9aookDnddDJit=`4;y<3urz7lDq+c1y;>moz>S; zm?2{ON@IE~&v66t(OgE7<0=OB-#fJzNtpby<(jfv&mvcTFC(jR!MzaK+yAEiI7K)D4yc(WOGJ@OZ&W>s~HG#$7 z=0_dbZkvuv=_FAgLuq>8FmziGg5R56NbgiwbMrVPhCxr&=0p0EcXy-T%*gW18_olx zU|0|Ds`Qoi9~^O1v}XXt%8U@1eTJxVy){Oj2@g-C0}DaAC>eI_G)V>zR*pR2YV>h1c}Qhw>k9|+F78w?83V4o z-we4oITAcTCxRRFv_6JEUb(#!3f}pya@8e?9!ciVvq|_v1@zZ12&kgF=BGY3m7MLyUc+duR1uNTEmePy!T{32DO}GB z{~8Yy1nP_}CXCoDf21xWiKyEf%~kWzXw~oB88A&?dp4bHVy5`$1l^xMl4^Vsa(p6c zIpoTE z#xPtv6tx{xI4RRwFD&HO2uh&T{Gz>|G=>N1A?Zcnpr(tetTz0I6F{t&*RB|7Amae; z_KaebQs$@5An^ZK2XtLnEqb#5$eRr<2nf;tpf*`{WT48@5hR`hDtO;y_TS}WxiMXCxYL=o0o5!y|v)>Ue@ zRZV{D&)Sa_8Q(MBtcj8=P@}P%FWb7)yc0ikFVhYB1Uw=l=%4YK%?}QiViK%M8H)2! z(_)FFQ$VaYyo+L?X;qJ`=+|WN&BJ3JY+T0ZkttMck11rJ1xca}>fo zUhqk~M^68P=`bi@>M*eom={TV)F!d2OJ_{k0i=#iZGA=aWSBap(K|l2(z_nkj_#-3 zYy|N(yi~DNlTxNdlS73m>1dVnI!Cut@JNnS?|6kf(>N$D+N7G&0(6aEpy}1zLU{5n z$Tn>r7AO0p^y6-+FfOX!19Uni`KD8fh4!ZGtPbg6tnKW~a@pwZv?zJoGc!0>hN|vu z{wp~9r>-vD@BT1s)7U`P{~0AlX1LrB@pH8#fbopiKprrz8u23e6> zphoRL+pfL^Y`IiVM&EkY?`o6WyxF}6F~-fn9$pDCPl3Y#GbO9wgC8wi1tNzYu@cQa zry|N-U9AC(HMTk(#cpfuO-&Uz95vNiz>Wm!l50HMOf`a8A={&9u|+~F?LoFqF1!eA ztx=5{@cs5kMk6PPep_3wBugV*&ULkIj3q5lFHdomNRdYi&+I>8$<-+i;#5e>T4}{5 z85$it%nThOYz18~bP$=pg*>&2!7CO{4CcVP7BIs`J~tlGzyInTJ)G5PbYaZP0Jq0k z6-)o@gT^^CJcoV`qZBv5b_-S4q#T~wU{;>A}d46vw-M$894aa$L+c;Xn+58s=hX z)QEoFXP-9WUioR*DGg^2arlF@x_{)io1IR4F$Y>2ia~Q|$YV7w?O?=Oh4ABPsaRld z5i($Ahf({S9moWa$iSd98GMxX0RJ%TT;%^MM#gxHvrDScR*s}@J5$ElXo~QBz&#Kw zp8JLTp9%Wcza!|rwQE=0Ccbq4NL*3dK*G)VBCag+G*Ow6nR^sF3UGnWIz`sRQ1*bF z3=q`k0eLEB=BY9uO7550Hnv{jPG61f(bA}2BFjY&l9p*M9&yps6z2GS>?@(86*5Ragr~u`KS|#I!WSEH-d31h%ld_q}K2F`F7INB%Cp6K>Tq zt40w9;AGfacVRF)7T*rR0M+xn|2A@LmBKVtt`vMPSG$l`Lx=vV)(iwAqKMtcc=!2j4 zWEC#b*9v9?FB1c5OR(-M`j*B()?ksa(%~;eXUO%AA+c927YG9BfD}j}sPM%Qj`d)A z2g+X<9l?aM}@18H+8F}T!F8QL{a;(-~4X{eXKL>Ono*+@}J%4}TO=p0=&HwkO0 z9D^(;XQ5g05d~h$lQ!t3G}m+#ar;x|Jiy9jbN8EtddD4*Ttc)7m-Y-4VEcMhDxt(t z^@?s^{^7jUIMAR*i--LWK@hCSvCKf`n366#nG4l z?l$Hyd-VFNmDiGvcQ@fB^~_ekFRbt z7*tV}f6_!`~f7NZSXUS%b#yLaA zE}8kRs63M_4~Dq$^%c5EonKPRIY#b+%D*soStn1`i>Gn=T%SiCBi9iwG8PbK<~n`0 zW7>ULe+q1!%D(21RDzAs*3#Ux7`B6bw=&X8mr*{G^pYxaKjTgIIulIZ{>NYXNX1(` zFZzg|%ur}w;lj`RZ@$O}oP)1gNg3lypy+0voaTziWQ(x;leAmusN&TsxF$ZJ{>kUJh(s!q-BxRTIT~;&Vx0d7cW>jYz%m8~CN9~2KU1|k z*k1LG;~P6)`nX(kltTG11rUvUtRPoou~NJV3ap%$#XC zQC5t%{pi)p&C1WQuz|8{ysfZ$oBzqi1{|#jZfBy82*OFRP&=-YG*PcXer_g-^L~;e z+Xo7yy1QBFsbfope}Ad7=LXs(1IJz>a;}v<5qE|f&%$KLo?{FnYXPrMoC=3VW5+DF#rYLIB)JKxVXcF5!Rpx`foD%Fmbbb zgjx+h#F2=o7*?@piboTbhBO#6$lxYJ-R1@kJY-kKO4zV*K4XOh4QgNK8F&$ju;eNXyXIvzW=aZrohMIqq^_z;F3L4D=%zw{|&F$cL!XlfZ)A%H7D ztD{c?$RFwV2@4!aCg<*#sQQV;aWXi+1si*XL|H|bebW7CLE)zyxBU_-8fQ(;J*kM_ z5e)|Y!Aw|yNYSL}p;{>RaV;vk1xnNWgy^fip+mcQ3X=*8fr~EfdHfr4FL(?;4U-3!l2MKOI3`7wxLVVRt z-a!TXO#ew>n~LL-IeH^jL2c>>*sh0mI&PBaQ>swLGVJhH+tP!wRkhe4}Wn@O(s>{bmarg@0B9 zqDFjf=IHtw-#n^Kx|hGiriq%T7G$J4{R&!{8C{p^4y;r4e>b^J_M2A2W^3*Hs3zYP zn&g9zVc!ncO;FW9ID-cLN@Y-M4*feo^p9F_6cXCpobv{g;OKq`0eIO2)y=^`TM6!d zClP5TAsLJS>olDbWWNdvbM$Wd*o$SQCy2n>Py1nS~LN z21Xk$p4XJR#E)U$2WW$oeCXyR)PiuFa-haO((m8;pFM+8)kmw`zgENryN1hnz2fBg-s1#9EJKUXta zJ47|AQWe}!oD}>^y3`RIn*nW70g_W(R<@Y7{)a~NZ1xl49?0KcAO774m63`83b9QR zGvniw`ZfHBNM0ROxI*ryXukbH$Hy=U*T$Pbq>|gIAk9i{)OqGLfLfjNNr2^1wJqvY zl1Qg9d9cIWCUUPg?mT$VwJ9C6Q7KT*rAWm(#YyoVnXs=BuDIX0nNVdd^VOZeQ9BBU z;<|?*BDu)_1@tbQ8EXv<=oh=uS@HsdtYp7AF~Trl*;e#~nU`g~b+a((z^FhJf6Dkv z5CjkMX-M3)AmM|+AvT_gUOLe(?+nqR_0pc>hXvmTK_j2)+N-+~e|vrl-k(n{4w>I9 zM>*?8^Z#&lj@_9>TbNCyl2mNlwrv{~+qU_}wpqy=t76->t%_}<)3^J}y-H;AywJ*v8N7KogDi4M* zZwYyr?>kS2+p9-pMh|d; z@}Mh<_zf{-p5ar~;3jj^8eCOa(UTB_KDX!sSK4ApuM6|BR?`&jC(A>$DmvoHx9P7r zZ-M-QYA@f0RB@roNb|yL(+6jFR9tl?+GQ$3QI)Vi#5XI~PP!ko@lJ^}&sNSZ z9627g9cFju%9l<&nz+xizR|Hx6hvn(%2C)|Q_ivZ_QPXboqBH-xTtiM4w{vlg2#xr zLji1Zx;qauZ~N<;xz7+VG?Utqm6OTqu%R5}Rh}+hE@J6{9bQpm{97j*nar z7TUntFNd;FrpfMMUdbk07><(@3lQeBp#iq)lnTH%)U|%-3$n~xvY4SLVU-u>OrYYT zIjfJq+-1oxWXw0Va42q^NCRjIk&_>py^LqidYP-qDh7aX=qwwum^UbRrFTYg8AN$I zdDt{~{3g7-I$CLisXw8u@C`YL^)Q@Yz#>5s3#^OW-Qq|@{00ic+JYqn%gBz`Oa4dw)KOY}7}`JV0myZY_R@luWZ zcn{pScuzhLeL?iPa1j&@BRh2kmT)?#GSK-4Ws@Qq@5*NeNtlY90AJkJeI)30&e?BM z54owSbMVbM=+>mHB8ENv(amQ=@!(sAwaGq~vW$>7pg{By&NcIo z;4nMF%~^lqE>z}zli8f~`P3SyjRmSr|F{qTjWW6vMh?Co1kE?4*tf4hbd6&6{EGJZ zr?8}wF^3Ho$F$&!(ZHSAXy%XJm^Jm3=A1~$g0lz%ovjnUsH$93wke~9Jlh0R1-<<_ z5SzxXN^m1sNMTco#>>)F1;DV#KKf+I=i-I1C>2~528u(CHak;ZEWMnRe5B;0rtR_6 zg57=+4HqtfEV;ybo-c*I+UOx>vZZj=ZW-i~y$UjdM$Asrxd0YveqeK6Y!BIYeOxTJ z>CyyyXfrg9FPRR5sVg0GcxR=R2C}xA9QFvGz9&z4Nncpm&h!u8A7I0wN{mk{BDw-1 zW(VX8m-R+c#qW95Q#=iA@&XywQ(TYh@dY{LgskU-v_km75w`Ogmo6R+5|6W!t}xFs zbAe7;Z9X|OBcF7_xO3EJXMN1&g+EkPIDY~@UqRo$k#@{++>UPM8==R(>%$4|-``UR zVgT#*`4{R)=lZT46_9N`a&X$;RaJO!(=81wg6=Gzkq$YH;4h&%K1=Gz=8v$*5RNIK zYOwPl8tD?RreOBq=^C&m-?v#+Sktq*B-htIt zWXA_VF^#ULBppz+1pC;!mo{7#gDm?O5=8q#MOb_0N7)x}1PC;mFBrSF_}fgEp%b{_ zBAgI0l{kv#Z;r4^R;{Z5zkZ3we#0)CvpGG-%Ay{Ah+zEh2=VuyFgiW|q|R{Odt6mP z@4h`9o3uDh`#A!l0mxw0Q5N>f3?cj3^6i}{bl{B$o*Sp4`xr;jr?bl&rDoAd?rRYF$5aB>}ixlXwi5jZku_1%R zx!5ZAbI6NwO@t|#8o%8I3*Q`JaIm-OB7Zlv!uF05;p((qoc$tRKJDeuk+0*Iu$^#b zOkg`D*5N!{o(#A?_L$c;;Gg;EwMr4J{w^%lXe&9Jk;qW$7;H zVY)2X*l!u@J|@!a41>~@1Z-%7>xqu*7Ed2wb*iJ|sbLd~F|@v~ny zSf=I=y@0jY1yrAg3=!X&bmJLmn}i9(7=wSg_tm1IEl9qg^Y626$=Yy(MU?fKhwJuv zNMuE335%^2S0o^HFho|`fZ}?mXF$P+fW?!_Z_h9BqNqX{{lXr|d@8wp<4fIQ^4k+Y z9@FTq%B0Z{MCsw|S0?yWXR$qzJIUqMN48UIF5pCo{`}rD@C3UOS&l|*aAdRU(`m`l zj%n7U*IEuZjZ8~<{{>}C`Mn#2|DFQ7neJ^lXgTm0qYWF_ohVeb4iv+V>MrbpjmjBr zz`WzCZDB?#%{JRj@oYp3&!#HNPW_#iG}t8Ta^4=76q|zw#a12aMvB{(XNHS!b3SGZAML+!Q`T?)5y5}5#%f7;fVKUc0qXD< zLYk_b@i(%vLhekBq07+`#l%D^BYB-zV^HTlJ(g5e-%yDfqd%ws6gS_45%4D%^Z28T z70V_wKY!#j_2o`~awQ-D_+gQUB>~%P)DpPP^jk5eAheCuy|9&LvI$hRH#Sf=^mudy zk^U!-frf{fM(A5Mkup~);PH6A?k$jLlZp1mikB00ieG^B#H7{OdgI)2INtkz}BD-x4k@^Yl-IESmYf**nG81hlRlt~Fw7AX_ zq%%BhS=D%>JHyKi1-+zW#dHlz#WMGvyn1$YNKu5D>)~F=5uJ!BfOzdVn1}#%cm{I8 zrunC80ZAH9U6w7eM;RLRUqSog*>$6E_lw$s2tEQhJ7Z|&4sopOE~@h4m6mg%bet4e-@ zykUY(rkgk9?9TyffgY>DD@b3UE~JbJ1`@e98d7DVG(ML7k!O+?`n8&J%1{K_zR#qu zK8x;K=nTa`H?fX@m?(6Zd2p1XV*bNxw1XFQc5#v_@`Se%K=x3(hb?#J4k`nRh>1va zdx<~SI60zLib$I!dLlUih%0R#-7u_ihef`K2efzVlavy-yg-GyONLe|FI0P?Y?N9_ zPJEzpYK(As z@Uoe#tI*oy3UDX8{P#CA91v8!s@S~G30Wff2um*r6$c7d3Xwjn147`;zXbL6Rh_V? zK@}m%b8}>-(BvcxG*i>eV1rZr#KQl|02M9>=$wHB9Wy1FRr5&lm~aaypvP_TsTN%) zQFMW!Si=G$Uc~O`IdApm=NNjSDx5N%qK-={9?G%~8ATvensbhL{SIBM{WEK*p?+i1 zhka+;7C8$Ctq!o{&027CVYEw+Cg=43pRd55MgH6eLwO8A5 zZV68t#S_`HFjhPQ&Xg)X)t5?^g$iRaeQ|C6~MB z)1<)H-H-rC_TV-NuG9CGIX+O7Ws+pOk@z%2kr1+Va7|QIUOa#h7}sj8796eJ(z&!~SHjQnW5B{vz6EgKy-Jy3vMU41IV4uXfr4zl<=qVg9{2J&U0+P;qkW zYRumynT^&?F4h5z(P_c8*6LuriL2A3#ie>9qMW~bWfhvJI~8Y|G&2?znmZWhlZ+_n z{Q2dFEHCP6?bZceIE_{%a9b4civ8o%`og2}&_I&3BQt`GOpVeU?!@{Bn=K7mtjD6h zPzw*U3E@*i7W)~|W1)HG92n@#e?cJ#~atIq)K`X~B#n|RMgnN9wKz@ZXlgbp^ z`BiDt$4|MBozqm3#Z?zqod98i_D7AJH64~nKObQ-j6le{*e*Ac0*nt~%jTGvJ~?Bn zd8W!loJaoyaT!CGWLD#2^+1e%zipIp2jwW|6A^$g;jV7OU1R>%V(|^}xZofHs;Gr~ zwv;o<9Kgf>6oZ(M>#+AlcvX+ZBx$g_x>9^UI8=$Zbhn?gdM^x{{JJi5_$D|s&R9(Z zd%k1co6hHC*@$r3?M~U*No~dGIU`ex-X`VVYCtaPI#OQ6>C9O~Yx-FEa z)S@)9-5ren(e0A@m~(}NZDUYhlfaKQLx1n7yZ>#x|E4zx-0NKMAC9|q70Xu53<7o37b%0H znp*EQPurtkgcOQ1bUS}G|4b4Nrq=Bx;(Ge9- zqYe+RqpsVjSvt0M7yPD9*PuN6uLvNVb{Dlfaq2dz5^K`~%v+^&w}f7ID=8sFU$kPWYrW-x!c0G%IdqqWIc0cueQK|FfV;^ z#?~C0%fCCF>zB4j@=D9~5*dw_`6iejnrSi19efw%KxCNkVl6p7Ur8mZ={v0w3!}QU$```L#d0=;dKSZJ01~YY6R;1*23KSn^=WwN=2k z;>-fuF{_MBwCj%2R9Od|oUG4$+tcg*!Ge}mXO8h zCEA|5chOJSigm`4ugI%W*Jr-|r;0vuo`mGPq+6T~7*i z@)uWj!h}Rx1WuQ&NAFXuN1s!j9B)tEfOqg6F&_*;gnVdn&{7drf5#4>0W(N0>kz2o zb#ZnFStAtyd>(oa+6>e}cp*-ctBY1~_fhi5-)w&M?lQh-p8_ox=I(Q^#I_~7AZtJ!?ico0pWq$x#LpSK#C zoffS}VY?3&T1?A>CcnBSEsjvVuEk!lzB?d3HM?wP8-+4F7GG5^|M|3LKq?0DAAYy6>8bdY!L6c;onua}cxw^W`ce#lt zQCJW-A9D_0ZBC8yXM20(Bh_(rBe;Ofp0RJTfUwL!>_^HOC>#`tsg?Ye+2`?mtKZ=R zC=`u7#RIXFuEOziIIQXos61rDc@<09!JH02Mhx0AK%|ip`Pf4^%amJWklL-~F7<_DCM`V_Ux+9Dw8qTwZ!OrS8uC7*fj-_-R5vg#K5jpLsJ&5tFJg5$M z)*9q^R@*UnSUrF$wJ7v&OY6bQ+z5YZu>G5M`Dmv&F>5| zo`@Sj(q7fcp+xL68l93<9lcR6b?#vCC3U)OPVGvp9KxVGQ?mOJtIB#K8S?lu0<%=9 z_cxr@#_QRO9;u6j9pDNNNTuA{7?7ovwoF!yPS z^vs(93g3-wsm!_s@vES<^N(K{`9=OqyKffWjK6z;bPX^AVa*O2JM{D_Cy1C25S`$- zmKU@;t?ii93T@Dw+@~kkwnt6N-8xWIZ`|a#R95x5uul-j1isK9FopY_A~qqn0Bj@4Ei^42))RheU`|?L|iV~FD6EN zdJ)@f-|cdIDWQtND^tRh2f2(HO@kkDSozxJ8$ErryV$*^rCY7~)O5D^w{tnm9#8>c zVvydOF5I^J`oq2vvv7*}KIH|%#Bjx*Hv=?-ND?Q$fJ)`-MU`-Pki>>P_WT2c{JJ`^ z?@Jx`)q+^uHyHi3@|tV+t`9>v7gYR6f+w&-cY@`fV=}jA!sd%=V8Z4dwSiGS*zwkz zl0Zy5_(y|hxMAVHcv!P2#!vylJb(kDP8iNHmU#03Il5OlT)KeZmm5DVJs8Gbfb)$l zCdMAx`DKBf^DF8-3@YD*eBPlvClJ#$0t$7ub7lx-#Sx8^JV=&W=(3LwO;Q$us$l5+ zhcJQ&jsmq1E0&2kl#@PO{wSeyY1uuqOfW&&$ zSY#*=5X^7LkMcjE8~HdIAVvKNfO3TO^@r|d|DOgkgpDk5^bev{Oh`nvL?h!!ZG^=- zqR@sz+MJ7@nkCut1|>7msl_C^QuCoMB|%dI4F>ZqP&qTj-dCFfB)Lt~cUwd`141VV zUdwkK+vWmSCnr;CL4o<94Ds8&J@0c%r02^SZ0^B7++R=R z6MPiM{9@ek>H(R%^OD5tLmhnN@yUvin{cy=$Uh97ZT>h-(*mP!fYv=y!EkdrPUI7^ z2>64g7thpxcPHxy5FP>t=ebrT2gnIwV_@AU25L@FwM18kUFQdfer%5R23*DhdoMq` z5`Y1J5(WeT`PK#w8p16{EpE!z(-x~YkA=2at2cbe=x&*-9e+(PM6`k_d@mxg|keiwfM>|5{c*|OM_-@S2VRe2a7abhy zy&0NcqVj^v)WfmJq@51QM(yJ+t33Mk=g&42li5V225m#vT1zXS9b*T8b5F8bHnj|o zoce~0Ds5r|7Q{7|9pWB_(WB%MdbW&NAb8V6Q!(rA-ykTwy=rnhpCfKjz;D6?zg3(u{6og+F za!#}X=m}mr2MIOS@_Q9H;e6byZbs%@&SRREOn0t}NuL>O5jl4{U$oH`l5{B&$pC5u zL;e)!a}pDIBodkG06X#r*y-ArM~)&IO{x=7HP09J>LMdo28c;nzpmDZXgUU|iL`pJ zY}Pj}ij~_{n8NK7ExJ$5)|x#qm2Yf)!m!-|qlW>)OYGziMrBeKuC9^EJtW43vL^j) z@1c@wg|$Z@;dW@)XI@|u zsyh8oQj%*b<#<|p>Nh_-@`>Y;sQpF~$2gHQ`G9*86F0`>OWqNqP%(P>CWgv`3b@e# zDl`F8_6H)8v_i_`*x+2Iha*$4U(Xn_&txd4K5b;4HS@G4bA2)!`NSnYxK~~Ni>s)f<$y>HPy0n zP@-XqI&j}*X{ngkuk7w*)RZ4LyJkXV>P>a;;)#&9j8+ZG+IPFT&Q6H0o_owVMslrV zMOk-w#+kcp^|P~YT{mao(VpkQFDKT7Zc3e>QKsO40oQp^l~*L&^Y@^VzLG1zQ{Ye< za%vXGGiN>ptz%G&Q2Dc%G{g>%$tjkT>}rz}_18ubXV%K!b$h0{KA1+ZBw4xRcTn1W z_uwwPx~BObQMs^1_ARf(-gBdqsJeNK5Vk4{y>nv-EPkx5$fIMWS7AS#!Bnu4Y^^$8 z>ZRC=!?)21*okfJp3Zh4aNfBA1=}n#UI(^@x=~jWVj(~ckuOB_83CXc>yF-rMEmkg z6?B$m_jJ9!&>Q@;AKC>E-y4rb_jbK~v2U7rH@ums2*oiSS%H%z6^m>nR31g-na8~@ z!O~CjAs{&u1BW%?{un6Ni8314)}6r)w0=;oj&p~&f1;c9hU!+t@+W|P=;MRT9f2j9 zmDwWwwAeZIs#BUtoZ1(F?9JWmO)31y38NV2m2=!Gv}n4iRqi93PR?}){21RpI2=5V zfmJwL=WyLt!B>fE);~fEu7bZ(XIe5UHD$gqnhmf%%~Yk#KKhD;%#URxj(;9pUza8;|JHSWFHd(HxNoGK^!| zXq=6*_WF>^hL1N6&>lMIw2s@PZl=aiHWS9JoR_R=t%J5PQGPe_`&nc-EL(z9P>dUtnaMAF&gJ~O}E_`aYTq8XxjZXz`Vkj=WGtJy!yyA-B#Mkd?i zaMj%Y6y(!=d)sp4RO9i&Dc|BZ7B3_e>s>+OJAEmr-4Re{bQMCJh!TymzYUwJjL)*i zex$~v&@Y;Ws@ylyhq{U^4v#RW8?w+eVi=JBGheg>fnnLd)2=ww$dO8LP|H$eDk%V% zKY$3GAH+Ndkis|tAyVa${Oc##L=nzGd~=HWw;b#+IqC)XA}~Le(!0!}=^UOFtSY~l zh;iNvt#>J7@kkLTN`-k^vIxwMoE9-O(DD=(A%Q_Y3EaE}89{}aPxyYA*UyIPbWVAp zXmBw^eLlFy&t@R&gqAQ-R_LJua-0*dlB|kF%$~fIwEgxk`Ru?YRGXX)XOAnOvq=5< zz%ec?E3%EXwN26^Y6tM#orGwb4Ys%;P?c0;DhW7SBW`N+6A?&`Q>IoZMns;oU?uvC zG&eA8sJU!wkOnJK0O;G!XJDNjAlQjx-olGlOl`O~Jlf*&zG`(o+}acX06rlU#a9JE zdAQx+L~D!%hwpm4kvuf}%xylu4`JlbU> z%K^LE9o}OhIOi03jGe+RCHFnx7STy#V?{6P)3!6{ zZsQ!U7`H6C;Khvt^JsW9GGj`eA}>W6T`|>^U_{izpGk2D&1U;bk}aWkUPl$b*Tvsv``&p_F!+S(IFNVZECuO5}i{Gmw5xc!0(D z&GQEqL?25gHWt`wba-34Kcc2?W~Ikd+o94S1*w^SbRvKdFiFS{9%7R+u7=EpiX>AL z;}@73f_+iPu%#AI6%F7b?k@v0sXBKCBC5J*?q+e%ADbL?*sGKJ;XWOL_ni8;LgN@P zBmb`G%(dn?)5#<*V= z*k9>;-@offx2V&}_pE%z3_*0J&HvkqF9$W&YkWs&=ShMVvVhSZ#sg$1Ng!IyLU7@L zj7+1Hl5|;2XjF0F3O0{yC?lfT-UQgUL*r7b>rG2P-{o)H>-pD>P+GIEJUt?^H9-4y1SkRjg>ciaAdjAi>E{}9;t!y9wtX%NCf&tv_Z5t*hYcYZS%1q*xHp;=0EUMS%T`ra?x`2s5_^Upl3QN@3zN zu3}Zf*#z|2_Rx|HF_LMIn%AmFl`Z4kh3*j2QyJluGV`^NhiPL^Ww~g;~_rL^&Y?OX#^O$ChHW z@u5ATQ2>*Ux*&ApU9k@Md6Upaq`wdq8}Vq;Kh{9t;XWtrBFesay+t{iG>v&Itlo%M zW|s_hUu<+3nBk2eM|qwpZ+@QHWMGjeap`tf0}hWUG9Sg~8{SXkOqwj7&`Yv{zqFz6 z05tXXMClwD28l|Oi@E}wCt3U^4Rdhbe%mq+Gyn2tAiO1$$)%|x z4YC$(w0a?JzL0Imin;o@WMXETH#0d3yKYe>Kf#V$lQQ)`N4-L`EsgXqd5G~?GMQsm zKLIpdwVlsuO3SN7xtTX)wB#mT*>44eY|0Qre^WGR-(yqgT*xMRL3I-3+b0qTCnSmG zGU@6CcTVDl($^zB0LeqV*st+rsjK2PA-H6FU9W56z#Pjev>)q=LrwhbZfGyYFU;nm%8+K7-}s!A{5bmE$>zM^ISyL)=FY?t<-u;`xYm z-XWHu=3$NQW9F?QkF7=T)(^3p$1Ic>w@|yLcIjSGPDo^2ubih`_cjof5p-v=y08TwA5Loo-Bs8P9cag=MPWfVTsNFy4Zx%tmn5Kr9{K~BIqy2Em z{*1x6adB~7n)j5ylK-|Wc-B7A zaHm$TEu`>hES3z(RZ5>(r|#uNyG}Azj%B+RnD0MaHt`Ir?ECdSy3BWl;7yF7GrGgG zlu)x*4$7_TzJdMsi>%+?d&T;OM9jbQL8>I68(vbj9SMM!y@*gu36hf3W4rjQj%;(p zP33i+_xb0$(jI#VvMjW&hTGv}*HdUl?sm03$P=ggjH`pRL!vBQ3wy91Lv|hbol0@= zmr;w^zvN$J=dQd55l|h}qQ|`SD4&)l#fR`ykanaYM&!%c4(b%Vl!ghj$`J;b1CBeZ zG9H&98R&qn8Jk2W0jZ*|q5EuwZ|;3)fi&6bXsXIRe(kx<4Vyq2)mC3MoJ<-B{CSl# zgER0^epBNnt(1BeEHJuos-L_sen#ft+irwJ-^^p*h1~W<2g$J>fuOa|QZ-?wVhaik zShQ?{lxwhQGOte^Tz9Zp=w5G<>=ISy0o*a}wf{SrA`adzMiW&4G=(jT0{6DzG z|9w?)ldymsHEnkkRm{z#;vCitz3>K0pw2$Ar7%uoL*CyFE&4&dnI-f{$axdlU&R@W z84gHS?(e@ko{KxK3F!k=HOCK<5V~G4-m&P@TBc37Gd2px!?N#koV)6~*FWVCp8$TJ z;63BdMOKVwvN22r_nT?4q5qb_uyzO354!Y2tdlv_>Z!K$MKU`Vz2UMWDoaC*>9t8= zhpkKJx)W^h%3CRp8-NwwdpOD}K7P#mZbJgQC8viE9yFtCMwTJ@y(3=M@^KihBn1G? zd_pV!>I3F^EnuFR=J@Xp;H~msB&FC?WY9McSG>X;cy$j><*2q=STU?KrhZ2IXOtP# zEAunko7iHkqCXt0xsmzv0Z#gkTqkEbA?gY#IPNW}3^-Ps79EjZ#CRk(UjYX*UQ@JM zmKTNeTIKEH){1%5D{(lazN>wfjWqya8t5kDu;?skJHzHcU8Cb+c<`&;robnrf}Ni} zl)(BnSZ03ZYbJif*|9touYiUKHp0NbP7k48QhwEV(d?5BMZbP}YR-^(=*JZJu`4nJ z{{G;=>6?(2x1zx6>&8HyzWXWjqSC&b)dL!>%ug}v?dS(N3TX3rcDgV|m@~i+?pK&% zc{Q-3>0CY{qC)S;O*C1JUj}8S%wFQ{QWd?~EUJ$>qhz!4XpJSNY$77MH(I(vy8&&s zvn`q;O`2;)%UD|UG$Z3H692h(xPB552uHJ@LD3gVsWsM(*m z^^&i%bl1t^ms{L1C3NCBIDc`DM-)oN_**&|^i8=5Fi_x@UM6N@gr29C(1Imj!J~5sMLpq_Eaw+R?PS2-!qo9W1SNQ z(U(|_un`bTD8%Cwy@nEg#NcNTl>fMiZMEAxlxA#dzM?l`UAhC%^^H#D#Lcv)<09Je zT}Qju6C3af1&TxBkVzs?k-}0)6Y9ly>Glg}l2(Amx(osLg~H|nELf)Xhm+4nkOc6! zt(AG@{;|Lv6LPB}I%Q&$)u7n*a%T*qKW@3l2R$L$v|WL#?44$H<-4|p|H9jXSc`)m z(L`|S3pq!OpKStM;Skb4Bl1uqt>X?94woBZD9M8n5Nn~_AYbWR14r@89v8!o%L7_O z!^d5Tro0?qyt%x(%hOm~y)=!EmfAN|eLah&OcQRss>5|YA~E6g6S$5X)7^zw)7Y7Q z^hvnFbaqDEg6U4#@X;Yi{K?>_|p^f&O=K zeZPpTF8ZGHcaT);$_;4zo(9<#ZXKitq=bwNqzJ!I)Sd%NJl)mt06`!tZgJU#y$eIQ z`udL;euz^P(?~RY`q0dIODz(reu6b-VX$k?CvmA(cj=ZMYCdKv#W95*VM9+2iI{vM zhEB}V6sWOZjx4Y{7|FP5Imxav$K@7Wc!nZUqbMPM%4>6)YUlkH z>L5`IZkmB1CJaYUjzWbNlDgimt8@jPCteVD1y1k67~~6)F@3V!DEUAbXV+`zX&Ar| z-Ar>in;rT%uV$>?-AymsgUqF;&G?Jr(MdtYNJ>f);R!NaI^H{OU?l#4!VSVj;0z@L zV%ktYYHVzMkPoBEPfoSbIcpsPBEH-xe2 z){A43b8jwwE!M2kn0krmM!|M58H<@(phmAwKA0~0&@*6!qfB8(>S!d($z{vP<~pS9 z8Xcw-W&X?5Y(QsElZl7zrP5WOtPRwmEiYWWIyK%0P~t|ixLJ!bQPN5|Z7;63NoG!i3Pp^8{9(45j*s@-^U7 zK<`mkK@FRg#_Ph#68zkD%Fb&a19gTD*Q$7@+{i?ei3MKC@~O$m#tu%T+LVVGbIZr` z`_3y~w5!iXi0x1+<%~w=qemqKkg8hRqs7bu4jN91i_nl4RP+1cnM!l+$_HnZ_?eu> zE6R;7HtEbACuTReQP7We1E~32l;( zi3m|3_2TuN5LOYKa}eV;5(TVkI&n(aD+lynz}5u8EzPGfW$hgkJD8~`@%##m zB(ghdSsyy?HC>Y!wfV_6q!r;D^q9pDz4fV2oH{(=E23|;j|9ejV5v(wlagOZ;h^c|J8U^=W42ZufS# z+)Mys=D6-3tMt<{+BJ!_s=$IULh7eVjf?DAy5Y$04zHXjt5GEOe#OxwwhD63#m2vY z$TJOvR-2W@BHvu_&;>jR&QIOYgj$cM8)gxw(F$bV5?Gq`2RUv2e_8aJ^2r_0?&CV# z(g-n9fN@JkldH~q+n4>AD-Yqaap4uG(Sw*!xpG%VCsMQcVBFZtlFB>RPrrG+ht~Tn z@zBrOIhxN@ZF&qg>&3L#Qo3k+M$#5QW!e53I+hnNlF)Iw>9O_LWNE{(%6E=Ff@d^{ zF*~GobQSVq&Re?|A+TujIc89+VtaFwpWvcntij)hH*9Pk$Uoy&yeO#4;gd`UJ5=6G zNAoBm@h~WuHJx0}7zJBr)$&J-0~U`1^U(Ojdav<%D*|4f!($~elBwjM8G18-ahlPy z62c^%b83}k0>7K>_%233HFUNKhwr=#+R+b|fSc_VV3BS?@M+y<>v(c^uy`7Tr+>@r z-h3gV=g$m%uTy*iS0o|$fGVXs^E)@8D6B!E(2~_$w&~X-t)Gxl+Hbf{)<83N%W3pG8hJ>#epVw{yIND*tM?{NU|m_eKv%=q*0 zk+B=@q{~r<=A^$vb#viCXnxepxY+)1CQ_6-lA4q3@|PjE-s|OglER)hjP;^$^w;b zqsSTJzvULWTP=KjVsT733cN~}n0|LVPy^Kkd6nyaj zXUgm1sGm#dl_XIEE$P8ch45@Ye0nz$4YEkVGqoKZv?v#c@Ic zQZKQDd}uHkmZlE;9(0wH4=MUME)6K)u@)9|7VKKA&|06(@twqV|C2O|DjWL{JKZ!={wiJ^4L__RHv{0O+t?Ol^ zSkDT$w5&CQc(pTa?tUsm4g00BHdl2z^-<)hWwypPpjN0xb97?fIHKk(!2%uNqs@@* z+^Bxzb~JB;S#DG7Z+T3pe2thwVr^CIaiNec!J1h1R6Qf?!estmEw|kS+XcDUX-tUC zR-7ny+uKI23_z)0b#X#l|_n_2} zpW`mo0Z+IHO;V~BgPhb^I?;%LXbtQ2=0%CA;R8x$o9>j3TmHVvB=0T?mOUmKd}r8N zy)EPTsE6VrrK#{>+{+$WmUlW?z-vK{^| zlMxT`HgmhIhhej{30~)w)IeuUl*%qcNnCt`)pAzrHcNf){FA@DoXbuCEZYOga_N@R zEdSYvo1p<#5eorLn1ir3GiRR)K`~9y1x$KRe9MdYpK$Kmh<;@nIzIJ#GcZIP`?to6 za|p=XHpZ9F%{lZL?rr!Pob=&_e@h}4&WaLM<>ql>R`y_Zt6uOg?~-vUEO8cD(S9mn z!ef?}tGZb`ruW%*OpTBRggn2N1TB97KIufuG^oX`SPjRJZ_+aE+SVY;Plv4XKwS^OP z#bb4g4jl)9-8;yD^@3_+&?HgqiDHXG5yY%;5-ENOsXwU2Wv+5OzaBnGAI>mhSF=$#FNGgLovo4;cmlW%@xTXdeqdxXxXj)-LJ`cg2wTA` zHT$H}GGPys&B~c4v2V`2S?33m+kixtF!vijsEY+K?8M#mfUy6 zyZYWP#Q(EZxJ;7)id6pJL}~Z8TE%AN@vunZa8(N-bPuND;8Ho+%!RnZ;3pv7>5g04 zgk1z+@T(_^A^q9F=aB&G=HJvMM39*Ep2*SE#`i#JZ+UY8z}GcW4>>zbjS`i?1TEJF z+c+PN40)$bftm5(5H(SEkq??J3iJ4)u5c5Q3z=bl5y0ogXJ$LeOXA!!o2of5_S!8( z?J%3XewGFe`4ERVSxz9?uawx8bUi`MuNvtdAp~714@!RismRWQ7?uo!*Al$#=&|A0 zYwp5h%+yBzS$F=SXa=~Df|TqUxEZ0qJ2dBXCWs5QjN&&f7yqy;+c?VBYgh;h=s(5+ z&DOQA1JqTmZBrmFqpl|$cleRRtIQs?b*}Kl=alkim$BP!#PyOC*)7D}EoHe1AL!2n z-!z@R5MyKBXah3M5UHj=4px?Gc5I8O8-fBZM@q&dv#_k1%_4D89Y%Yxb$*>WKoxD5 z;mR6~LA<>l1HM+B&m#!?7Q`YUqY5^OWs9#&02CIF?}9S(xs1NOSxgKj@33+jm}1j* zEAJuyr>nC7ilbZpHX7U=76|U{9w4}Da0wpVoyFZ3cemgKcXxLWF2O^9VEOjV|JJ=9 z?^I3I?#@%aXLkF{oH?ia*N!+qXKk$^8eu*kCqB$!y>p>T_l`Q{eJ=aNTi8_3OG*g& zyorS95UVX~s5&>Pk`J`;tOy%<0lXiq0 zj-~9?H&eR=0KI4@r>-0SVerkaJ=kPXnCNr z7<;%ef+T!%AF}8Wym%a(adLlX|HpE4Le{Z$HiR%Stq__Z5BgwSLE=6Rj})aGBx4@G zv0xFT-J@vFE&$qhypE3dI}*aLRHT0@0Nq1BF%3R1rBZc}?`S}-0vuiDkKGwJrKLTx zSp_VNE!VB_ff?9rPo#6&1y))3eEqjD68H9&M-W)l1U z{1r{50HIZqS=+FwvBzJ*`TTTIFw2tyTW9B>iUiAh;N|AvZWh^Q^^ca(uLiqg z#nhhEn2=FCf{?x(mb2qQz6Prg?`KSgMs>&2C^HDRYP8!IUu62!B`4A{0Lo0+_rmNw zTbN~A#%U&@7K>SycJwwM{4{TDM>yN6K7P^S00ejwIrlGqTTw@-An;6X-Gxa(H$F4U z<=@@51JVtPHD;A@U&PZv5ywO=wQ}b3t*Z-ogLa2X;5NrGj-*vhSdv@l)4Vz}TP600J1UQgL8&MmHv0$M%zt3R z^IL8+=@?3E7x`Wn`8aK}xY*mSpb0V^(ZstkfLaOXjGHONB)fNgQekEr5en=Xv&@Ci zKg}aV)!<6i$Ly!9%`ho2LvLzFxOTeu`*2uS?j9@p<8OpyCPNUYUviH zWLMQ2NHwSrf3uXutNs!Y+Gjg+Nw$z>#Z;_?_I2Sa&p|rBH^on+$ag9^#;VEopX;uQ z%qg?TK7C@D3~Q~y6#IJpAb9bOFoP(WQ?U&s{Z{6?mW@@roUoz+=sN#47 z={?6<4GCQ?Qq8lUiuvFJ+Toe|HgRwS1@))MbBbMV#Mg+ywZQ#T9DY(p_I)NC+=+rm zRxC8dEZ#|xk=|L7f$8T8kVy)$h9}~`lszkPMWOrv5S8u-qIbk@+{Ist1iGuR#1?92 z^>lu0z@TCz6humGFm}hnY^%#!?~YARuj1eyYv7q6374^)h+~G{*LtZ!p$v6^(B%ow`C^DA#nJ zjuDg`^XJ9)?Y~2vony4dV$n%47Wrs*O+M9~A>82DyIX~bNqdbvtnr_We-t%bUzN9! zB~Xg24-D*j-#h&-&aiA>_wEVj?i_~DJpGifp1Ou&+ye;Nn6tEq5eGb%^wL+@p~hRm zM3pETHD|cY2(;_UdB=r5lQ-m+f2o8n6Go z1d)1cFKoP^RvHlR;4EjYkXK?#xcl4@-+7QxJ6SkRnH`<0{2LLE@D@pWD5y)D{q7N# za*wRpAKwc9nqF#>YC=pQ4%vp}_B!-4+!(LlM{FkO$};d6>l+a2nRJrZ z04ccZU+b-u16>8h;Bne5c>U}D9_iGgiGj9C^U7d_19zg$B^(0!@HqL8tI+i*31npm zQ(^ct8AhzVpYWC@GA?lK1q!ZX>~H5E#4tk!@cbY|lCC$JCLrbBjVE4YtvmZ2)t%>d zcl$x74JO9IIPoWr(8$o!y6Q+F@+_`KCq_CZ?WLK`WHC)UeFQY)hZ;WI$T91yRRg^? zI0&Lk#nS4(I4^VQhqh)=5DkXzyRI|Zt!dVsa%So+Sf{W;caKuUQ$;#0W2U(0<|m%R z#p{}cYh?2ua1)Iva&d#dl%iJ*7n^X1j!2h|WlXNlSJnqUg3m-pTkD=K( zBM?=cqb}PvDq@#bB*W4orBdJM?gPxkvt=e6)KE%XSGCI~8#5C>aNnoy&!a$nztsM| z?ZeX#zcCDk*cG&7e%5WcO{YuZLnHrh6#Vie5&VlelU0Lk!&fCW5%+g=Pr4Ytl~q&O z`}ZC7@NYFrsdy%bQ`Qn^DY!W*2;POpYo@NhbID?7jbnqxVZ?6tz< z5^nrR`C^tPkJ(@`g&a%MHW$!V1g)3;s zr>gTp&9iWq;p%c3?iw^tlZ5${ATLRb%%g+^Xl5ff%b&Dm?Ls7vo zL{mxH`|;;|K3U)?5a#K0FS>`dr&v_21y_tD(Tvk_g=A$a-!y|5$Bp}g&76HlV)b6A z;cU`ndT~Yo!hj_&6+YW9SO#$EnJ0}toH-io_t_8sQtMEMkrS=}`|`j$@bHfbq#K71 zem0y3SBT3ja@=j9OybrXqbj0P*pyrQK-cL&ha-WL{~lI?-kxX}W`3Z02{lA%2gzdx zk`ggo7F>iNBu!y6kzSM;hh-}}z2PX@!W`_%UEn0cflgk`k@6WTDaef@yF373%BstN3a?GTFNstg zzR3N`UAK|lhavBpe}vT2Ft&GbzGjh4Na#~<;1s3s-i4ZwqEp6FdUxaLI}Az9pOwbc z{YtZc!ciSpl*2#Fpo(!>KCvnfU55vr@DTw7+0FplgTkY3qw zC0}Qbu*}e0;g@Dyo2px}X`aRpBsucyEDO*nrQ@@_J5*Xx3mpZPR|U2soAf&8WZF(U zPB@{pWg!syoD$Y7l5||d)Lco9KaZPgw3W9ll<#b?$}dk&PR=lq2n}^6VYUAvsWlqeWE|w&nLw|SR16`h!<#7{?kxxbl(N9n>8w@|6F-PMD>~*=rA@g;@ z+fjPep3#nMU{vQ0q#QR6_pyqcCD56MKF?l&$2fvTewv6fl~Y&GeFz*06YruO?7blt zoE6WZW!}XcBw(i4`+1gE6GMNRi$Pr>th37Vaht3Y&P+{*T`YZgi=s8`p~Sv|blskT zOJlUB1l?qwxNT5;v!KTNWAYuMSYaGCy#f5H+Z+Y2)K^6-%41J%%!0I728`c-X4xdd zd8;@moeK4%R{k~FmPYk`q8;qb6`=QtB0%-Omt?<-rCAkzKn-G-Kk_DG5c>!8NO-!-t|{iSP=&E7NJAZuo~*`1$MkDiavJzUDik+pm8XU6KgWk|Js zeTp4_|H)?EVLau+^Bq}AMeMpVQMI2D!Iw>TjXrMIC;w8aB7XybHV7BvdEH#Tr<&^Nq2?z{19uHQM@%=_j|OM zwi;Kjq)2!sh~cH8qU#*WEf!Y!ZJ5a!T@IqAi)N46ia%=#<2+2il!|$Fk?Ld?R41)k zUH6Jd1w_^o{BqRBM?QZTdY2QCm#))kyxcF=G(By34?C5Mb*(pq+hlcB4ZI69X5co{ zzK@PPCup*GJgl2d_a=WwW`fLuPrX+$oY5Y zZxl3au`TkK&>DU3l=u+iBB5@x*&MMp%1PnmPUZ8aE#L2)#m?+CK0>plizh1dE_G(d zZH01?%pV{Tgq=)L5{nZ}DS>Jn7ScYHQ~}^3s}3|zp~iCu=>8ZRWTPHY1>ey=rjWO? zdq?i`1vj4|mn$8&`nRZ^IaDDXNjtZ@+sX5WxHzR(k;;(2#**DZd){%3v;o+qv|HWq z?gO(czYALlvWLN?<*f)3+az+5wJWrHcr>`R2;+y~xe{1e%P?-0bYQOH2 zCvqi5lnGaQ`$~%U94M-#4N>SQ?K=8T7otnUHp?igOtRCeF7Te*j+cz84G6AUC643& zyPnFvxp-PBI5w#e#GL#An7Azq)~p%oZqROMDih_^FykB$+T{hA(TF~X5 z6inmPyC$MKFg_X30O!Ua)`K@6O|Qa2b%Wf!rORI8GTXn+=bwz$jM*05OgMiZVlaL; z2q6$$cyj-NR>WXd`(F0WqwjuR55w5|hBo+=5@dMrXF1F}@@hl=J&aGJR)nkhACfI% zI1+PbESeLVLtf?>WcX{SD^ClpF;@bH*6LleYMfLSfulA2*N+j4MExh*S7Rfrfk(6a z21jQ0PJeXm(5A2-e8W6N2FR_`en^7%PAaq;r_^M%HrPw+%SL_T}qTP$a!7*jE8;T|C)N&`P9 zb_gvJTWX)}jzG;$b*MKo0?877{Wx&_nW9$MxzAdec(o^ zK=mhAG3dBj)9?I4`1pK=y=lPEVdP@xYx9xke#6ZTx3h<&lM(9zZ?u!@BZ;S9w1>uk zPUk(Q(0o)7Q%5PoG!e>h4d$jIs%?qpgPx)@DELN#FOeI`&wn2GO zvTN_G5Om-s#3?6Ee9O!oNo|xuQAb&EjBdTs0u53=WR&AmKY4HsEEAD=0c2gxuN=W3j@2#ht2H?RO?;w4QFPg}B!frtO$F3y7Bh+1 zssM59Hv5K68-V9y|HrSAWQ2BOq2c{*td$)#k9v;9t!8y6TixbYCByO)^*O!ur*tEC2{;4S#r^pse3p4 zeJy;m=FjJn)!V@?lXR3}ZUk@24d2zX^egD`X??@;i9D`?@QkT_>#*IAX|WEO=5X2< zQ`i+QLs1pUluyh(2Y2KV=l~BtY2G{Fjns!#bs_ASPf7!hg6GLY0^VVo6pZc`h#DwS z7%fM&#q~*9-3CS@FGv~r-F7_s7~qw^AHdJ^h_=JebX#8cXGH3dJek^=WJAF}_p>kwwCLsAZ9cs5`TEn50=Uy6Q>~Bzx$Q)8^8BY#kq{vL`gaAsR z4x?Aer0*ln?Gq70*ck;gf^G)FM8xw2Yc&MJ3c3PHx!CSC}u%2w5m3r`6!JF;HnqIP10wvEP-y>&+YM*pX~~S!w?P&OUNEls5ePsxRiqI-&c_K zXp1YM%L}i&eCw-53Vs?eU|($kFR-xxU0^}aCIR}dFYw`D@Zb|Ib`nHWV=Bmxm^m_%`=kLxXb4a0mBDiDo z*PK>B$x;Jp+73K=7Yz|i44SCRA{UKiJmce=aW@ua6N}1$5%=u;OTJAbb_}N?0ZP|j z>YicB8R0C@sT4TGzR?{YFqeorIj&`RyW&ksN$Heg1tzh?wvbrf6XXx+93aQmVvb;E zhEV5cJ$!P>ov>mrJ>PXS2ZtMy#fJrSpg(y5w*=)b>Eh`|i1_MjsCS5fQ>#}r(!SzLSmr6*v4O}HAlh~=MUiGP7_2A-bj@n#9B7F)? znIl)3wv}nzpKSKH*O^dQ4*qj<#)N=?hX=={i9$F`YN2d`-NPCrk}C&PK^JmQl<8yr z0FQnh5>6`vADoH?Qx`usM~_$~rd``kspjayF0Md5#`3_zTofslj3m;NOk{|}W4Ce4 zqUEMkA{MMn#=POOz1}>1(p73teF^OLg)>5I<{F4!oC&St)`@8ttxNwNBom`rQrJQ` z$OYMWCNmwr8~Wji;v@x#;+_l*^$5Mry+I|HGA^lY76mt3T; z#cf*{NmLy)jSyUAbPIHU9Pge%}GQ7!MVjn}MLb&Yjn*r}iWK~m8`$9j&-lN1KnzoiEHcnbV1(wv=o z*RL?`a@M^sD2g;3aPpWf(cjK89%Waj%5D?4pisO_ceawWaWylgSII{6`yw#S;OChPuy1S$CR2u6&87r4R0*oYx{4o%$9{vu z^KO)aZ_h`HSm+pN=By@#x69v_Fq5CJHjMtRAZ?y4PFH0nGw0=~l#F!|dmUn7^L>h~ zgz4iVH=e8!rR$tAGPgGj6mh+BOZ!2qlY$P;+C7)oSCq2Yq=EOou2&%It;h@MiVx@D@XE`F8WbfI!MsO7#yu!SuqSSh=%Q zU3s5;2I=_6)e&amIQ+YJ1N#geWd;o$ImjY&23zx7QUopNc9t9YF=E5*g=-9iP&{C# zOx}MugeuBd72lsEySnGE)qysAZ&vYL`)17XJE<_QwnEva^q|Ba$-rFq4-Fsq^vH8o z$r|0z*Tj>ufL=XiY*7ypZr5L{i#?K)spo=r{H?RGDSa`4Nor0c0r#6M@*5V2yC4JN$GW~B3Yc{Q1>)(aWg>_GoWVQZ>A zP!onAkmF9NOF)~@@=BB>nGR3KXyO%X>BERck2S)A3NQ+T32Ze*3RqQW>BAhkz+9QE zH(m`A@<$=Fffye4d)dmIoj4|lcvf~-a@TOxa9vzpRH`BNzC-udzELOPzpnf83jD;? zy6D!HBdH2KKNZnSf__mp z*|1`fnM1?5jk#eTnJc4eG|Q(lg|IuTd{mDstF)=|*SJsdJybWNdN-a`Jy1)E19lt} za3Fo^0W(%CJeGN~@9MMt)<7x;RE03XHIn2e}50jt4q62xIb4 zTq~2iM5HTF($==l|DM8=2l9&43t?^&)Zs0|m^gAQCBak6D=aw=J1f;TiL$PlJwl19 zh|`iBE!}<5uiVrrU~;uZ;4q&a;N zTgacEBuvg1&+Ck=S1Rog!YYW@HABcts?MGzlKA-XQKj6ugWb}rV%uTH9@&lJ3y)-c5@)n{Kyq|DU0eIjzD$hw)>u??vo)=H3Mx zkv7SjZ;U~TAGo(Ck$Ilo4{|teJ9qj@M86*x91)r2s{PGX?NtiIQ^BG{OpTfgZnmd^(;WeD zFwC@SIoiN8y96CTiHe~>v08dLoUYD?5HOQqbv{?^08NVE3?ONI*z_n^8t>uP~nNUWi z@5F{d^>l?CfzZ!dT0a*b?1Drew!zkUrB0+Two+BO;j+`mbP#%$)I!!?G#oE?z%I_5 z$^cJH6}4B4I@njiF5X4jb;)-a;+;@7Tr#2Fd zxG@Akrx;bQ=kMR(Lv%HU;LqDMt2y>{ojS>QXr(r2KjAxpeAeSL559=o^`3`r&^Hs@ z5L3}VZM++-Cu<&e*Vbz7DT<82W1n)>pWyVY4xA6u#zhgNNwU6Fp3Dy2pmL{)ajE5~ z-?VXs0?Yf!@2tk*i+9o7o2IGaMZtPwvvW`a=_64(f`F{1&3Y{UWZwG*;$1BFmQJ|= zJO^;08@YoyP}brk$~%YYB)KG`;rv99D8KXdY(|1^(C^djh-c&QC9Zcr4;qd1(+>`{ znq-3|tY3sRVXUao>nert%@I~RgWNw?beLJ0WEu#Xc=<>=(i#AehbN4zl^hu^*TPqU zI>p}}>YRR3gg$>P7CWyuqQq3dhSSxJDzxely^&1hVB%H~#E7PfbPvTO9^|06P$zi6 ztzOdCAAZ_5ZQpmF+B2Q2h>jc z+@|KHZBhL~g!Aw9D$qI-CNESpWjoUZna69R^RPEKfa9PbEZLlWds{lWg$iYXj*-=K zy@TJT=?h2s@{4Xix@vug(a5X#G2AYa*;?<5Z#FiA){sw{S*qKPgSr%Grbv76(~;ni zYFbPvl|S%W()m;0V|B0BI98osG!vaGgAS4#jCWIcssm$!PZCaYOHvTgqWA0Wg%NL# zM7$sZ<^r12jGsRgN_86!c zipisAmYWA(hz}&s@On;ZsSvK;l&{SpP&RC~&aW+Z7{lYHIFCSOkU#MUK3f*C5z3Gl z1xTcGnuN@;H0z6wTiMke;jXlHq!S39*IBwRhsLj!vI_lJOohU+;0RUxLPOs0fx`2L zwzaUGCLSsfww_hUg)rXBDsD>J@TooIX4#2j3loV-#Lu;y=2Z_56m|l$?ba|Xwx0kU zg(7-@TocLDwrTHcTWk3Y-0Ih6 zc7HJ8Kyb^yA6emORq)dKB!i%#`ui}}QdK>xos_E1V$QXwAgnIi4;6v;mIQL0)P1D{ zR%{$TrVTtR@;~({c_rUbouB*lYOq!u+b=yTmBkA<8&+|80%o#JB6T3h6QS zxMM2}vCT2!*|*yfR5eAC290*Os~C%%N#ed^clYVK)fbH}sWRjY$JRjmx2R?3)uj@` zwYEG1Bjlb9xs*TQIdF)idV9Ps?B)LVFpnTk;mih)`N4YbbIGpqeuGD8Q5 zA;@UUinB)%kcKx z3iC+!GI4$c5BWNgmO^*(TJb27oAt=;g1p)4mhi5A?P6oPmJWZ57B80TS&u4=SIZ5bkMIkOK zRsMo|?kwAg=>z(USTSq#Dtu|HnAHV+N-EXUe0hI3NB^qohO>N{PVOpIl!K|g=wX|4 zCh)o|%vi#lSY6%Fg*G2HCx>yelrU+t;cjDY#-doWVlNRMSxTeQdnYJwgefn9by|zc z;3&NQ3;r{7(;n1q9>J&26!S({Cbk?TL%4!5NY~3Nk%quM@(#JN`&yLTLfGM1dJ3a4 zk8h)NmZ_gzr0Mt%J5 zi|f$=d$?{0szD-qUVNcM&Q575^mn9#mZt~(XbvLl?W_dg3=M!#w8~PJPwZ)8Z+`wSUzKx9s)w+`o94gn*&Y}Vsi)7Pyv%W(PD^I6F-ri z#_ufhXR6d+n>exO035*#)#Rq};R3X~ig*~G*tm1*B*Ighth3fDuMQ#1xsrG^Nh+@Q zmpr06By!-=$0WA28@nP+6>}1TgeRtYL#;cbpSCq_jbrob=H8J-*SN5jNs_5auuN^Z zKL-Kp1GIAb)SzB=Xbu&2%O_0@`&)S9jyeL4x)Ll(ZnG68Y4T_ICVxIG%OQhnG^%N} z5x_Aq{^G!TykcZh%*hv;pC)h(Romg4Qogl1UbxB%Jz7QFyoN0yR%v)bGa*k$-GN=F zYMpiv=&OCMAa%|hM>8UEY)`uRG{SE=d<<>0mZ$*nbj`CwHxlCE`qGxhnzP&QoGGxjpnBiG0Z_|XYmXKaKnA5e9W)$A&GW_!4oCa+M?dGno9jX4fWh0jg1 z!?~-r=kDP}Z=tcGusCsnm3#tI8QZvuR>uKXj->r!(lPCN;i(Zp8Hh`!1`fbmxsD;_ zDG=T@k4LsPdVJ|M!_Cp>l#M7JEq>3k>z^JV?W4we$%D+Q(HgakXC}7Et({9=i}7Qn z(FboYHM3C@);P-~PN{=V;UtK1V4bHl8!EEvRIMK)!Zt2!5epRE|DEXR&FhM2GHbj^Z%`du zdn3I`AEw$oMs}pxnru;i!{ixeIsCnuIhLZ#++R^Ry4+{oApg#b=>A?%Jfy~ITguS@vgrw`**j#dy4Vd#K=?b0+v-oBNPmx zfOlOsUVsIuV5^ex7vqQbZi2ARSGwi?Qa3GB%)?sbST;)rr-XaE)H8RPjk`fv?JM^lE;nww>{D?TK)xr26=2X0idDnW*UYn;oBgYA z0*)vrHD*BK6HP2SgX@Ere@!nDMjZ%`fN5SqQe6ufqAg+((pf!d|AM#$q?rSTn0p2}N@Zizl6wAV1R10N_Atr>O z=5;b_7ojL#mu6heo`UgD6vy8sEgj{%x`>369ocNoAruT8O~B;Q;_~#4JeD8O5&|IN zw7H-Ch^OZ4Cs}rQOya$xr==8A>|c-Opiz6SimZy^=2CLT2%(WyCwf3_8my{&hjWQJ zath{7Lv)_ISF{O4Hz_KtPECdM>ZHOZE1aH1I(v?JF!kUk@m1;M0K(|;rs6p5lU>Z~ za?6UI^a*U3TVR($1yij!MY`S)iN*{`4J)rPQHme4HS+xwJ$!T~ow&#JRY{cG5%LeZ zYk5(ZIlY;NhPGr3>rdK84pS7tqLb-}-#Ydu5F|x?+LPu!&2Kii|0H4g;oq4?=drT% zeO{AGZlTGeE-$Z&e<9h2Zlj;%j^f^3Nx9o(XbCUJBpWD0i?gkHO$^&_;JJcZFru-- zEmypSdHJj8&*#~1F?D)h`X=BXJLfMNIn*5zG^A+5qp{C@W;JJ2z`{aoAO(Vy_dII6 zDRqe7RHq_+yhDz(Gbz@ae%&_PsjwtfRB{K-D(=KGTt11(cvl=boOYX%(PH{n(sG$HDae;^!p zPONrH7diaW6{L@HzpjUFzx%Zi%JwiC_Q1svn{%U(Fy}M6i0ZXTQ=j>g#`DL1rE3mY zxXo)`>>u=$CW|bjQlDNzG27^S)Q*^IgSoQ$K!kp=8eMixX?Tb}qiLar-%|m2LWCu( zRmRA`BhF{UPa1mjtZ@y5elgn)-#*F1-AU_Ar*!apdG=kPDQlf4@puNzIIXQ{lSJf&rdWbaDgilTLo5nBYQRQfv zscb7JhWpxbXuj4ml0DRfZ?Ux&<7%a?i1WX{0asLt%XAt@Jj1ir!}c1#HxeqLi5)3e z%g_wvN%afAY`rJC#1_isaVn2V_$1WV#;gH5Yi_vzncem%%o##u_J=ZSPcqu&UIvqa0qOqZObM56N5?Dr zDOepeT69d@pB+_1{mYj0MrdLS^!BsugFX2NHfc5&tkY2Z5-lkHHkwuI3k+>h@-Gl>MEDZ!5@gUpWPvbGd%h*HSVyvWlvRcKdXZl2X zYW3I49wC>nmg=(ui@=!|L(CIpfl;9>nvO)RtesV;DwZ|eN36A@7X-<#TSHUnzx{#e zKZ^0OKWl{pKMcDfP>8uVxs5siN+0#Yd_Rxywf47ngn5Yg!eyI2_m=mRfs*t6zpLN) zN=q|u_ioygB}QMgg={*H9+PiF8ulfsI=85zC13o!t$dS&zKTDAJ1{gI1$Gq zUKEWD$E31rugD`XgHF{@_5;BC8iN}6p7xmmXvrL;Su>N)$)XFo}Lr~1OJte z2eRsFTU=eDRIKfiyrkej-=!rQm?W9C%)Ccac745-b=)4Zl9iX~B6!x{$XC;E3kAMn)?hzKy(FkAMk>o;|RLvzJwHI>b=wCRJABi&?y08+2l!8 zzsGNBYJ0W+)99qQM)}oZ1--%4Ggg8>j^vRcHa1~Yk)IOvc6=f^>v*u?!2?{AW0FQM zhwI@|jB2=@8&T(oql-wN#9${YKD@yCMG?cy@85`OEC}yfpih2Afxs8XTG|@pcAJv^8%^8x*kwll6!LHoifzy(k zGpfjG%8JqItmuh>gs6m;GJ=XSVv6ZV>%UVPJ-1<9*1$`u5{aE$Kz*f!576t?v!5H>_(xbmBGj?SOAl*6?KiY zDX;VA)7QPHg^o7BWdOH<5NUlN&U)I4G--;!&xph=IVA_7k)xEioR}@?y-e~{j=Ei! zWi1Tb^G>a$yIqlRKtxR42^)t`qD|jMo6@$fg3D8be@xO!Cx;lb)$BOHQZVIO1T~B@ zinDmf)<&oE(aTIA%6T{=7KB`Dz}88$%MKD#8DFQytns@WG`NDuTNCw-59ytQ1;cvN zn#sat@=XOo?RYD{ICXwTe8F+nGtmEL&1J4ZwII8R)g*DA2z+<1SquTY8FJl_g#9g9 zNYnE^aIv4Hyb$qAk^sFGu8oDY(JuW zKYw1ve9)fib*nBum=s_a7!k%A!1amNEx8NsbB^jq#AV8n0mwCSkNGcS`0O7mr%TU{ zLOq;#0 zEbtf^10*saiy+u=Ju3u$rw&RSP=ibZ9S*QTc7X5)c_EL$>i*P#idFL!XmAkWDOmHL z3C#U(BZLIy58;56APGUV6X*a-l~>;a@X{T zdlM`D2K<=;23bIVW^n=kjXnjt*xR$b%mzbBfHxt3Z>((P{XIkU?Em|^{|0}7d%nmU zkiGCPNIK69_;;Yp|2UfxFhmZDn#Tpao$mM_0s%49@^|uM8w~w-U<7VB{x`sVKgfDP9`Mf{H#Q*>R delta 48830 zcmZ6SQ*b3*w5`*zZQHifv2ELScCZ{p zQFDHCuHF`?_8O=}J_NAsyreh;1cXk~cXcGfi>z~kdyhMhZxD%uNzFb8H~t10H_#9e zODP=Rc!B!93p(gu*c0hgQnFeCO$&WI<7ZFLy(cQKaq?~mTqm-uie*!H3USgDE-+%yVn)QK(M8k<|aAP{YS2& z6Yj5GiW^hpOGoEMAfSNFH&o5KmWqi25BgxVCsvYPZWSG!zl0!Lgzh(9Tpc2MicLJi zsfsA|E`ynH<*Zde7c?m&}xuK;t&kYEm7Ff;fxKd^V{GGCyQJQq0crerl+dG!4 zCl{(S+-D~#R#{VZ_cOyyy0tQ>PW>6~Io6VGt1YoIs|#=7`iVbK`~x)()nlFczgN3< z;Tt%t(=7D-q3SSP!A^g7|~`J0JA0oem_U^DFYrc&t_XWncBa&#=CYzZ+MFg{fi(M$c5nn-C^< z>09)`BZ`hz;(4%$Eh2P*rw2bI`ZHb?`XvLEe{6M>_jZ&YV71dTrH8p<2gcl-;#~Th zFc5)ttc23A*$cQQ{8=qBeWWU0%z!PCz5SG|;7D68kF!k$DK23Tl@d<73u_k&PEj4g z1g2L~vD?{TZ1yj)fcR`%zQ(S|PcR@T+~m6|x6Te+OiLV%QLTKr0ktNzz^oSZ4aOs0 zX5KdITJ7qpdw#eLN`-M&kak<9_#nkG3f#;BQtNA2CDUsgN=(&aLO_dk43Oyl`Y(!$ zF(X>v(4|#QCp&gfRJTgrbXz|=u5+FO3xwlrkG?xNZrUJ5<76E)NgFSFR?ACj91maUBS5F;36d8C}Qa?wm;bOKS&pF>(WBonH86Oq90ldt7Wx4P(Z zVj)`MjnUHtcL;}#dj0v0_!^9>LI-FclE&0qd3R4mbjIR0O60`okwDDxSK6~}Un=IW z@Gn`T8$M)szf0(u5+RU^LByYrZ;&47DUU~DQRYU!;dLgbxNR&I(Rgcp?0J@{iiK4<(xm{NbOynj+F5tc(+X+Etk4r72ZeQ zmZ~#m`Qv#)l_N6?`3hKL0fgBrtFra%)7<~zcWN7}gc3>T2g`_$NK(1GO9egfWl9!3 z!1nL!0Xu@T$eayLm0d&na3*g6IoUA(2mH5eTDj4)z-V&!O3y>yP|{oq_FylJ7v*B_ z!VK=60e#3-)ig0#x}=Eql;jB&%2y%&n^*ZSI5kBmXlw`wczD$RVp1H!F4W4V5qW~8$d|sCP1_Ul!TL{Q65Ha= z1KK81__Iw4Bkncl4|5T*cHd45*aTu=8otK~I>Dv!nkLHKJl{cuN>OprK6$uwwR=1> zXc)YlXE7T431&(S4?vB+Jth7JSBJ{ zF@g8&h`dI?^6Yr*uX`LD#q;h+oXHj)*(vSMt=RhTXC7%_;F*Kc^iLGIvST3rXJkyP zAQhyM5e1PX?3%krS~2WIuIE~$K{OXk*IR&MEp2L=0KZ!XN47& zC5sa{eKL}L*`f!>h^)L0^^K3-Evr`11QW(6G_J-fN#zsve|v_NuAdS0KhM(=r_3TS z0ku}VMRc>X{yjalvHvjpz1#l-!yqoM9~buL5N%@EEcu@wO@EX6N<-!X-)6{g+r1C? zOH~$zl!L0ayiXw@9zjP`|AD z2T+h2_A*#L5k)JqO|`jym1mo_yTO_8-4ekDFAQjWriar9Zw%b*`YzqyR2{!42W}6L z?V@Qv$X36FhO_M_uzVtXY4HGcWYtYM?@n;PAc6L~6SQCB1OA`h`0q1nn?pN#uRz03 zW05aXSG`xbz~Q0N>^IrA1>L1dU6juhjpG9C2e#@r-KEbQhc9CuQ_U$LefZbH5Kx~# z`XBF|H`YJPT_RkceQT_pLtVd3cZ{W)qPWuK0da&ZOfWSih@qUq+hLruY#V#ycQ=ZC zW2t(!lKpXg^H5?Xg`%>)&A5l80Z>Wb++0g18<(JaP)5p9D?}ZJ_ocT_-pS>z;rU#d}!z6;R=`6pn$Kmfk zeNy{HeLqViBVgv)>`osz*M5&dkLGzz*J=A;FTTWI@BDH-ERy;+7L>cb$PA1u?H!ZP z=&mKb{T#cN-yu-#-ogvCwHbNphI`yc4e7s21wi+|meX0SqA`5Or3pg83UM(hMP`iBnhp9&*43vh)obmZ zn|uMWh9fmM&+i?$ov)drN(M^Ijx#Q#H$uFhdcLF+PYmZ?6eqcS6mE)wfF>t2T zFpP#Ih?b9|2wlUoI^*5SV}YuEzBk0S%sccO9~XhroG*>i6F`6_j9g5LaXw@6Jef&~ z)uG!XUk_GdghaDiBgB9W58hyWR3I72#^YPurQm{5y?jy-0MI6;*=c8$hv2)Yim=Su zczGPYd~l};YSqAG0d8JcX<8!nwslr}y6EV4juA2?oS9h0P`q_N?(OUN`ThQYDB@EJULARU-P&}a7~nAtBm>S%nXd|bS_QGeN%S# zN93=Wy9Ea6ww};xCVSh@8J?;t~TkXc$_}&iNR(lm+5T+_V7p$ zlMyyVU^_|ZWXy&?zyx(Kit@7VaxC>MjUbaY?NK>WsVSqLI0y1=D{Lsk^ee5zjzcMU zO7d>ZgHB+8#xYj;D11AlsZ>`Wuhn@5W02Q|M}$+;kxJ*pNQz%h9NHd%A$WXCNvy2$ zbcy03UUvlbXFaE+7AphBhE~f~Vt`YpuD{bKdw$y;rAaYG>U-Y$J?OIWNOaA9g0*ue zeXDsi(ZulGs)6FeX^joeNuUxtz`)q;i6IT{b5jYp8&Bl%Cw-8UT+aY?i^{qZ!GZ>9 zh>d^E55_WMKE~<%@+TO13_}v+EWk(wqIMFXEddi}d^1#`FWE?AZ#iP|8%r;BP#GsR zRH@pAsuq|6eTdo}RNQfeHr{J5r$-!`_a9*e$bjA>P={n&!n8kkn=A+2^T_ufxTH3P zZr*|Jbdt_3*Z7{nfvgBaP;4d&o$j2&Xm-HSQ9#LS6_ z6sv_w)Whdu$#7t+1;u`VXH_feh@7~HHHpPTiQjfQ*LkWXr%MbI)+AkYdyK_`ajzg1 z0jK>$D!ZnEXB9|3P+V)nLuVQ%Npq~PXc9Q5C&x3|6RIMuo`x`**vhgnfE6a2aCJld zTR1CAR@cJVfCdR+-{IL}W%$f%B}>bu$FTv6zrTzigCP@jmTxMVzy%#!>*6Tw)YPIc zcf%HdJ|4wogjt6K( zW`52H4bbKIqm6&pl_e%%q~ta3+sC4h<0k0ooyT^}(aq0zTv0yY=I=6c+iV=(YTv}? zx;~fX^fco1GUC+ZZRr#7)!td`s7Nl5%$%H(sasw9Jj2qU&s^lZcbq1k)sc#j8-r6{ zIn+VVp%q z08~I4rBgQFH0ts?cBuXH*m;*)u#~nXzb|yFN2sMXou;j5;97rWQMVM6D_|V`8OupX zwV%mNXa@k#_psvx|<6J7uh{#%AikLFlFa*^4w!ru6 zduiM|(*p2FhQf3ub#Ga-H@XrMYqb2+qBH9ujZNX;jpn;~x(AWQ+E?d9NwWsA4nuM6 z&OGBnYQqD3XZ4c(xaH2hh&XelIf(M6(&{?L<1?(;&t@`D_xW;e@mxgGdHq-1D0~~! z7Bd3vfKfUnXUci4BA%9h1S*grFGG6}Kf%9W6-KH}JmuwuX~3Y5OF%6Bxw)FQ7z)c$ zvwoZj_5-fhN$nJcYY!(N{=Q~xdoH)eqaJbd?|(PzYFe}SU}|%0-~R!f&XF;)*X3>T z{2KLMtI@9lHF5mzY_z5A)%sQ!1)j<(nR@}|f4fW2Npad55pQ$;j03Ij@2inoyFHxL zCrAECt|glt)o7gk>PDQY%JF0L?L6aXx;Lzw>#Lds+z?Z^yW)K0S6kdW%%EBS*;WjB zca~Rd!>abR+1Ot-?+L*3x;?e^b=@Zd2CgIE5*J06RTCGV>!Pj|LNdM=2MW)&j!z~!5xY`(_h+_c_f=6PfP@KETibjmi|zM(d_B6mS9hx0WAOCGW$g#B{JL@H z!Y2IS1b#*QMF-l-tOA> zf{!YJ_&1a~C<;BXH-p^>X%PgwH|CKm>g$%N0l@gAEb^O|gWk%Dp%~$ffVtl(Js`57 z53N)MNd%QkoV_8FZz?<*2^Dw2i_sDuAymw?qCe2)IX%s|mK1+QUJj<#lXqSVUdw@x znxt1Z#|pY%ThhdQA8a?6!KMsE!o>vBI$@K7pXl1+`a==&W41u`p_XB#)|$E?+!ti; zJ|Ji|W-efZfhi&1*35DArtmy@oiT`<56*Q}R0C79&qa!Jy%Uxu7SUn^K5@W+1J?@2gMv z8P!MX<)}%_zwmj1KU4Ofn(SjDAsMQBF0j@(It1D_Kq5~fv|kwHKhUJcJh{|w6iW*?a3?>H!X(-$x9s4m0#TQJ3Xg*9d~|qqhf_H2CIX{rL*cs%cXnsk}67R^xx=Wl)+bgUK`||J)qPby_MuvP}N@ zIY+%w6i?KVcGuf=>JIyLeVi|>;)5$39RAAB6VZtq#3HPm&{I$y_(T5vC-Su|V_Ric;RI|L44Wd8EFNM&k_=n$9=J4i2R zKwUP7a*k`N+z0I&kpi{kXBd{ezd%y#&?=WjlGMDDDs>JOiBs(K5KpYJKE#G?)v~A7 zDOC?xY&_=%Fz{^I10`0U;L%v9c1MqqYxb<}({^^JdRihy(4_mMZm}BE!{7IJKkP4a zfcbR`%F1d?;(dz2eFb}&Vi=(k`$o~4*1wBz?Am`Y|GR};RAJjnsl!(V&URU2P)j5& z2G2(@?Ejg}{I^_;c3MIi2i_Xqb3r|M3KsrW%YM5HIK`rEs*W)l<8PhUvGGii9^fz+ zM&bly6q_+kSNvUnIl&uX*Gh}`%Q#lbbOD~L*GRQ+h&wRj)n?Sm>(S>$N=@+FsyphZ z*A9z?v%eqT)3Twi#8O=_VZxKNQbo*$FNi)8`s@b(e{>``k{!bY76Rf32?B!af9OaL z0R>RU&=c!~AXrp?e*J=n7nL7llxJ3ob2aqwJ7YJsOurVnXS4f3n8fmYM6X|4O})+! zMOAsl^-^blg>{g}Le4SeLY56J2Kr2xQ5#pUOU19kjLtJaH#5+e zR|w-3EF{D{oA$D4VAk26dbC*fUDiyuR2OZ;5we)PvFcnvr5~jfxa6^D?lI(ZwHx`J z(wN?=9C{znpfr44d^N85G#>j%S4SYr2s9bVKFbns($3J;J%l;Rmaik|5g3Kh75X4nsm z>diPLSS^_0(UaA4lh^-#$`%^G#l$+97>K%lNF6iCiS!+%_bOlmslm=`E5J@|U+xN? zIo56yEOF6wHL$C-JZ`n3^;$c}dD7U}9@XNCKBYd2lLtsEtI^h&sq$lyj4W%iPWdru zMyvXdJcz1Agz2X|r~9T4@o)y%3zW8Q)P!qewSDJs@3fU3GU)Xr@`zy~B~8aBq@ZXR zGP;~(&W&ZV#l~~d`7w7%1k@OZ1-T9W4kl)2kA#AhakPW&3wK2l?t2d>soPM|e~0b+ zc|-V0YC@q3%AM~C(Ov3C?{nRS>#*PD;rT@8)4z=1onA72BF;!*bz{F-%4OhfbvH>ELht9S5F3Uv@nZ(^6wHZo|!7R`&gVw1SexO{HwFnNjC48ud-9#Q#>I)Y_obuU=hC0jB!Hi z@?`-FO;#?IW9BcZsifv{yd=@0*(*6b-tp~elpMIa#ukAZU<`Qku3dzXc<{E2sxI3o z#KWQp_YY#tu|a)B<;+)0c?^4G<5Xb**+s=GgNr1hy%jn8Z)0RZf9;vwF5NbiY4>du ziitaN--T3r&4SwyA#|t>F)1gr(FnT$F)dN->S2Lr&&{8xm&3G|gA2kOR6K#Crjfx= zVJ=b~8u1oJKp=)`G4|HbxLhF;`V2FrXz#C(AR>IY_?wE_eOlHI#5nGS~BEwKz+;Un)%8-GlL@z=*Siv6;EcyX#3Hk>|cLyPjF=$`3hpbMF z7epU<#ZTeW)OmAc4Dy9@r#<`w!&x33@OX>N?e9FR8gF~pkLlV$ST2iE-xo@CI7PZ1 zm~QJV7eh)(V19}FB`z#GR?PVY63#!FnX}a51E%wgRswqDZ)WytD1bc;upOmhww`)C{7JWn&ZU)jbQVi&}|Nh^A;eSv)bd~>>(SNA^n|ewSIWN#q zl~5e}OD4&9`AMUez76bP2yM7#aHYL4G*KO;1y7EkY88%9&)RLJ@ARqLOYPt=KPvht z%Of(V!Sv^6_gLqYZ=dJ!eXQQK>~UUPWhUKcp}-5|cD~Bb!#&YhV=(Qh z+}$K! zBIWMrzF71dcUQ^wb_nWN$FRkZqx)pZjt4s%@V4rMIz5$hQubV3Ue&XP~pxb1g=6}d6v@&j5!y7Nu_{I1x zHl#ly)%Yc#AwT#M!+GESk^{cqW@hxpt6kUb-4PC&r3DbIP!Qh4QsDjyqQUHS8S`T! z&vQ>a?Fdo;*-h26GjB&Vi$J)4&p(L3za9-(9AVyiSNA9R8yRh%j5CaHKajE|3&cL) z`y{8x;+ITL!Jd=jnukKdGcz?zq#rY5;*W=;)+YPGAWTg%z)jkgq6##ydgLAq$~a}a z{u}Ba{ynJ@wlC~wMxvnPz$>hN=1MrF6f&hwqz`g}{4_bJ^I`xKm!9^SX5su-eFlSJ zpr$T74qIUuT-BGr;w^fVCo(XC`FX(Ql19R#T-)cF?SGeR7qt#4<%s`Hi0~mG82>Y) ztA}lhtO|Mxkcu3b7#E2kb?=#U)QCZ@3nhk~rOZrt-4Qv=xx>uUlxM+3N@JFO1mM$9DV5he`hD%wWQ| zRdU{{RZzFK(~#ofy+&t`QolV84W?0O29sYnS;OOjw#N(50jHr~BzW^;)DfpgMLKP$ zdf?({k4UQr!&T$f28Rdrm4y*}P?vnNad_|3*&{g|)bZcM8gctfR>_}9&boXy?;CE>f(Mrk~gf||B479N$bFuPIKdErCgz>+Yh||@6 z#AG^m7=c>A;n7=noPyt2h@w@ynvCFGd7Od~+t1h9(zjf)0^M7fw>N3PYpU>PvejcM znQ3}#-~OmBPN4F-N4rtq^(e{x3sG>O=5YJ1GtAE2L3i?h;o6r#_U9;qF*uB@<`p5h zde@c__Rce8o#gLTUQ)f}2W;0n+STR7nc!l(ZDTjEL6;m|pqgxD&|O4;khD>w>Ci6c zyL3dBlA-@R;rs!CB)OK<_8QOXHp$uf#GhHdBoN^Xk-1(> TkST!!$v~9HM$GOLLSp_sBNo*#;Hfaoz za2>H~mT;?ujDZAnYO1|K9>;!q{6$r3f5Ot6zsy0CTSS)J=@^FnB`Hxv?0B)wJwJhb zs3H;qtV$URRTCW@V}kf7HvxeDxC6)XQ&^&M0h>#8z=4}UeViul6;jOD@ig4XM1I5ic4Hz!uA^w zPKAc$uhlgEnS$2 z^3^2ML;F;JWfmjj3{VUhkKxsBZWuCu(tt{Eft?sAt;LaTyiv4EQ9(I0J z|9~;#SJj^BYx{2vv={G*KH`LyU13zADXg)Ms)R;3emXjCVZ3%lwl7=oMmYoU`E*$? zK+u_bRb3PF=9keavxg(CD?HM=>N^ZV>E0YZTFJUCyEae!$sVkcy&T z3LA(-6w}7C?#OdKGyn;;$rK6~NL3aa&D%TSd7j-xSI$b6RcZ`DDl=n$lh$%9mSNTy zvQS=(rtw+_I3$1Nk7ax*%|`=iVmFJls464NvODC?FN7qxc$$=$@m80M=u*mY2suvU zp|L2i<1Wq~`9+8XEy^Y8yC}Aai>Mg5Frx31g=#C})V9{Wj3;WjuY>{@BMq|kD0CE} zD3oiH_e0IQlXEmx7-sJ09Nal1HylAusfjU|+1Hp_6e+|-se}#b^-(|`i#ponD8Z14 zKZ931MWtI#5@+sIghH5HG7?RP0Fr za<(2@wtjr5(_nT7bUTy9&)NjTvoKFyEcsq_6STVyfQ9M(2=zpv>-)T(<_>#CO#7nO zQ}3pyFh5#M-To^380Y}*ZQKLICORA4F`b*dlkAz-l#!%K%?EWt zYjxrg*Qz`NeVxxT#tAsXH6I56UQS-UeEFQ)Wsx5H`MhO*Zb3jx@!RR{X*} zN)ghvGaURz*?5R0c;FW<%Z}j&eJ1XpJ)=>w^62~_ELj{)7ovUPbPpNrbs8EV`B zSj*C8TDLU=@AGqBoQ`TlUO6!5IvCg@MjlR1FTZ1NyGR@7_U~Qs1}y-(;yBjMtKP%? zPOb;MJqHgTVo^Q+orJH@cUvwC@mgf_*^CfGyhEW#>V?Ox!v2mR2Q?N9-tuftNfNva z0s_)LfKY%B#A~$UD<*l0LnzS2Q&<=UoMU0BJLJAN(GmKJ22lqjpW6~K^eO-34yP1{ z1T$vL0=BNo>Oo2-o_+SvCn;;+HbW}1YGLaV-lKKVCEB|JX*kd?xX{UL6;BPWnSMqh z=bNsaTN(6Cfdmc%UTZqEBK6fkVFO3ZcgHhoV6|adp@iKTwnJnyeax`_A}pY}bcyVd zPnD9MPtkB^K6RFT#Xs_2g&+7Lbr%>@zc_Yw9BaCk$4%>iE?noLx zFve%Y_aGh^={tnFO%=!+KUZL#`90Q@lN{~tn*iu7^#Q^~DpxPSoHB$uC>ZJ=*H~F9 zG3q|Ph8lfM-rx!TXDT==p#`}!uiRwRJ@k&>egflkkI&TAC9~v^C*ha(<01ArhzRNl z%Ptdo%kUyJ`~Eu@YE`Mlce?61=?tfKV4T5a=v=yQuD2LW;HgD@6*>{;EcciZIJC)5Y76j!`U_J2VO_C_3Yq{@pFUanv6g0 zCu0c;juT9qQdOGLIFHgLt-pX0(6s-xog>7T!`qk3xvVWNb#^GW^QTGTRyE1Y-~)oG z{3%lAbZF+$<8P!ZyE69*#}`tesI$3|@A|j_*Wz(0;LXdaVpD%0CKQ)|)b3%GGy(y$ zHO1uN0AlwKYF{MZ&dv6YO{NNVxpJIT_K=`(0&h*y426ssvpM-%ynkr_yS6>K)bmWq zV-*M9_0%9jiK{)Dv~%(*2%US0=D?Btj=&U3BY+>v_2 zEd$YMG>+dEql0(|2W~CdIu+KQQ)4=)W52jcjx{5`yQ#r-hT$4bI4gQztv)*4{02h( zZ@GCj_^*rqAxT~|2nfpmL6XyxXuv$(j{t04f?(BW&Iy|z=mu_vc5^bT%qU1jY6u~S z2+ooj--#rkExntCoM36WE-slfs-HmZV22u=GKXHR7~iU78f@o_y7l*%e?NY`>%Lar z@U`S@Iilf5R(j8}Ij|XvY?My$PCifpq)O)|K2#vT-|}jNS&(6t@+twTN^{#PEJ`(qH~gh2 zFY(n3vgHx3*keL8wngq91^PUIf(95x?AFQ$qBj{1Vw2i7YioU(yrNb589xToD;m4{ z?rW=}T|1{cY-flB$_q$M((?d!Gb#(LvaAV$R{5tzmQXS(HE z01j(uhK5mQm`}~SiL1Y6jQR_yW+ZASj7^aBw8!M!_+;uC2~T+`>Xrb*1jk^Dl`)o# zbq1o^tSoT^hEs_VFGHte)YPNXL0^g2h=g}pfZ;{XqpiZjQUZ4A9}=B}0p}vh7FS!@ zz?5z`5%mI&o`rHeU^SbsQi~d?&x|105gU6EfwR;pmpj6cp!mau!8V`6xjE9WJ3IHJ zk_ld9Bj|?6KnWJl#j3$f$c@!Co!Ul!P`5Mu-Q{O3S-zn4G0l84N6jgI z@NmgWPz+A?eq#Sy^hs8H0g1&Iv|M+irOWU{j=934!E$>fkPs>0_t=I@){&^y`vR-s z@QRmkLc<0AZ81h_GL?+z0#6u>1`;K{Kc7I&CHLF z@e?2YuL;s|j|ngs^|ca-#Pbfs+gZ2^A_yjL&bXCC5u3l`WMvovry|d++*tztVtuBn zUp$pQpzPn-0ttib5Aa^g;!XFOLAqX_gumw8VOj*MJF3l9MtORw_qyVDq6>CH7EMH) z*2~#cGr6CU-j?!n7HqSfd*IeXCw+zlskU{1itw z_V0f<1#f9E)72O_Ry78XSzuZXTICBP`fl>0_8*HZ0B@n9nJyhRWY&3qNSm5avyhkp z<{lI9A(^NXN1iGQ9Ookyf6jr(;DW4F&RUjqIYH6jx)E1w=I!vTwjU&48aRC z@4({#@Og8?pK};#K%phXC#TTabVP6+Os6}|g8Q?Tm0q$gf>|51h?2)K3Z6LM+^%mN zB~1K)x!^Tcm221Xc&uE zfXRfbfWJ|lrz|Wu-+MJvdp)e-VY5!3kbQCrq&$b#BsmU(Y`M`f^GL{XX+{d$QGU2v z^`hs~tLAGUn)QKim34(Q;PP}iR&0P3#Z|S+Y>}lEhDr?2QS5j-zE5x$FB-I`jbHT$TH z+U~Zm9(f(A(hcT8&{cM)SM^bXwE~+W=sibo!sv>_^?P~Ic=00lkF4Q*iTIK2(47(D z8t&ILQkqDq(KAK1`}07P20*;_-8Ey)fZylcQ8_N3eRhQ1kh-sHM{MhRodEePP>F=g zw<8Ha90}uY>A5;-v?G)hZH&4~u@pq?DDX_Bta3$J;9)gS4bUQwC?6|6IV6=qH8xM% zDIMNvBK!9n{VUwOjTCN?)9PMMxvvZ z3UiFqhnt=-nY@XD+?<1apV~J$c9Uj0PZ9G%hfK%ds@S_7$G5UfTMLt$5$wIPZcqna zUbb}%x~d6ka6JoXaAmjw+PJv5B#Zw(%y6~oJF5NX+4m3xcyd)0QItY@D?xv9|4t~K zUC0fX54)cQ9x9fMpLC$7-TQ^py+-Z?vr19q(_5yLGQi(iKN4gx?&xhgU zlNK`wOCHZ?`ROFuM);vRA{Jm4p9%8~Ron}oF{B@Jho3~)p(dcYBV>Ro(_uKrJQ(Rym%1+g{Y*Yjj#@J?uEG$X7Kg!}JOe9kjf)Ok7Euupw53u2e z60M+7I6#RopS&i){#1|j9ul@on!KT8YkYfnWfq&3UqGZmm@Ahl_94l=dlPpW;su4n z5pOtfL$Wn)Ba|4S8RHN&shumKdlkEm72#e5AS0cbIdF&W{X5wX^KEk~L&1_E7kN&_ zC2=G@iXq&WVH2y^B`aswM1Php+5~rh=Hcib(Ai^5A=+uQ$CMQ*?;JadQo~wJmm-tQ zV6@ZN<)gVULh=;HLd3^izuwy9Zu}7c)J6ncVaMS%F20$sDe>=V=^6%4`kf$_yL(6Z z%`YE3+lXX`vg&fc6h}>~3|eH*r8WxI2(gG@@>B;f_jYeIR_El3*CNYcirm>rB(P6C zXRqoNkm9j!b!}-QVOTn=7YVO8_P<{W=y7kIIwzDTAG2Dv5%4%{Y{Xf4)RC@`tW^Vh z>A8tLzx{!k<~}IzO+G75R-L<8;1TFsEz8OJ^p1+`=PnwHC7P&YrUA8 z>arg9?OJg4&AQ!GQk~wc@RSi15kGV^D0Lk_wcry71ys zXW_-rZM99{O3P+F|S0*ClC}R&$P|WDFJ8MqYinZ6! z(rTC?)OvOB;)}bsG4L>AnjvaI%*EL=ay1|*0(t9@92-u#yn$Fkuu2TmX>h(EH$(fM zwXlz;F#TM{i9%1AaMy39&9lXd7+4a)IkK0&QvQRViYQvvu%_VEAe&Q2jAyg1+3kN?zhc7EW zUsG11uOR^bdA+jJHvSN-u?)QSuvV2dHO`pp<6`L3H{jG+KXCZ@Tg##z&rGs+OXW;` zbOA6XHSWMiEPO)LyCu`@ER~yNwiu4Bwa~MdSXXV@UJHOvb}qR(P9StgKFqYcH<+C6 zaH(LntVNPpcw`@IEj8zRgwz7-x%Zk6Pg-gGh*cw0c5Z1@vSAMm#sd<`Sp6)L-OSBw zrHV?;`&K;Y?>!KG2EJ0dl}#G_UeS&6#y?ScQ5R;lqRqf{t}&1=lQOE3h+zpmy<#2e zDh&%g9X-OYXHeyWFBYKiM;Y1?lVg_PBzTd=fGNy&xdU^Sz#t)bM5 z30sLe$6G%OkE0lLs0y4I)X1!))|?Lh2DrmbOlHKSmoX%(fYf z2{I0=b~RQag+OV`&I$^_B9XbRjI^Jb)e6F%DeYRU%VsPEKWTxcTFiL!Br(&5jb`68 zQ%=%(T7TtvF^-=ZlTS(5U3v!RG<)!U;JTy;zS|gp@*6x-P>&V6Xlw}>7ML- zU77_g|9lg*GbrGDK-#B2kd zE|sz^ueyP5?Y{J1mG|B~vU_Y}qZ!(L`L1ejOo33~6H6u@*COT%P92FY+I6pSer zM+YYLQu@G>alSh~^rbbdW(yptQuLz~=yoWosM zUD0Q&H&^}Hov2bv;~Fh>jB^kfUcjiXjn7{no(?WN8O%ntK|OMSN-_TPwkEghL%Ymx^IqA zs8fm{R1(zaTn%Ej_;$#Wd`}Dig5|Os&Q{*RNJlW~MLLS%1;5Wl|+;6BHrr73nfGB zb9Q79krrv+Wn)-Xx;nv9SS`iD(mlVZrnzJ{JhuO;X0TXFoJ!FyWO?`O<*a+ak7(6u zQ1P9JtMHyGbVOEL^iKlAXUDi|dH1J}VBJoc+_+}4{O%ryKe%F=e{m2nevH2ThcnWA(YjV((#VmvM58emiFH19Kp$D>AGA1| z63V5l(vO+qCP+Db>BhcVGU^;Pf_)7CHILC!g%R~#p1Nkg@=EIq<5$sbg7z1}uc~|1 zHj(dI~n7|i03}JXu|Ba$4D{+M`kIPQ=9?93(2UIV|R}d`3r*wXM zQ(8fQDH-xnK_z$sG0mYLTf5-;0FQyqwxvrddIC58%!L&*oNf7VSa55rNE@1eUfsL9 zPzU70;yVu9_;l&{4>Vah-ONDm@xgaKr@v+o%uM`qa}vi$&yddG6C%w#Hmc`k>{8F; zPr<2Tr*7tik)_hN8L8taW2H%1W1TdU?O2W=Hbq25LXkV%-|l4#*B<-d{i1+*M_vLY zr7oABlYbS4o4lX0Nn5cbJz~l9wU28!Z_hzLOuxK!AAANr(Q&1)TrYyXY9Qw6ov8>ga2Xi zV8=IPSn5=(Vie0g)k9sks)~k`4??96`kZAnc@0F|_nOIFuE|Tg)!OtVIGT+WIx`ES z_?kJYBj8F0t)oJ3Gpwer111Fg;-C6(V>3`DB~>uRl6O@bXd3H85(GY!CJG$O$lqHZ z56Co+zI#=Td#xj6Sq-C4Z$cz5vhZo;7;ygE`$MR=cle`R3JsGEG0vQH%~KdaUr8Rj z?M*TYUmboPJp6L1)4$ zqKm$e_BvivQ~7;**^e#pC-`9qV-2w16MJGcxg}{T@nIckjDi>}e3FKuX0k>eZh&-8 zH#Y%QN?Po0eH0c0w(Zip5PIsi$F`FHOwU>C`ivxrhT4d)(Q5pC*^vewv#)hFRC3P2&v|; z9)NFox99wPFzyt|{BS+gK7yggC$YRe9c?z&W{2}%Y1E@vBW?R^K}&{>4#@Mv{+JV3 z^zXU^SPy^lj#zY98zd`MeOj@`GwY#p0b$P*?W*6%jt@i?4j^O6l#(3`8Q9%tLR(kXA6FT*B_$ff?#Y)f`B6w|;laThHhoSv&MB&tUW*aF=&K$;c_v11(pqP9Lq0KSFnQ_E5X&|D$!V8NM{rC%(YMeM%Ft1U{5xkz0Q_JL z;MemXd!yfed}wv<;9z6oBs^!80RmX8IH#24C!-m={39MX&if?VWhHr$6^j|}Qk6^a z^Nt2@&QRCP*SSOPi!ADV9g{cO$=ab7tuc+`{hOk!{`R>!YMX*qq8zs*S!u%Bt|uU4 z1S+r}X3fI4W~DAL-cHm>v$l{I8-%x20B5#AZ0K9$+)rR~i4p})UTN`KUu^rY%)oZ6 z+q0LuWa@rX)JoY8Mo6puuFS}n&1~V*H-F=Gg8ds0&pz3be|)WndEdsr&uI?B0|QkS zwe)Ho{igLMrUISitaq`>ZJ*F811_M?cR5~bx=sDCH~%$bzrj^=zsPZ+LCIki0$3T; zjQc!3A2}UZ@IMz9<*i@l&IPUP@Cse3m;j$ZS^NSxB+88jeZ$zIlPzUUWUoYMf0F{H z=oSb1l6}|f!97Hg`@yF^EXpTbsr4SVR$wxKJLYYUH^iA!`T`Kcj)s1I~BGg)hq5${vDV{GyKdxjNs#JN`1KXdFDzK>RavD!I1U3dGnl zE|i5H6)f4?F8x&Eb`uNU_(6#QQ%6T~JSuyJl}}Zfs{XROiA=xr&>GNz^+69);11=+ zpYT?%5;Xc2{|*SX-hCs(0v?$(x&jQ`(a5~uC@)<|{;69ap8b&2-`tsWrZ z$Itp`IAKqIQ;Q!SVL8#3=yM;B$5nhiRS#4ca!e;6vb2 z6PRc?6*ifS5jZoRedi@TMaX^hS~UEI)XK@NenX+~3GjZGj}~R`#eb;X@|EcEw{0=b z_@q_)CD8KA;0v?)FQAGEy({dCu84DcoZ2m?Si(QNcDX7|W!?P8AH zfHM0~++|KjFRVP@CGDFITVL^P5ta8N`U|S|RFtPgEXC3DSS1z`ImFpaF5o(R)sHj3R!J?oik3qs z5ezqg88#Wvn@WqQ?lY3EQJOAo7@?% zgTvk^;u{*YEy8;K?DsAz(RRg_J%ULibgP#C^!Mzy=iD6uha}gxI6ykXgSV1ck{pt` zfaw|UvG6dNUbhdn=r={%hbN+|>&w$`JiN9VdDwor%d=_RKWsVI(;fAjP?h%ZADxkX z+l?`Xu;Ye`v>{Gs4{{LGg8fbayMwY)331XCfR8`(3aBGJ zDI-)hjKlUasVG^{qUyna7|DW7ppXcl%*Z~f5;Ll$E%Q*LzQX~3v4;i;{(=50h&*YA zL)K`B&Hj-BaOONv0lvQ8;dT+_6C`%cKvL|f{WnU<6a#5QbVIp?Hk0gs3&zbFgvOLW zxM8hA4XiT{Fm0Z|)}2dU0aEpBy?U|gbqDrqN(T7cw#$4H~t zg?P~Sp^l)kmvH_yKdeiJLWm zCUspf0jyO(XZ$$*aT+O*;%2U){Z}1Ik62s9-A=bR~UoNz+&D;Iw|BX4T-&V1Wz(3+4P#_?VpV0e(4e8At4NYtkWaT&k zTT9Lr4o%JL+KnEe))(vh+R}o^V1$i-(FwdMWa0_s+SlL`WFx+%`V?;S_FsXY;J4S5h}EzMflB#(OicYrg2(iU+vYnw{OypY$cb6LQj znK2FeTVPaVvN3UiT_e7s=?|IxAnQFNazc^$Mf_r;sNKKoX&@uv?oGbh1;y6}dbu0M ztG-72dNr7Z@ejEj*0Ot}LmCU3nVTQvR9{Rl>qTP0L| zoE@YSdhxq0if(dpeYHCuG=2qpoA$0BJ@6))Aot2BB10T1Orl9wGu&?qAs!K(zW-f3AL;3El$e5$fmSf73(U5|@ ziJk9rSXtQ6dl@mf_6$Z8=}2u&BAe&{yKyp8OonJ68Bt7n@|m8}Y&Ncr8e%%dQJJgr zdShf7VEj1UkpA3SRQMmvtLYY-DV|O6CR{@4M~Sp6;X%jvX8n`nS}n6vdk5oiDO;K4 z!d%^<@5=s_MW6l4LtVO!=>;XgtZ`*ZXlPl~&H0wL3%)bVe$U3I1`V8cs?oXhcY87S za3r0G*aOQ&1fFZzAcGB{Xq-R?d|tJo(tzy^aD3Sn$f0ebSG77s-QFM=hWkD*OxxMW z%+! zkqtw(%*H0o9#uxWQi~gN9D7t(5DP~tJsmLlk@@s%ly+c>X387^Qu9=&KH`a}yGHf^ zU|Osz7%4WAMi}$?6}V=h0+2u+4}>WS@3I>uqxR8r;x(;O?9K_U2k|VW zwP5!jca44B9W`3dGDD!|_W9-^F_mutV6WvhD}2ZL74)lgPyV$bfDU1mY|3;@MtY8t%frkZP_xH#_32 zi6`D4AE(Cj)pp~epgdEsvm|T{G|#}cgAtzi$})_(XM}(u7|@td>+Gp&xvy>Mv-T=e zw15A8#Ofh8>3Xrr(wN4>ee;h3lsCMALB(b7w$vy%Dqw3ADotO|@dhQV41=4H5CDae zG<;2m{UC?+N>%Uf#^w$KbX)N$a}iyEFLnaaS5@UoV^{zeE*)nF@A4&KG zsKIwaqV!y{&;RQ2@H*Rb0&GWE*?ja)E|D8f>jOx!}!(S0KD+&Kn*Fp<95a)0X;ZSjC44#hI`)D6|J{x{H z^PE>WP%yz5QCHg9yOw~{<$kjFMBGa?p$O!V5|byCofTf~ufK`og(s!`=8?5qIwa;dh1FrZl<7<=pkFz%# zADHeJ#xF=u#*mm>fXJ`6v=NZH%%aXvQu_2bu6HCFuhWe~QiNDOXWH|t~((B}5GFJspUrY_JWBmKShD;Fz zflNFncjABiTf9&}Kn(wnbakl{0JN30RZ+j#BsmCi$Gcc0v?nx6;$fuH7>UQA&ajr4gDFWZqLI$Rz%sefL#_VK!E z>SaL#Zx;37kvde=uB_2(_uKuwNFsUvUK769%9ee7e= zUx=Z|NheW9F(gZ%LL=#3i8SF}f<w4c0rSd3xKw>jLQ)JZ<{f&4B=e7!3So*v-h z-@K}2SHg)lThZ$apcJtfiwYC6z1<*x;2Vno@ITbh6cpU`k4Nv*Dn zNg>#7XmM3-14woIX}qqHlZp>YsU6k2QKVG$RF{DNMnG%gf+ z&`Q7P3?4H+T~GGwfyH55&@9;{l3}QuCI*-odXiU{;0-XHIc+Vt0k9+*P}XR-oX0t} zG=g?HSJ`DJ=Q^@Lj)2L3=9X>ShW+3ei4Cpfmdzs1%$nhreuL$ga%cqmfJTSGCx zv|?V;mY5{3i+)!d0YG(j6eFPgt4mb0#5ya8{0pIlhwW+<9sREl22<*bjcv|Ir>n=6 zWzC`%gY95+Qk!ijuD9gQ&Hjwfvz=b>R%_qr9y_WIkzGs1Db;~1lbMuB91w#yhr{B6 zYofj6Rjdxf(~BL4=H>?Iq7fXRpyli{qvag+Bq$Um=bCxj@2U&VC${klBVb$@cZ{BFQ z)f=Hob>{|I5+H)GHc>Frlv`hJzRd9cdb<2dy&gSC*>+Dq9MCX}t&y!eTZ=HgjUUOa zTBph-X_8Ud0T@EIq!ElO^Lv_VQK6Xg-tejrPW{*Tj{zyEgbunpI9nAaDfDQA}R@a0g~ z{EWAk6M&)i^2Lei35)S|Djpm5%Yl>soiW&fWu6#;K$QDP)7=4n>aZ>e_I*sBK=b0n zJhTyvGg=XcoYoMzAslJ7k?(f_0@l;7FT*$RnqBqCV@VyOMm}d`v_m?>M?|-@bzx_e zHE~bd&hV65CdwuIKx*+b(x%2}hsS+$ojz-tzl;PsPAd%342m>zeQrtXP2RxzCC_4d zUJv^Jy{Vss(-VvPsSvzDB-H6}1HLy-mIL_${POdOt_U-8cbTCOR2fxa(ImWutL>AC zkD@nqS?`oBi5Tzx67(XNW_D@kR&ETkGdD0lLO>dJ-lr^hUE^` zbx#Vmu06@j2{*krpqJ@70i6m}V}E8o9F^-l>p~imixlS;s7em#s>?1b(nHF>pUcI> zJL1hn){x3t%=d_Wca)-42;}(1r*C*gk}O(FLxmTvJiV4-GH!fTnszn6u;I{P=k)aL z=xSKcJG7_XkFP7D*j?ZPN10(WeF?v)?^sb|R}}V(C!>Oa;Y7DN08cskN!48X`(hWf z>o5$!Um-wBp?6`ZL$6nP^pN_f-6YrDz*_;Nj>27r)mI8SudBq;v+B)V#2k!`3dvi zp}PJR0z3i<1eEwgj#2#oI!iNy21rs5^FsN_K&#?tJTzfPF>4vafIx-f8QogrQE`+v z-xX+atQLkfrLDcfD$y#?c3CWiJPXfs*Bc4DJae}4yKsSZ$%*G$07GKVWH&j0Mpt+?$(%p zVD_$y;}sqIx4e#M^A2be!I6L$VKMmSFOksM~fm4)b{Ie|(5{3B_>xtW2F ze}09Hb&ZFMNq}{ZBM%2x>=_&^PAc*)S6;F-otd6j!{kmN>Kk3NshWpvY`1 z8b<`>v?C-E&XsEkV7%tcT77pM2C^5Zr_UWumZp7+(B)-#%MpnZD5MZrj4n_UM%p&u z9#iMISm#GD6xgK9%5CY;;;wKM!M8+5`DJCAz?PAzoLsTYfJK|}1PDW3`LK#I=4ygN z=Re&F^_N$mgft4dB`g!xn3|&8s_Ha^NFRWE!a}H>u`rZ6l$PW;0wwwsRl1Xj3ntiz zOJ$f(DzC!yWV3J_=+a};+O4ka9AARMeYUX+-o&uI>%}iT|=(tsa6{TB)_*DmWA)|h2ZCMtTqNI5l?l1|M z{6_3l;!LpR)5Q?cm}+Ly0+Gj6ytbk;q&Aj*|NFNt(_ui180nzZp>6#DVT)ROdH_qe zZ24z+x(EFoL*=fiPQjOsgL`RW%OloQ6$6yyNU;StfQL(RScLr#ja{@E9MMBC+S0Vr znN3l8nE{-fy)g71H*|NzO{q5MxVnZj|5wv-J45t-UQ;0Zl*i?JqB)JPNyeEyhlsgX{uh! zlie90!Q%if3&pe#GEE+@UPt7c#$n1bV_c*mrgp~?vt-&hS+KONZG6)d*46a9&HKFF zg_?TGv%HpZaWw@eRJ*mkVR2&%K=#U@&vy}U?-_dL>)UoqMCx%3yNub_@lSu&k1sC zQ_>#Kai)}Rs)>m7WcYUDLjVve$?lsdVo3c*VKli4`5a~sggU!S9LSUJ`0_JWDD!#$u4k<&{UZK*&OBBxlT6s>)@AI zr0O>p@+2p9DPJ$TK|H+xr(augc&5(8>5{c+-j=RQ%}Q{7z1S4(QH5keNH)c7QQn3q z`TdnmcE#0bamCeAFJ(pn3U=v9c>vFutn@WOW*ma%2Xlb!6z`=2!zBlW;6R~M_W~!u z;X_kM2~SAi^&X`RH{)2`oM884Vby2vmwqI7qDp}I+cAt&i?;_G_(HYjxe z=ma6@AQC4ca3Ye7k_M(UTLlFWJ0Om`Gni3_-P{DFm?2*b1<{9J|1}$1=L@p6srBtpZ>|cE{iQ-$MGvjGSgLE ze!=bm_bzhfr`qxw8oiqLKFLz=is3`XhkXYcvYM;G1n2DmR zf?jh&mn(6)P@4pktx&d(zj7CB#XxmB*|qItakHOA8mE@Fj!%ryC!d~WZjfL% zhG|3DmK9{~7*un#2$g}OXlWku|Z2F9C7dJ*#A#Ph%d#}f^9 zA9~W!Gz~qzFM-wCy0H&l3ck2YFszvpov?C)l4J*XZ=7dq^{akBm5$9^`{ga({x#<6 zKX-1kbkqIu!$}yf)nC7OCa)-(xa@GB2stX6t+q}+HGSvYoRWzosa5EZvTflL!;Z8K zr@7dr=}tIJwxH%~Ofau@JY%((auNhfM3Can*Efd-R7=iV(=GARj7GiCGVTT#TI=;r z_U!>Mu$5_u2+ex~wFLKz&H2a!Bms!()ZBry>R~ zc9I2Ok&67Cm=swGV!m{cm&*`>%R~Bi$R!96c6C~r@q3>)>YHTf)Zu7+^N9c5z$`54 z_C=;bh1<0FXs!89O(Gz4x`RY0J(b5Q*@3Mo#-`Q^^dTsR>Shu3%K)3Yk!%E5Lp1zy zOIjkGreL~CF+2Z5OpUnX4JIxIBQzUTirC_>l0(qyHssyS0yMRWpa)dtf-iBY3g(H7 z92MoCtnJ#~4te>Yd}VjPki?4VI4BM_@)OJv{`6t~;#cxR>mZ(?1_96D?7Ea8HG`J_ z?aS?X3^U05l&MYt`LB33JCX* z`Oxpef+8dyj-rE9HY1y@6VahHZC}-)Aw>9$u!StT4eFTx;!fEDYpoJhg6ArWD{!8y1X&3v{z5whXcm;J zG@~gWI7d*fexwiC|LC5MMAlTK&{!2KEyA4p>Sc>8{-lY?vf9cvsYO|-&Pi`xmgf;5 zE3+oR9+*CfoLV#h%&dc_8IZ!v5E6w+6%`*HiOe(7^~mNVV@`FK)EdLJj8d=fCaKc3 zCiWHPW?vc|w@60!MlEacW}8tLh;(?6Kv`cV8D6YB;@PySr{nF2h%M_@Wq=3p5U^KF z%7m%cWW=_Xslr@}k|VS8%N$t)!+V)j3QNxCnw82btfSHaWm?HFl&jPke74z4TZUA^ z;EFbZzlhla>pVpcL`Qxvt*ahmoUpV1e4?ieavp+rQ?>8v$7;EOZ+z7u@?JuV5J6yYWyX|LWDYW8}Y|*lEvFZW2XP zj$P&Ft{>w}lbu$Xvpi8e_iE(pGbWq^uVd~>FK8D4Fm%>S??UHOYQ2mdN#hm#k`vE6HQ%MzG?)QyyQLb4?g08Xq&G|J zhmCd40ryyE;9K$>w&aIp4>rs{_?>D%pFe*K^(@&TF4*z`JjC*WNH{!%o1Q0}P_9e3 zpWMO?Q~zNcajK=3HIco%;k!8ryA%qA(0U{Q0U>^W`v{PXH?6g;|V`8;qun86C=W`+o0XS6skud z(GjGF06?f$3R83$CeG+z=!Vp=$8$~t1j>|r>D^%(Nehdz9v#HS6!(u}STZTTw6|q@ z#$3&O$)_STu%hlsmEP*MyN&mW)iU5j#UNv$zqt=F>PVNqQ#}(RKg?@M=$Gj3O2D@99Mx6vn#DuoX<^Tc=naT^(1` zcc^VSwY*!6e++*$)PqlE4NR7W!&y5lxUTU154uARCIOqE=O_uNk)@PSL9dP-M$sWQ8hPZ)>=~D`e}g&KeFu72QxrGYV6( zIPLW!e}r)8vEr%BQc67H+LLE$ddDDboAZRy*{RkDnz)+Is0q3T!1PbK?gyg3MN&7c zsbmOP7`UX<3Zc+uT&-1_-m&50u-59UY^F<%sKMK73|bQ%LXHf1-T#kFjmUFMD;^vO zC=KpEEW(^UUIOPTEFkMg?fIXRtP(LN>O_%Y2lw!Q`x&EbqJtzBDnPafNca^>3g?Q} zsfw$_>1ZC$e{_AqbUs!c6(KY#_ZLn&zYhwVzmvZUultItnXJ`I7bk>GXD?-M{c-Ez zy(a*CzvK9&+}YxCGmsi{MYGAc7;tZ&_QIO|eJpmLB((>906=XH(%!jC4{tzi22&Xe zit$BHpO*#1vJiF=7v{Uu4Cjv6i&N}G^&ys%IxM3iNQ+4hdm)=t9G)g?CeO0^ig#5U zQ$^by7PFXZ2#qA_v8a|9StCC$_u!S;!7b4o7vpNPM+}gA2q|6}nMyKeo&6%kUq`IP zW?ghlBg?>c0T5$3Fe%QCduY4$Nk{~5DQnJ3H7Be89jmm;SdYAj6u>#(&*wl!*pJcqQkn3aI6bOsd3DBiFH zE{ubp8i$D01f5QSB`>FvjLzbv)rrzu)oC?Z+D!MH4PdhJuxvW}^aa*Ys8$}S$70>5 zIlt`B#xSm|$%~N|lJ~K1?ggEti9yGktX@-FB-C5X98(3cl|TrhXsl6OA+j-2q|JG5 zB{rnMBbx5jI(Ou8mW#EP75`YUW>N;>NOL5A;Kw1kE?)A!D8p%LavU-+(-!(OqvBkU z3kBgk4{*zBC^)OFVa=04UbY;ly#}vqF8RfQul=d64k@!%fwB)r@6lRVH=&S5Y4Z1p;`>v1X zR8u7-LWjmBgm;RqDo8d&Ovo*SnOIJ)?7((?c<2b+d8ddEFf}M>wJzwEMV#7sdC1AM zKLXtzuXd7!rh9kQ^c^Ne_K3YFwDJ|_1`z4cn`R4*^9XNjDmU^Ak~}ak^9nW=$2Cxg zO13)MgX1CN8{zzI9efB2o1_ngtroBxxGVsCy@zeD#~nB!7XZ<6=%kB^!nGCo$JqZ4=VLu08SK|b#%r9G%JTrg$)y{uXy`w*WIQz=~HCv5pMky zXV7E#Gi*R|n~?J=a2XX=Z(0jbgFz4YO;Ijy1n@hdFh~ zJ&x_;dW7h~$7?*GhU_C;Pjo;sN5JWul>78tL&}-^^yHdPi&ClfE{Wc>04}cm)UNDK zk&bfRA-Env5+5;eA{o$&9UC<-MaRCYEd*MxX}0Gljt__Nd56bon^Wjxc(?J=A2QhY z0%K{TD1}1Pb_;3Py~H2x!#?=nCG=|?K3WN9pw%pRx@~v0+m6oUn_edKPWc{al2GKG zt#K_s8- z>3IA?kd<}X?fD+~3Cy-HKHWh^5knw_MS@Ok?3#CNt5R!6WjCaBudS!ZE%7#7h;QP5 zs6Zhs^^5b~t))alv!J57q|!?*D(4ntAX9 z>3KB3klKaPf&$`~oHk@3uoUmAfH_#5K zxwAr=!@rZrn0;Z6Ye{HS>5P7cgPaC0*B5Vdw}uYRo-gkY_}{^zNK>IK^T=p(0ihE0 zg7m@m(!I5Q8#>HXLcOVe+#tiE?hf)np&uXsd`Jm~#f(3TLchmnv*x38NX$noJV@iG zD&h*e>jlwO$u@`9qRGbS(iFT7?v(e1Fk{G{LR@$cpRv;E)?+BnqV zcPe^g34=NF4UWE&Y%1OK(7cH1qNo#YT>c4^O41rVodZH(9oJu9PgL_ z4)~D$M>7Q47)eljq9C0Xq2-3o@*Ie2|MF)os~yKSPKeG zI-NT6Czl>rY;U{CQMmFtSol7iTlM56QbN)grc2#jwkR04vB0nw=J$J90}+}-d!86a z(vkFmu>M-pjG*K*Pzo zIP>-P;y>0Si2~-NNkP)BlXw{g>n@!J%^=?5q@sW*R?`(&7v;0$4%V3zmVw){_44G@ zOB|Gq`aImIvUsE0Lm#1|YaLP5h+ejQu7n27A_d4uu`;O;D~m|J4(1jaR)RD@0{hz{ z2T?BkG$om^{8_9-O(mtK)3;5TL@!8GIdNVWzg=ghAb0X~npTEebFrd2_4t z-c%>Y)0g^rKe?bN#^;gtDQ~YgAKz@K)4zAgxd>F$It4x-!ZZ{|XI;CgF{%PeRs&pjz$qfar2#c2kGI zt4cXBgjUC`dj2x{!HAGkfKEN`pOcKK=CAM&d4~3{2#_0L0t2ju5XrkzPIYg^wO~>3 zG0eh>dG`OOTdj}4H(&76t)~3-AEEny{u(v&$bhOOTls|_vi-XS=i(Fshaz+aCO`lt zF)u%qQPCT6Oa#R_tdr5r8gHqZ)lKw{_*K`NFia{x>^}``o|cM&`nUi_S09WQ1h<7nB!92|YDy zIKUW!8>0+Nx*kJ2jtfV61DlXR2drvl05&Wo9TpQu-2_EWcoM>M8ISjFvfkF^g}q;HJgr%M;#d4~H{uQg|`Lq#}~ zv=?7;zul8MHl7Iy{HdfKhbD!ZziTfkLV z8;}l=s&Y>$yv5(!@&IB|W~(n}K($yP(zYw`wTQq)&e0@)4dk{Ma6bX ztQ>=(Of;#J%a;kQZ5hTZLA+&XTfbgPL;GR;Hari)ux+CYBKWd9WQlW2L<3fWA1#@fMqo znN^11czOr+0G8pZ)uL3oi#V<781i_UX;XWZR(mRYRDA9PqTbwfMsjm8K-;BbmV(uX zi2^4$PjjtYtHv5EO52qn}R@~BXq$7Jy>voVzu1#gkY`R(%yTT;V^njqtPOv zCTnaH^ywF?s%n%}xq)`O;+V5mac7y^D;d1+QaR)q*T1@FX<-`r7}Xw;-LxnfVE6)M0k$vj-((8D zg=xkCH#n6Pkx|-S`AHmG49Bef3;15j`9KnT#jh?zM`1dCHQFnNyaIGHdEdS>J&%%$ zy0C))Og1knp)NftP1=+uPhKu|;gpLX1FXk*)Kcw})p_!}1nP`11eFQelv(Sl+`?3% zy4Qy6FW6)7z&kxmz@>d`w!iTC*Q#t%i-PC6wR256gub~DTBcG%aXig~uKPB(UYS}G zyxSV}{^tXP>FUAC?kB=)&!BAPAT`lrF7yI-BzRzc=fWsLPU-XGi)V}(yXAF>Uf2-e zFFm^aW#2dRl#hGP`hroEt}ubNke}|zFB6<%)_&te?*S}DKocp>fOzZb<2yUVtaw*5 zLK&ufE$^f4kgxv@);r2fpU*2OD9n85Bl&#w+oC+=l`!3~C3Yy1o=1C3pV4XKBlk4w z=U6{8*bBp>BSBvNY8-By9^sD1dcGHa=BZiOIb3ADYGfTN$to9&8o=@VTq!G3fB8g_ZM=MX2#G;cpt!u7YkNp6876T*3JoEsg z^f|M+^VpnpB;A4WiI28}Rf&*E4obU2aV zLxYIdlUck@X499KF9iDCAQroWe$ny%kz7jd`Ozg@@T#qq1?2_B1=F#1lIp>>!6I#r z)3KNC!hN-KN!9KE?ySeZfZ-2#QE1&iy;)Az(k?N9W4B$fn9p21h|M^V?6G;B$X&B3 z*R2P#bZs-jGv*AoD`>tWdxF0y_FPsTMwVAkgvB*L#kGR6?A=PsyAOPS|C`4iCkx~Y zwObrEZ1d`?X+5%cvzdyUI+7SF#=T)m%-&G&Oa_V=AuqH6P>2&lnC%$wU+ISmuIfaq z*x=1zBJQn}TeB?HHX)3_76AmAO}rd0@9oPUo4*!kSrVh4tR<>Uqipk;JhvAKLfMtN z`uCOEh?7kEdudU`Yh`c;j7E0~nd-u5@mR)FzCV}%#yK!Xt6D{&dj_A4m~01291oXu z7jHsK=Bs*uIx;KDdKi|0FD}pYU^xOb(>e3xpyFc@_5?e0&2^x{@|-*$B5~p9=|*TP z%T2PKOwd|?kj3g*$b&h1bt`~3tTN>8xs7GkNJthg7WT3%+f&A4<(qm1W5P&0Q8wv9Zg9UJ8A{*BY z-DDuxnK#nbz%GZRw6C`GX4%xd>LAxW{iTqFIv4e7^PiUU<-eaj-g$i9cXFzD&COhY z{FAj|{2wP7Kf^qm-A*6V*F)dg0AK;`?W8(zx9VI{!U7b4F|rI6p+=mJhW$6ecf7-!`u;ZuzdHXZ^vAruauD$UKET9JyfwyhsqKG-0+4%nP(r*9NEzbj z26rlh4VYm?lMbxvDfZ#5J}3~FgtJPU%QT*BE;`bH!q?;^pO7YpD%KG@1DvZ0^F1UyZ_0{UYy`smYj&Hi0j|OoQ9V8g2=evZ>vwoO7Fc zn|}R(%0ulnrjDz)#Z#gg>TSa{D;uOGT4XtDj=&wN zR&aJ|cXVOsX(NgL#NOt7x*cl{c+;dDPUoXsfC}1R!_)#v)6V4gqXhD1VF6+O)DJ$S?g-ck`*eeOm_{2GbOJ+o#pfwWi z%6-~2|7`!M_V0i#8YPwQB5xzLK7&46`A3sB|6f;U0Tf5mtzq2VWpQ_RcMA}LYjAfb z$Ob1^fF-y)!3pl}?(PJ)KybPH<=(pApTDcNs%!R{*_rJ#(>>kiyeAe=G!N@MK6)$x zr95gwep7_c2J{bX*=gsr=AK)zVJWsJm@S!gV!MfOa!m>@eZ$JZ zUn<~(%h1l%!*)@meY2>wg!xOHWxA2!PfjaS(3d-{*t=UA6q-PSUfXq5&NYhki^1T= zq<6=(g#kdf?r!IJ%N8@pmpohnRWi>C_aD0IT{x}=30{b#Ged-YD_V>K=Fa_<6IXAa zCpPp`gURvq$$kuQ)l{`-$jy}UA$$u+@SIWIaFzBhAgAw954|hw?s=NA-}bjT&w04` z6r@z$TAL7#`}IeN6~5pD^%s=hhJX1Je;j} z*p#aP$>e^0MPzl#=ug8!iO;Do=4xCTBE?XZz)}VMF0RqdkVy8%Yu_pN3ijx@9YcM* zvO$S+fP!~+Gv39}Ro&&aRZ#LK(gc-Q`v1_OWB>BaO-F2OV%=F#`!{1phw=C8_x)~J zg8le7CD_6=aEgM$g*n#uidkbr3|4L43{hek!0k>NhXU`<00k91i_;;6ISbuG--?!; zMXk$M0Pmw7-|q)|vb<>zBC^}hlc6dFP5~BzP};b$UCis)zUh`%m|FTWUGqVfRd$cW z>BVYp$1lJKQ(Ly2;G6`vCE;*E#lc7UUhX_*2cnpGS~5$?pdyJjme&psJO^DQ-U8e@ z;N7)yMCI<>(;M~sEc6}^9JmN%9V(-u@^<+eAZ}ySB0B|GRs0H$wm;olr!5q;<6wZ5 ze3|79|Au+-f%)}ZpeLGFOl#<3cJl^6!sR(*gqJ59o*$UO>nQf z2B(JjU5*8zM!A+6&9ttjv{_KYkQdqoXep6Gc2#$Nb;PM#x`i>{c$uT~-UMk!B9vh& z7)jkZT-yXC#(3@oKCF0~UG$hnSD;~=pfWW2>+Bsi#VheCfrUS!V=)_Jua4?wRp6VA zP}rd4ufOAyBp^OWB9fGV$Ch{C&|TsGjK{WM1LXwbfpq7HftlY7RcDRCGboe1Efpu` zIpiIj_=)Z0W)Yu1(+06|sBm{12Wg|1r}C_;&){vcwOtJmzmOZr=TTktpl^k9W)-uz z5i1QOfwO+APCjHV*8PzP$O*y4<=E5gQ;Ps`)6ADKVx=Tm$uNvda}f9Q!1b?qpejjo z46@%WfwFE7Qe%CwAl95woq|w}UYV??~Ujzc>)N1#WpK;mJ3^byME0rBbn_0rx z8_PI5sn`e;lB^}{J;dEV-!8s{&@8aH5HL=2ZqBPTd!f$~kz0JB1-0f$)?2xys~K29 z^j>#_ZLAmXvB(~dN4I?=ppD(_$#K;2y^Rf+jEmy@)Mu%8E7eQ>;TLa}L_Z=bGQn(h z1ThhCV!JJjCR$FS{NeRB!sL509@p6IhIrV!J5Tm^q-Oj9`DS5mO;X?aE)CU_O)P6g z*rJc?O{aA;B`Wj!P2bsmPGqxn{VF^tbn`q>E%aeyYpHM&7c zjBlU7Nqjd3Fx!NagC{H^t`N{1n;rv@CqqPl_xVO4)?+^BToy^=376^-F`f zyb+(~(pvSX(QctTEXYVkcalqv2bxgYnxDoR^8uU51m+jMR*%dK*Nrse?SJXNH*{6I zrGU-*JJ`Jc8Q3y{hUf5r6`E%%*x*;EM#Gjf&l=p^fGv1a%oZFZo}pqSEDmbav^nl> zVB?t{eq-y;^M$W+SBTHM=*rSD)ZpBz78&|UdNXMz1wzJfBlaGUJR+HkhX8 zhN)V$%(c`H9W?N^^uI>8{6_Iptg-FXfXp4g(vY6|)3_`18*6WGsk3*JUu92@eT#=J zFAcO%h2=bss`q@+(EH&Em)4$8niuktAx}ciuSSnaC-!YWDc>_atrfR*obeYe)ioca zhUlZ6ajs^6d9`a925xF8{|W&{m=L5go?U6T8M_s$(LeRL5|uR1(J`qRp#%5Kz zLkdkm+p8@kv}v^7v`>QS$!xz!2)L}MCQM32 zUWa!V!Gxzwlt9@4vT0QbmXf4j%;+)Walzl#k`Ha?1iN`-$PwyV#)*(Y z@1>n(f0#pp2(iSHrpFR0X&mI9312Zf?qQ>;!zN$pcYGJ%=+bdO6$kCY7yE6=w~O;% zvmq3{nW$m}2!o$FKKmddb4oLh6zC^ZGn>fBMqV?NE9&fCCSxto*vn55KJu~*A~g^I zv0y#2C74^oX?x$#yN<1EU3cAx62bE-)M_?5E9(7e!^21W(8O&(rRRhsaEb4*fS9&$a4DhkS(gfv%csCczV31#*QZt;?)#cp;7sI`0??_z4*u+3ekx23Bhh3)~9%HU3ahZZ~b zDaT7`E;2+JuI{SFg5ju*r6*Y??AdS%<%Oqgn?NFJvEs2RL@&_74j9=^_%Vvbm1S`) zNthJ^joWxD<=3 z%j@@J=e}rHKhVR}b1*Aff<8q&)r7Iq7LaU4|8jTp=227ahLYc|Cl?G5!<%afBMgzS zCJ9Hj9VD0=6Yu=Znu{M+GjZRbED(B*#r>*rD_Ni1ao2TZduP6(lVjc5LSqXXb%RG- zfJ&6-m4guY72u+~!!2mj9G4*i48M#)fTX%P-IeqVRm=N^@tI={{-czwn)&m$44NZ@ zOd+Wqs2w`b*gdZsL&=rQ3oM5Se0i6SPJW6k3DNdg-Ut{v4$7%am|PMMnj3)3{v5Qh zy;W_boV3emv3ImWZDwwiGA50sCw(iOnx|a>MW_7agrTW%)~nTf6{luX4MCKo?M}Zv z^Gp?r@8~Cn*;g3OnKoFHml-$0Y_V+O4%t7N*~zr3JqVSMy6uRiMS zFA7fxcL+Nz)tJfPF#qoB0N03l2P4(N@U4mPMo4n=T17gw-*RnVEgfiQbFo`s*D)Mo zvVA)|q_ggliLlPOHdVD&|3Q+EGK)HhesEkTT%yE>4NHPA0UQi{*L=Jzn%^=a6ffB8 zS580oQx;{s8BHXS@aM)&spcK6QYY?jGa0?$j`v~Pc`JT&ND2y)Ek@T;&`Zui?rudRO9J{e^W(3>~|GB}BIr zi3O?9ZR5$Mkw$7go4V3rP$JdEyN?(C1E>X8?Y z(W*&GswjR&wM8GXIJ7sJ_#IEjj1fR9ZF2ixnV#jQEl(4`Pp1s9v(-Nz|4&rZYHjP}>?wv70a2YnNX%dWO6uUGPhK{yJV;-sxc3rB6lhI{PfK`5slsg5Gbx zrKd_j7y;%?&n|1TE=|;lzvsJ9gXXm!%Q>%k6yF+p$VdJdI>^)_gpW%StJ~R&K%c0` zXgsBtkRr*dT|4g~h_FTc86VN??`blpvq9XNHh6?c_}v^RSDy@FBywBo#4q)(5{{pc zl$}031sJx~?N1H*GL9)l#F`KYx)Q|yjE`kTF*Pc+_AWOgWJ1ASFU-DJRgt&kG9G5~f(0}A=(k#de@h;e-yl@V~vPbWD-lROXa77|*sAUJVr zF1))L@o8mQgPT6{%Av!xDJuOrxf>?@(uDG|K%ksBpkSn;lDbeZ5;K)fTxgx1$6uj) zP2lpwFCs+13&Ard#|$Yc=T0C5W|m{9gnraG;=}?I2o0 zpu^Eg6|Y2UvV~j}Ah{|ks~8rXXjKQGc)H9GBZdO_1x!P2F!c>qBeVN()NQq@v@RVka3F?~P6iym8F^ z+!WvAuy$l3yVH_tW0j&@#xcV&#;D&{(2_u~elB7lL&Rqj8vpoTA)^E@4tqeb^=g8x z_kRb6#tV4BOpQrZu<5>9wpbkL<^0Mwu8+{kEgbqFj=(^XKufDk#L*ph=s*wRG;dY% zZ0rNVs%!`^Md8bHb7Z~bM>{p(e#_$__1yUSu;IJe=JVrYr~flbPKXCG;+TH*jVYy* zX?IYdOuH~|O+ll=3T={dA^}`OVp)$vm`+Faq_GCDO=_6^gLPLX! zOr_8Wt-X3vc*_#owmfN;7zJo%&Xz7uQ6I z%>P6+L@(F%H2Y=cSmd0yu=chs>0|O_Vn?gHYBLQU9A9h%=zkrrBd6^^m zx_IcZ#w}X+iztGj3+)%g0>TfAK3w+C2V>8`r1Dwju&eZze)iQjT5DFo5pPfaVd0eX z_)SzdrwC8f&mBCGchClWN#!QB_6gja5o`uQK=M$tkOj!IJ%$2#RZobf7?;ECyj%C+ zO)A*{<6`f?lqPYq-z#mq^`WfpH1-M1gR|3Stw{{Nm2BfS)8q(zM3JcuMqyf@>WF54O`*@cIwupV@hmTvA}y zng%yEAAp_r*pG`qnK~^c&H>R9-M3Y|w|rO0R`eeJUXF0mVc9$w#%3mq z>@MTN?;vA!6BzIcOSH<%xipbyyG!WFJjcR~kgA<^leZY6(e5J2F4R3mQ|>8p zL~0cMRQX8%DC(=YZcP4+oh$61ngO(Q{}~#xR`Wi$>{EQiQtBIdVVPhDf>biP{hdnV*nLmj}(#mN~r zq&d)`X*un&@PaKG|v$4OwVJlbrc83x*U5K>b$E2ntIntve>jr58v3YOx zm63gYPV=pEWtQ}dM9$AQKgT{EoRd?w=>+@=spubHr%$pE_9ck2b}nLbw!jS1-R@+9 ze$ipSLGVWOT5{F-B^$T#0=Ww$W`eVSwX6e;4Ao?*Lg`dO(Q-*}z%kam6JN=&us*b8 z_)YS2QJ+1XV@wt+p{s9&=sb|XnFQT*E1X{HW31fbXk#;Dq%ez^m{Q#+FOfTSQ zODioD$}#ln!n#FN^5~~j6VrZ|n2haTd4&f^NvZmtiZqV++VAwMc4UvYu2Cq?iBN{W zSv|u}>aOv++GW|Tz>)MLhwFrG=Z}f1*2HF$$HUI@w#&eY_-E7zNci?!EfDsMs>Da0 z;4zean<+XM{h6M`YP5&bR6k*;ei{rkUh=g~kcU=1J!rU|3253+jeSYZU`cewO3B)^ zINb0zGIEVkRA?0Wz?Zq4Q*EXmc2aS?x{(m)%b97@s2XWTHnU-qx#3h>C&KW~D22hKh)`A!a&~kfBT|3rA~o-&)Jy<-Cx#LDX_{`V zEz*%%54OU%#td)j;7_~@wDs1$;W&zAA7eYl7Y$#W|*Kx;BJ*IyL* zlsZeQealrX30Rt1$MS5f@CL*M#G_MHtJ}-qJ;+{Q1A!WBZ{p^bsf+BbF<)=Qbj=R}K~u*=1PfE551S=GNPfQLl-5XuVMhoH z_m)>l2poQwk_w?#k~hcrTdge5*k8M}LQ6&=Lan^w!qlP6e_ho6;_UZ^{;IE&@9HNz z{iDv?)YP$eXtcMxyQd<)RqYEaCs!BsUK`K%&Dhf1Q9#w7GiZ;%hi4g4vf+a&q8FCr zR~538yk$=!HC+^Z7d ziGv_v);+F0F1=mKOteb^vaj&^g+y8|)6p8epffUTREu=2`Lt3@o(UUa0 z*KJQd=gJ@x&#ivo+Ffttn#U-|B}Qe`J6EN?O+Et8u1&i5x*ti&qvgFN-gPt4;<0-T zyTO2)p8=hgfU0HElXbDYz(-O1LqCeDxD6Sij~PI@6Ys*WCp?aE=Zu)mD}#2@7fOC> z^qq^gex)kf2)iBzSc{xIOg|*>ObbmR)Y>H;)l`-`i!B@Cm2aotz6)xKWDz%tK z4&_Q9m)Sz3^BB@#(AAa-Q+fo$rcq{xqtI-VQtxVSyrGG&Ydv9%?({$tVIw!1#7=*4 z>WMLBWyG|#0b~88XfK(X-u=`c zQK#7yTx(huTslphSPo^jQE4m`W#0vG$`wx?EZFUrM_3J@vS2Ogah5g7)*R}o$NKw# z#MRI|S=(%5I1M_PYGunB9iMUN(TP`TgX0-_>O_h9*mbfwlqDIyCuQMBr|L`LF`paa zpuq1cr5Ml@o}{dK#N#RPBG(hO2sY?V*GnR(F;R_MT4xfXj$HKR2yKvt5UGorI5k*F z`za_^g(y<8iTTD2&oU^b{QPDQun3`1FTA_mTcg zE>NrZOk3N#{e>&B2NB3IPB=p}3ul4ecC;2nyM8$Fn2P(*l zVr@p4liwW5*Y!uzf`&+O>mi&ffFroW@z31#T5x80rFO0Js7p&WVJ+lyW}&-k5AAD? zQJnEUX+%ky@s&+5Ir(8j;u%sT$NWR)E&;9t+*DkNVXj#i<$(+HCg+xEX$}&DmzhcR zxwCOB?ki!l9J@`Q&Zh=JyMlKO@)(#6CDac7hSWH>caULGBj#pGnO@YO-nF^n zi9-#>V@tOmxKYaKLTV$cCCMoF@?Jkq92JA(Q{Yu!E_kn=G9e>?nN}^>phi%FMZe7BQGe-57(LCdm`Z>eeMjsh5}Jx&?Zm1BbVKA4G#M@TG*iya=Aj#mp~z{f4&KW<}t{dcNBF^=(xl zG}^Z`G@f1_h^DqW<_y+@$iwqmwp!5H%pEGi%KXYOZ&+ zB1yw}f|&ODR4Jod^_M!Kerb&CxT63kcK}f!COz+A@(ok)*GM2__O=zO=zyxtn%R_U z^WxU7?x=ckM7KYd%lT2w(ROs#4~~_sgL}B(rw1Q|)XKISC@2Ok17>6tQ3@OW0m2xb z0JX!;KuVelS?HC9R0^C{LlXXS^-z9WHs6zeuA)Ip9C)*SO9X_e8N;<8N^<*pjYvGU zFR7h2myOiKw#UFRXK?HXufAR+m5rTpIVzo8nnRk-TJ%7a7wqm?KX`RSgUB6u{rlA@ zTxU;}?Q@!W-|u(C;lnljSZE&jd~!W5Tm)fxnP-*+t6$*KqtaKmYb--_P_7tiHGWvg z?$g1QpZUdn60C;~ zf?=g<`PY?toOjkAG6QXHi#z2|;UiVAWjVaOodpkvO`9GvUYxse?zxi-x3~O%=F2@C z9g%!sducn0?XQTHKu5fFX>-mgpA;<7)D`$*Q-QiVpwJ%b-tr8n_ zp6;o+I(QN@hU7XNL$+^i%G?>#<7Yc-YdVVyI+S_9C`B#)q#Y{i!`X)FZWH9Uv9r~y zOiOdg6awA$#WBEuoA|Dz5qv7WZ{nxwC$oVJHJi96lnS zbdLfM{wLGO9&b`qKqYVK*o0@KNO6}6G_vMQvvU?z%1T~*<-~7^c>NSK)L(lPE_`N( zW2(~Dc(v4%iaZCQ)hq80vA5ul^30@4!-IpO0gYtFvX+!N4pZ0VI&R!4erG3%bkPrJ zyo5#H+EyzZEwbXFLBET12B{zsJMZW0BJ-|*63>E9G5t}MB%-%W!(GVW{3R%mCr4}y z*cqI@$@ zrO#?4j03V;Ww?$Foc88yRgL&8?pp@gdwh&B>CfW`DKn=1L26|*9_ZLGhdm(+(axH{ z#%opiPvM=BQ@VA*X%Rz8yM^AeY;ukleK#`mog{BA@!BuJ-&qoUMb%#Uo0)d*=%;p8 zVlR>}?KqmJd#!cFe9~eQ8k6gceu-2NtVLDDCRsDY6u2FU-k_vsoEO7K;FQ5*m-B+; zrd60v3Jt1f(?2m>5HwHmq>Ze%bMFuUuON0MT{F zLVhy|P3$GM&-YthLVfwi-f_NUk&$42vh3oWx|+J~XJ4ClQa)rc{quuoUU$>{z#=KX z51#^xoTl!HR6{psrL0JI=$^BB5JkAJF!t|e3utu(o3{}*qqcWQ1y3+y;0rgQcNj#t z>jzHmZl}00P9eM9W!@j%?XjKb{wQnbwZNpeM5y?=_WqWxslHmOlcJ9wXS^SC-dUnQleg z;7QAy!%BQ&d@MzZw#1t_&Rv!iZVjJ^_q2_fyg11f2rrqsP;$ioJ_SjbwYpV0!7_pi z(iD@vu3>sk@EWx$vf`vGDzhC7ybTBFaK#ZU!*8c$D2JMIf`)rQGKoy$;}#T{E$zD7%0 zmsmvU>;?<=sI@L80Pg!8y)@Ye_{==^uQ>=`3sth@g_=_G3>00m$}3d`;x&bH-j*u-Kt2f^=PVycT*Wp&w`Ma1{eE%88AzH720tyiY?0kYXMfrMAA zClUE-NAq6oBfI+z1<;YpC#ZJ0ID5__39jC(P*&bdI2-lDlN;(IfYgLoC~otVZCToH z2}j2~ZN;kD*__#&;P!>^%ooUoPgtFH4+5XyM9{xo3sv`VDEtO^ae{NV_`*DHFLZUv zIFiUEsqBALBNKs7zoX-jRTIgMui))#|NM&hi6cRU5Kkv#;0{ZhjvIf%HuE~ld{cSpK@Rr8GD7fv&oq#~Y= z<)SzA@Y?s`ZcqH5Q;)zG(!L8eDF3>M(%c?6Y$E;mu7v&|_CQiS*S)zX3uq1j*aceh zRP=^anAS0AeN?vcK(dA2N#ZTOX)o+&9z+*oUl}=SAafS({(iV3Iv%l*H>;$v>P0d@j28S0R{{`)%wQLW}-Rl|b14 zSgo_FdYiu@2&V*Ct+&*n=o5KoGJ~{e+;ukj`^4YCb)ra=rx7At;X3sZ zP2FsyaFFW!pp8R4{5FLWD@j`cSG{?rPVU!*vVFz-)WDw6HMaNaASmm(mKc#bOkzC@ z`Ex7mh7T}Ps|Mc9?F_1LJ|Lm)X(O;Sb;x#7u{wn}4U`g4oE9JPR1PUwvUOR0YaRg^ z+lE*+?OZZVF=-vWLWTV16&CW=EoAx$A$gM`L{xIh?;V|PaU;Z+&27I==?7C@4+f8< zi_>?_F&o>4H>{hAVtd+$gU^C5A*w-2SQ}`0ru+=XGK&x?H0Ev;PaL0++j!(VGdrZ5 zsDw9*Z6I2KbR17YA-8vPk@Hi(>3i0dc+8~|;cHQ01Bg^V*tnofEx|RtiMB`8@NAU~ zP?vyg0yoeP!x0_~0#XB{#bmMWA9NoPxt*u&JK$F~WF1dWVINBsxfId_tQ|yvNi)|6 z5Ssc6Z0wfT-cMJU0d3HhZ+- zop}&shj@yuOy;$dUdvdEh2O%}@wK=eGD56O2EWqQ@dZ1QOx=(;os0cfh_Jr5V@e`; z=5`Phw9&*0Tm}+r=hrZ}5D&3 z4*aZ|XT)Bg2F0N7**8>ku)|Qe0x_cT_SBMG7y?Uxt<`<`l)WlB{p+Q)H6f!42HmyZ z!c@DugxN|txUx{IR|Z(wU}dpMx+|y-jKL_*j+Ct$Hkzr}WOF}(Kt}=>Kbf|VvAeie z0h-orv?>8SU#hlibpi|_{GT-xHr(K}N6(!G@q{gi||SV*0Q?_)#GESDFQ^Igq- z78=}v`9qndQO5~_I`zM2w(9gqw@KNV0_0mE>F(2jDEfjbngIT+V5^ZqdqmuOK zZGx|9{4ZVRzgveQy1A(FAgQ(SHq=F0E}d%?PT!YXs_%CXWI_a^#TBxo zDd}~?E81w=E0Hut&dcn$<=3NGDOMpIwi7o1dAnm{z|m@76%NN17&5APltezx$UaWU zN4!gcRra2|wEP}*z;l9lr7ZbrBtu)tN<#_X&gn=0egh`I7ADT5COEg-ntZSK4|T`Pjgg zT*W8&dR(5T!?5OrN=?-l^X0|5W(A1VImvDb;ryiUE3`L14DyxE@9>Tn5rBf?cjxu;gsd~$*Vw(k+oY}>`a^3o$^=2*Bm$9_>FFiJ};a9wfhIB_wdb(SodBzY=z;sdP zOQtp_bb9^}XsmPHH$g|;<63f^Zn;Y}LT%i(U@r?l?e4ove^{|DbQ<&XpL|>mJX$!< z2x3WB*t~tR=N8Wz-IJGYi16(${2523dt5R2@vV#A`d`}Ny3`NYYU6Onwi1JIpM$g~ zY1MX`f}PdV_`-oR)u0-{KJ+-Amo%EDrbFfz6KsE7MkDG()bKs(&LYJD1&vQO#M24E<8umJTQ_7l%iJju>nZdPz~a zeE9lEz8KD1{!fL1`au*JO)=8@nfo)l`FUGHD=dQ-ac;9d!lrnLX`<;V+@1%AN=>Ng z4qU>E-FxbH2aDCpJ(?K>aS4~&_SwxbTE}&$a&V56?P0#uC~~cNSw^z*v8GSX%y*)c zhBcK#M4llop8FG>TIX$EnG+!#JqM8Vex6)=B+nI_5~U!qaqw_)&zIxN6MK|lbqz7w zPC#1-iV>|szOYLMNi;=R@pP2jCCBSv3e66GEQV9UJ+^nECuz@MW)+15Wb6*X zmi$*tnI6Ouq|Nh*0@WP_#SDO7f}~%NR)`9)h3b~heInE?TB##6^M@NI1uY2&V_y@- z+viF~Y7w2gNvasvjh;Z#8);mcBzzfxk( zKf!Rk*Co!ZeB=ethJP2 zE1&d%Uhy@scVhK0Rp}y%e}0V(=@#$PTf)bxl{##bK;raV9u4m7$^B zx*cQxI^`~f$UOx*FHeEDLAtb(CXY0SSSC<4Ms^)YPx`yI1OY=f-r~2xW?UDi($fb+ zDBMj6SeZcyc8GN;?lzcn$1MxlM8l0#YkzKz>T{4yE|0L6j{ zM4F`SxX<~b7$@xyj?bJtWi7-}@Jr1Uc*?x++Z5pHeDiyQG{$6NFn|AE?HxssB>E$J zcHVwiWV!-#LEPu80TyOj7yW>Xk|w6kyS>_i9biW!JQ9}gn7xa(L{aVNZOX>4kW%fC z6Bee!Qlwr}qtR_7%a(xTpdZ7`QdOT~tE2Zh3#k=*u#9vixIZpwVCdJMB(7;=;sK{^ z?cEM=S9uu{DwcRG9XaKO+z6t5gn|SFu!+MWt7ww9sK5`E3|JH1sHr6uy;J__n zq7T$ywQ+V*CwkAG!j_RNO{#(+ogotFo|Ykk-`1smL)TklxiCxJIk<{?RSMp9dEm;_;>9tEGu zX-<)8ZlBNYhE8O-OP+fqZ96z(9~QjPrV`@vF4Qf!j>LQ z*SRhI2;Hru20qX;5#MYj1VNY7pqoFNB@aBH#Vj{;3y^(#`!83iCYzAp1% z$s86i`)?tI1jV!Af+~lZh=a^6^TA)tgFjUN0^phs+#kjd9ub5!f)5yR5_Cfc-=5;& z19JZb3d8>&NCEO4!2qz*bu*&n2GoYY#SkknNwj~2W^btK^^1&*)~Q% z3St<=00cy^V;X^D4EB-1j*x%tyh8B5I}?(FBEcX`>8EuA@ad?5Pv`Gz`$7i793uuz zk5U6p%56bKNZ|7&fPermWdBEDrvyX7FbAhPZXmTWJSb8Y@X_%=z%f!lW*Nc#9QYJ? z(IFrx|6>=hlL#vqq6V#xVE{6bo5iWY4RXr>eqj9t0eHX=0Q8;@4#YH$0r>yipx~z( z*; z(Nq=&w~spbx;g&ZCCdUtVNVFEvLOPoPGJE4UoaQkPC|cy4Ax*!|9`U43p0RVr= zX8i@Fd4M4*u!7=XWAP5*-LQvVO81^ngb_ZKvs@#g|baxp=h za~OcX+`aw+>9hZUG>fc&|8|A>m$S?RLzEz@Q9QuEQxE^^FY_;iSr5ACq6FJt2ex8l@c$+F7j)4Ha#?!!zdc6x5*}1 Date: Fri, 28 Dec 2018 12:37:57 -0800 Subject: [PATCH 4/5] fixed the refresh policy on ldap group refresh, so as not to apply to tables other than of type policy. --- .../org/apache/spark/sql/hive/SnappyHiveExternalCatalog.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 c7c1c4bc6d..c4bb5fcd3f 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 @@ -553,7 +553,8 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, def refreshPolicies(ldapGroup: String): Unit = { val qualifiedLdapGroup = Constants.LDAP_GROUP_PREFIX + ldapGroup - getAllTables().foreach { table => + getAllTables().filter(_.provider.map(_.equalsIgnoreCase("policy")). + getOrElse(false)).foreach { table => val applyToStr = table.properties(PolicyProperties.policyApplyTo) if (applyToStr.nonEmpty) { val applyTo = applyToStr.split(",") From 563c39e6ed07dbfe80346765d3b32b67c04b1bf2 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 29 Dec 2018 02:47:28 +0530 Subject: [PATCH 5/5] [SNAP-2818] trim the JOB_DESCRIPTION property in Spark jobs (#1227) - previous changes set SparkContext.SPARK_JOB_DESCRIPTION property to the query string for SnappySession.sql executions but this can exceed 32K and property will fail in serialization, so trim it to 100 size with "..." continuation like done in SQL tab display - added large view test to ViewTest and enhanced it to accept a generic "String => DataFrame" closure so that the same can be used for scala tests for SnappySession as well as dunits for JDBC Statement.execute; added the same tests to DDLRoutingDUnitTest using this Note: the above test is unable the reproduce the original issue with CREATE VIEW but it does reproduce for a large query string - disallow CREATE INDEX creation on column tables without experimental-features property - clear catalog cache in shutdown to avoid its accidental use by subsequent tests --- .../cluster/DDLRoutingDUnitTest.scala | 63 ++- .../cluster/QueryRoutingDUnitTest.scala | 1 + .../cluster/QueryRoutingSingleNodeSuite.scala | 4 +- .../spark/sql/store/SQLMetadataTest.scala | 6 +- .../SplitClusterDUnitSecurityTest.scala | 6 +- .../cluster/SplitClusterDUnitTestBase.scala | 18 +- .../apache/spark/sql/CachedDataFrame.scala | 7 +- .../org/apache/spark/sql/SnappySession.scala | 29 +- .../apache/spark/sql/collection/Utils.scala | 10 +- .../columnar/impl/ColumnFormatRelation.scala | 9 +- .../org/apache/spark/sql/execution/ddl.scala | 5 +- .../sql/hive/SnappyHiveExternalCatalog.scala | 15 +- .../api/JavaCreateIndexTestSuite.java | 2 + .../scala/io/snappydata/SnappyFunSuite.scala | 24 +- .../apache/spark/sql/store/MetadataTest.scala | 23 +- .../org/apache/spark/sql/store/ViewTest.scala | 397 +++++++++++------- .../test/java/io/snappydata/hydra/smoke.sh | 2 - gradle.properties | 7 + 18 files changed, 387 insertions(+), 241 deletions(-) diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala index 2d1bef88a2..8d789bfec7 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/DDLRoutingDUnitTest.scala @@ -19,10 +19,12 @@ package io.snappydata.cluster import java.sql.{Connection, DriverManager, SQLException} import com.pivotal.gemfirexd.internal.engine.{GfxdConstants, Misc} +import io.snappydata.SnappyFunSuite.resultSetToDataset import io.snappydata.test.dunit.{AvailablePortHelper, SerializableRunnable} -import org.apache.spark.sql.SnappyContext import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.store.ViewTest +import org.apache.spark.sql.{Dataset, Row, SnappyContext, SnappySession} class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { @@ -72,12 +74,12 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { val conn = getANetConnection(netPort1) // first fail a statement - failCreateTableXD(conn, tableName, true, " row ") + failCreateTableXD(conn, tableName, doFail = true, " row ") createTableXD(conn, tableName, " row ") tableMetadataAssertRowTable("APP", tableName) // Test create table - error for recreate - failCreateTableXD(conn, tableName, false, " row ") + failCreateTableXD(conn, tableName, doFail = false, " row ") // Drop Table and Recreate dropTableXD(conn, tableName) @@ -167,7 +169,7 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { vm2.invoke(classOf[ClusterManagerTestBase], "stopAny") val props = bootProps.clone().asInstanceOf[java.util.Properties] - props.put("distributed-system-id" , "1") + props.put("distributed-system-id", "1") props.put("server-groups", "sg1") val restartServer = new SerializableRunnable() { @@ -185,7 +187,7 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { var s = conn.createStatement() s.execute(s"CREATE TABLE $tableName (Col1 INT, Col2 INT, Col3 STRING)") insertDataXD(conn, tableName) - var snc = org.apache.spark.sql.SnappyContext(sc) + val snc = org.apache.spark.sql.SnappyContext(sc) verifyResultAndSchema(snc, tableName, 3) s.execute(s"ALTER TABLE $tableName ADD Col4 INT") @@ -207,21 +209,21 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { s.execute(s"insert into $tableName values (1,1)") s.execute(s"ALTER TABLE $tableName add constraint emp_uk unique (Col1)") try { - s.execute(s"insert into $tableName values (1,1)") + s.execute(s"insert into $tableName values (1,1)") } catch { case sqle: SQLException => if (sqle.getSQLState != "23505" || - !sqle.getMessage.contains("duplicate key value in a unique or" + - " primary key constraint or unique index")) { + !sqle.getMessage.contains("duplicate key value in a unique or" + + " primary key constraint or unique index")) { throw sqle } } // asynceventlistener s.execute("CREATE ASYNCEVENTLISTENER myListener (" + - " listenerclass 'com.pivotal.gemfirexd.callbacks.DBSynchronizer'" + - " initparams 'org.apache.derby.jdbc.EmbeddedDriver,jdbc:derby:newDB;create=true')" + - " server groups(sg1)") + " listenerclass 'com.pivotal.gemfirexd.callbacks.DBSynchronizer'" + + " initparams 'org.apache.derby.jdbc.EmbeddedDriver,jdbc:derby:newDB;create=true')" + + " server groups(sg1)") s.execute(s"ALTER TABLE $tableName SET ASYNCEVENTLISTENER (myListener) ") var rs = s.executeQuery(s"select * from SYS.SYSTABLES where tablename='$tableName'") @@ -287,7 +289,8 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { var cnt = 0 while (rs.next()) { cnt += 1 - rs.getInt(1); rs.getInt(2); + rs.getInt(1) + rs.getInt(2) } assert(cnt == 5, cnt) @@ -296,7 +299,9 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { cnt = 0 while (rs2.next()) { cnt += 1 - rs2.getInt(1); rs2.getInt(2); rs2.getInt(3); + rs2.getInt(1) + rs2.getInt(2) + rs2.getInt(3) } assert(cnt == 5, cnt) @@ -324,6 +329,36 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { dropTableXD(conn, tableName) } + def testViews(): Unit = { + val netPort1 = AvailablePortHelper.getRandomAvailableTCPPort + vm2.invoke(classOf[ClusterManagerTestBase], "startNetServer", netPort1) + + val session = new SnappySession(sc) + ViewTest.createTables(session) + + def newExecution(): String => Dataset[Row] = { + val session = new SnappySession(sc) + val conn = getANetConnection(netPort1) + val stmt = conn.createStatement() + resultSetToDataset(session, stmt) + } + + val conn = getANetConnection(netPort1) + val stmt = conn.createStatement() + ViewTest.testTemporaryView(resultSetToDataset(session, stmt), newExecution) + ViewTest.testGlobalTemporaryView(resultSetToDataset(session, stmt), newExecution) + ViewTest.testTemporaryViewUsing(resultSetToDataset(session, stmt), newExecution) + ViewTest.testGlobalTemporaryViewUsing(resultSetToDataset(session, stmt), newExecution) + ViewTest.testPersistentView(resultSetToDataset(session, stmt), checkPlans = false, + newExecution, restartSpark) + ViewTest.dropTables(new SnappySession(sc)) + } + + private def restartSpark(): Unit = { + ClusterManagerTestBase.stopAny() + ClusterManagerTestBase.startSnappyLead(ClusterManagerTestBase.locatorPort, bootProps) + } + def createTableXD(conn: Connection, tableName: String, usingStr: String): Unit = { val s = conn.createStatement() @@ -421,7 +456,7 @@ class DDLRoutingDUnitTest(val s: String) extends ClusterManagerTestBase(s) { s.execute("CREATE EXTERNAL TABLE airlineRef_temp(Code VARCHAR(25), " + "Description VARCHAR(25)) USING parquet OPTIONS()") } catch { - case e: java.sql.SQLException => + case _: java.sql.SQLException => // println("Exception stack. create. ex=" + e.getMessage + // " ,stack=" + ExceptionUtils.getFullStackTrace(e)) } diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala index fa4ec20a2d..4e507e39be 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/QueryRoutingDUnitTest.scala @@ -761,6 +761,7 @@ class QueryRoutingDUnitTest(val s: String) TPCHUtils.createAndLoadTables(snc, true) + snc.setConf(Property.EnableExperimentalFeatures.name, "true") snc.sql( s"""CREATE INDEX idx_orders_cust ON orders(o_custkey) options (COLOCATE_WITH 'customer') diff --git a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala index 39194fa11a..4600a926fa 100644 --- a/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala +++ b/cluster/src/test/scala/io/snappydata/cluster/QueryRoutingSingleNodeSuite.scala @@ -25,7 +25,7 @@ import io.snappydata.{SnappyFunSuite, SnappyTableStatsProviderService} import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.SnappySession -import org.apache.spark.sql.store.{ColumnTableBatchInsertTest, MetadataTest} +import org.apache.spark.sql.store.ColumnTableBatchInsertTest class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll { @@ -707,7 +707,7 @@ class QueryRoutingSingleNodeSuite extends SnappyFunSuite with BeforeAndAfterAll val connSession = allSessions.head // skip the "isCached" checks with JDBC since session is different for JDBC connection ColumnTableBatchInsertTest.testSparkCachingUsingSQL(sc, - MetadataTest.resultSetToDataset(connSession, stmt), connSession.catalog.isCached, + SnappyFunSuite.resultSetToDataset(connSession, stmt), connSession.catalog.isCached, df => connSession.sharedState.cacheManager.lookupCachedData(df).isDefined) stmt.close() } finally { 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 37c4a2e4af..68d9812f98 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 @@ -45,7 +45,7 @@ class SQLMetadataTest extends SnappyFunSuite { val conn = DriverManager.getConnection(s"jdbc:snappydata://localhost:$netPort") try { val stmt = conn.createStatement() - MetadataTest.testSYSTablesAndVTIs(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testSYSTablesAndVTIs(SnappyFunSuite.resultSetToDataset(session, stmt), netServers = Seq(s"localhost/127.0.0.1[$netPort]")) stmt.close() } finally { @@ -58,7 +58,7 @@ class SQLMetadataTest extends SnappyFunSuite { val conn = DriverManager.getConnection(s"jdbc:snappydata://localhost:$netPort") try { val stmt = conn.createStatement() - MetadataTest.testDescribeShowAndExplain(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testDescribeShowAndExplain(SnappyFunSuite.resultSetToDataset(session, stmt), usingJDBC = true) stmt.close() } finally { @@ -71,7 +71,7 @@ class SQLMetadataTest extends SnappyFunSuite { val conn = DriverManager.getConnection(s"jdbc:snappydata://localhost:$netPort") try { val stmt = conn.createStatement() - MetadataTest.testDSIDWithSYSTables(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testDSIDWithSYSTables(SnappyFunSuite.resultSetToDataset(session, stmt), Seq(s"localhost/127.0.0.1[$netPort]")) stmt.close() } finally { diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala index 946179e55e..e7ec9299a5 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitSecurityTest.scala @@ -681,8 +681,7 @@ class SplitClusterDUnitSecurityTest(s: String) s"CREATE TEMPORARY TABLE ${t1}temp AS SELECT id, name FROM $schema.$t1", s"CREATE GLOBAL TEMPORARY TABLE ${t1}tempg AS SELECT id, name FROM $schema.$t1", s"CREATE EXTERNAL TABLE $schema.${t1}ext USING csv OPTIONS(path " + - s"'../../quickstart/src/main/resources/customer.csv')", - s"CREATE INDEX $schema.idx ON $schema.$t1 (id, name)") + s"'../../quickstart/src/main/resources/customer.csv')") .foreach(executeSQL(user1Stmt, _)) // user gemfire2 of same group gemGroup1 @@ -698,7 +697,6 @@ class SplitClusterDUnitSecurityTest(s: String) s"select * from $schema.$t2", s"delete from $schema.$t1 where name like 'two'", s"drop table $schema.$t1r", - s"drop index $schema.idx", s"select * from $schema.$t2").foreach(executeSQL(user2Stmt, _)) // user gemfire1 @@ -724,7 +722,7 @@ class SplitClusterDUnitSecurityTest(s: String) s"CREATE INDEX $schema.idx4 ON $schema.$t1 (id, name)") .foreach(sql => assertFailures(() => { executeSQL(user4Stmt, sql) - }, sql, Seq("42500", "42502", "42506", "42507"))) + }, sql, Seq("42500", "42502", "42506", "42507", "38000"))) // Grant DML permissions to gemfire4 and ensure it works. executeSQL(user1Stmt, s"grant select on $schema.$t1 to ldapgroup:$group2") diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index d951199642..cf435524e9 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -32,7 +32,7 @@ import com.pivotal.gemfirexd.internal.engine.Misc import io.snappydata.test.dunit.{SerializableRunnable, VM} import io.snappydata.test.util.TestException import io.snappydata.util.TestUtils -import io.snappydata.{ColumnUpdateDeleteTests, Constant} +import io.snappydata.{ColumnUpdateDeleteTests, Constant, SnappyFunSuite} import org.junit.Assert import org.apache.spark.sql.catalyst.InternalRow @@ -296,11 +296,11 @@ trait SplitClusterDUnitTestObject extends Logging { netServers, locatorId, locatorNetServer, servers, leadId) // next test metadata using JDBC connection stmt = jdbcConn.createStatement() - MetadataTest.testSYSTablesAndVTIs(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testSYSTablesAndVTIs(SnappyFunSuite.resultSetToDataset(session, stmt), hostName = "localhost", netServers, locatorId, locatorNetServer, servers, leadId) - MetadataTest.testDescribeShowAndExplain(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testDescribeShowAndExplain(SnappyFunSuite.resultSetToDataset(session, stmt), usingJDBC = true) - MetadataTest.testDSIDWithSYSTables(MetadataTest.resultSetToDataset(session, stmt), + MetadataTest.testDSIDWithSYSTables(SnappyFunSuite.resultSetToDataset(session, stmt), netServers, locatorId, locatorNetServer, servers, leadId) stmt.close() @@ -417,8 +417,10 @@ trait SplitClusterDUnitTestObject extends Logging { SnappyContext.getClusterMode(snc.sparkContext) match { case ThinClientConnectorMode(_, _) => // test index create op - snc.createIndex("tableName" + "_index", tableName, Map("COL1" -> None), - Map.empty[String, String]) + if ("row".equalsIgnoreCase(tableType)) { + snc.createIndex("tableName" + "_index", tableName, Map("COL1" -> None), + Map.empty[String, String]) + } case _ => } @@ -427,7 +429,9 @@ trait SplitClusterDUnitTestObject extends Logging { SnappyContext.getClusterMode(snc.sparkContext) match { case ThinClientConnectorMode(_, _) => // test index drop op - snc.dropIndex("tableName" + "_index", ifExists = false) + if ("row".equalsIgnoreCase(tableType)) { + snc.dropIndex("tableName" + "_index", ifExists = false) + } case _ => } } 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 61cac1d3e9..f70d898e80 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -106,6 +106,9 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti @transient private[sql] var currentLiterals: Array[ParamLiteral] = _ + @transient + private[sql] var queryShortString: String = _ + @transient private[sql] var queryString: String = _ @@ -288,7 +291,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti try { didPrepare = prepareForCollect() val (result, elapsedMillis) = CachedDataFrame.withNewExecutionId(snappySession, - queryString, queryString, currentQueryExecutionString, currentQueryPlanInfo, + queryShortString, queryString, currentQueryExecutionString, currentQueryPlanInfo, currentExecutionId, planStartTime, planEndTime)(body) (result, elapsedMillis * 1000000L) } finally { @@ -613,7 +616,7 @@ object CachedDataFrame else Utils.nextExecutionIdMethod.invoke(SQLExecution).asInstanceOf[Long] val executionIdStr = java.lang.Long.toString(executionId) localProperties.setProperty(SQLExecution.EXECUTION_ID_KEY, executionIdStr) - localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, queryLongForm) + localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, queryShortForm) localProperties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, executionIdStr) val startTime = System.currentTimeMillis() 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 b49f812901..d7c8fa3516 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -1835,8 +1835,8 @@ object SnappySession extends Logging { * data to the active executions. SparkListenerSQLPlanExecutionEnd is * then sent with the accumulated time of both the phases. */ - private def planExecution(qe: QueryExecution, session: SnappySession, sqlText: String, - executedPlan: SparkPlan, paramLiterals: Array[ParamLiteral], paramsId: Int) + private def planExecution(qe: QueryExecution, session: SnappySession, sqlShortText: String, + sqlText: String, executedPlan: SparkPlan, paramLiterals: Array[ParamLiteral], paramsId: Int) (f: => RDD[InternalRow]): (RDD[InternalRow], String, SparkPlanInfo, String, SparkPlanInfo, Long, Long, Long) = { // Right now the CachedDataFrame is not getting used across SnappySessions @@ -1845,7 +1845,7 @@ object SnappySession extends Logging { val context = session.sparkContext val localProperties = context.getLocalProperties localProperties.setProperty(SQLExecution.EXECUTION_ID_KEY, executionIdStr) - localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, sqlText) + localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, sqlShortText) localProperties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, executionIdStr) val start = System.currentTimeMillis() try { @@ -1870,8 +1870,8 @@ object SnappySession extends Logging { } } - private def evaluatePlan(qe: QueryExecution, session: SnappySession, sqlText: String, - paramLiterals: Array[ParamLiteral], paramsId: Int): CachedDataFrame = { + private def evaluatePlan(qe: QueryExecution, session: SnappySession, sqlShortText: String, + sqlText: String, paramLiterals: Array[ParamLiteral], paramsId: Int): CachedDataFrame = { val (executedPlan, withFallback) = getExecutedPlan(qe.executedPlan) var planCaching = session.planCaching @@ -1906,7 +1906,7 @@ object SnappySession extends Logging { case _ => true } else true // post final execution immediately (collect for these plans will post nothing) - CachedDataFrame.withNewExecutionId(session, sqlText, sqlText, executionStr, planInfo, + CachedDataFrame.withNewExecutionId(session, sqlShortText, sqlText, executionStr, planInfo, postGUIPlans = postGUIPlans) { // create new LogicalRDD plan so that plan does not get re-executed // (e.g. just toRdd is not enough since further operators like show will pass @@ -1922,14 +1922,15 @@ object SnappySession extends Logging { case plan: CollectAggregateExec => val (childRDD, origExecutionStr, origPlanInfo, executionStr, planInfo, executionId, - planStartTime, planEndTime) = planExecution(qe, session, sqlText, plan, paramLiterals, - paramsId)(if (withFallback ne null) withFallback.execute(plan.child) else plan.childRDD) + planStartTime, planEndTime) = planExecution(qe, session, sqlShortText, sqlText, plan, + paramLiterals, paramsId)( + if (withFallback ne null) withFallback.execute(plan.child) else plan.childRDD) (childRDD, qe, origExecutionStr, origPlanInfo, executionStr, planInfo, childRDD.id, true, executionId, planStartTime, planEndTime) case plan => val (rdd, origExecutionStr, origPlanInfo, executionStr, planInfo, executionId, - planStartTime, planEndTime) = planExecution(qe, session, sqlText, plan, + planStartTime, planEndTime) = planExecution(qe, session, sqlShortText, sqlText, plan, paramLiterals, paramsId) { plan match { case p: CollectLimitExec => @@ -1993,6 +1994,7 @@ object SnappySession extends Logging { def sqlPlan(session: SnappySession, sqlText: String): CachedDataFrame = { val parser = session.sessionState.sqlParser + val sqlShortText = CachedDataFrame.queryStringShortForm(sqlText) val plan = parser.parsePlan(sqlText, clearExecutionData = true) val planCaching = session.planCaching val paramLiterals = parser.sqlParser.getAllLiterals @@ -2007,7 +2009,7 @@ object SnappySession extends Logging { session.currentKey = key try { val execution = session.executePlan(plan) - cachedDF = evaluatePlan(execution, session, sqlText, paramLiterals, paramsId) + cachedDF = evaluatePlan(execution, session, sqlShortText, sqlText, paramLiterals, paramsId) // put in cache if the DF has to be cached if (planCaching && cachedDF.isCached) { if (isTraceEnabled) { @@ -2026,12 +2028,13 @@ object SnappySession extends Logging { logDebug(s"Using cached plan for: $sqlText (existing: ${cachedDF.queryString})") cachedDF = cachedDF.duplicate() } - handleCachedDataFrame(cachedDF, plan, session, sqlText, paramLiterals, paramsId) + handleCachedDataFrame(cachedDF, plan, session, sqlShortText, sqlText, paramLiterals, paramsId) } private def handleCachedDataFrame(cachedDF: CachedDataFrame, plan: LogicalPlan, - session: SnappySession, sqlText: String, paramLiterals: Array[ParamLiteral], - paramsId: Int): CachedDataFrame = { + session: SnappySession, sqlShortText: String, sqlText: String, + paramLiterals: Array[ParamLiteral], paramsId: Int): CachedDataFrame = { + cachedDF.queryShortString = sqlShortText cachedDF.queryString = sqlText if (cachedDF.isCached && (cachedDF.paramLiterals eq null)) { cachedDF.paramLiterals = paramLiterals 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 baab7912c9..4ec0b01665 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 @@ -20,7 +20,7 @@ import java.io.ObjectOutputStream import java.lang.reflect.Method import java.net.{URL, URLClassLoader} import java.nio.ByteBuffer -import java.sql.DriverManager +import java.sql.{DriverManager, ResultSet} import java.util.TimeZone import scala.annotation.tailrec @@ -41,6 +41,7 @@ import org.apache.commons.math3.distribution.NormalDistribution import org.eclipse.collections.impl.map.mutable.UnifiedMap import org.apache.spark._ +import org.apache.spark.executor.InputMetrics import org.apache.spark.io.CompressionCodec import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD @@ -56,7 +57,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, analysis} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.columnar.ExternalStoreUtils.CaseInsensitiveMutableHashMap -import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, DriverWrapper} +import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, DriverWrapper, JdbcUtils} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SnappySessionCatalog import org.apache.spark.sql.sources.{CastLongTime, JdbcExtendedUtils} @@ -678,6 +679,11 @@ object Utils { def createCatalystConverter(dataType: DataType): Any => Any = CatalystTypeConverters.createToCatalystConverter(dataType) + def resultSetToSparkInternalRows(resultSet: ResultSet, schema: StructType, + inputMetrics: InputMetrics = new InputMetrics): Iterator[InternalRow] = { + JdbcUtils.resultSetToSparkInternalRows(resultSet, schema, inputMetrics) + } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisUtc: Long, tz: TimeZone): Int = { // SPARK-6785: use Math.floor so negative number of days (dates before 1970) 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 60157eb4ff..b730edec6d 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 @@ -23,7 +23,7 @@ import scala.util.control.NonFatal import com.gemstone.gemfire.internal.cache.{ExternalTableMetaData, LocalRegion} import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer -import io.snappydata.Constant +import io.snappydata.{Constant, Property} import io.snappydata.sql.catalog.{RelationInfo, SnappyExternalCatalog} import org.apache.spark.rdd.RDD @@ -518,8 +518,13 @@ class ColumnFormatRelation( indexColumns: Map[String, Option[SortDirection]], options: Map[String, String]): DataFrame = { - val parameters = new CaseInsensitiveMutableHashMap(options) val session = sqlContext.sparkSession.asInstanceOf[SnappySession] + // only allow if experimental-features are enabled + if (!Property.EnableExperimentalFeatures.get(session.sessionState.conf)) { + throw new UnsupportedOperationException( + "CREATE INDEX on column tables is an experimental unsupported feature") + } + val parameters = new CaseInsensitiveMutableHashMap(options) val parser = session.snappyParser val indexCols = indexColumns.keys.map(parser.parseSQLOnly(_, parser.parseIdentifier.run())) val catalog = session.sessionCatalog 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 591bdde9c7..b6e73eeafc 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 @@ -367,9 +367,10 @@ case class SnappyCacheTableCommand(tableIdent: TableIdentifier, queryString: Str if (isOffHeap) df.persist(StorageLevel.OFF_HEAP) else df.persist() Nil } else { + val queryShortString = CachedDataFrame.queryStringShortForm(queryString) val localProperties = session.sparkContext.getLocalProperties val previousJobDescription = localProperties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION) - localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, queryString) + localProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, queryShortString) try { session.sessionState.enableExecutionCache = true // Get the actual QueryExecution used by InMemoryRelation so that @@ -386,7 +387,7 @@ case class SnappyCacheTableCommand(tableIdent: TableIdentifier, queryString: Str }.get val planInfo = PartitionedPhysicalScan.getSparkPlanInfo(cachedExecution.executedPlan) Row(CachedDataFrame.withCallback(session, df = null, cachedExecution, "cache")(_ => - CachedDataFrame.withNewExecutionId(session, queryString, queryString, + CachedDataFrame.withNewExecutionId(session, queryShortString, queryString, cachedExecution.toString(), planInfo)({ val start = System.nanoTime() // Dummy op to materialize the cache. This does the minimal job of count on 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 c4bb5fcd3f..6662dfe07a 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 @@ -140,7 +140,7 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, } catch { case he: Exception if isDisconnectException(he) => // stale JDBC connection - closeHive() + closeHive(clearCache = false) suspendActiveSession { hiveClient = hiveClient.newSession() } @@ -236,12 +236,13 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, } override def listDatabases(): Seq[String] = { - withHiveExceptionHandling(super.listDatabases().map(toUpperCase)) :+ SYS_SCHEMA + (withHiveExceptionHandling(super.listDatabases().map(toUpperCase).toSet) + SYS_SCHEMA) + .toSeq.sorted } override def listDatabases(pattern: String): Seq[String] = { - withHiveExceptionHandling(super.listDatabases(pattern).map(toUpperCase)) ++ - StringUtils.filterPattern(Seq(SYS_SCHEMA), pattern) + (withHiveExceptionHandling(super.listDatabases(pattern).map(toUpperCase).toSet) ++ + StringUtils.filterPattern(Seq(SYS_SCHEMA), pattern)).toSeq.sorted } override def setCurrentDatabase(schema: String): Unit = { @@ -730,7 +731,8 @@ class SnappyHiveExternalCatalog private[hive](val conf: SparkConf, override def close(): Unit = {} - private[hive] def closeHive(): Unit = synchronized { + private[hive] def closeHive(clearCache: Boolean): Unit = synchronized { + if (clearCache) invalidateAll() // Non-isolated client can be closed here directly which is only present in cluster mode // using the new property HiveUtils.HIVE_METASTORE_ISOLATION not present in upstream. // Isolated loader would require reflection but that case is only in snappy-core @@ -800,7 +802,8 @@ object SnappyHiveExternalCatalog { def close(): Unit = synchronized { if (instance ne null) { - instance.withHiveExceptionHandling(instance.closeHive(), handleDisconnects = false) + instance.withHiveExceptionHandling(instance.closeHive(clearCache = true), + handleDisconnects = false) instance = null } } diff --git a/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java b/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java index a0d3fcb1e4..ac7d06afe9 100644 --- a/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java +++ b/core/src/test/java/io/snappydata/api/JavaCreateIndexTestSuite.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; +import io.snappydata.Property; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -44,6 +45,7 @@ public class JavaCreateIndexTestSuite implements Serializable { @Before public void setUp() { + snc.setConf(Property.EnableExperimentalFeatures().name(), "true"); List dummyList = new ArrayList(); for (int i = 0; i < 2; i++) { DummyBeanClass object = new DummyBeanClass(); diff --git a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala index b57b0fede1..9361e16db7 100644 --- a/core/src/test/scala/io/snappydata/SnappyFunSuite.scala +++ b/core/src/test/scala/io/snappydata/SnappyFunSuite.scala @@ -17,6 +17,7 @@ package io.snappydata import java.io.File +import java.sql.Statement import scala.collection.mutable.ArrayBuffer @@ -27,10 +28,15 @@ import io.snappydata.test.dunit.DistributedTestBase.{InitializeRun, WaitCriterio import io.snappydata.util.TestUtils import org.scalatest.Assertions +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, EqualNullSafe, EqualTo, Exists, ExprId, Expression, ListQuery, PredicateHelper, PredicateSubquery, ScalarSubquery} import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, OneRowRelation, Sample} import org.apache.spark.sql.catalyst.util.{sideBySide, stackTraceToString} -import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row} +import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils +import org.apache.spark.sql.row.SnappyStoreDialect +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SnappySession} // scalastyle:off import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome, Retries} // scalastyle:on @@ -234,6 +240,22 @@ object SnappyFunSuite extends Assertions { assert(query.queryExecution.executedPlan.missingInput.isEmpty, s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}") } + + /** + * Converts a JDBC ResultSet to a DataFrame. + */ + def resultSetToDataset(session: SnappySession, stmt: Statement) + (sql: String): Dataset[Row] = { + if (stmt.execute(sql)) { + val rs = stmt.getResultSet + val schema = JdbcUtils.getSchema(rs, SnappyStoreDialect) + val rows = Utils.resultSetToSparkInternalRows(rs, schema).map(_.copy()).toSeq + session.internalCreateDataFrame(session.sparkContext.makeRDD(rows), schema) + } else { + implicit val encoder: ExpressionEncoder[Row] = RowEncoder(StructType(Nil)) + session.createDataset[Row](Nil) + } + } } /** 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 e79042c7b0..c922e40b95 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.store -import java.sql.{SQLException, Statement} +import java.sql.SQLException import java.util.regex.Pattern import com.gemstone.gemfire.internal.shared.ClientSharedUtils @@ -25,14 +25,10 @@ import com.pivotal.gemfirexd.internal.engine.diag.SysVTIs import io.snappydata.SnappyFunSuite import org.scalatest.Assertions -import org.apache.spark.executor.InputMetrics import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.execution.columnar.impl.ColumnPartitionResolver -import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils -import org.apache.spark.sql.row.SnappyStoreDialect import org.apache.spark.sql.types._ -import org.apache.spark.sql.{AnalysisException, Dataset, Row, SnappySession} +import org.apache.spark.sql.{AnalysisException, Dataset, Row} /** * Tests for meta-data queries using Spark SQL. @@ -118,21 +114,6 @@ object MetadataTest extends Assertions { getLongVarcharTuple("GATEWAYSENDERS"), ("OFFHEAPENABLED", 0, "BOOLEAN", false), ("ROWLEVELSECURITYENABLED", 0, "BOOLEAN", false)) - def resultSetToDataset(session: SnappySession, stmt: Statement) - (sql: String): Dataset[Row] = { - if (stmt.execute(sql)) { - val rs = stmt.getResultSet - val schema = JdbcUtils.getSchema(rs, SnappyStoreDialect) - val dummyMetrics = new InputMetrics - val rows = JdbcUtils.resultSetToSparkInternalRows(rs, schema, dummyMetrics) - .map(_.copy()).toSeq - session.internalCreateDataFrame(session.sparkContext.makeRDD(rows), schema) - } else { - implicit val encoder: ExpressionEncoder[Row] = RowEncoder(StructType(Nil)) - session.createDataset[Row](Nil) - } - } - def testSYSTablesAndVTIs(executeSQL: String => Dataset[Row], hostName: String = ClientSharedUtils.getLocalHost.getCanonicalHostName, netServers: Seq[String] = Seq(""), locator: String = "", locatorNetServer: String = "", 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 2e4b3f7e07..3dce18c0b2 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 @@ -17,17 +17,63 @@ package org.apache.spark.sql.store +import java.sql.SQLException + import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem +import io.snappydata.SnappyFunSuite.checkAnswer import io.snappydata.{Property, SnappyFunSuite} +import org.scalatest.Assertions import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoinExec} -import org.apache.spark.sql.{AnalysisException, Row, SnappySession} +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SnappySession} /** * Tests for temporary, global and persistent views. */ class ViewTest extends SnappyFunSuite { + override def beforeAll(): Unit = { + super.beforeAll() + ViewTest.createTables(this.snc.snappySession) + } + + override def afterAll(): Unit = { + ViewTest.dropTables(this.snc.snappySession) + super.afterAll() + } + + test("temporary view") { + ViewTest.testTemporaryView(snc.snappySession.sql, () => new SnappySession(sc).sql) + } + + test("global temporary view") { + ViewTest.testGlobalTemporaryView(snc.snappySession.sql, () => new SnappySession(sc).sql) + } + + test("temporary view using") { + ViewTest.testTemporaryViewUsing(snc.snappySession.sql, () => new SnappySession(sc).sql) + } + + test("global temporary view using") { + ViewTest.testGlobalTemporaryViewUsing(snc.snappySession.sql, () => new SnappySession(sc).sql) + } + + test("persistent view") { + ViewTest.testPersistentView(snc.snappySession.sql, checkPlans = true, + () => new SnappySession(sc).sql, restartSpark) + } + + private def restartSpark(): Unit = { + stopAll() + val sys = InternalDistributedSystem.getConnectedInstance + if (sys ne null) { + sys.disconnect() + } + } +} + +object ViewTest extends Assertions { + private val columnTable = "viewColTable" private val rowTable = "viewRowTable" private val numRows = 10 @@ -35,275 +81,306 @@ class ViewTest extends SnappyFunSuite { private val viewTempMeta = Seq(Row("ID", "int", null), Row("ADDR", "string", null), Row("RANK", "int", null)) - override def beforeAll(): Unit = { - super.beforeAll() - val session = this.snc.snappySession + private def getExpectedResult: Seq[Row] = { + (0 until numRows).map(i => Row(i, "address_" + (i + 1), i + 1)) + } + + private def tableExists(executeSQL: String => Dataset[Row], name: String): Boolean = { + try { + executeSQL(s"select 1 from $name where 1 = 0") + true + } catch { + case _: Exception => false + } + } + + def createTables(session: SnappySession): Unit = { session.sql(s"create table $columnTable (id int, addr varchar(20)) using column " + "options (partition_by 'id')") session.sql(s"create table $rowTable (id int, addr varchar(20)) using row " + s"options (partition_by 'id', colocate_with '$columnTable')") val rows = (0 until numRows).map(i => Row(i, "address_" + (i + 1))) - snc.insert(columnTable, rows: _*) - snc.insert(rowTable, rows: _*) - } - - private def getExpectedResult: Seq[Row] = { - (0 until numRows).map(i => Row(i, "address_" + (i + 1), i + 1)) + session.insert(columnTable, rows: _*) + session.insert(rowTable, rows: _*) } - private def tableExists(session: SnappySession, name: String): Boolean = { - val identifier = session.tableIdentifier(name) - session.sessionCatalog.isTemporaryTable(identifier) || - session.sessionCatalog.tableExists(identifier) + def dropTables(session: SnappySession): Unit = { + session.sql(s"drop table $rowTable") + session.sql(s"drop table $columnTable") } - test("temporary view") { - val session = this.snc.snappySession - + def testTemporaryView(executeSQL: String => Dataset[Row], + newExecution: () => String => Dataset[Row]): Unit = { val tableMeta = Seq(Row("ID", "int", null), Row("ADDR", "varchar(20)", null)) - checkAnswer(session.sql(s"describe $columnTable"), tableMeta) - checkAnswer(session.sql(s"describe $rowTable"), tableMeta) + checkAnswer(executeSQL(s"describe $columnTable"), tableMeta) + checkAnswer(executeSQL(s"describe $rowTable"), tableMeta) val expected = getExpectedResult val showResult = Seq(Row("", "VIEWONTABLE", true, false)) // check temporary view and its meta-data for column table - session.sql(s"create temporary view viewOnTable as $viewQuery from $columnTable") + executeSQL(s"create temporary view viewOnTable as $viewQuery from $columnTable") - assert(tableExists(session, "viewOnTable") === true) - checkAnswer(session.sql("describe viewOnTable"), viewTempMeta) - checkAnswer(session.sql("select * from viewOnTable"), expected) - checkAnswer(session.sql("show views"), showResult) - checkAnswer(session.sql("show views in app"), showResult) - checkAnswer(session.sql("show views from app"), showResult) + assert(tableExists(executeSQL, "viewOnTable") === true) + checkAnswer(executeSQL("describe viewOnTable"), viewTempMeta) + checkAnswer(executeSQL("select * from viewOnTable"), expected) + checkAnswer(executeSQL("show views"), showResult) + checkAnswer(executeSQL("show views in app"), showResult) + checkAnswer(executeSQL("show views from app"), showResult) // should not be visible from another session - val session2 = session.newSession() - assert(tableExists(session2, "viewOnTable") === false) + val executeSQL2 = newExecution() + assert(tableExists(executeSQL2, "viewOnTable") === false) // drop and check unavailability - session.sql("drop view viewOnTable") - assert(tableExists(session, "viewOnTable") === false) - assert(tableExists(session2, "viewOnTable") === false) + executeSQL("drop view viewOnTable") + assert(tableExists(executeSQL, "viewOnTable") === false) + assert(tableExists(executeSQL2, "viewOnTable") === false) // check the same for view on row table - session.sql(s"create temporary view viewOnTable as $viewQuery from $rowTable") + executeSQL(s"create temporary view viewOnTable as $viewQuery from $rowTable") - assert(tableExists(session, "viewOnTable") === true) - checkAnswer(session.sql("describe viewOnTable"), viewTempMeta) - checkAnswer(session.sql("select * from viewOnTable"), expected) + assert(tableExists(executeSQL, "viewOnTable") === true) + checkAnswer(executeSQL("describe viewOnTable"), viewTempMeta) + checkAnswer(executeSQL("select * from viewOnTable"), expected) - assert(tableExists(session2, "viewOnTable") === false) - session.sql("drop view viewOnTable") - assert(tableExists(session, "viewOnTable") === false) - assert(tableExists(session2, "viewOnTable") === false) - - session2.close() + assert(tableExists(executeSQL2, "viewOnTable") === false) + executeSQL("drop view viewOnTable") + assert(tableExists(executeSQL, "viewOnTable") === false) + assert(tableExists(executeSQL2, "viewOnTable") === false) } - test("global temporary view") { - val session = this.snc.snappySession - + def testGlobalTemporaryView(executeSQL: String => Dataset[Row], + newExecution: () => String => Dataset[Row]): Unit = { val expected = getExpectedResult val showResult = Seq(Row("GLOBAL_TEMP", "VIEWONTABLE", true, true)) // check temporary view and its meta-data for column table - session.sql(s"create global temporary view viewOnTable as $viewQuery from $columnTable") + executeSQL(s"create global temporary view viewOnTable as $viewQuery from $columnTable") - assert(session.sessionCatalog.getGlobalTempView("viewOnTable").isDefined) - checkAnswer(session.sql("describe global_temp.viewOnTable"), viewTempMeta) - checkAnswer(session.sql("select * from viewOnTable"), expected) - checkAnswer(session.sql("show views"), Nil) - checkAnswer(session.sql("show views in global_temp"), showResult) - checkAnswer(session.sql("show views from global_temp"), showResult) + assert(executeSQL("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "VIEWONTABLE", true, true))) + checkAnswer(executeSQL("describe global_temp.viewOnTable"), viewTempMeta) + checkAnswer(executeSQL("select * from viewOnTable"), expected) + checkAnswer(executeSQL("show views"), Nil) + checkAnswer(executeSQL("show views in global_temp"), showResult) + checkAnswer(executeSQL("show views from global_temp"), showResult) // should be visible from another session - val session2 = session.newSession() - assert(session2.sessionCatalog.getGlobalTempView("viewOnTable").isDefined) - checkAnswer(session2.sql("describe global_temp.viewOnTable"), viewTempMeta) - checkAnswer(session2.sql("select * from viewOnTable"), expected) + val executeSQL2 = newExecution() + assert(executeSQL2("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "VIEWONTABLE", true, true))) + checkAnswer(executeSQL2("describe global_temp.viewOnTable"), viewTempMeta) + checkAnswer(executeSQL2("select * from viewOnTable"), expected) // drop and check unavailability - session.sql("drop view viewOnTable") - assert(session.sessionCatalog.getGlobalTempView("viewOnTable").isEmpty) - assert(session2.sessionCatalog.getGlobalTempView("viewOnTable").isEmpty) + executeSQL("drop view viewOnTable") + assert(executeSQL("show views in global_temp").collect().isEmpty) + assert(executeSQL2("show views in global_temp").collect().isEmpty) // check the same for view on row table - session.sql(s"create global temporary view viewOnTable as $viewQuery from $columnTable") - - assert(session.sessionCatalog.getGlobalTempView("viewOnTable").isDefined) - checkAnswer(session.sql("describe global_temp.viewOnTable"), viewTempMeta) - checkAnswer(session.sql("select * from viewOnTable"), expected) + executeSQL(s"create global temporary view viewOnTable as $viewQuery from $columnTable") - assert(session2.sessionCatalog.getGlobalTempView("viewOnTable").isDefined) - checkAnswer(session2.sql("describe global_temp.viewOnTable"), viewTempMeta) - checkAnswer(session2.sql("select * from viewOnTable"), expected) + assert(executeSQL("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "VIEWONTABLE", true, true))) + checkAnswer(executeSQL("describe global_temp.viewOnTable"), viewTempMeta) + checkAnswer(executeSQL("select * from viewOnTable"), expected) - session.sql("drop view viewOnTable") - assert(session.sessionCatalog.getGlobalTempView("viewOnTable").isEmpty) - assert(session2.sessionCatalog.getGlobalTempView("viewOnTable").isEmpty) + assert(executeSQL2("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "VIEWONTABLE", true, true))) + checkAnswer(executeSQL2("describe global_temp.viewOnTable"), viewTempMeta) + checkAnswer(executeSQL2("select * from viewOnTable"), expected) - session2.close() + executeSQL("drop view viewOnTable") + assert(executeSQL("show views in global_temp").collect().isEmpty) + assert(executeSQL2("show views in global_temp").collect().isEmpty) } - test("temporary view using") { - val session = this.snc.snappySession - + def testTemporaryViewUsing(executeSQL: String => Dataset[Row], + newExecution: () => String => Dataset[Row]): Unit = { // check temporary view with USING and its meta-data val hfile: String = getClass.getResource("/2015.parquet").getPath - val airline = session.read.parquet(hfile) - session.sql(s"create temporary view airlineView using parquet options(path '$hfile')") - val airlineView = session.table("airlineView") + executeSQL(s"create external table airlineTemp using parquet options (path '$hfile')") + val airline = executeSQL("select * from airlineTemp limit 1") + executeSQL(s"create temporary view airlineView using parquet options(path '$hfile')") + val airlineView = executeSQL("select * from airlineView limit 1") - assert(tableExists(session, "airlineView") === true) + assert(tableExists(executeSQL, "airlineView") === true) assert(airlineView.schema === airline.schema) - checkAnswer(session.sql("select count(*) from airlineView"), Seq(Row(airline.count()))) - assert(airlineView.count() == airline.count()) + checkAnswer(executeSQL("select count(*) from airlineView"), + executeSQL("select count(*) from airlineTemp").collect()) // should not be visible from another session - val session2 = session.newSession() - assert(tableExists(session2, "airlineView") === false) + val executeSQL2 = newExecution() + assert(tableExists(executeSQL2, "airlineView") === false) // drop and check unavailability - session.sql("drop table airlineView") - assert(tableExists(session, "airlineView") === false) - assert(tableExists(session2, "airlineView") === false) - - session2.close() + executeSQL("drop table airlineTemp") + executeSQL("drop table airlineView") + assert(tableExists(executeSQL, "airlineTemp") === false) + assert(tableExists(executeSQL2, "airlineTemp") === false) + assert(tableExists(executeSQL, "airlineView") === false) + assert(tableExists(executeSQL2, "airlineView") === false) } - test("global temporary view using") { - val session = this.snc.snappySession - + def testGlobalTemporaryViewUsing(executeSQL: String => Dataset[Row], + newExecution: () => String => Dataset[Row]): Unit = { // check global temporary view with USING and its meta-data val hfile: String = getClass.getResource("/2015.parquet").getPath - val airline = session.read.parquet(hfile) - session.sql(s"create global temporary view airlineView using parquet options(path '$hfile')") - val airlineView = session.table("airlineView") + executeSQL(s"create external table airlineTemp using parquet options (path '$hfile')") + val airline = executeSQL("select * from airlineTemp limit 1") + executeSQL(s"create global temporary view airlineView using parquet options(path '$hfile')") + val airlineView = executeSQL("select * from airlineView limit 1") - assert(session.sessionCatalog.getGlobalTempView("airlineView").isDefined) + assert(executeSQL("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "AIRLINEVIEW", true, true))) assert(airlineView.schema === airline.schema) - checkAnswer(session.sql("select count(*) from airlineView"), Seq(Row(airline.count()))) - assert(airlineView.count() == airline.count()) + checkAnswer(executeSQL("select count(*) from airlineView"), + executeSQL("select count(*) from airlineTemp").collect()) // should be visible from another session - val session2 = session.newSession() - assert(session2.sessionCatalog.getGlobalTempView("airlineView").isDefined) - checkAnswer(session2.sql("select count(*) from airlineView"), Seq(Row(airline.count()))) + val executeSQL2 = newExecution() + assert(executeSQL2("show views in global_temp").collect() === + Array(Row("GLOBAL_TEMP", "AIRLINEVIEW", true, true))) + checkAnswer(executeSQL2("select count(*) from airlineView"), + executeSQL("select count(*) from airlineTemp").collect()) // drop and check unavailability - session.sql("drop table airlineView") - assert(session.sessionCatalog.getGlobalTempView("airlineView").isEmpty) - assert(session2.sessionCatalog.getGlobalTempView("airlineView").isEmpty) - - session2.close() + executeSQL("drop table airlineTemp") + executeSQL("drop table airlineView") + assert(tableExists(executeSQL, "airlineTemp") === false) + assert(tableExists(executeSQL2, "airlineTemp") === false) + assert(executeSQL("show views in global_temp").collect().isEmpty) + assert(executeSQL2("show views in global_temp").collect().isEmpty) } - test("persistent view") { + def testPersistentView(executeSQL: String => Dataset[Row], checkPlans: Boolean, + newExecution: () => String => Dataset[Row], restartSpark: () => Unit): Unit = { val expected = getExpectedResult // check temporary view and its meta-data for column table - checkPersistentView(columnTable, rowTable, snc.snappySession, expected) - // check the same for view on row table - checkPersistentView(rowTable, columnTable, snc.snappySession, expected) + checkPersistentView(columnTable, rowTable, checkPlans, executeSQL, newExecution, + expected, restartSpark) + // check the same for view on row table with new session since old one would not be valid + val newExecuteSQL = newExecution() + checkPersistentView(rowTable, columnTable, checkPlans, newExecuteSQL, newExecution, + expected, restartSpark) } - private def checkPersistentView(table: String, otherTable: String, session: SnappySession, - expectedResult: Seq[Row]): Unit = { - session.sql(s"create view viewOnTable as $viewQuery from $table") + private def checkPersistentView(table: String, otherTable: String, checkPlans: Boolean, + executeSQL: String => Dataset[Row], newExecution: () => String => Dataset[Row], + expectedResult: Seq[Row], restartSpark: () => Unit): Unit = { + executeSQL(s"create view viewOnTable as $viewQuery from $table") val viewMeta = Seq(Row("ID", "int", null), Row("ADDR", "varchar(20)", null), Row("RANK", "int", null)) val showResult = Seq(Row("APP", "VIEWONTABLE", false, false)) - assert(tableExists(session, "viewOnTable") === true) - checkAnswer(session.sql("describe viewOnTable"), viewMeta) - checkAnswer(session.sql("select * from viewOnTable"), expectedResult) - checkAnswer(session.sql("show views"), showResult) - checkAnswer(session.sql("show views in app"), showResult) - checkAnswer(session.sql("show views from app"), showResult) + assert(tableExists(executeSQL, "viewOnTable") === true) + checkAnswer(executeSQL("describe viewOnTable"), viewMeta) + checkAnswer(executeSQL("select * from viewOnTable"), expectedResult) + checkAnswer(executeSQL("show views"), showResult) + checkAnswer(executeSQL("show views in app"), showResult) + checkAnswer(executeSQL("show views from app"), showResult) // should be visible from another session - var session2 = session.newSession() - assert(tableExists(session2, "viewOnTable") === true) - checkAnswer(session2.sql("describe viewOnTable"), viewMeta) - checkAnswer(session2.sql("select * from viewOnTable"), expectedResult) + var executeSQL2 = newExecution() + assert(tableExists(executeSQL2, "viewOnTable") === true) + checkAnswer(executeSQL2("describe viewOnTable"), viewMeta) + checkAnswer(executeSQL2("select * from viewOnTable"), expectedResult) // test for SNAP-2205: see CompressionCodecId.isCompressed for a description of the problem - session.conf.set(Property.ColumnBatchSize.name, "10k") + executeSQL(s"set ${Property.ColumnBatchSize.name}=10k") // 21 columns mean 63 for ColumnStatsSchema so total of 64 fields including the COUNT // in the stats row which will fit in exactly one long for the nulls bitset val cols = (1 to 21).map(i => s"col$i string").mkString(", ") - session.sql(s"CREATE TABLE test2205 ($cols) using column options (buckets '4')") + executeSQL(s"CREATE TABLE test2205 ($cols) using column options (buckets '4')") val numElements = 10000 val projection = (1 to 21).map(i => s"null as col$i") - session.range(numElements).selectExpr(projection: _*).write.insertInto("test2205") + executeSQL( + s"insert into test2205 select ${projection.mkString(", ")} from range($numElements)") - checkAnswer(session.sql("select count(*), count(col10) from test2205"), + checkAnswer(executeSQL("select count(*), count(col10) from test2205"), Seq(Row(numElements, 0))) - // should be available after a restart - session.close() - session2.close() - stopAll() - val sys = InternalDistributedSystem.getConnectedInstance - if (sys ne null) { - sys.disconnect() - } + // test large view + val longStr = (1 to 1000).mkString("test data ", "", "") + val largeViewStr = (1 to 100).map(i => + s"case when $i % 3 == 0 then cast(null as string) else '$longStr[$i]' end as c$i").mkString( + "select ", ", ", "") + assert(largeViewStr.length > 100000) + var rs = executeSQL2(largeViewStr).collect() + assert(rs.length == 1) + executeSQL2(s"create view largeView as $largeViewStr").collect() + rs = executeSQL("select * from largeView").collect() + assert(rs.length == 1) - session2 = new SnappySession(sc) - assert(tableExists(session2, "viewOnTable") === true) - checkAnswer(session2.sql("describe viewOnTable"), viewMeta) - checkAnswer(session2.sql("select * from viewOnTable"), expectedResult) + // should be available after a restart + restartSpark() + executeSQL2 = newExecution() + assert(tableExists(executeSQL2, "viewOnTable") === true) + checkAnswer(executeSQL2("describe viewOnTable"), viewMeta) + checkAnswer(executeSQL2("select * from viewOnTable"), expectedResult) - checkAnswer(session2.sql("select count(*), count(col10) from test2205"), + checkAnswer(executeSQL2("select count(*), count(col10) from test2205"), Seq(Row(numElements, 0))) try { - session2.sql("drop table viewOnTable") + executeSQL2("drop table viewOnTable") fail("expected drop table to fail for view") } catch { - case _: AnalysisException => // expected + case _: AnalysisException | _: SQLException => // expected } // drop and check unavailability - session2.sql("drop view viewOnTable") - assert(tableExists(session2, "viewOnTable") === false) - session2.sql("drop table test2205") + executeSQL2("drop view viewOnTable") + assert(tableExists(executeSQL2, "viewOnTable") === false) + executeSQL2("drop table test2205") + + // test large view after restart + rs = executeSQL2("select * from largeView").collect() + assert(rs.length == 1) + executeSQL2("drop view largeView") // check colocated joins with VIEWs (SNAP-2204) val query = s"select c.id, r.addr from $columnTable c inner join $rowTable r on (c.id = r.id)" // first check with normal query - var ds = session2.sql(query) + var ds = executeSQL2(query) checkAnswer(ds, expectedResult.map(r => Row(r.get(0), r.get(1)))) - var plan = ds.queryExecution.executedPlan - assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) - assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + if (checkPlans) { + val plan = ds.queryExecution.executedPlan + assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) + assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + } val expectedResult2 = expectedResult.map(r => Row(r.get(0), r.get(1))) // check for normal view join with table - session2.sql(s"create view viewOnTable as select id, addr, id + 1 from $table") - ds = session2.sql("select t.id, v.addr from viewOnTable v " + + executeSQL2(s"create view viewOnTable as select id, addr, id + 1 from $table") + ds = executeSQL2("select t.id, v.addr from viewOnTable v " + s"inner join $otherTable t on (v.id = t.id)") checkAnswer(ds, expectedResult2) - plan = ds.queryExecution.executedPlan - assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) - assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + if (checkPlans) { + val plan = ds.queryExecution.executedPlan + assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) + assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + } - session2.sql("drop view viewOnTable") - assert(tableExists(session2, "viewOnTable") === false) + executeSQL2("drop view viewOnTable") + assert(tableExists(executeSQL2, "viewOnTable") === false) // next query on a join view - session2.sql(s"create view viewOnJoin as $query") - ds = session2.sql("select * from viewOnJoin") + executeSQL2(s"create view viewOnJoin as $query") + ds = executeSQL2("select * from viewOnJoin") checkAnswer(ds, expectedResult2) - plan = ds.queryExecution.executedPlan - assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) - assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + if (checkPlans) { + val plan = ds.queryExecution.executedPlan + assert(plan.find(_.isInstanceOf[HashJoinExec]).isDefined) + assert(plan.find(_.isInstanceOf[BroadcastHashJoinExec]).isEmpty) + } - session2.sql("drop view viewOnJoin") - assert(tableExists(session2, "viewOnJoin") === false) + executeSQL2("drop view viewOnJoin") + assert(tableExists(executeSQL2, "viewOnJoin") === false) } } diff --git a/dtests/src/test/java/io/snappydata/hydra/smoke.sh b/dtests/src/test/java/io/snappydata/hydra/smoke.sh index c7d502100e..5847d56c52 100755 --- a/dtests/src/test/java/io/snappydata/hydra/smoke.sh +++ b/dtests/src/test/java/io/snappydata/hydra/smoke.sh @@ -38,9 +38,7 @@ mkdir -p $resultDir shift $SNAPPYDATA_SOURCE_DIR/store/tests/core/src/main/java/bin/sample-runbt.sh $resultDir $SNAPPYDATA_SOURCE_DIR -r 1 -d false io/snappydata/hydra/cluster/startDualModeCluster_smoke.bt -sleep 30; $SNAPPYDATA_SOURCE_DIR/store/tests/core/src/main/java/bin/sample-runbt.sh $resultDir $SNAPPYDATA_SOURCE_DIR -r 1 -d false io/snappydata/hydra/smoke.bt -sleep 30; $SNAPPYDATA_SOURCE_DIR/store/tests/core/src/main/java/bin/sample-runbt.sh $resultDir $SNAPPYDATA_SOURCE_DIR -r 1 -d false io/snappydata/hydra/cluster/stopDualModeCluster.bt diff --git a/gradle.properties b/gradle.properties index 905f38f193..e097267349 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,10 @@ +# Gradle daemon has been disabled due to two reasons: +# 1) It frequently fails after a few runs due to OOME. +# 2) Messes up buildOutput.log by writing to it multiple +# times, increasing by one in every run i.e. first run +# will be good, then second run will write each line twice, +# third run thrice and so on. Clearing the loggerService +# explicitly makes no difference. org.gradle.daemon=false org.gradle.warning.mode=none #org.gradle.parallel=true