From 1b90b2fa8ffb40f95474f0c7e5cd201184c54581 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 23 Jan 2025 17:07:03 -0500 Subject: [PATCH 1/7] Upgrade to hive 4.0.1 --- .../beam_PreCommit_Java_HCatalog_IO_Direct.yml | 15 --------------- .../beam_PreCommit_Java_IOs_Direct.yml | 15 --------------- sdks/java/extensions/sql/hcatalog/build.gradle | 5 +++-- sdks/java/io/hcatalog/build.gradle | 17 +++++------------ .../hcatalog/test/EmbeddedMetastoreService.java | 17 +++++++++-------- sdks/java/io/iceberg/build.gradle | 3 ++- sdks/java/io/iceberg/hive/build.gradle | 4 +--- .../sdk/io/iceberg/catalog/HiveCatalogIT.java | 5 +++-- .../catalog/hiveutils/TestHiveMetastore.java | 16 ++++++++-------- 9 files changed, 31 insertions(+), 66 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 5c3cf29419c2..eb0dcbcc7206 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -87,10 +87,6 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action - with: - java-version: | - 8 - 11 - name: run HCatalog IO build script uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -98,17 +94,6 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - # TODO(https://github.com/apache/beam/issues/32189) remove when embedded hive supports Java11 - - name: Test HCatalog IO on Java8 - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :sdks:java:io:hcatalog:test - arguments: | - -PdisableSpotlessCheck=true \ - -PdisableCheckStyle=true \ - -Dfile.encoding=UTF-8 \ - -PtestJavaVersion=8 \ - -Pjava8Home=$JAVA_HOME_8_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 03ff102861c7..98b9247f664b 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -86,10 +86,6 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action - with: - java-version: | - 8 - 11 - name: run Java IOs PreCommit script uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -98,17 +94,6 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -Dfile.encoding=UTF-8 \ - # TODO(https://github.com/apache/beam/issues/32189) remove when embedded hive supports Java11 - - name: run Java8 IOs PreCommit script - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :sdks:java:io:hcatalog:build - arguments: | - -PdisableSpotlessCheck=true \ - -PdisableCheckStyle=true \ - -Dfile.encoding=UTF-8 \ - -PtestJavaVersion=8 \ - -Pjava8Home=$JAVA_HOME_8_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/sdks/java/extensions/sql/hcatalog/build.gradle b/sdks/java/extensions/sql/hcatalog/build.gradle index e8abf21b7c3e..c6051df66a04 100644 --- a/sdks/java/extensions/sql/hcatalog/build.gradle +++ b/sdks/java/extensions/sql/hcatalog/build.gradle @@ -25,8 +25,8 @@ applyJavaNature( ], ) -def hive_version = "3.1.3" -def netty_version = "4.1.51.Final" +def hive_version = "4.0.1" +def netty_version = "4.1.82.Final" // in sync with hive-exec's dependency /* * We need to rely on manually specifying these evaluationDependsOn to ensure that @@ -43,6 +43,7 @@ dependencies { implementation project(":sdks:java:core") implementation library.java.vendored_guava_32_1_2_jre + testImplementation library.java.junit testImplementation project(":sdks:java:io:hcatalog").sourceSets.test.output // Needed for HCatalogTableProvider tests, // they use HCat* types diff --git a/sdks/java/io/hcatalog/build.gradle b/sdks/java/io/hcatalog/build.gradle index d07904f3465e..8aeeba2a3230 100644 --- a/sdks/java/io/hcatalog/build.gradle +++ b/sdks/java/io/hcatalog/build.gradle @@ -38,7 +38,7 @@ def hadoopVersions = [ hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} -def hive_version = "3.1.3" +def hive_version = "4.0.1" dependencies { implementation library.java.vendored_guava_32_1_2_jre @@ -64,6 +64,10 @@ dependencies { testImplementation library.java.hamcrest testImplementation "org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version:tests" testImplementation "org.apache.hive:hive-exec:$hive_version" + // datanucleus dependency version should be in alignment with managed dependencies of hive-standalone-metastore + testRuntimeOnly 'org.datanucleus:datanucleus-api-jdo:5.2.8' + testRuntimeOnly 'org.datanucleus:datanucleus-rdbms:5.2.10' + testRuntimeOnly 'org.datanucleus:javax.jdo:3.2.0-release' testImplementation "org.apache.hive:hive-common:$hive_version" testImplementation "org.apache.hive:hive-cli:$hive_version" testImplementation "org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version" @@ -105,14 +109,3 @@ hadoopVersions.each { kv -> include '**/*Test.class' } } - -project.tasks.withType(Test).configureEach { - if (JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 && project.findProperty('testJavaVersion') != '8') { - useJUnit { - filter { - excludeTestsMatching "org.apache.beam.sdk.io.hcatalog.HCatalogIOTest" - excludeTestsMatching "org.apache.beam.sdk.io.hcatalog.HCatalogBeamSchemaTest" - } - } - } -} diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/test/EmbeddedMetastoreService.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/test/EmbeddedMetastoreService.java index f68f969f29b9..1f0774a92c92 100644 --- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/test/EmbeddedMetastoreService.java +++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/test/EmbeddedMetastoreService.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.DriverFactory; import org.apache.hadoop.hive.ql.IDriver; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.processors.CommandProcessorException; import org.apache.hadoop.hive.ql.session.SessionState; /** @@ -58,11 +58,11 @@ public EmbeddedMetastoreService(String baseDirPath) throws IOException { String testWarehouseDirPath = makePathASafeFileName(testDataDirPath + "/warehouse"); hiveConf = new HiveConf(getClass()); - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.POST_EXEC_HOOKS, ""); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, testWarehouseDirPath); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, true); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, testWarehouseDirPath); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, true); hiveConf.setVar( HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); @@ -75,9 +75,10 @@ public EmbeddedMetastoreService(String baseDirPath) throws IOException { /** Executes the passed query on the embedded metastore service. */ public void executeQuery(String query) { - CommandProcessorResponse response = driver.run(query); - if (response.failed()) { - throw new RuntimeException(response.getException()); + try { + driver.run(query); + } catch (CommandProcessorException e) { + throw new RuntimeException(e); } } diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 8613a4b22c6a..ae5250e2cb60 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -40,7 +40,7 @@ hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} def iceberg_version = "1.6.1" def parquet_version = "1.12.0" def orc_version = "1.9.2" -def hive_version = "3.1.3" +def hive_version = "4.0.1" dependencies { implementation library.java.vendored_guava_32_1_2_jre @@ -76,6 +76,7 @@ dependencies { testImplementation "org.apache.iceberg:iceberg-common:$iceberg_version" testImplementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") testImplementation ("org.apache.hive:hive-metastore:$hive_version") + testImplementation ("org.apache.hive:hive-standalone-metastore-server:$hive_version") testImplementation "org.assertj:assertj-core:3.11.1" testRuntimeOnly ("org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version") { exclude group: "org.apache.hive", module: "hive-exec" diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index 7d93a4026775..dabdc485f62f 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -1,5 +1,3 @@ -import groovy.json.JsonOutput - /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -27,7 +25,7 @@ applyJavaNature( description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" ext.summary = "Runtime dependencies needed for Hive catalog integration." -def hive_version = "3.1.3" +def hive_version = "4.0.1" def hbase_version = "2.6.1-hadoop3" def hadoop_version = "3.4.1" def iceberg_version = "1.6.1" diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/HiveCatalogIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/HiveCatalogIT.java index acb0e36b4b01..24a91e041b71 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/HiveCatalogIT.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/HiveCatalogIT.java @@ -95,11 +95,12 @@ public void catalogCleanup() throws Exception { @Override public Map managedIcebergConfig(String tableId) { - String metastoreUri = hiveMetastoreExtension.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS); + String metastoreUri = + hiveMetastoreExtension.hiveConf().getVar(HiveConf.ConfVars.METASTORE_URIS); Map confProperties = ImmutableMap.builder() - .put(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUri) + .put(HiveConf.ConfVars.METASTORE_URIS.varname, metastoreUri) .build(); return ImmutableMap.builder() diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java index 01fc54ed59cd..d165a30494a3 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java @@ -39,7 +39,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HMSHandler; import org.apache.hadoop.hive.metastore.IHMSHandler; import org.apache.hadoop.hive.metastore.RetryingHMSHandler; import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; @@ -68,10 +68,10 @@ public class TestHiveMetastore { // create the metastore handlers based on whether we're working with Hive2 or Hive3 dependencies // we need to do this because there is a breaking API change between Hive2 and Hive3 - private static final DynConstructors.Ctor HMS_HANDLER_CTOR = + private static final DynConstructors.Ctor HMS_HANDLER_CTOR = DynConstructors.builder() - .impl(HiveMetaStore.HMSHandler.class, String.class, Configuration.class) - .impl(HiveMetaStore.HMSHandler.class, String.class, HiveConf.class) + .impl(HMSHandler.class, String.class, Configuration.class) + .impl(HMSHandler.class, String.class, HiveConf.class) .build(); private static final DynMethods.StaticMethod GET_BASE_HMS_HANDLER = @@ -120,7 +120,7 @@ public class TestHiveMetastore { private HiveConf hiveConf; private ExecutorService executorService; private TServer server; - private HiveMetaStore.HMSHandler baseHandler; + private HMSHandler baseHandler; private HiveClientPool clientPool; private final String hiveWarehousePath; @@ -230,7 +230,7 @@ private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf con throws Exception { HiveConf serverConf = new HiveConf(conf); serverConf.set( - HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, "jdbc:derby:" + DERBY_PATH + ";create=true"); baseHandler = HMS_HANDLER_CTOR.newInstance("new db based metaserver", serverConf); IHMSHandler handler = GET_BASE_HMS_HANDLER.invoke(serverConf, baseHandler, false); @@ -247,8 +247,8 @@ private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf con } private void initConf(HiveConf conf, int port) { - conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://localhost:" + port); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, hiveWarehousePath); + conf.set(HiveConf.ConfVars.METASTORE_URIS.varname, "thrift://localhost:" + port); + conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, hiveWarehousePath); conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "false"); conf.set(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES.varname, "false"); conf.set("iceberg.hive.client-pool-size", "2"); From 80e5184a76a59b77bd894634977500b30c8a2611 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 24 Jan 2025 11:42:10 -0500 Subject: [PATCH 2/7] trigger tests --- .github/trigger_files/IO_Iceberg_Integration_Tests.json | 2 +- .github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index b73af5e61a43..34a6e02150e7 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 1 + "modification": 4 } diff --git a/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json b/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json index 53d94cfc4f1e..1bd74515152c 100644 --- a/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json +++ b/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 4 } \ No newline at end of file From 301b5ae6feff51e4b2666704ac791788df544f94 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 27 Jan 2025 14:44:44 -0500 Subject: [PATCH 3/7] Drop Hadoop 2 support for HCatalogIO --- .../core/src/main/java/org/apache/beam/sdk/coders/Coder.java | 3 ++- sdks/java/io/hcatalog/build.gradle | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 6bcdea0c0ab6..0a3650ca133b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -198,7 +198,8 @@ public static void verifyDeterministic(Coder target, String message, Iterable } } - public static long getEncodedElementByteSizeUsingCoder(Coder target, T value) throws Exception { + public static long getEncodedElementByteSizeUsingCoder(Coder target, T value) + throws Exception { return target.getEncodedElementByteSize(value); } /** diff --git a/sdks/java/io/hcatalog/build.gradle b/sdks/java/io/hcatalog/build.gradle index 8aeeba2a3230..d3bdd8f10765 100644 --- a/sdks/java/io/hcatalog/build.gradle +++ b/sdks/java/io/hcatalog/build.gradle @@ -29,8 +29,8 @@ applyJavaNature( description = "Apache Beam :: SDKs :: Java :: IO :: HCatalog" ext.summary = "IO to read and write for HCatalog source." +// hive 4.x is compatible with Hadoop 3.x; Hive 3.x has been EOL as of Oct 2024 def hadoopVersions = [ - "2102": "2.10.2", "324": "3.2.4", "336": "3.3.6", // "341": "3.4.1", // tests already exercised on the default version From e21a71c68262bb5ba23d3baf6085ad98affc799d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 27 Jan 2025 15:32:30 -0500 Subject: [PATCH 4/7] update to hive-metastore 4 --- sdks/java/io/iceberg/build.gradle | 2 +- .../catalog/hiveutils/TestHiveMetastore.java | 90 +- .../catalog/hiveutils/TestTxnDbUtil.java | 525 ++++++++++ .../resources/hive-schema-3.1.0.derby.sql | 267 ------ .../resources/hive-schema-4.0.0.derby.sql | 898 ++++++++++++++++++ 5 files changed, 1489 insertions(+), 293 deletions(-) create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java delete mode 100644 sdks/java/io/iceberg/src/test/resources/hive-schema-3.1.0.derby.sql create mode 100644 sdks/java/io/iceberg/src/test/resources/hive-schema-4.0.0.derby.sql diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index ae5250e2cb60..2be55fe46a81 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -74,7 +74,7 @@ dependencies { // Hive catalog test dependencies testImplementation project(path: ":sdks:java:io:iceberg:hive") testImplementation "org.apache.iceberg:iceberg-common:$iceberg_version" - testImplementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + testImplementation ("org.apache.hive:hive-iceberg-catalog:$hive_version") testImplementation ("org.apache.hive:hive-metastore:$hive_version") testImplementation ("org.apache.hive:hive-standalone-metastore-server:$hive_version") testImplementation "org.assertj:assertj-core:3.11.1" diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java index d165a30494a3..cd7fd13f456a 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java @@ -30,23 +30,28 @@ import java.nio.charset.StandardCharsets; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.SQLException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Future; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HMSHandler; +import org.apache.hadoop.hive.metastore.HMSHandlerProxyFactory; import org.apache.hadoop.hive.metastore.IHMSHandler; -import org.apache.hadoop.hive.metastore.RetryingHMSHandler; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; +import org.apache.hadoop.hive.metastore.api.GetTableRequest; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.hadoop.Util; import org.apache.iceberg.hive.HiveClientPool; +import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TServer; import org.apache.thrift.server.TThreadPoolServer; @@ -58,7 +63,7 @@ * HiveMetastoreExtension} instead. * *

Copied over from Iceberg's + * href="https://github.com/apache/hive/blob/branch-4.0/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java">Iceberg's * integration testing util */ public class TestHiveMetastore { @@ -76,10 +81,22 @@ public class TestHiveMetastore { private static final DynMethods.StaticMethod GET_BASE_HMS_HANDLER = DynMethods.builder("getProxy") - .impl(RetryingHMSHandler.class, Configuration.class, IHMSHandler.class, boolean.class) - .impl(RetryingHMSHandler.class, HiveConf.class, IHMSHandler.class, boolean.class) + .impl(HMSHandlerProxyFactory.class, Configuration.class, IHMSHandler.class, boolean.class) + .impl(HMSHandlerProxyFactory.class, HiveConf.class, IHMSHandler.class, boolean.class) .buildStatic(); + // Hive3 introduces background metastore tasks (MetastoreTaskThread) for performing various + // cleanup duties. These + // threads are scheduled and executed in a static thread pool + // (org.apache.hadoop.hive.metastore.ThreadPool). + // This thread pool is shut down normally as part of the JVM shutdown hook, but since we're + // creating and tearing down + // multiple metastore instances within the same JVM, we have to call this cleanup method manually, + // otherwise + // threads from our previous test suite will be stuck in the pool with stale config, and keep on + // being scheduled. + // This can lead to issues, e.g. accidental Persistence Manager closure by + // ScheduledQueryExecutionsMaintTask. private static final DynMethods.StaticMethod METASTORE_THREADS_SHUTDOWN = DynMethods.builder("shutdown") .impl("org.apache.hadoop.hive.metastore.ThreadPool") @@ -89,13 +106,15 @@ public class TestHiveMetastore { // It's tricky to clear all static fields in an HMS instance in order to switch derby root dir. // Therefore, we reuse the same derby root between tests and remove it after JVM exits. private static final File HIVE_LOCAL_DIR; + private static final File HIVE_EXTERNAL_WAREHOUSE_DIR; private static final String DERBY_PATH; static { try { HIVE_LOCAL_DIR = createTempDirectory("hive", asFileAttribute(fromString("rwxrwxrwx"))).toFile(); - DERBY_PATH = HIVE_LOCAL_DIR + "/metastore_db"; + DERBY_PATH = new File(HIVE_LOCAL_DIR, "metastore_db").getPath(); + HIVE_EXTERNAL_WAREHOUSE_DIR = new File(HIVE_LOCAL_DIR, "external"); File derbyLogFile = new File(HIVE_LOCAL_DIR, "derby.log"); System.setProperty("derby.stream.error.file", derbyLogFile.getAbsolutePath()); setupMetastoreDB("jdbc:derby:" + DERBY_PATH + ";create=true"); @@ -127,9 +146,16 @@ public class TestHiveMetastore { TestHiveMetastore(String hiveWarehousePath) { this.hiveWarehousePath = hiveWarehousePath; } + /** + * Starts a TestHiveMetastore with the default connection pool size (5) and the default HiveConf. + */ + public void start() { + start(new HiveConf(new Configuration(), TestHiveMetastore.class), DEFAULT_POOL_SIZE); + } /** - * Starts a TestHiveMetastore with the default connection pool size with the provided HiveConf. + * Starts a TestHiveMetastore with the default connection pool size (5) with the provided + * HiveConf. * * @param conf The hive configuration to use */ @@ -143,7 +169,6 @@ public void start(HiveConf conf) { * @param conf The hive configuration to use * @param poolSize The number of threads in the executor pool */ - @SuppressWarnings("FutureReturnValueIgnored") public void start(HiveConf conf, int poolSize) { try { TServerSocket socket = new TServerSocket(0); @@ -153,7 +178,14 @@ public void start(HiveConf conf, int poolSize) { this.hiveConf = conf; this.server = newThriftServer(socket, poolSize, hiveConf); this.executorService = Executors.newSingleThreadExecutor(); - this.executorService.submit(() -> server.serve()); + Future ignored = this.executorService.submit(() -> server.serve()); + + // in Hive3, setting this as a system prop ensures that it will be picked up whenever a new + // HiveConf is created + System.setProperty( + HiveConf.ConfVars.METASTORE_URIS.varname, + hiveConf.getVar(HiveConf.ConfVars.METASTORE_URIS)); + this.clientPool = new HiveClientPool(1, hiveConf); } catch (Exception e) { throw new RuntimeException("Cannot start TestHiveMetastore", e); @@ -169,13 +201,7 @@ public void stop() throws Exception { server.stop(); } if (executorService != null) { - executorService.shutdownNow(); - try { - // Give it a reasonable timeout - executorService.awaitTermination(10, TimeUnit.SECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + executorService.shutdown(); } if (baseHandler != null) { baseHandler.shutdown(); @@ -215,9 +241,6 @@ public void reset() throws Exception { Path warehouseRoot = new Path(hiveWarehousePath); FileSystem fs = Util.getFs(warehouseRoot, hiveConf); - if (!fs.exists(warehouseRoot)) { - return; - } for (FileStatus fileStatus : fs.listStatus(warehouseRoot)) { if (!fileStatus.getPath().getName().equals("derby.log") && !fileStatus.getPath().getName().equals("metastore_db")) { @@ -226,6 +249,19 @@ public void reset() throws Exception { } } + public Table getTable(String dbName, String tableName) throws TException, InterruptedException { + return clientPool.run(client -> client.getTable(new GetTableRequest(dbName, tableName))); + } + + public Table getTable(TableIdentifier identifier) throws TException, InterruptedException { + return getTable(identifier.namespace().toString(), identifier.name()); + } + + public R run(ClientPool.Action action) + throws InterruptedException, TException { + return clientPool.run(action, false); + } + private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf conf) throws Exception { HiveConf serverConf = new HiveConf(conf); @@ -249,20 +285,24 @@ private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf con private void initConf(HiveConf conf, int port) { conf.set(HiveConf.ConfVars.METASTORE_URIS.varname, "thrift://localhost:" + port); conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, hiveWarehousePath); + conf.set( + HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, + "file:" + HIVE_EXTERNAL_WAREHOUSE_DIR.getAbsolutePath()); conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "false"); conf.set(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES.varname, "false"); conf.set("iceberg.hive.client-pool-size", "2"); - // Setting this to avoid thrift exception during running Iceberg tests outside Iceberg. - conf.set( - HiveConf.ConfVars.HIVE_IN_TEST.varname, HiveConf.ConfVars.HIVE_IN_TEST.getDefaultValue()); + // set to false so that TxnManager#checkLock does not throw exception when using UNSET data type + // operation + // in the requested lock component + conf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, false); } - private static void setupMetastoreDB(String dbURL) throws SQLException, IOException { + private static void setupMetastoreDB(String dbURL) throws Exception { Connection connection = DriverManager.getConnection(dbURL); ScriptRunner scriptRunner = new ScriptRunner(connection, true, true); ClassLoader classLoader = ClassLoader.getSystemClassLoader(); - InputStream inputStream = classLoader.getResourceAsStream("hive-schema-3.1.0.derby.sql"); + InputStream inputStream = classLoader.getResourceAsStream("hive-schema-4.0.0.derby.sql"); try (Reader reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8)) { scriptRunner.runScript(reader); } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java new file mode 100644 index 000000000000..6894d363495c --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java @@ -0,0 +1,525 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg.catalog.hiveutils; + +import static org.apache.hadoop.hive.metastore.DatabaseProduct.determineDatabaseProduct; + +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStream; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.SQLTransactionRollbackException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Scanner; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.DatabaseProduct; +import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo; +import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility methods for creating and destroying txn database/schema, plus methods for querying + * against metastore tables. Placed here in a separate class so it can be shared across unit tests. + * + *

Copied over from Iceberg's + * * integration testing util + */ +public final class TestTxnDbUtil { + + private static final Logger LOG = LoggerFactory.getLogger(TestTxnDbUtil.class.getName()); + private static final String TXN_MANAGER = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; + + private static int deadlockCnt = 0; + + private TestTxnDbUtil() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + /** + * Set up the configuration so it will use the DbTxnManager, concurrency will be set to true, and + * the JDBC configs will be set for putting the transaction and lock info in the embedded + * metastore. + * + * @param conf HiveConf to add these values to + */ + public static void setConfValues(Configuration conf) { + MetastoreConf.setVar(conf, ConfVars.HIVE_TXN_MANAGER, TXN_MANAGER); + MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, true); + } + + /** + * Prepares the metastore database for unit tests. Runs the latest init schema against the + * database configured in the CONNECT_URL_KEY param. Ignores any duplication (table, index etc.) + * So it can be called multiple times for the same database. + * + * @param conf Metastore configuration + * @throws Exception Initialization failure + */ + public static synchronized void prepDb(Configuration conf) throws Exception { + LOG.info("Creating transactional tables"); + Connection conn = null; + Statement stmt = null; + try { + conn = getConnection(conf); + String s = conn.getMetaData().getDatabaseProductName(); + DatabaseProduct dbProduct = determineDatabaseProduct(s, conf); + stmt = conn.createStatement(); + if (checkDbPrepared(stmt)) { + return; + } + String schemaRootPath = getSchemaRootPath(); + IMetaStoreSchemaInfo metaStoreSchemaInfo = + MetaStoreSchemaInfoFactory.get(conf, schemaRootPath, dbProduct.getHiveSchemaPostfix()); + String initFile = metaStoreSchemaInfo.generateInitFileName(null); + try (InputStream is = + new FileInputStream( + metaStoreSchemaInfo.getMetaStoreScriptDir() + File.separator + initFile)) { + LOG.info( + "Reinitializing the metastore db with {} on the database {}", + initFile, + MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY)); + importSQL(stmt, is); + } + } catch (SQLException e) { + try { + if (conn != null) { + conn.rollback(); + } + } catch (SQLException re) { + LOG.error("Error rolling back: " + re.getMessage()); + } + // This might be a deadlock, if so, let's retry + if (e instanceof SQLTransactionRollbackException && deadlockCnt++ < 5) { + LOG.warn("Caught deadlock, retrying db creation"); + prepDb(conf); + } else { + throw e; + } + } finally { + deadlockCnt = 0; + closeResources(conn, stmt, null); + } + } + + private static boolean checkDbPrepared(Statement stmt) { + /* + * If the transactional tables are already there we don't want to run everything again + */ + try { + stmt.execute("SELECT * FROM \"TXNS\""); + } catch (SQLException e) { + return false; + } + return true; + } + + private static void importSQL(Statement stmt, InputStream in) throws SQLException { + Set knownErrors = getAlreadyExistsErrorCodes(); + Scanner s = new Scanner(in, "UTF-8"); + s.useDelimiter("(;(\r)?\n)|(--.*\n)"); + while (s.hasNext()) { + String line = s.next(); + + if (line.trim().length() > 0) { + try { + stmt.execute(line); + } catch (SQLException e) { + if (knownErrors.contains(e.getSQLState())) { + LOG.debug("Ignoring sql error {}", e.getMessage()); + } else { + throw e; + } + } + } + } + } + + private static Set getAlreadyExistsErrorCodes() { + // function already exists, table already exists, index already exists, duplicate key + Set knownErrors = new HashSet<>(); + // derby + knownErrors.addAll(Arrays.asList("X0Y68", "X0Y32", "X0Y44", "42Z93", "23505")); + // postgres + knownErrors.addAll(Arrays.asList("42P07", "42P16", "42710")); + // mssql + knownErrors.addAll(Arrays.asList("S0000", "S0001", "23000")); + // mysql + knownErrors.addAll(Arrays.asList("42S01", "HY000")); + // oracle + knownErrors.addAll(Arrays.asList("42000")); + return knownErrors; + } + + private static String getSchemaRootPath() { + String hiveRoot = System.getProperty("hive.root"); + if (StringUtils.isNotEmpty(hiveRoot)) { + return ensurePathEndsInSlash(hiveRoot) + "standalone-metastore/metastore-server/target/tmp/"; + } else { + return ensurePathEndsInSlash(System.getProperty("test.tmp.dir", "target/tmp")); + } + } + + private static String ensurePathEndsInSlash(String path) { + if (path == null) { + throw new NullPointerException("Path cannot be null"); + } + if (path.endsWith(File.separator)) { + return path; + } else { + return path + File.separator; + } + } + + public static void cleanDb(Configuration conf) throws Exception { + LOG.info("Cleaning transactional tables"); + + boolean success = true; + Connection conn = null; + Statement stmt = null; + try { + conn = getConnection(conf); + stmt = conn.createStatement(); + if (!checkDbPrepared(stmt)) { + // Nothing to clean + return; + } + + // We want to try these, whether they succeed or fail. + success &= truncateTable(conn, conf, stmt, "TXN_COMPONENTS"); + success &= truncateTable(conn, conf, stmt, "COMPLETED_TXN_COMPONENTS"); + success &= truncateTable(conn, conf, stmt, "MIN_HISTORY_WRITE_ID"); + success &= truncateTable(conn, conf, stmt, "TXNS"); + success &= truncateTable(conn, conf, stmt, "TXN_TO_WRITE_ID"); + success &= truncateTable(conn, conf, stmt, "NEXT_WRITE_ID"); + success &= truncateTable(conn, conf, stmt, "HIVE_LOCKS"); + success &= truncateTable(conn, conf, stmt, "NEXT_LOCK_ID"); + success &= truncateTable(conn, conf, stmt, "COMPACTION_QUEUE"); + success &= truncateTable(conn, conf, stmt, "NEXT_COMPACTION_QUEUE_ID"); + success &= truncateTable(conn, conf, stmt, "COMPLETED_COMPACTIONS"); + success &= truncateTable(conn, conf, stmt, "AUX_TABLE"); + success &= truncateTable(conn, conf, stmt, "WRITE_SET"); + success &= truncateTable(conn, conf, stmt, "REPL_TXN_MAP"); + success &= truncateTable(conn, conf, stmt, "MATERIALIZATION_REBUILD_LOCKS"); + success &= truncateTable(conn, conf, stmt, "MIN_HISTORY_LEVEL"); + success &= truncateTable(conn, conf, stmt, "COMPACTION_METRICS_CACHE"); + try { + String dbProduct = conn.getMetaData().getDatabaseProductName(); + DatabaseProduct databaseProduct = determineDatabaseProduct(dbProduct, conf); + try { + resetTxnSequence(databaseProduct, stmt); + stmt.executeUpdate("INSERT INTO \"NEXT_LOCK_ID\" VALUES(1)"); + stmt.executeUpdate("INSERT INTO \"NEXT_COMPACTION_QUEUE_ID\" VALUES(1)"); + } catch (SQLException e) { + if (!databaseProduct.isTableNotExistsError(e)) { + LOG.error("Error initializing sequence values", e); + throw e; + } + } + } catch (SQLException e) { + LOG.error("Unable determine database product ", e); + throw e; + } + /* + * Don't drop NOTIFICATION_LOG, SEQUENCE_TABLE and NOTIFICATION_SEQUENCE as its used by other + * table which are not txn related to generate primary key. So if these tables are dropped + * and other tables are not dropped, then it will create key duplicate error while inserting + * to other table. + */ + } finally { + closeResources(conn, stmt, null); + } + if (success) { + return; + } + throw new RuntimeException("Failed to clean up txn tables"); + } + + private static void resetTxnSequence(DatabaseProduct databaseProduct, Statement stmt) + throws SQLException { + for (String s : databaseProduct.getResetTxnSequenceStmts()) { + stmt.execute(s); + } + } + + private static boolean truncateTable( + Connection conn, Configuration conf, Statement stmt, String name) throws SQLException { + String dbProduct = conn.getMetaData().getDatabaseProductName(); + DatabaseProduct databaseProduct = determineDatabaseProduct(dbProduct, conf); + try { + // We can not use actual truncate due to some foreign keys, but we don't expect much data + // during tests + + String s = databaseProduct.getTruncateStatement(name); + stmt.execute(s); + + LOG.debug("Successfully truncated table " + name); + return true; + } catch (SQLException e) { + if (databaseProduct.isTableNotExistsError(e)) { + LOG.debug("Not truncating " + name + " because it doesn't exist"); + return true; + } + LOG.error("Unable to truncate table " + name, e); + } + return false; + } + + /** + * A tool to count the number of partitions, tables, and databases locked by a particular lockId. + * + * @param lockId lock id to look for lock components + * @return number of components, or 0 if there is no lock + */ + public static int countLockComponents(Configuration conf, long lockId) throws Exception { + Connection conn = null; + PreparedStatement stmt = null; + ResultSet rs = null; + try { + conn = getConnection(conf); + stmt = conn.prepareStatement("SELECT count(*) FROM hive_locks WHERE hl_lock_ext_id = ?"); + stmt.setLong(1, lockId); + rs = stmt.executeQuery(); + if (!rs.next()) { + return 0; + } + return rs.getInt(1); + } finally { + closeResources(conn, stmt, rs); + } + } + + /** + * Utility method used to run COUNT queries like "select count(*) from ..." against metastore + * tables + * + * @param countQuery countQuery text + * @return count countQuery result + * @throws Exception + */ + public static int countQueryAgent(Configuration conf, String countQuery) throws Exception { + Connection conn = null; + Statement stmt = null; + ResultSet rs = null; + try { + conn = getConnection(conf); + stmt = conn.createStatement(); + rs = stmt.executeQuery(countQuery); + if (!rs.next()) { + return 0; + } + return rs.getInt(1); + } finally { + closeResources(conn, stmt, rs); + } + } + + public static String queryToString(Configuration conf, String query) throws Exception { + return queryToString(conf, query, true); + } + + public static String queryToString(Configuration conf, String query, boolean includeHeader) + throws Exception { + return queryToString(conf, query, includeHeader, " "); + } + + public static String queryToCsv(Configuration conf, String query) throws Exception { + return queryToString(conf, query, true, ","); + } + + public static String queryToCsv(Configuration conf, String query, boolean includeHeader) + throws Exception { + return queryToString(conf, query, includeHeader, ","); + } + + public static String queryToString( + Configuration conf, String query, boolean includeHeader, String columnSeparator) + throws Exception { + Connection conn = null; + Statement stmt = null; + ResultSet rs = null; + StringBuilder sb = new StringBuilder(); + try { + conn = getConnection(conf); + stmt = conn.createStatement(); + rs = stmt.executeQuery(query); + ResultSetMetaData rsmd = rs.getMetaData(); + if (includeHeader) { + for (int colPos = 1; colPos <= rsmd.getColumnCount(); colPos++) { + sb.append(rsmd.getColumnName(colPos)).append(columnSeparator); + } + sb.append('\n'); + } + while (rs.next()) { + for (int colPos = 1; colPos <= rsmd.getColumnCount(); colPos++) { + sb.append(rs.getObject(colPos)).append(columnSeparator); + } + sb.append('\n'); + } + } finally { + closeResources(conn, stmt, rs); + } + return sb.toString(); + } + + /** + * This is only for testing, it does not use the connectionPool from TxnHandler! + * + * @param conf + * @param query + * @throws Exception + */ + public static void executeUpdate(Configuration conf, String query) throws Exception { + Connection conn = null; + Statement stmt = null; + try { + conn = getConnection(conf); + stmt = conn.createStatement(); + stmt.executeUpdate(query); + } finally { + closeResources(conn, stmt, null); + } + } + + @SuppressWarnings("ClassNewInstance") + public static Connection getConnection(Configuration conf) throws Exception { + String jdbcDriver = MetastoreConf.getVar(conf, ConfVars.CONNECTION_DRIVER); + Driver driver = (Driver) Class.forName(jdbcDriver).newInstance(); + Properties prop = new Properties(); + String driverUrl = MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY); + String user = MetastoreConf.getVar(conf, ConfVars.CONNECTION_USER_NAME); + String passwd = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD); + prop.setProperty("user", user); + prop.setProperty("password", passwd); + Connection conn = driver.connect(driverUrl, prop); + conn.setAutoCommit(true); + + DatabaseProduct dbProduct = + determineDatabaseProduct(conn.getMetaData().getDatabaseProductName(), conf); + String initSql = dbProduct.getPrepareTxnStmt(); + if (initSql != null) { + try (Statement stmt = conn.createStatement()) { + stmt.execute(initSql); + } + } + return conn; + } + + public static void closeResources(Connection conn, Statement stmt, ResultSet rs) { + if (rs != null) { + try { + rs.close(); + } catch (SQLException e) { + LOG.error("Error closing ResultSet: " + e.getMessage()); + } + } + + if (stmt != null) { + try { + stmt.close(); + } catch (SQLException e) { + System.err.println("Error closing Statement: " + e.getMessage()); + } + } + + if (conn != null) { + try { + conn.rollback(); + } catch (SQLException e) { + System.err.println("Error rolling back: " + e.getMessage()); + } + try { + conn.close(); + } catch (SQLException e) { + System.err.println("Error closing Connection: " + e.getMessage()); + } + } + } + + /** The list is small, and the object is generated, so we don't use sets/equals/etc. */ + public static ShowLocksResponseElement checkLock( + LockType expectedType, + LockState expectedState, + String expectedDb, + String expectedTable, + String expectedPartition, + List actuals) { + return checkLock( + expectedType, expectedState, expectedDb, expectedTable, expectedPartition, actuals, false); + } + + public static ShowLocksResponseElement checkLock( + LockType expectedType, + LockState expectedState, + String expectedDb, + String expectedTable, + String expectedPartition, + List actuals, + boolean skipFirst) { + boolean skip = skipFirst; + for (ShowLocksResponseElement actual : actuals) { + if (expectedType == actual.getType() + && expectedState == actual.getState() + && StringUtils.equals(normalizeCase(expectedDb), normalizeCase(actual.getDbname())) + && StringUtils.equals(normalizeCase(expectedTable), normalizeCase(actual.getTablename())) + && StringUtils.equals( + normalizeCase(expectedPartition), normalizeCase(actual.getPartname()))) { + if (!skip) { + return actual; + } + skip = false; + } + } + Assert.fail( + "Could't find {" + + expectedType + + ", " + + expectedState + + ", " + + expectedDb + + ", " + + expectedTable + + ", " + + expectedPartition + + "} in " + + actuals); + throw new IllegalStateException("How did it get here?!"); + } + + private static String normalizeCase(String s) { + return s == null ? null : s.toLowerCase(); + } +} diff --git a/sdks/java/io/iceberg/src/test/resources/hive-schema-3.1.0.derby.sql b/sdks/java/io/iceberg/src/test/resources/hive-schema-3.1.0.derby.sql deleted file mode 100644 index 808c60585764..000000000000 --- a/sdks/java/io/iceberg/src/test/resources/hive-schema-3.1.0.derby.sql +++ /dev/null @@ -1,267 +0,0 @@ --- --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You under the Apache License, Version 2.0 --- (the "License"); you may not use this file except in compliance with --- the License. You may obtain a copy of the License at --- --- http://www.apache.org/licenses/LICENSE-2.0 --- --- Unless required by applicable law or agreed to in writing, software --- distributed under the License is distributed on an "AS IS" BASIS, --- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. --- See the License for the specific language governing permissions and --- limitations under the License. --- --- This file was copied from Apache Hive, at: --- https://github.com/apache/hive/blob/master/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-3.1.0.derby.sql --- --- This has been modified slightly for compatibility with older Hive versions. --- --- Timestamp: 2011-09-22 15:32:02.024 --- Source database is: /home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb --- Connection URL is: jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb --- Specified schema is: APP --- appendLogs: false - --- ---------------------------------------------- --- DDL Statements for functions --- ---------------------------------------------- - -CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ; - -CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ; - --- ---------------------------------------------- --- DDL Statements for tables --- ---------------------------------------------- -CREATE TABLE "APP"."DBS" ( - "DB_ID" BIGINT NOT NULL, - "DESC" VARCHAR(4000), - "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, - "NAME" VARCHAR(128), - "OWNER_NAME" VARCHAR(128), - "OWNER_TYPE" VARCHAR(10), - "CTLG_NAME" VARCHAR(256) -); - -CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); - -CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL); - -CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); - -CREATE TABLE "APP"."ROLE_MAP" ("ROLE_GRANT_ID" BIGINT NOT NULL, "ADD_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "ROLE_ID" BIGINT); - -CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "USER_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128)); - -CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "OWNER_TYPE" VARCHAR(10), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N'); - -CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000), "IS_COMPRESSED" CHAR(1) NOT NULL, "LOCATION" VARCHAR(4000), "NUM_BUCKETS" INTEGER NOT NULL, "OUTPUT_FORMAT" VARCHAR(4000), "SERDE_ID" BIGINT, "CD_ID" BIGINT, "IS_STOREDASSUBDIRECTORIES" CHAR(1) NOT NULL); - -CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL); - -CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."BUCKETING_COLS" ("SD_ID" BIGINT NOT NULL, "BUCKET_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SD_PARAMS" ("SD_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID" BIGINT NOT NULL); - -CREATE TABLE "APP"."SKEWED_STRING_LIST_VALUES" ("STRING_LIST_ID" BIGINT NOT NULL, "STRING_LIST_VALUE" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SKEWED_COL_NAMES" ("SD_ID" BIGINT NOT NULL, "SKEWED_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_LIST_ID_KID" BIGINT NOT NULL, "LOCATION" VARCHAR(4000)); - -CREATE TABLE "APP"."SKEWED_VALUES" ("SD_ID_OID" BIGINT NOT NULL, "STRING_LIST_ID_EID" BIGINT NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255)); - -CREATE TABLE "APP"."CTLGS" ( - "CTLG_ID" BIGINT NOT NULL, - "NAME" VARCHAR(256) UNIQUE, - "DESC" VARCHAR(4000), - "LOCATION_URI" VARCHAR(4000) NOT NULL); - --- ---------------------------------------------- --- DML Statements --- ---------------------------------------------- - -INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_VAL" FROM "APP"."SEQUENCE_TABLE" WHERE "SEQUENCE_NAME" = 'org.apache.hadoop.hive.metastore.model.MNotificationLog'); - --- ---------------------------------------------- --- DDL Statements for indexes --- ---------------------------------------------- - - -CREATE UNIQUE INDEX "APP"."ROLEENTITYINDEX" ON "APP"."ROLES" ("ROLE_NAME"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME"); - -CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."GLOBALPRIVILEGEINDEX" ON "APP"."GLOBAL_PRIVS" ("AUTHORIZER", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); - - --- ---------------------------------------------- --- DDL Statements for keys --- ---------------------------------------------- - --- primary/unique -ALTER TABLE "APP"."CDS" ADD CONSTRAINT "SQL110922153006460" PRIMARY KEY ("CD_ID"); - -ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); - -ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_PK" PRIMARY KEY ("SD_ID"); - -ALTER TABLE "APP"."SERDES" ADD CONSTRAINT "SERDES_PK" PRIMARY KEY ("SERDE_ID"); - -ALTER TABLE "APP"."ROLES" ADD CONSTRAINT "ROLES_PK" PRIMARY KEY ("ROLE_ID"); - -ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_PK" PRIMARY KEY ("SERDE_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID"); - -ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_PK" PRIMARY KEY ("DB_ID"); - -ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_PK" PRIMARY KEY ("ROLE_GRANT_ID"); - -ALTER TABLE "APP"."GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_PK" PRIMARY KEY ("USER_GRANT_ID"); - -ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "SQL110922153006740" PRIMARY KEY ("CD_ID", "COLUMN_NAME"); - -ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_PK" PRIMARY KEY ("TBL_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."SKEWED_STRING_LIST" ADD CONSTRAINT "SKEWED_STRING_LIST_PK" PRIMARY KEY ("STRING_LIST_ID"); - -ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_PK" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_PK" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID"); - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_PK" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX"); - -ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); - --- foreign - -ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK2" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_FK1" FOREIGN KEY ("ROLE_ID") REFERENCES "APP"."ROLES" ("ROLE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "COLUMNS_V2_FK1" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_FK1" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY ("SD_ID_OID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK2" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID"); - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; - --- ---------------------------------------------- --- DDL Statements for checks --- ---------------------------------------------- - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); - --- ---------------------------- --- Transaction and Lock Tables --- ---------------------------- -CREATE TABLE HIVE_LOCKS ( - HL_LOCK_EXT_ID bigint NOT NULL, - HL_LOCK_INT_ID bigint NOT NULL, - HL_TXNID bigint NOT NULL, - HL_DB varchar(128) NOT NULL, - HL_TABLE varchar(128), - HL_PARTITION varchar(767), - HL_LOCK_STATE char(1) NOT NULL, - HL_LOCK_TYPE char(1) NOT NULL, - HL_LAST_HEARTBEAT bigint NOT NULL, - HL_ACQUIRED_AT bigint, - HL_USER varchar(128) NOT NULL, - HL_HOST varchar(128) NOT NULL, - HL_HEARTBEAT_COUNT integer, - HL_AGENT_INFO varchar(128), - HL_BLOCKEDBY_EXT_ID bigint, - HL_BLOCKEDBY_INT_ID bigint, - PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) -); - -CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); - -CREATE TABLE NEXT_LOCK_ID ( - NL_NEXT bigint NOT NULL -); -INSERT INTO NEXT_LOCK_ID VALUES(1); - -CREATE TABLE AUX_TABLE ( - MT_KEY1 varchar(128) NOT NULL, - MT_KEY2 bigint NOT NULL, - MT_COMMENT varchar(255), - PRIMARY KEY(MT_KEY1, MT_KEY2) -); - ---1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK ---This is a good candidate for Index orgainzed table - --- ----------------------------------------------------------------- --- Record schema version. Should be the last step in the init script --- ----------------------------------------------------------------- -INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '3.1.0', 'Hive release version 3.1.0'); \ No newline at end of file diff --git a/sdks/java/io/iceberg/src/test/resources/hive-schema-4.0.0.derby.sql b/sdks/java/io/iceberg/src/test/resources/hive-schema-4.0.0.derby.sql new file mode 100644 index 000000000000..775ae9c22ac2 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/resources/hive-schema-4.0.0.derby.sql @@ -0,0 +1,898 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- +-- This file was copied from Apache Hive, at: +-- https://github.com/apache/hive/blob/master/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql +-- +-- This has been modified slightly for compatibility with older Hive versions. +-- +-- Timestamp: 2011-09-22 15:32:02.024 +-- Source database is: /home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Connection URL is: jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Specified schema is: APP +-- appendLogs: false + +-- ---------------------------------------------- +-- DDL Statements for functions +-- ---------------------------------------------- + +CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ; + +CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ; + +-- ---------------------------------------------- +-- DDL Statements for tables +-- ---------------------------------------------- +CREATE TABLE "APP"."DBS" ( + "DB_ID" BIGINT NOT NULL, + "DESC" VARCHAR(4000), + "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, + "NAME" VARCHAR(128), + "OWNER_NAME" VARCHAR(128), + "OWNER_TYPE" VARCHAR(10), + "CTLG_NAME" VARCHAR(256) NOT NULL DEFAULT 'hive', + "CREATE_TIME" INTEGER, + "DB_MANAGED_LOCATION_URI" VARCHAR(4000), + "TYPE" VARCHAR(32) DEFAULT 'NATIVE' NOT NULL, + "DATACONNECTOR_NAME" VARCHAR(128), + "REMOTE_DBNAME" VARCHAR(128) +); + +CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +CREATE TABLE "APP"."TBL_COL_PRIVS" ("TBL_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_COL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."PARTITION_KEY_VALS" ("PART_ID" BIGINT NOT NULL, "PART_KEY_VAL" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."DB_PRIVS" ("DB_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DB_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."DC_PRIVS" ("DC_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "NAME" VARCHAR(128), "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DC_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT, "WRITE_ID" BIGINT DEFAULT 0); + +CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); + +CREATE TABLE "APP"."PART_PRIVS" ("PART_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."ROLE_MAP" ("ROLE_GRANT_ID" BIGINT NOT NULL, "ADD_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "ROLE_ID" BIGINT); + +CREATE TABLE "APP"."TYPES" ("TYPES_ID" BIGINT NOT NULL, "TYPE_NAME" VARCHAR(128), "TYPE1" VARCHAR(767), "TYPE2" VARCHAR(767)); + +CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "USER_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."PARTITION_PARAMS" ("PART_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."PARTITION_EVENTS" ( + "PART_NAME_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256), + "DB_NAME" VARCHAR(128), + "EVENT_TIME" BIGINT NOT NULL, + "EVENT_TYPE" INTEGER NOT NULL, + "PARTITION_NAME" VARCHAR(767), + "TBL_NAME" VARCHAR(256) +); + +CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "COLUMN_NAME" VARCHAR(128) NOT NULL, "TYPE_NAME" VARCHAR(4000) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128)); + +CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "OWNER_TYPE" VARCHAR(10), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N', "WRITE_ID" BIGINT DEFAULT 0); + +CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."PART_COL_PRIVS" ("PART_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_COL_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000), "IS_COMPRESSED" CHAR(1) NOT NULL, "LOCATION" VARCHAR(4000), "NUM_BUCKETS" INTEGER NOT NULL, "OUTPUT_FORMAT" VARCHAR(4000), "SERDE_ID" BIGINT, "CD_ID" BIGINT, "IS_STOREDASSUBDIRECTORIES" CHAR(1) NOT NULL); + +CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL); + +CREATE TABLE "APP"."TAB_COL_STATS"( + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TABLE_NAME" VARCHAR(256) NOT NULL, + "COLUMN_NAME" VARCHAR(767) NOT NULL, + "COLUMN_TYPE" VARCHAR(128) NOT NULL, + "LONG_LOW_VALUE" BIGINT, + "LONG_HIGH_VALUE" BIGINT, + "DOUBLE_LOW_VALUE" DOUBLE, + "DOUBLE_HIGH_VALUE" DOUBLE, + "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), + "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), + "NUM_DISTINCTS" BIGINT, + "NUM_NULLS" BIGINT NOT NULL, + "AVG_COL_LEN" DOUBLE, + "MAX_COL_LEN" BIGINT, + "NUM_TRUES" BIGINT, + "NUM_FALSES" BIGINT, + "LAST_ANALYZED" BIGINT, + "CS_ID" BIGINT NOT NULL, + "TBL_ID" BIGINT NOT NULL, + "BIT_VECTOR" BLOB, + "ENGINE" VARCHAR(128) NOT NULL, + "HISTOGRAM" BLOB +); + +CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."BUCKETING_COLS" ("SD_ID" BIGINT NOT NULL, "BUCKET_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."TYPE_FIELDS" ("TYPE_NAME" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "FIELD_NAME" VARCHAR(128) NOT NULL, "FIELD_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."NUCLEUS_TABLES" ("CLASS_NAME" VARCHAR(128) NOT NULL, "TABLE_NAME" VARCHAR(128) NOT NULL, "TYPE" VARCHAR(4) NOT NULL, "OWNER" VARCHAR(2) NOT NULL, "VERSION" VARCHAR(20) NOT NULL, "INTERFACE_NAME" VARCHAR(256) DEFAULT NULL); + +CREATE TABLE "APP"."SD_PARAMS" ("SD_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."SKEWED_STRING_LIST_VALUES" ("STRING_LIST_ID" BIGINT NOT NULL, "STRING_LIST_VALUE" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_NAMES" ("SD_ID" BIGINT NOT NULL, "SKEWED_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_LIST_ID_KID" BIGINT NOT NULL, "LOCATION" VARCHAR(4000)); + +CREATE TABLE "APP"."SKEWED_VALUES" ("SD_ID_OID" BIGINT NOT NULL, "STRING_LIST_ID_EID" BIGINT NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."MASTER_KEYS" ("KEY_ID" INTEGER NOT NULL generated always as identity (start with 1), "MASTER_KEY" VARCHAR(767)); + +CREATE TABLE "APP"."DELEGATION_TOKENS" ( "TOKEN_IDENT" VARCHAR(767) NOT NULL, "TOKEN" VARCHAR(767)); + +CREATE TABLE "APP"."PART_COL_STATS"( + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TABLE_NAME" VARCHAR(256) NOT NULL, + "PARTITION_NAME" VARCHAR(767) NOT NULL, + "COLUMN_NAME" VARCHAR(767) NOT NULL, + "COLUMN_TYPE" VARCHAR(128) NOT NULL, + "LONG_LOW_VALUE" BIGINT, + "LONG_HIGH_VALUE" BIGINT, + "DOUBLE_LOW_VALUE" DOUBLE, + "DOUBLE_HIGH_VALUE" DOUBLE, + "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), + "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), + "NUM_DISTINCTS" BIGINT, + "BIT_VECTOR" BLOB, + "NUM_NULLS" BIGINT NOT NULL, + "AVG_COL_LEN" DOUBLE, + "MAX_COL_LEN" BIGINT, + "NUM_TRUES" BIGINT, + "NUM_FALSES" BIGINT, + "LAST_ANALYZED" BIGINT, + "CS_ID" BIGINT NOT NULL, + "PART_ID" BIGINT NOT NULL, + "ENGINE" VARCHAR(128) NOT NULL, + "HISTOGRAM" BLOB +); + +CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255)); + +CREATE TABLE "APP"."FUNCS" ("FUNC_ID" BIGINT NOT NULL, "CLASS_NAME" VARCHAR(4000), "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "FUNC_NAME" VARCHAR(128), "FUNC_TYPE" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "OWNER_TYPE" VARCHAR(10)); + +CREATE TABLE "APP"."FUNC_RU" ("FUNC_ID" BIGINT NOT NULL, "RESOURCE_TYPE" INTEGER NOT NULL, "RESOURCE_URI" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."NOTIFICATION_LOG" ( + "NL_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256), + "DB_NAME" VARCHAR(128), + "EVENT_ID" BIGINT NOT NULL, + "EVENT_TIME" INTEGER NOT NULL, + "EVENT_TYPE" VARCHAR(32) NOT NULL, + "MESSAGE" CLOB, + "TBL_NAME" VARCHAR(256), + "MESSAGE_FORMAT" VARCHAR(16) +); + +CREATE UNIQUE INDEX "APP"."NOTIFICATION_LOG_EVENT_ID" ON "APP"."NOTIFICATION_LOG" ("EVENT_ID"); + +CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."KEY_CONSTRAINTS" ( + "CHILD_CD_ID" BIGINT, + "CHILD_INTEGER_IDX" INTEGER, + "CHILD_TBL_ID" BIGINT, + "PARENT_CD_ID" BIGINT, + "PARENT_INTEGER_IDX" INTEGER, + "PARENT_TBL_ID" BIGINT NOT NULL, + "POSITION" BIGINT NOT NULL, + "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, + "CONSTRAINT_TYPE" SMALLINT NOT NULL, + "UPDATE_RULE" SMALLINT, + "DELETE_RULE" SMALLINT, + "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL, + "DEFAULT_VALUE" VARCHAR(400) +); + +CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ( + "PROPERTY_KEY" VARCHAR(255) NOT NULL, + "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, + "DESCRIPTION" VARCHAR(1000), + "PROPERTYCONTENT" BLOB +); + +CREATE TABLE "APP"."WM_RESOURCEPLAN" (RP_ID BIGINT NOT NULL, NS VARCHAR(128), NAME VARCHAR(128) NOT NULL, QUERY_PARALLELISM INTEGER, STATUS VARCHAR(20) NOT NULL, DEFAULT_POOL_ID BIGINT); + +CREATE TABLE "APP"."WM_POOL" (POOL_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, PATH VARCHAR(1024) NOT NULL, ALLOC_FRACTION DOUBLE, QUERY_PARALLELISM INTEGER, SCHEDULING_POLICY VARCHAR(1024)); + +CREATE TABLE "APP"."WM_TRIGGER" (TRIGGER_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, NAME VARCHAR(128) NOT NULL, TRIGGER_EXPRESSION VARCHAR(1024), ACTION_EXPRESSION VARCHAR(1024), IS_IN_UNMANAGED INTEGER NOT NULL DEFAULT 0); + +CREATE TABLE "APP"."WM_POOL_TO_TRIGGER" (POOL_ID BIGINT NOT NULL, TRIGGER_ID BIGINT NOT NULL); + +CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, ENTITY_TYPE VARCHAR(128) NOT NULL, ENTITY_NAME VARCHAR(128) NOT NULL, POOL_ID BIGINT, ORDERING INTEGER); + +CREATE TABLE "APP"."MV_CREATION_METADATA" ( + "MV_CREATION_METADATA_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TBL_NAME" VARCHAR(256) NOT NULL, + "TXN_LIST" CLOB, + "MATERIALIZATION_TIME" BIGINT NOT NULL +); + +CREATE TABLE "APP"."MV_TABLES_USED" ( + "MV_CREATION_METADATA_ID" BIGINT NOT NULL, + "TBL_ID" BIGINT NOT NULL, + "INSERTED_COUNT" BIGINT NOT NULL DEFAULT 0, + "UPDATED_COUNT" BIGINT NOT NULL DEFAULT 0, + "DELETED_COUNT" BIGINT NOT NULL DEFAULT 0 +); + +CREATE TABLE "APP"."CTLGS" ( + "CTLG_ID" BIGINT NOT NULL, + "NAME" VARCHAR(256) UNIQUE, + "DESC" VARCHAR(4000), + "LOCATION_URI" VARCHAR(4000) NOT NULL, + "CREATE_TIME" INTEGER); + +-- Insert a default value. The location is TBD. Hive will fix this when it starts +INSERT INTO "APP"."CTLGS" ("CTLG_ID", "NAME", "DESC", "LOCATION_URI", "CREATE_TIME") +VALUES (1, 'hive', 'Default catalog for Hive', 'TBD', NULL); + +-- ---------------------------------------------- +-- DML Statements +-- ---------------------------------------------- + +INSERT INTO "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID", "NEXT_EVENT_ID") SELECT * FROM (VALUES (1,1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_EVENT_ID" FROM "APP"."NOTIFICATION_SEQUENCE"); + +INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_VAL" FROM "APP"."SEQUENCE_TABLE" WHERE "SEQUENCE_NAME" = 'org.apache.hadoop.hive.metastore.model.MNotificationLog'); + +-- ---------------------------------------------- +-- DDL Statements for indexes +-- ---------------------------------------------- + +CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("AUTHORIZER", "TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("AUTHORIZER", "DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."DCPRIVILEGEINDEX" ON "APP"."DC_PRIVS" ("AUTHORIZER", "NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DC_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME","CAT_NAME"); + +CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("DB_NAME", "TABLE_NAME", "COLUMN_NAME", "CAT_NAME"); + +CREATE INDEX "APP"."PARTPRIVILEGEINDEX" ON "APP"."PART_PRIVS" ("AUTHORIZER", "PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."ROLEENTITYINDEX" ON "APP"."ROLES" ("ROLE_NAME"); + +CREATE INDEX "APP"."TABLEPRIVILEGEINDEX" ON "APP"."TBL_PRIVS" ("AUTHORIZER", "TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUETABLE" ON "APP"."TBLS" ("TBL_NAME", "DB_ID"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME"); + +CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."GLOBALPRIVILEGEINDEX" ON "APP"."GLOBAL_PRIVS" ("AUTHORIZER", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_TYPE" ON "APP"."TYPES" ("TYPE_NAME"); + +CREATE INDEX "APP"."PARTITIONCOLUMNPRIVILEGEINDEX" ON "APP"."PART_COL_PRIVS" ("AUTHORIZER", "PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("TBL_ID", "PART_NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUEFUNCTION" ON "APP"."FUNCS" ("FUNC_NAME", "DB_ID"); + +CREATE INDEX "APP"."FUNCS_N49" ON "APP"."FUNCS" ("DB_ID"); + +CREATE INDEX "APP"."FUNC_RU_N49" ON "APP"."FUNC_RU" ("FUNC_ID"); + +CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID"); + +CREATE INDEX "APP"."CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "APP"."KEY_CONSTRAINTS"("CONSTRAINT_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_RESOURCEPLAN" ON "APP"."WM_RESOURCEPLAN" ("NS", "NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_POOL" ON "APP"."WM_POOL" ("RP_ID", "PATH"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_TRIGGER" ON "APP"."WM_TRIGGER" ("RP_ID", "NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "ENTITY_TYPE", "ENTITY_NAME"); + +CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); + + +-- ---------------------------------------------- +-- DDL Statements for keys +-- ---------------------------------------------- + +-- primary/unique + +ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_PK" PRIMARY KEY ("TBL_COLUMN_GRANT_ID"); + +ALTER TABLE "APP"."CDS" ADD CONSTRAINT "SQL110922153006460" PRIMARY KEY ("CD_ID"); + +ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_PK" PRIMARY KEY ("DB_GRANT_ID"); + +ALTER TABLE "APP"."DC_PRIVS" ADD CONSTRAINT "DC_PRIVS_PK" PRIMARY KEY ("DC_GRANT_ID"); + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); + +ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); + +ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_PK" PRIMARY KEY ("PART_GRANT_ID"); + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_PK" PRIMARY KEY ("SD_ID"); + +ALTER TABLE "APP"."SERDES" ADD CONSTRAINT "SERDES_PK" PRIMARY KEY ("SERDE_ID"); + +ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_PK" PRIMARY KEY ("SD_ID", "COLUMN_NAME"); + +ALTER TABLE "APP"."PARTITION_EVENTS" ADD CONSTRAINT "PARTITION_EVENTS_PK" PRIMARY KEY ("PART_NAME_ID"); + +ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_PK" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME"); + +ALTER TABLE "APP"."ROLES" ADD CONSTRAINT "ROLES_PK" PRIMARY KEY ("ROLE_ID"); + +ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_PK" PRIMARY KEY ("TBL_GRANT_ID"); + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_PK" PRIMARY KEY ("SERDE_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."NUCLEUS_TABLES" ADD CONSTRAINT "NUCLEUS_TABLES_PK" PRIMARY KEY ("CLASS_NAME"); + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID"); + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_PK" PRIMARY KEY ("DB_ID"); + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_PK" PRIMARY KEY ("ROLE_GRANT_ID"); + +ALTER TABLE "APP"."GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_PK" PRIMARY KEY ("USER_GRANT_ID"); + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_PK" PRIMARY KEY ("PART_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."TYPES" ADD CONSTRAINT "TYPES_PK" PRIMARY KEY ("TYPES_ID"); + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "SQL110922153006740" PRIMARY KEY ("CD_ID", "COLUMN_NAME"); + +ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_PK" PRIMARY KEY ("PART_COLUMN_GRANT_ID"); + +ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_PK" PRIMARY KEY ("PART_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_PK" PRIMARY KEY ("PART_ID"); + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_PK" PRIMARY KEY ("TBL_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST" ADD CONSTRAINT "SKEWED_STRING_LIST_PK" PRIMARY KEY ("STRING_LIST_ID"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_PK" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_PK" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID"); + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_PK" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX"); + +ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_PK" PRIMARY KEY ("CS_ID"); + +ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_PK" PRIMARY KEY ("CS_ID"); + +ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_PK" PRIMARY KEY ("FUNC_ID"); + +ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_PK" PRIMARY KEY ("FUNC_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."NOTIFICATION_LOG" ADD CONSTRAINT "NOTIFICATION_LOG_PK" PRIMARY KEY ("NL_ID"); + +ALTER TABLE "APP"."NOTIFICATION_SEQUENCE" ADD CONSTRAINT "NOTIFICATION_SEQUENCE_PK" PRIMARY KEY ("NNI_ID"); + +ALTER TABLE "APP"."NOTIFICATION_SEQUENCE" ADD CONSTRAINT "ONE_ROW_CONSTRAINT" CHECK (NNI_ID = 1); + +ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("PARENT_TBL_ID", "CONSTRAINT_NAME", "POSITION"); + +ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD CONSTRAINT "PROPERTY_KEY_PK" PRIMARY KEY ("PROPERTY_KEY"); + +ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID"); + +ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); + + +-- foreign +ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK2" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_FK1" FOREIGN KEY ("TYPE_NAME") REFERENCES "APP"."TYPES" ("TYPES_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_FK1" FOREIGN KEY ("ROLE_ID") REFERENCES "APP"."ROLES" ("ROLE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "COLUMNS_V2_FK1" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_FK1" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY ("SD_ID_OID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK2" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_FK" FOREIGN KEY ("TBL_ID") REFERENCES TBLS("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_FK" FOREIGN KEY ("PART_ID") REFERENCES PARTITIONS("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID"); + +ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_FK1" FOREIGN KEY ("FUNC_ID") REFERENCES "APP"."FUNCS" ("FUNC_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_PK" PRIMARY KEY ("RP_ID"); + +ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_PK" PRIMARY KEY ("POOL_ID"); + +ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_FK1" FOREIGN KEY ("DEFAULT_POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_PK" PRIMARY KEY ("TRIGGER_ID"); + +ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK1" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK2" FOREIGN KEY ("TRIGGER_ID") REFERENCES "APP"."WM_TRIGGER" ("TRIGGER_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_PK" PRIMARY KEY ("MAPPING_ID"); + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_PK" PRIMARY KEY ("TBL_ID", "MV_CREATION_METADATA_ID"); + +ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +-- ---------------------------------------------- +-- DDL Statements for checks +-- ---------------------------------------------- + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); + +-- ---------------------------- +-- Transaction and Lock Tables +-- ---------------------------- +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL, + TXN_AGENT_INFO varchar(128), + TXN_META_INFO varchar(128), + TXN_HEARTBEAT_COUNT integer, + TXN_TYPE integer +); + +INSERT INTO TXNS (TXN_ID, TXN_STATE, TXN_STARTED, TXN_LAST_HEARTBEAT, TXN_USER, TXN_HOST) + VALUES(0, 'c', 0, 0, '', ''); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(256), + TC_PARTITION varchar(767), + TC_OPERATION_TYPE char(1) NOT NULL, + TC_WRITEID bigint +); + +CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS (TC_TXNID); + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID bigint NOT NULL, + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(256), + CTC_PARTITION varchar(767), + CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL, + CTC_WRITEID bigint, + CTC_UPDATE_DELETE char(1) NOT NULL +); + +CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION); + +CREATE TABLE TXN_LOCK_TBL ( + TXN_LOCK bigint NOT NULL +); +INSERT INTO TXN_LOCK_TBL VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint NOT NULL, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(256), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + HL_HEARTBEAT_COUNT integer, + HL_AGENT_INFO varchar(128), + HL_BLOCKEDBY_EXT_ID bigint, + HL_BLOCKEDBY_INT_ID bigint, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(256) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_TBLPROPERTIES varchar(2048), + CQ_WORKER_ID varchar(128), + CQ_ENQUEUE_TIME bigint, + CQ_START bigint, + CQ_RUN_AS varchar(128), + CQ_HIGHEST_WRITE_ID bigint, + CQ_META_INFO varchar(2048) for bit data, + CQ_HADOOP_JOB_ID varchar(32), + CQ_ERROR_MESSAGE clob, + CQ_NEXT_TXN_ID bigint, + CQ_TXN_ID bigint, + CQ_COMMIT_TIME bigint, + CQ_INITIATOR_ID varchar(128), + CQ_INITIATOR_VERSION varchar(128), + CQ_WORKER_VERSION varchar(128), + CQ_CLEANER_START bigint, + CQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + CQ_POOL_NAME varchar(128), + CQ_NUMBER_OF_BUCKETS integer, + CQ_ORDER_BY varchar(4000) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT bigint NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + +CREATE TABLE COMPLETED_COMPACTIONS ( + CC_ID bigint PRIMARY KEY, + CC_DATABASE varchar(128) NOT NULL, + CC_TABLE varchar(256) NOT NULL, + CC_PARTITION varchar(767), + CC_STATE char(1) NOT NULL, + CC_TYPE char(1) NOT NULL, + CC_TBLPROPERTIES varchar(2048), + CC_WORKER_ID varchar(128), + CC_ENQUEUE_TIME bigint, + CC_START bigint, + CC_END bigint, + CC_RUN_AS varchar(128), + CC_HIGHEST_WRITE_ID bigint, + CC_META_INFO varchar(2048) for bit data, + CC_HADOOP_JOB_ID varchar(32), + CC_ERROR_MESSAGE clob, + CC_NEXT_TXN_ID bigint, + CC_TXN_ID bigint, + CC_COMMIT_TIME bigint, + CC_INITIATOR_ID varchar(128), + CC_INITIATOR_VERSION varchar(128), + CC_WORKER_VERSION varchar(128), + CC_POOL_NAME varchar(128), + CC_NUMBER_OF_BUCKETS integer, + CC_ORDER_BY varchar(4000) +); + +CREATE INDEX COMPLETED_COMPACTIONS_RES ON COMPLETED_COMPACTIONS (CC_DATABASE,CC_TABLE,CC_PARTITION); + +-- HIVE-25842 +CREATE TABLE COMPACTION_METRICS_CACHE ( + CMC_DATABASE varchar(128) NOT NULL, + CMC_TABLE varchar(256) NOT NULL, + CMC_PARTITION varchar(767), + CMC_METRIC_TYPE varchar(128) NOT NULL, + CMC_METRIC_VALUE integer NOT NULL, + CMC_VERSION integer NOT NULL +); + +CREATE TABLE AUX_TABLE ( + MT_KEY1 varchar(128) NOT NULL, + MT_KEY2 bigint NOT NULL, + MT_COMMENT varchar(255), + PRIMARY KEY(MT_KEY1, MT_KEY2) +); + +--1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK +--This is a good candidate for Index orgainzed table +CREATE TABLE WRITE_SET ( + WS_DATABASE varchar(128) NOT NULL, + WS_TABLE varchar(256) NOT NULL, + WS_PARTITION varchar(767), + WS_TXNID bigint NOT NULL, + WS_COMMIT_ID bigint NOT NULL, + WS_OPERATION_TYPE char(1) NOT NULL +); + +CREATE TABLE TXN_TO_WRITE_ID ( + T2W_TXNID bigint NOT NULL, + T2W_DATABASE varchar(128) NOT NULL, + T2W_TABLE varchar(256) NOT NULL, + T2W_WRITEID bigint NOT NULL +); + +CREATE UNIQUE INDEX TBL_TO_TXN_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID); +CREATE UNIQUE INDEX TBL_TO_WRITE_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_WRITEID); + +CREATE TABLE NEXT_WRITE_ID ( + NWI_DATABASE varchar(128) NOT NULL, + NWI_TABLE varchar(256) NOT NULL, + NWI_NEXT bigint NOT NULL +); + +CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE); + +CREATE TABLE MIN_HISTORY_WRITE_ID ( + MH_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), + MH_DATABASE varchar(128) NOT NULL, + MH_TABLE varchar(256) NOT NULL, + MH_WRITEID bigint NOT NULL +); + +CREATE TABLE MIN_HISTORY_LEVEL ( + MHL_TXNID bigint NOT NULL, + MHL_MIN_OPEN_TXNID bigint NOT NULL, + PRIMARY KEY(MHL_TXNID) +); + +CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID); + +CREATE TABLE MATERIALIZATION_REBUILD_LOCKS ( + MRL_TXN_ID BIGINT NOT NULL, + MRL_DB_NAME VARCHAR(128) NOT NULL, + MRL_TBL_NAME VARCHAR(256) NOT NULL, + MRL_LAST_HEARTBEAT BIGINT NOT NULL, + PRIMARY KEY(MRL_TXN_ID) +); + +CREATE TABLE "APP"."I_SCHEMA" ( + "SCHEMA_ID" bigint primary key, + "SCHEMA_TYPE" integer not null, + "NAME" varchar(256) unique, + "DB_ID" bigint references "APP"."DBS" ("DB_ID"), + "COMPATIBILITY" integer not null, + "VALIDATION_LEVEL" integer not null, + "CAN_EVOLVE" char(1) not null, + "SCHEMA_GROUP" varchar(256), + "DESCRIPTION" varchar(4000) +); + +CREATE TABLE "APP"."SCHEMA_VERSION" ( + "SCHEMA_VERSION_ID" bigint primary key, + "SCHEMA_ID" bigint references "APP"."I_SCHEMA" ("SCHEMA_ID"), + "VERSION" integer not null, + "CREATED_AT" bigint not null, + "CD_ID" bigint references "APP"."CDS" ("CD_ID"), + "STATE" integer not null, + "DESCRIPTION" varchar(4000), + "SCHEMA_TEXT" clob, + "FINGERPRINT" varchar(256), + "SCHEMA_VERSION_NAME" varchar(256), + "SERDE_ID" bigint references "APP"."SERDES" ("SERDE_ID") +); + +CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SCHEMA_ID", "VERSION"); + +CREATE TABLE REPL_TXN_MAP ( + RTM_REPL_POLICY varchar(256) NOT NULL, + RTM_SRC_TXN_ID bigint NOT NULL, + RTM_TARGET_TXN_ID bigint NOT NULL, + PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID) +); + +CREATE TABLE "APP"."RUNTIME_STATS" ( + "RS_ID" bigint primary key, + "CREATE_TIME" integer not null, + "WEIGHT" integer not null, + "PAYLOAD" BLOB +); + +CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME); + +CREATE TABLE TXN_WRITE_NOTIFICATION_LOG ( + WNL_ID bigint NOT NULL, + WNL_TXNID bigint NOT NULL, + WNL_WRITEID bigint NOT NULL, + WNL_DATABASE varchar(128) NOT NULL, + WNL_TABLE varchar(256) NOT NULL, + WNL_PARTITION varchar(767) NOT NULL, + WNL_TABLE_OBJ clob NOT NULL, + WNL_PARTITION_OBJ clob, + WNL_FILES clob, + WNL_EVENT_TIME integer NOT NULL, + PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION) +); +INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1); + +CREATE TABLE "APP"."SCHEDULED_QUERIES" ( + "SCHEDULED_QUERY_ID" bigint primary key not null, + "SCHEDULE_NAME" varchar(256) not null, + "ENABLED" CHAR(1) NOT NULL DEFAULT 'N', + "CLUSTER_NAMESPACE" varchar(256) not null, + "USER" varchar(128) not null, + "SCHEDULE" varchar(256) not null, + "QUERY" varchar(4000) not null, + "NEXT_EXECUTION" integer, + "ACTIVE_EXECUTION_ID" bigint +); + +CREATE INDEX NEXTEXECUTIONINDEX ON APP.SCHEDULED_QUERIES (ENABLED,CLUSTER_NAMESPACE,NEXT_EXECUTION); +CREATE UNIQUE INDEX UNIQUE_SCHEDULED_QUERIES_NAME ON APP.SCHEDULED_QUERIES (SCHEDULE_NAME,CLUSTER_NAMESPACE); + +CREATE TABLE APP.SCHEDULED_EXECUTIONS ( + SCHEDULED_EXECUTION_ID bigint primary key not null, + EXECUTOR_QUERY_ID VARCHAR(256), + SCHEDULED_QUERY_ID bigint not null, + START_TIME integer not null, + END_TIME INTEGER, + LAST_UPDATE_TIME INTEGER, + ERROR_MESSAGE VARCHAR(2000), + STATE VARCHAR(256), + CONSTRAINT SCHEDULED_EXECUTIONS_SCHQ_FK FOREIGN KEY (SCHEDULED_QUERY_ID) REFERENCES APP.SCHEDULED_QUERIES(SCHEDULED_QUERY_ID) ON DELETE CASCADE +); + +CREATE INDEX LASTUPDATETIMEINDEX ON APP.SCHEDULED_EXECUTIONS (LAST_UPDATE_TIME); +CREATE INDEX SCHEDULED_EXECUTIONS_SCHQID ON APP.SCHEDULED_EXECUTIONS (SCHEDULED_QUERY_ID); +CREATE UNIQUE INDEX SCHEDULED_EXECUTIONS_UNIQUE_ID ON APP.SCHEDULED_EXECUTIONS (SCHEDULED_EXECUTION_ID); + +--HIVE-23516 +CREATE TABLE "APP"."REPLICATION_METRICS" ( + "RM_SCHEDULED_EXECUTION_ID" bigint NOT NULL, + "RM_POLICY" varchar(256) NOT NULL, + "RM_DUMP_EXECUTION_ID" bigint NOT NULL, + "RM_METADATA" varchar(4000), + "RM_PROGRESS" varchar(10000), + "RM_START_TIME" integer not null, + "MESSAGE_FORMAT" VARCHAR(16) DEFAULT 'json-0.2', + PRIMARY KEY("RM_SCHEDULED_EXECUTION_ID") +); + +CREATE INDEX "POLICY_IDX" ON "APP"."REPLICATION_METRICS" ("RM_POLICY"); +CREATE INDEX "DUMP_IDX" ON "APP"."REPLICATION_METRICS" ("RM_DUMP_EXECUTION_ID"); + +-- Create stored procedure tables +CREATE TABLE "APP"."STORED_PROCS" ( + "SP_ID" BIGINT NOT NULL, + "CREATE_TIME" INTEGER NOT NULL, + "DB_ID" BIGINT NOT NULL, + "NAME" VARCHAR(256) NOT NULL, + "OWNER_NAME" VARCHAR(128) NOT NULL, + "SOURCE" clob NOT NULL, + PRIMARY KEY ("SP_ID") +); + +CREATE UNIQUE INDEX "UNIQUESTOREDPROC" ON "STORED_PROCS" ("NAME", "DB_ID"); +ALTER TABLE "STORED_PROCS" ADD CONSTRAINT "STOREDPROC_FK1" FOREIGN KEY ("DB_ID") REFERENCES "DBS" ("DB_ID"); + +CREATE TABLE "APP"."DATACONNECTORS" ("NAME" VARCHAR(128) NOT NULL, "TYPE" VARCHAR(32) NOT NULL, "URL" VARCHAR(4000) NOT NULL, "COMMENT" VARCHAR(256), "OWNER_NAME" VARCHAR(256), "OWNER_TYPE" VARCHAR(10), "CREATE_TIME" INTEGER NOT NULL); +CREATE TABLE "APP"."DATACONNECTOR_PARAMS" ("NAME" VARCHAR(128) NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); +ALTER TABLE "APP"."DATACONNECTORS" ADD CONSTRAINT "DATACONNECTORS_KEY_PK" PRIMARY KEY ("NAME"); +ALTER TABLE "APP"."DATACONNECTOR_PARAMS" ADD CONSTRAINT "DATACONNECTOR_PARAMS_KEY_PK" PRIMARY KEY ("NAME", "PARAM_KEY"); +ALTER TABLE "APP"."DATACONNECTOR_PARAMS" ADD CONSTRAINT "NAME_FK1" FOREIGN KEY ("NAME") REFERENCES "APP"."DATACONNECTORS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; +ALTER TABLE "APP"."DC_PRIVS" ADD CONSTRAINT "DC_PRIVS_FK1" FOREIGN KEY ("NAME") REFERENCES "APP"."DATACONNECTORS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +-- Create stored procedure packages +CREATE TABLE "APP"."PACKAGES" ( + "PKG_ID" BIGINT NOT NULL, + "CREATE_TIME" INTEGER NOT NULL, + "DB_ID" BIGINT NOT NULL, + "NAME" VARCHAR(256) NOT NULL, + "OWNER_NAME" VARCHAR(128) NOT NULL, + "HEADER" clob NOT NULL, + "BODY" clob NOT NULL, + PRIMARY KEY ("PKG_ID") +); + +CREATE UNIQUE INDEX "UNIQUEPKG" ON "PACKAGES" ("NAME", "DB_ID"); +ALTER TABLE "PACKAGES" ADD CONSTRAINT "PACKAGES_FK1" FOREIGN KEY ("DB_ID") REFERENCES "DBS" ("DB_ID"); + +-- ----------------------------------------------------------------- +-- Record schema version. Should be the last step in the init script +-- ----------------------------------------------------------------- +INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '4.0.0', 'Hive release version 4.0.0'); From 1f0ebd6e8aa0b70e7f869b89921e579f62a144e1 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 27 Jan 2025 16:31:38 -0500 Subject: [PATCH 5/7] Fix checkstyle in copied codes --- sdks/java/io/expansion-service/build.gradle | 1 - sdks/java/io/iceberg/hive/build.gradle | 3 +-- .../sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java | 2 +- .../beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java | 2 +- 4 files changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 433000922b61..3d24385b7b4d 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -62,7 +62,6 @@ dependencies { // For writing to GCS runtimeOnly library.java.bigdataoss_gcs_connector // HiveCatalog - runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") runtimeOnly project(path: ":sdks:java:io:iceberg:hive") // BigQueryMetastoreCatalog (Java 11+) runtimeOnly project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow") diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index dabdc485f62f..9ab8f26d1bc7 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -28,13 +28,12 @@ ext.summary = "Runtime dependencies needed for Hive catalog integration." def hive_version = "4.0.1" def hbase_version = "2.6.1-hadoop3" def hadoop_version = "3.4.1" -def iceberg_version = "1.6.1" def avatica_version = "1.25.0" dependencies { // dependencies needed to run with iceberg's hive catalog // these dependencies are going to be included in io-expansion-service - runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + runtimeOnly ("org.apache.hive:hive-iceberg-catalog:$hive_version") // analyzeClassesDependencies fails with "Cannot accept visitor on URL", likely the plugin does not recognize "core" classifier // use "core" classifier to depend on un-shaded jar runtimeOnly ("org.apache.hive:hive-exec:$hive_version:core") { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java index cd7fd13f456a..d730b47bf420 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java @@ -257,7 +257,7 @@ public Table getTable(TableIdentifier identifier) throws TException, Interrupted return getTable(identifier.namespace().toString(), identifier.name()); } - public R run(ClientPool.Action action) + public T run(ClientPool.Action action) throws InterruptedException, TException { return clientPool.run(action, false); } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java index 6894d363495c..a41a56f233f3 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestTxnDbUtil.java @@ -324,7 +324,7 @@ public static int countLockComponents(Configuration conf, long lockId) throws Ex /** * Utility method used to run COUNT queries like "select count(*) from ..." against metastore - * tables + * tables. * * @param countQuery countQuery text * @return count countQuery result From 929fa9fe8946b90a0e51c277cb26971021faafa6 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 27 Jan 2025 16:56:21 -0500 Subject: [PATCH 6/7] Fix path --- .../catalog/hiveutils/TestHiveMetastore.java | 38 ++++++++----------- 1 file changed, 16 insertions(+), 22 deletions(-) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java index d730b47bf420..b92d91018462 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/hiveutils/TestHiveMetastore.java @@ -33,6 +33,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -43,10 +44,7 @@ import org.apache.hadoop.hive.metastore.IHMSHandler; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; -import org.apache.hadoop.hive.metastore.api.GetTableRequest; -import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.ClientPool; -import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.hadoop.Util; @@ -63,8 +61,8 @@ * HiveMetastoreExtension} instead. * *

Copied over from Iceberg's - * integration testing util + * href="https://github.com/apache/hive/blob/branch-4.0/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java"> + * Hive metastore Iceberg integration utils */ public class TestHiveMetastore { @@ -106,7 +104,6 @@ public class TestHiveMetastore { // It's tricky to clear all static fields in an HMS instance in order to switch derby root dir. // Therefore, we reuse the same derby root between tests and remove it after JVM exits. private static final File HIVE_LOCAL_DIR; - private static final File HIVE_EXTERNAL_WAREHOUSE_DIR; private static final String DERBY_PATH; static { @@ -114,7 +111,6 @@ public class TestHiveMetastore { HIVE_LOCAL_DIR = createTempDirectory("hive", asFileAttribute(fromString("rwxrwxrwx"))).toFile(); DERBY_PATH = new File(HIVE_LOCAL_DIR, "metastore_db").getPath(); - HIVE_EXTERNAL_WAREHOUSE_DIR = new File(HIVE_LOCAL_DIR, "external"); File derbyLogFile = new File(HIVE_LOCAL_DIR, "derby.log"); System.setProperty("derby.stream.error.file", derbyLogFile.getAbsolutePath()); setupMetastoreDB("jdbc:derby:" + DERBY_PATH + ";create=true"); @@ -202,6 +198,12 @@ public void stop() throws Exception { } if (executorService != null) { executorService.shutdown(); + try { + // Give it a reasonable timeout + executorService.awaitTermination(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } } if (baseHandler != null) { baseHandler.shutdown(); @@ -241,6 +243,9 @@ public void reset() throws Exception { Path warehouseRoot = new Path(hiveWarehousePath); FileSystem fs = Util.getFs(warehouseRoot, hiveConf); + if (!fs.exists(warehouseRoot)) { + return; + } for (FileStatus fileStatus : fs.listStatus(warehouseRoot)) { if (!fileStatus.getPath().getName().equals("derby.log") && !fileStatus.getPath().getName().equals("metastore_db")) { @@ -249,14 +254,6 @@ public void reset() throws Exception { } } - public Table getTable(String dbName, String tableName) throws TException, InterruptedException { - return clientPool.run(client -> client.getTable(new GetTableRequest(dbName, tableName))); - } - - public Table getTable(TableIdentifier identifier) throws TException, InterruptedException { - return getTable(identifier.namespace().toString(), identifier.name()); - } - public T run(ClientPool.Action action) throws InterruptedException, TException { return clientPool.run(action, false); @@ -285,16 +282,13 @@ private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf con private void initConf(HiveConf conf, int port) { conf.set(HiveConf.ConfVars.METASTORE_URIS.varname, "thrift://localhost:" + port); conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, hiveWarehousePath); - conf.set( - HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, - "file:" + HIVE_EXTERNAL_WAREHOUSE_DIR.getAbsolutePath()); + conf.set(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, hiveWarehousePath); conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "false"); conf.set(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES.varname, "false"); conf.set("iceberg.hive.client-pool-size", "2"); - // set to false so that TxnManager#checkLock does not throw exception when using UNSET data type - // operation - // in the requested lock component - conf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, false); + // Setting this to avoid thrift exception during running Iceberg tests outside Iceberg. + conf.set( + HiveConf.ConfVars.HIVE_IN_TEST.varname, HiveConf.ConfVars.HIVE_IN_TEST.getDefaultValue()); } private static void setupMetastoreDB(String dbURL) throws Exception { From c366f1aaf658b30d84f57e07d92916ac0f3c518a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 27 Jan 2025 17:41:02 -0500 Subject: [PATCH 7/7] add hive-iceberg-handler --- sdks/java/io/iceberg/hive/build.gradle | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index 9ab8f26d1bc7..a6e0267a1e26 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -33,7 +33,10 @@ def avatica_version = "1.25.0" dependencies { // dependencies needed to run with iceberg's hive catalog // these dependencies are going to be included in io-expansion-service - runtimeOnly ("org.apache.hive:hive-iceberg-catalog:$hive_version") + runtimeOnly ("org.apache.hive:hive-iceberg-handler:$hive_version") { + // prefer to use unshaded hive-exec + exclude group: "org.apache.hive", module: "hive-exec" + } // analyzeClassesDependencies fails with "Cannot accept visitor on URL", likely the plugin does not recognize "core" classifier // use "core" classifier to depend on un-shaded jar runtimeOnly ("org.apache.hive:hive-exec:$hive_version:core") {