From 79ea478b046a61650389bef5ad8e8fa7d9e5ae3e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 Aug 2024 16:33:40 -0400 Subject: [PATCH 1/8] iceberg hive support and integration tests --- .../IO_Iceberg_Integration_Tests.json | 2 +- .../IO_Iceberg_Integration_Tests.yml | 4 +- sdks/java/io/iceberg/hive/build.gradle | 77 ++ sdks/java/io/iceberg/hive/exec/build.gradle | 58 ++ .../io/iceberg/hive/IcebergHiveCatalogIT.java | 173 +++++ .../testutils/HiveMetastoreExtension.java | 68 ++ .../iceberg/hive/testutils/ScriptRunner.java | 203 +++++ .../hive/testutils/TestHiveMetastore.java | 273 +++++++ .../resources/hive-schema-3.1.0.derby.sql | 726 ++++++++++++++++++ .../sdk/io/iceberg/IcebergCatalogConfig.java | 37 +- .../IcebergReadSchemaTransformProvider.java | 56 +- .../beam/sdk/io/iceberg/IcebergUtils.java | 103 ++- .../IcebergWriteSchemaTransformProvider.java | 58 +- .../beam/sdk/io/iceberg/RecordWriter.java | 19 +- .../iceberg/SchemaTransformConfiguration.java | 69 ++ .../io/iceberg/WriteGroupedRowsToFiles.java | 12 +- .../io/iceberg/WriteUngroupedRowsToFiles.java | 16 +- .../sdk/io/iceberg/IcebergIOReadTest.java | 16 +- .../sdk/io/iceberg/IcebergIOWriteTest.java | 40 +- ...cebergReadSchemaTransformProviderTest.java | 4 +- ...ebergWriteSchemaTransformProviderTest.java | 5 +- .../beam/sdk/io/iceberg/ScanSourceTest.java | 33 +- settings.gradle.kts | 4 + 23 files changed, 1846 insertions(+), 210 deletions(-) create mode 100644 sdks/java/io/iceberg/hive/build.gradle create mode 100644 sdks/java/io/iceberg/hive/exec/build.gradle create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java create mode 100644 sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java create mode 100644 sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 3f63c0c9975f2..bbdc3a3910ef8 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": 2 + "modification": 3 } diff --git a/.github/workflows/IO_Iceberg_Integration_Tests.yml b/.github/workflows/IO_Iceberg_Integration_Tests.yml index 20d1f4bb60fd3..025c622eccfa9 100644 --- a/.github/workflows/IO_Iceberg_Integration_Tests.yml +++ b/.github/workflows/IO_Iceberg_Integration_Tests.yml @@ -75,4 +75,6 @@ jobs: - name: Run IcebergIO Integration Test uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:java:io:iceberg:integrationTest \ No newline at end of file + gradle-command: | + :sdks:java:io:iceberg:integrationTest \ + :sdks:java:io:iceberg:hive:integrationTest \ \ No newline at end of file diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle new file mode 100644 index 0000000000000..cca3726ec388d --- /dev/null +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -0,0 +1,77 @@ +import groovy.json.JsonOutput + +/* + * 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. + */ +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.iceberg.hive', + exportJavadoc: false, + shadowClosure: {}, +) + +description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" +ext.summary = "Runtime dependencies needed for Hive catalog integration." + +def hive_version = "3.1.3" + +dependencies { + // dependencies needed to run with iceberg's hive catalog + runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") + runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + runtimeOnly library.java.bigdataoss_gcs_connector + runtimeOnly library.java.hadoop_client + + // ----- below dependencies are for testing and will not appear in the shaded jar ----- + // Beam IcebergIO dependencies + testImplementation project(path: ":sdks:java:core", configuration: "shadow") + testImplementation project(":sdks:java:managed") + testImplementation project(":sdks:java:io:iceberg") + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testRuntimeOnly library.java.snake_yaml + + // needed to set up the test environment + testImplementation "org.apache.iceberg:iceberg-common:1.4.2" + testImplementation "org.apache.iceberg:iceberg-core:1.4.2" + testImplementation "org.assertj:assertj-core:3.11.1" + testImplementation library.java.junit + + // needed to set up test Hive Metastore + testImplementation ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") + testImplementation project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + testRuntimeOnly ("org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version") { + exclude group: "org.apache.hive", module: "hive-exec" + exclude group: "org.apache.parquet", module: "parquet-hadoop-bundle" + } +} + +task integrationTest(type: Test) { + group = "Verification" + def gcpTempLocation = project.findProperty('gcpTempLocation') ?: 'gs://temp-storage-for-end-to-end-tests/iceberg-hive-it' + systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ + "--tempLocation=${gcpTempLocation}", + ]) + + // Disable Gradle cache: these ITs interact with live service that should always be considered "out of date" + outputs.upToDateWhen { false } + + include '**/*IT.class' + + maxParallelForks 4 + classpath = sourceSets.test.runtimeClasspath + testClassesDirs = sourceSets.test.output.classesDirs +} \ No newline at end of file diff --git a/sdks/java/io/iceberg/hive/exec/build.gradle b/sdks/java/io/iceberg/hive/exec/build.gradle new file mode 100644 index 0000000000000..581f71ddedd1f --- /dev/null +++ b/sdks/java/io/iceberg/hive/exec/build.gradle @@ -0,0 +1,58 @@ +/* + * 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. + */ +plugins { + id 'org.apache.beam.module' + id 'java' + id 'com.github.johnrengelman.shadow' +} + +dependencies { + implementation("org.apache.hive:hive-exec:3.1.3") + permitUnusedDeclared("org.apache.hive:hive-exec:3.1.3") +} + +configurations { + shadow +} + +artifacts { + shadow(archives(shadowJar) { + builtBy shadowJar + }) +} + +shadowJar { + zip64 true + + // need to shade "com.google.guava" to avoid Guava conflict + relocate 'com.google.protobuf', getJavaRelocatedPath('com.google.protobuf') + relocate 'shaded.parquet', getJavaRelocatedPath('shaded.parquet') + relocate 'org.apache.parquet', getJavaRelocatedPath('org.apache.parquet') + + version "3.1.3" + mergeServiceFiles() + + exclude 'LICENSE' + exclude( + 'org/xml/**', + 'javax/**', + 'com/sun/**' + ) +} +description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive :: Exec" +ext.summary = "A copy of the hive-exec dependency with some popular libraries relocated." diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java new file mode 100644 index 0000000000000..7933368f771f8 --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java @@ -0,0 +1,173 @@ +/* + * 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.hive; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.hive.testutils.HiveMetastoreExtension; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.thrift.TException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +public class IcebergHiveCatalogIT { + private static final Schema DOUBLY_NESTED_ROW_SCHEMA = + Schema.builder() + .addStringField("doubly_nested_str") + .addInt64Field("doubly_nested_float") + .build(); + + private static final Schema NESTED_ROW_SCHEMA = + Schema.builder() + .addStringField("nested_str") + .addInt32Field("nested_int") + .addFloatField("nested_float") + .addRowField("nested_row", DOUBLY_NESTED_ROW_SCHEMA) + .build(); + private static final Schema ROW_SCHEMA = + Schema.builder() + .addStringField("str") + .addBooleanField("bool") + .addNullableInt32Field("nullable_int") + .addNullableInt64Field("nullable_long") + .addArrayField("arr_long", Schema.FieldType.INT64) + .addRowField("row", NESTED_ROW_SCHEMA) + .addNullableRowField("nullable_row", NESTED_ROW_SCHEMA) + .build(); + + private static final SimpleFunction ROW_FUNC = + new SimpleFunction() { + @Override + public Row apply(Long num) { + String strNum = Long.toString(num); + Row nestedRow = + Row.withSchema(NESTED_ROW_SCHEMA) + .addValue("nested_str_value_" + strNum) + .addValue(Integer.valueOf(strNum)) + .addValue(Float.valueOf(strNum + "." + strNum)) + .addValue( + Row.withSchema(DOUBLY_NESTED_ROW_SCHEMA) + .addValue("doubly_nested_str_value_" + strNum) + .addValue(num) + .build()) + .build(); + + return Row.withSchema(ROW_SCHEMA) + .addValue("str_value_" + strNum) + .addValue(num % 2 == 0) + .addValue(Integer.valueOf(strNum)) + .addValue(num) + .addValue(LongStream.range(1, num % 10).boxed().collect(Collectors.toList())) + .addValue(nestedRow) + .addValue(num % 2 == 0 ? null : nestedRow) + .build(); + } + }; + private static HiveMetastoreExtension HIVE_METASTORE_EXTENSION; + + @Rule public TestPipeline writePipeline = TestPipeline.create(); + + @Rule public TestPipeline readPipeline = TestPipeline.create(); + + private static final String TEST_CATALOG = "test_catalog"; + private static final String TEST_TABLE = "test_table"; + private static HiveCatalog catalog; + private static final String TEST_DB = "test_db"; + + @BeforeClass + public static void setUp() throws TException { + String warehousePath = TestPipeline.testingPipelineOptions().getTempLocation(); + HIVE_METASTORE_EXTENSION = new HiveMetastoreExtension(warehousePath); + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), + TEST_CATALOG, + ImmutableMap.of( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, + String.valueOf(TimeUnit.SECONDS.toMillis(10))), + HIVE_METASTORE_EXTENSION.hiveConf()); + + String dbPath = HIVE_METASTORE_EXTENSION.metastore().getDatabasePath(TEST_DB); + Database db = new Database(TEST_DB, "description", dbPath, Maps.newHashMap()); + HIVE_METASTORE_EXTENSION.metastoreClient().createDatabase(db); + } + + @AfterClass + public static void cleanup() throws Exception { + HIVE_METASTORE_EXTENSION.cleanup(); + } + + @Test + public void testWriteReadWithHiveCatalog() { + TableIdentifier tableIdentifier = + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE)); + catalog.createTable(tableIdentifier, IcebergUtils.beamSchemaToIcebergSchema(ROW_SCHEMA)); + + String metastoreUri = + HIVE_METASTORE_EXTENSION.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS); + + Map confProperties = + ImmutableMap.builder() + .put(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUri) + .build(); + + Map transformConfig = + ImmutableMap.builder() + .put("table", tableIdentifier.toString()) + .put("config_properties", confProperties) + .build(); + + List inputRows = + LongStream.range(1, 1000).mapToObj(ROW_FUNC::apply).collect(Collectors.toList()); + + writePipeline + .apply(Create.of(inputRows)) + .setRowSchema(ROW_SCHEMA) + .apply(Managed.write(Managed.ICEBERG).withConfig(transformConfig)); + writePipeline.run().waitUntilFinish(); + + PCollection outputRows = + readPipeline + .apply(Managed.read(Managed.ICEBERG).withConfig(transformConfig)) + .getSinglePCollection(); + PAssert.that(outputRows).containsInAnyOrder(inputRows); + readPipeline.run().waitUntilFinish(); + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java new file mode 100644 index 0000000000000..52de1b91a216a --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/HiveMetastoreExtension.java @@ -0,0 +1,68 @@ +/* + * 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.hive.testutils; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; + +/** + * A class that interacts with {@link TestHiveMetastore}. + * + *

Trimmed down from Iceberg's + * integration testing util + */ +public class HiveMetastoreExtension { + private HiveMetaStoreClient metastoreClient; + private TestHiveMetastore metastore; + + public HiveMetastoreExtension(String warehousePath) throws MetaException { + metastore = new TestHiveMetastore(warehousePath); + HiveConf hiveConf = new HiveConf(TestHiveMetastore.class); + + metastore.start(hiveConf); + metastoreClient = new HiveMetaStoreClient(hiveConf); + } + + public void cleanup() throws Exception { + if (metastoreClient != null) { + metastoreClient.close(); + } + + if (metastore != null) { + metastore.reset(); + metastore.stop(); + } + + metastoreClient = null; + metastore = null; + } + + public HiveMetaStoreClient metastoreClient() { + return metastoreClient; + } + + public HiveConf hiveConf() { + return metastore.hiveConf(); + } + + public TestHiveMetastore metastore() { + return metastore; + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java new file mode 100644 index 0000000000000..6853d4e3230fb --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java @@ -0,0 +1,203 @@ +/* + * 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.hive.testutils; + +import java.io.IOException; +import java.io.LineNumberReader; +import java.io.PrintWriter; +import java.io.Reader; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; + +/** + * Tool to run database scripts. + * + *

Copied over from Iceberg's + * integration testing + */ +@SuppressWarnings({"OperatorPrecedence", "DefaultCharset"}) +public class ScriptRunner { + + private static final String DEFAULT_DELIMITER = ";"; + + private final Connection connection; + + private final boolean stopOnError; + private final boolean autoCommit; + + private final PrintWriter logWriter = new PrintWriter(System.out); + private final PrintWriter errorLogWriter = new PrintWriter(System.err); + + /** Default constructor */ + public ScriptRunner(Connection connection, boolean autoCommit, boolean stopOnError) { + this.connection = connection; + this.autoCommit = autoCommit; + this.stopOnError = stopOnError; + } + + /** + * Runs an SQL script (read in using the Reader parameter) + * + * @param reader - the source of the script + */ + public void runScript(Reader reader) throws IOException, SQLException { + try { + boolean originalAutoCommit = connection.getAutoCommit(); + try { + if (originalAutoCommit != this.autoCommit) { + connection.setAutoCommit(this.autoCommit); + } + runScript(connection, reader); + } finally { + connection.setAutoCommit(originalAutoCommit); + } + } catch (IOException | SQLException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException("Error running script. Cause: " + e, e); + } + } + + /** + * Runs an SQL script (read in using the Reader parameter) using the connection passed in + * + * @param conn - the connection to use for the script + * @param reader - the source of the script + * @throws SQLException if any SQL errors occur + * @throws IOException if there is an error reading from the Reader + */ + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private void runScript(Connection conn, Reader reader) throws IOException, SQLException { + StringBuilder command = null; + try { + LineNumberReader lineReader = new LineNumberReader(reader); + String line; + while ((line = lineReader.readLine()) != null) { + if (command == null) { + command = new StringBuilder(); + } + String trimmedLine = line.trim(); + boolean fullLineDelimiter = false; + if (trimmedLine.startsWith("--")) { + println(trimmedLine); + } else if (trimmedLine.isEmpty() || trimmedLine.startsWith("//")) { + // Do nothing + } else if (!fullLineDelimiter && trimmedLine.endsWith(getDelimiter()) + || fullLineDelimiter && trimmedLine.equals(getDelimiter())) { + command.append(line, 0, line.lastIndexOf(getDelimiter())); + command.append(" "); + Statement statement = conn.createStatement(); + + println(command); + + boolean hasResults = false; + if (stopOnError) { + hasResults = statement.execute(command.toString()); + } else { + try { + statement.execute(command.toString()); + } catch (SQLException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + } + } + + if (autoCommit && !conn.getAutoCommit()) { + conn.commit(); + } + + ResultSet rs = statement.getResultSet(); + if (hasResults && rs != null) { + ResultSetMetaData md = rs.getMetaData(); + int cols = md.getColumnCount(); + for (int i = 0; i < cols; i++) { + String name = md.getColumnLabel(i); + print(name + "\t"); + } + println(""); + while (rs.next()) { + for (int i = 0; i < cols; i++) { + String value = rs.getString(i); + print(value + "\t"); + } + println(""); + } + } + + command = null; + try { + statement.close(); + } catch (Exception e) { + // Ignore to workaround a bug in Jakarta DBCP + } + Thread.yield(); + } else { + command.append(line); + command.append(" "); + } + } + if (!autoCommit) { + conn.commit(); + } + } catch (IOException | SQLException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + throw e; + } finally { + conn.rollback(); + flush(); + } + } + + private String getDelimiter() { + return DEFAULT_DELIMITER; + } + + private void print(Object obj) { + if (logWriter != null) { + System.out.print(obj); + } + } + + private void println(Object obj) { + if (logWriter != null) { + logWriter.println(obj); + } + } + + private void printlnError(Object obj) { + if (errorLogWriter != null) { + errorLogWriter.println(obj); + } + } + + private void flush() { + if (logWriter != null) { + logWriter.flush(); + } + if (errorLogWriter != null) { + errorLogWriter.flush(); + } + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java new file mode 100644 index 0000000000000..e3af43d58c65f --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/TestHiveMetastore.java @@ -0,0 +1,273 @@ +/* + * 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.hive.testutils; + +import static java.nio.file.Files.createTempDirectory; +import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; +import static java.nio.file.attribute.PosixFilePermissions.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +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 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.HiveMetaStore; +import org.apache.hadoop.hive.metastore.IHMSHandler; +import org.apache.hadoop.hive.metastore.RetryingHMSHandler; +import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; +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.protocol.TBinaryProtocol; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransportFactory; + +/** + * A Hive Metastore implementation for local testing. Not meant to be used directly. Use {@link + * HiveMetastoreExtension} instead. + * + *

Copied over from Iceberg's + * integration testing util + */ +public class TestHiveMetastore { + + private static final String DEFAULT_DATABASE_NAME = "default"; + private static final int DEFAULT_POOL_SIZE = 5; + + // 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 = + DynConstructors.builder() + .impl(HiveMetaStore.HMSHandler.class, String.class, Configuration.class) + .impl(HiveMetaStore.HMSHandler.class, String.class, HiveConf.class) + .build(); + + 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) + .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") + .orNoop() + .buildStatic(); + + // 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 String DERBY_PATH; + + static { + try { + HIVE_LOCAL_DIR = + createTempDirectory("hive", asFileAttribute(fromString("rwxrwxrwx"))).toFile(); + DERBY_PATH = HIVE_LOCAL_DIR + "/metastore_db"; + File derbyLogFile = new File(HIVE_LOCAL_DIR, "derby.log"); + System.setProperty("derby.stream.error.file", derbyLogFile.getAbsolutePath()); + setupMetastoreDB("jdbc:derby:" + DERBY_PATH + ";create=true"); + Runtime.getRuntime() + .addShutdownHook( + new Thread( + () -> { + Path localDirPath = new Path(HIVE_LOCAL_DIR.getAbsolutePath()); + FileSystem fs = Util.getFs(localDirPath, new Configuration()); + String errMsg = "Failed to delete " + localDirPath; + try { + assertThat(fs.delete(localDirPath, true)).as(errMsg).isTrue(); + } catch (IOException e) { + throw new RuntimeException(errMsg, e); + } + })); + } catch (Exception e) { + throw new RuntimeException("Failed to setup local dir for hive metastore", e); + } + } + + private HiveConf hiveConf; + private ExecutorService executorService; + private TServer server; + private HiveMetaStore.HMSHandler baseHandler; + private HiveClientPool clientPool; + private final String hiveWarehousePath; + + TestHiveMetastore(String hiveWarehousePath) { + this.hiveWarehousePath = hiveWarehousePath; + } + + /** + * Starts a TestHiveMetastore with the default connection pool size (5) with the provided + * HiveConf. + * + * @param conf The hive configuration to use + */ + public void start(HiveConf conf) { + start(conf, DEFAULT_POOL_SIZE); + } + + /** + * Starts a TestHiveMetastore with a provided connection pool size and HiveConf. + * + * @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); + int port = socket.getServerSocket().getLocalPort(); + initConf(conf, port); + + this.hiveConf = conf; + this.server = newThriftServer(socket, poolSize, hiveConf); + this.executorService = Executors.newSingleThreadExecutor(); + this.executorService.submit(() -> server.serve()); + this.clientPool = new HiveClientPool(1, hiveConf); + } catch (Exception e) { + throw new RuntimeException("Cannot start TestHiveMetastore", e); + } + } + + public void stop() throws Exception { + reset(); + if (clientPool != null) { + clientPool.close(); + } + if (server != null) { + server.stop(); + } + if (executorService != null) { + executorService.shutdown(); + } + if (baseHandler != null) { + baseHandler.shutdown(); + } + METASTORE_THREADS_SHUTDOWN.invoke(); + } + + public HiveConf hiveConf() { + return hiveConf; + } + + public String getDatabasePath(String dbName) { + return hiveWarehousePath + "/" + dbName + ".db"; + } + + public void reset() throws Exception { + if (clientPool != null) { + for (String dbName : clientPool.run(client -> client.getAllDatabases())) { + for (String tblName : clientPool.run(client -> client.getAllTables(dbName))) { + clientPool.run( + client -> { + client.dropTable(dbName, tblName, true, true, true); + return null; + }); + } + + if (!DEFAULT_DATABASE_NAME.equals(dbName)) { + // Drop cascade, functions dropped by cascade + clientPool.run( + client -> { + client.dropDatabase(dbName, true, true, true); + return null; + }); + } + } + } + + Path warehouseRoot = new Path(hiveWarehousePath); + FileSystem fs = Util.getFs(warehouseRoot, hiveConf); + for (FileStatus fileStatus : fs.listStatus(warehouseRoot)) { + if (!fileStatus.getPath().getName().equals("derby.log") + && !fileStatus.getPath().getName().equals("metastore_db")) { + fs.delete(fileStatus.getPath(), true); + } + } + } + + private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf conf) + throws Exception { + HiveConf serverConf = new HiveConf(conf); + serverConf.set( + HiveConf.ConfVars.METASTORECONNECTURLKEY.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); + + TThreadPoolServer.Args args = + new TThreadPoolServer.Args(socket) + .processor(new TSetIpAddressProcessor<>(handler)) + .transportFactory(new TTransportFactory()) + .protocolFactory(new TBinaryProtocol.Factory()) + .minWorkerThreads(poolSize) + .maxWorkerThreads(poolSize); + + return new TThreadPoolServer(args); + } + + 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_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()); + } + + private static void setupMetastoreDB(String dbURL) throws SQLException, IOException { + 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"); + try (Reader reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8)) { + scriptRunner.runScript(reader); + } + } +} diff --git a/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql new file mode 100644 index 0000000000000..b1a4c158ebb9f --- /dev/null +++ b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql @@ -0,0 +1,726 @@ +-- +-- 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"."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"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT); + +CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +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); + +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" VARCHAR(4000)); + +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'); + +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 +); + +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 +); + +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 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)); + +CREATE TABLE "APP"."WM_RESOURCEPLAN" (RP_ID BIGINT NOT NULL, 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 +); + +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"."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 UNIQUE INDEX "APP"."UNIQUEINDEX" ON "APP"."IDXS" ("INDEX_NAME", "ORIG_TBL_ID"); + +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 INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME"); + +CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_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" ("PART_NAME", "TBL_ID"); + +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" ("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"."IDXS" ADD CONSTRAINT "IDXS_PK" PRIMARY KEY ("INDEX_ID"); + +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"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_PK" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); + +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"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("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"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK3" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +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"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_FK1" FOREIGN KEY ("INDEX_ID") REFERENCES "APP"."IDXS" ("INDEX_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_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"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REBUILD IN ('Y','N')); + +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, + 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 +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + 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 NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID 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(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 COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) 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_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) +); + +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(128) 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_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) +); + +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(128) 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_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); + +-- ----------------------------------------------------------------- +-- 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/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 2956d75a266e7..5307047354b8b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -19,19 +19,27 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; -import java.util.Properties; +import java.util.Map; +import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; +import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @AutoValue public abstract class IcebergCatalogConfig implements Serializable { @Pure + @Nullable public abstract String getCatalogName(); @Pure - public abstract Properties getProperties(); + @Nullable + public abstract Map getCatalogProperties(); + + @Pure + @Nullable + public abstract Map getConfigProperties(); @Pure public static Builder builder() { @@ -39,15 +47,32 @@ public static Builder builder() { } public org.apache.iceberg.catalog.Catalog catalog() { - return CatalogUtil.buildIcebergCatalog( - getCatalogName(), Maps.fromProperties(getProperties()), new Configuration()); + String catalogName = getCatalogName(); + if (catalogName == null) { + catalogName = "apache-beam-" + ReleaseInfo.getReleaseInfo().getVersion(); + } + Map catalogProps = getCatalogProperties(); + if (catalogProps == null) { + catalogProps = Maps.newHashMap(); + } + Map confProps = getConfigProperties(); + if (confProps == null) { + confProps = Maps.newHashMap(); + } + Configuration config = new Configuration(); + for (Map.Entry prop : confProps.entrySet()) { + config.set(prop.getKey(), prop.getValue()); + } + return CatalogUtil.buildIcebergCatalog(catalogName, catalogProps, config); } @AutoValue.Builder public abstract static class Builder { - public abstract Builder setCatalogName(String catalogName); + public abstract Builder setCatalogName(@Nullable String catalogName); + + public abstract Builder setCatalogProperties(@Nullable Map props); - public abstract Builder setProperties(Properties props); + public abstract Builder setConfigProperties(@Nullable Map props); public abstract IcebergCatalogConfig build(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index ef535353efd01..df7bda4560dd5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -18,18 +18,11 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.service.AutoService; -import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Config; import org.apache.beam.sdk.managed.ManagedTransformConstants; -import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -44,11 +37,12 @@ * org.apache.beam.sdk.values.Row}s. */ @AutoService(SchemaTransformProvider.class) -public class IcebergReadSchemaTransformProvider extends TypedSchemaTransformProvider { +public class IcebergReadSchemaTransformProvider + extends TypedSchemaTransformProvider { static final String OUTPUT_TAG = "output"; @Override - protected SchemaTransform from(Config configuration) { + protected SchemaTransform from(SchemaTransformConfiguration configuration) { return new IcebergReadSchemaTransform(configuration); } @@ -62,38 +56,10 @@ public String identifier() { return ManagedTransformConstants.ICEBERG_READ; } - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class Config { - public static Builder builder() { - return new AutoValue_IcebergReadSchemaTransformProvider_Config.Builder(); - } - - @SchemaFieldDescription("Identifier of the Iceberg table to write to.") - public abstract String getTable(); - - @SchemaFieldDescription("Name of the catalog containing the table.") - public abstract String getCatalogName(); - - @SchemaFieldDescription("Configuration properties used to set up the Iceberg catalog.") - public abstract Map getCatalogProperties(); - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); - - public abstract Builder setCatalogName(String catalogName); - - public abstract Builder setCatalogProperties(Map catalogProperties); - - public abstract Config build(); - } - } - static class IcebergReadSchemaTransform extends SchemaTransform { - private final Config configuration; + private final SchemaTransformConfiguration configuration; - IcebergReadSchemaTransform(Config configuration) { + IcebergReadSchemaTransform(SchemaTransformConfiguration configuration) { this.configuration = configuration; } @@ -102,7 +68,7 @@ Row getConfigurationRow() { // To stay consistent with our SchemaTransform configuration naming conventions, // we sort lexicographically and convert field names to snake_case return SchemaRegistry.createDefault() - .getToRowFunction(Config.class) + .getToRowFunction(SchemaTransformConfiguration.class) .apply(configuration) .sorted() .toSnakeCase(); @@ -113,19 +79,11 @@ Row getConfigurationRow() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - Properties properties = new Properties(); - properties.putAll(configuration.getCatalogProperties()); - - IcebergCatalogConfig.Builder catalogBuilder = - IcebergCatalogConfig.builder() - .setCatalogName(configuration.getCatalogName()) - .setProperties(properties); - PCollection output = input .getPipeline() .apply( - IcebergIO.readRows(catalogBuilder.build()) + IcebergIO.readRows(configuration.getIcebergCatalog()) .from(TableIdentifier.parse(configuration.getTable()))); return PCollectionRowTuple.of(OUTPUT_TAG, output); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index a2f84e6475c9c..7abbe1a367cc4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -34,6 +34,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -319,63 +320,59 @@ private static void copyFieldIntoRecord(Record rec, Types.NestedField field, Row } } + private static @Nullable Object icebergValueToBeamValue(Schema.Field field, Record record) { + boolean isNullable = field.getType().getNullable(); + @Nullable Object icebergValue = record.getField(field.getName()); + if (icebergValue == null) { + if (isNullable) { + return null; + } + throw new RuntimeException( + String.format("Received null value for required field '%s'.", field.getName())); + } + + switch (field.getType().getTypeName()) { + case BYTE: + case INT16: + case INT32: + case INT64: + case DECIMAL: // Iceberg and Beam both use BigDecimal + case FLOAT: // Iceberg and Beam both use float + case DOUBLE: // Iceberg and Beam both use double + case STRING: // Iceberg and Beam both use String + case BOOLEAN: // Iceberg and Beam both use String + case ARRAY: + case ITERABLE: + case MAP: + return icebergValue; + case DATETIME: + // Iceberg uses a long for millis; Beam uses joda time DateTime + long millis = (long) icebergValue; + return new DateTime(millis, DateTimeZone.UTC); + case BYTES: + // Iceberg uses ByteBuffer; Beam uses byte[] + return ((ByteBuffer) icebergValue).array(); + case ROW: + Record nestedRecord = (Record) icebergValue; + Schema nestedSchema = + checkArgumentNotNull( + field.getType().getRowSchema(), + "Corrupted schema: Row type did not have associated nested schema."); + return icebergRecordToBeamRow(nestedSchema, nestedRecord); + case LOGICAL_TYPE: + throw new UnsupportedOperationException( + "Cannot convert iceberg field to Beam logical type"); + default: + throw new UnsupportedOperationException( + "Unsupported Beam type: " + field.getType().getTypeName()); + } + } + /** Converts an Iceberg {@link Record} to a Beam {@link Row}. */ public static Row icebergRecordToBeamRow(Schema schema, Record record) { Row.Builder rowBuilder = Row.withSchema(schema); for (Schema.Field field : schema.getFields()) { - switch (field.getType().getTypeName()) { - case BYTE: - // I guess allow anything we can cast here - byte byteValue = (byte) record.getField(field.getName()); - rowBuilder.addValue(byteValue); - break; - case INT16: - // I guess allow anything we can cast here - short shortValue = (short) record.getField(field.getName()); - rowBuilder.addValue(shortValue); - break; - case INT32: - // I guess allow anything we can cast here - int intValue = (int) record.getField(field.getName()); - rowBuilder.addValue(intValue); - break; - case INT64: - // I guess allow anything we can cast here - long longValue = (long) record.getField(field.getName()); - rowBuilder.addValue(longValue); - break; - case DECIMAL: // Iceberg and Beam both use BigDecimal - case FLOAT: // Iceberg and Beam both use float - case DOUBLE: // Iceberg and Beam both use double - case STRING: // Iceberg and Beam both use String - case BOOLEAN: // Iceberg and Beam both use String - case ARRAY: - case ITERABLE: - case MAP: - rowBuilder.addValue(record.getField(field.getName())); - break; - case DATETIME: - // Iceberg uses a long for millis; Beam uses joda time DateTime - long millis = (long) record.getField(field.getName()); - rowBuilder.addValue(new DateTime(millis, DateTimeZone.UTC)); - break; - case BYTES: - // Iceberg uses ByteBuffer; Beam uses byte[] - rowBuilder.addValue(((ByteBuffer) record.getField(field.getName())).array()); - break; - case ROW: - Record nestedRecord = (Record) record.getField(field.getName()); - Schema nestedSchema = - checkArgumentNotNull( - field.getType().getRowSchema(), - "Corrupted schema: Row type did not have associated nested schema."); - Row nestedRow = icebergRecordToBeamRow(nestedSchema, nestedRecord); - rowBuilder.addValue(nestedRow); - break; - case LOGICAL_TYPE: - throw new UnsupportedOperationException( - "Cannot convert iceberg field to Beam logical type"); - } + rowBuilder.addValue(icebergValueToBeamValue(field, record)); } return rowBuilder.build(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index b3de7a88c541d..3f0f88946d9ca 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -18,19 +18,12 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.service.AutoService; -import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import org.apache.beam.sdk.managed.ManagedTransformConstants; -import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -48,7 +41,8 @@ * outputs a {@code PCollection} representing snapshots created in the process. */ @AutoService(SchemaTransformProvider.class) -public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformProvider { +public class IcebergWriteSchemaTransformProvider + extends TypedSchemaTransformProvider { static final String INPUT_TAG = "input"; static final String OUTPUT_TAG = "output"; @@ -64,7 +58,7 @@ public String description() { } @Override - protected SchemaTransform from(Config configuration) { + protected SchemaTransform from(SchemaTransformConfiguration configuration) { return new IcebergWriteSchemaTransform(configuration); } @@ -83,38 +77,10 @@ public String identifier() { return ManagedTransformConstants.ICEBERG_WRITE; } - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class Config { - public static Builder builder() { - return new AutoValue_IcebergWriteSchemaTransformProvider_Config.Builder(); - } - - @SchemaFieldDescription("Identifier of the Iceberg table to write to.") - public abstract String getTable(); - - @SchemaFieldDescription("Name of the catalog containing the table.") - public abstract String getCatalogName(); - - @SchemaFieldDescription("Configuration properties used to set up the Iceberg catalog.") - public abstract Map getCatalogProperties(); - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); - - public abstract Builder setCatalogName(String catalogName); - - public abstract Builder setCatalogProperties(Map catalogProperties); - - public abstract Config build(); - } - } - static class IcebergWriteSchemaTransform extends SchemaTransform { - private final Config configuration; + private final SchemaTransformConfiguration configuration; - IcebergWriteSchemaTransform(Config configuration) { + IcebergWriteSchemaTransform(SchemaTransformConfiguration configuration) { this.configuration = configuration; } @@ -123,7 +89,7 @@ Row getConfigurationRow() { // To stay consistent with our SchemaTransform configuration naming conventions, // we sort lexicographically and convert field names to snake_case return SchemaRegistry.createDefault() - .getToRowFunction(Config.class) + .getToRowFunction(SchemaTransformConfiguration.class) .apply(configuration) .sorted() .toSnakeCase(); @@ -136,19 +102,11 @@ Row getConfigurationRow() { public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection rows = input.get(INPUT_TAG); - Properties properties = new Properties(); - properties.putAll(configuration.getCatalogProperties()); - - IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder() - .setCatalogName(configuration.getCatalogName()) - .setProperties(properties) - .build(); - // TODO: support dynamic destinations IcebergWriteResult result = rows.apply( - IcebergIO.writeRows(catalog).to(TableIdentifier.parse(configuration.getTable()))); + IcebergIO.writeRows(configuration.getIcebergCatalog()) + .to(TableIdentifier.parse(configuration.getTable()))); PCollection snapshots = result diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java index d7212783d1b6c..66b441b4e641a 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamRowToIcebergRecord; import java.io.IOException; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; @@ -34,6 +35,7 @@ import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; +import org.checkerframework.checker.nullness.qual.Nullable; class RecordWriter { @@ -42,23 +44,30 @@ class RecordWriter { private final Table table; private final String absoluteFilename; - RecordWriter(Catalog catalog, IcebergDestination destination, String filename) + RecordWriter( + Catalog catalog, IcebergDestination destination, String filename, @Nullable Schema dataSchema) throws IOException { this( - catalog.loadTable(destination.getTableIdentifier()), destination.getFileFormat(), filename); + catalog.loadTable(destination.getTableIdentifier()), + destination.getFileFormat(), + filename, + dataSchema); } - RecordWriter(Table table, FileFormat fileFormat, String filename) throws IOException { + RecordWriter(Table table, FileFormat fileFormat, String filename, @Nullable Schema dataSchema) + throws IOException { this.table = table; this.absoluteFilename = table.location() + "/" + filename; OutputFile outputFile = table.io().newOutputFile(absoluteFilename); + org.apache.iceberg.Schema writerSchema = + dataSchema == null ? table.schema() : IcebergUtils.beamSchemaToIcebergSchema(dataSchema); switch (fileFormat) { case AVRO: icebergDataWriter = Avro.writeData(outputFile) .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) - .schema(table.schema()) + .schema(writerSchema) .withSpec(table.spec()) .overwrite() .build(); @@ -67,7 +76,7 @@ class RecordWriter { icebergDataWriter = Parquet.writeData(outputFile) .createWriterFunc(GenericParquetWriter::buildWriter) - .schema(table.schema()) + .schema(writerSchema) .withSpec(table.spec()) .overwrite() .build(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java new file mode 100644 index 0000000000000..6e7a12aa15afc --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java @@ -0,0 +1,69 @@ +/* + * 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; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.checkerframework.checker.nullness.qual.Nullable; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SchemaTransformConfiguration { + public static Builder builder() { + return new AutoValue_SchemaTransformConfiguration.Builder(); + } + + @SchemaFieldDescription("Identifier of the Iceberg table.") + public abstract String getTable(); + + @SchemaFieldDescription("Name of the catalog containing the table.") + @Nullable + public abstract String getCatalogName(); + + @SchemaFieldDescription("Properties used to set up the Iceberg catalog.") + @Nullable + public abstract Map getCatalogProperties(); + + @SchemaFieldDescription("Properties passed to the Hadoop Configuration.") + @Nullable + public abstract Map getConfigProperties(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String table); + + public abstract Builder setCatalogName(String catalogName); + + public abstract Builder setCatalogProperties(Map catalogProperties); + + public abstract Builder setConfigProperties(Map confProperties); + + public abstract SchemaTransformConfiguration build(); + } + + public IcebergCatalogConfig getIcebergCatalog() { + return IcebergCatalogConfig.builder() + .setCatalogName(getCatalogName()) + .setCatalogProperties(getCatalogProperties()) + .setConfigProperties(getConfigProperties()) + .build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index c11263519442e..d2d861bb6cbdb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -26,8 +27,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.ShardedKey; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.iceberg.catalog.Catalog; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; class WriteGroupedRowsToFiles extends PTransform< @@ -75,17 +78,20 @@ private org.apache.iceberg.catalog.Catalog getCatalog() { return catalog; } - private RecordWriter createWriter(IcebergDestination destination) throws IOException { - return new RecordWriter(getCatalog(), destination, "-" + UUID.randomUUID()); + private RecordWriter createWriter(IcebergDestination destination, @Nullable Schema dataSchema) + throws IOException { + return new RecordWriter(getCatalog(), destination, "-" + UUID.randomUUID(), dataSchema); } @ProcessElement public void processElement( ProcessContext c, @Element KV, Iterable> element) throws Exception { + @Nullable Row peekFirstRow = Iterables.getFirst(element.getValue(), null); + @Nullable Schema dataSchema = peekFirstRow != null ? peekFirstRow.getSchema() : null; Row destMetadata = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); - RecordWriter writer = createWriter(destination); + RecordWriter writer = createWriter(destination, dataSchema); for (Row e : element.getValue()) { writer.write(e); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index a00f3de4bb4e1..110170d38c9ea 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -213,10 +214,12 @@ private org.apache.iceberg.catalog.Catalog getCatalog() { return catalog; } - private RecordWriter createAndInsertWriter(IcebergDestination destination, BoundedWindow window) + private RecordWriter createAndInsertWriter( + IcebergDestination destination, BoundedWindow window, Schema dataSchema) throws IOException { RecordWriter writer = - new RecordWriter(getCatalog(), destination, filename + "-" + UUID.randomUUID()); + new RecordWriter( + getCatalog(), destination, filename + "-" + UUID.randomUUID(), dataSchema); getWindows().put(destination, window); getWriters().put(destination, writer); return writer; @@ -227,7 +230,8 @@ private RecordWriter createAndInsertWriter(IcebergDestination destination, Bound * reached the maximum number of writers and should spill any records associated. */ @Nullable - RecordWriter getWriterIfPossible(IcebergDestination destination, BoundedWindow window) + RecordWriter getWriterIfPossible( + IcebergDestination destination, BoundedWindow window, Schema dataSchema) throws IOException { RecordWriter existingWriter = getWriters().get(destination); @@ -239,7 +243,7 @@ RecordWriter getWriterIfPossible(IcebergDestination destination, BoundedWindow w return null; } - return createAndInsertWriter(destination, window); + return createAndInsertWriter(destination, window, dataSchema); } @StartBundle @@ -255,7 +259,7 @@ public void processElement(@Element Row element, BoundedWindow window, MultiOutp IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); // Spill record if writer cannot be created - RecordWriter writer = getWriterIfPossible(destination, window); + RecordWriter writer = getWriterIfPossible(destination, window, data.getSchema()); if (writer == null) { out.get(SPILLED_ROWS_TAG).output(element); return; @@ -270,7 +274,7 @@ public void processElement(@Element Row element, BoundedWindow window, MultiOutp .setManifestFile(writer.getManifestFile()) .setTableIdentifier(destination.getTableIdentifier()) .build()); - writer = createAndInsertWriter(destination, window); + writer = createAndInsertWriter(destination, window, data.getSchema()); } // Actually write the data diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java index 3f31073b44486..fe4a07dedfdf9 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java @@ -21,7 +21,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import java.util.List; -import java.util.Properties; +import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -94,12 +95,17 @@ public void testSimpleScan() throws Exception { .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .collect(Collectors.toList()); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); PCollection output = testPipeline diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java index 02213c45e0756..2abe6b0934819 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java @@ -23,7 +23,6 @@ import java.io.Serializable; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.UUID; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; @@ -76,12 +75,17 @@ public void testSimpleAppend() throws Exception { // Create a table and add records to it. Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); testPipeline .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) @@ -110,12 +114,17 @@ public void testDynamicDestinationsWithoutSpillover() throws Exception { Table table2 = warehouse.createTable(table2Id, TestFixtures.SCHEMA); Table table3 = warehouse.createTable(table3Id, TestFixtures.SCHEMA); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); DynamicDestinations dynamicDestinations = new DynamicDestinations() { @@ -200,12 +209,17 @@ public void testDynamicDestinationsWithSpillover() throws Exception { elementsPerTable.computeIfAbsent(tableId, ignored -> Lists.newArrayList()).add(element); } - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); IcebergCatalogConfig catalog = - IcebergCatalogConfig.builder().setCatalogName("name").setProperties(props).build(); + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); DynamicDestinations dynamicDestinations = new DynamicDestinations() { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java index effb5cc4838e9..0311c31da4058 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -101,8 +101,8 @@ public void testSimpleScan() throws Exception { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - IcebergReadSchemaTransformProvider.Config readConfig = - IcebergReadSchemaTransformProvider.Config.builder() + SchemaTransformConfiguration readConfig = + SchemaTransformConfiguration.builder() .setTable(identifier) .setCatalogName("name") .setCatalogProperties(properties) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index a2cd64e239567..6b555e7e14d0f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; import static org.hamcrest.MatcherAssert.assertThat; @@ -89,8 +88,8 @@ public void testSimpleAppend() { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - Config config = - Config.builder() + SchemaTransformConfiguration config = + SchemaTransformConfiguration.builder() .setTable(identifier) .setCatalogName("name") .setCatalogProperties(properties) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java index 007cb028c665d..38a15cb2aa98f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java @@ -20,13 +20,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import java.util.List; -import java.util.Properties; +import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -65,9 +66,11 @@ public void testUnstartedReaderReadsSamesItsSource() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -75,7 +78,7 @@ public void testUnstartedReaderReadsSamesItsSource() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) @@ -107,9 +110,11 @@ public void testInitialSplitting() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -117,7 +122,7 @@ public void testInitialSplitting() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) @@ -153,9 +158,11 @@ public void testDoubleInitialSplitting() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); - Properties props = new Properties(); - props.setProperty("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); - props.setProperty("warehouse", warehouse.location); + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); BoundedSource source = new ScanSource( @@ -163,7 +170,7 @@ public void testDoubleInitialSplitting() throws Exception { .setCatalogConfig( IcebergCatalogConfig.builder() .setCatalogName("name") - .setProperties(props) + .setCatalogProperties(catalogProps) .build()) .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(simpleTable.name().replace("hadoop.", "").split("\\.")) diff --git a/settings.gradle.kts b/settings.gradle.kts index 4d4b93908a02f..65a55885afa72 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -365,3 +365,7 @@ include("sdks:java:io:solace") findProject(":sdks:java:io:solace")?.name = "solace" include("sdks:java:extensions:combiners") findProject(":sdks:java:extensions:combiners")?.name = "combiners" +include("sdks:java:io:iceberg:hive") +findProject(":sdks:java:io:iceberg:hive")?.name = "hive" +include("sdks:java:io:iceberg:hive:exec") +findProject(":sdks:java:io:iceberg:hive:exec")?.name = "exec" From 5f731410a4f4c48ee888eade43bb65b6b1993077 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 Aug 2024 17:05:39 -0400 Subject: [PATCH 2/8] split read and write tests; cleanup --- sdks/java/io/iceberg/hive/build.gradle | 13 +- .../io/iceberg/hive/IcebergHiveCatalogIT.java | 158 ++++++++++++++---- .../iceberg/hive/testutils/ScriptRunner.java | 6 +- 3 files changed, 140 insertions(+), 37 deletions(-) diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index cca3726ec388d..b81867ec90ca3 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -28,10 +28,11 @@ description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" ext.summary = "Runtime dependencies needed for Hive catalog integration." def hive_version = "3.1.3" +def iceberg_version = "1.4.2" dependencies { // dependencies needed to run with iceberg's hive catalog - runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") + runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") runtimeOnly library.java.bigdataoss_gcs_connector runtimeOnly library.java.hadoop_client @@ -45,18 +46,20 @@ dependencies { testRuntimeOnly library.java.snake_yaml // needed to set up the test environment - testImplementation "org.apache.iceberg:iceberg-common:1.4.2" - testImplementation "org.apache.iceberg:iceberg-core:1.4.2" + testImplementation "org.apache.iceberg:iceberg-common:$iceberg_version" + testImplementation "org.apache.iceberg:iceberg-core:$iceberg_version" testImplementation "org.assertj:assertj-core:3.11.1" testImplementation library.java.junit - // needed to set up test Hive Metastore - testImplementation ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") + // needed to set up test Hive Metastore and run tests + testImplementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") testImplementation project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") testRuntimeOnly ("org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version") { exclude group: "org.apache.hive", module: "hive-exec" exclude group: "org.apache.parquet", module: "parquet-hadoop-bundle" } + testImplementation "org.apache.iceberg:iceberg-parquet:$iceberg_version" + testImplementation "org.apache.parquet:parquet-column:1.12.0" } task integrationTest(type: Test) { diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java index 7933368f771f8..e7baae236bba4 100644 --- a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java @@ -17,8 +17,14 @@ */ package org.apache.beam.sdk.io.iceberg.hive; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -36,10 +42,28 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.AppendFiles; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.encryption.InputFilesDecryptor; import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; import org.apache.thrift.TException; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -60,7 +84,7 @@ public class IcebergHiveCatalogIT { .addFloatField("nested_float") .addRowField("nested_row", DOUBLY_NESTED_ROW_SCHEMA) .build(); - private static final Schema ROW_SCHEMA = + private static final Schema BEAM_SCHEMA = Schema.builder() .addStringField("str") .addBooleanField("bool") @@ -88,7 +112,7 @@ public Row apply(Long num) { .build()) .build(); - return Row.withSchema(ROW_SCHEMA) + return Row.withSchema(BEAM_SCHEMA) .addValue("str_value_" + strNum) .addValue(num % 2 == 0) .addValue(Integer.valueOf(strNum)) @@ -99,7 +123,18 @@ public Row apply(Long num) { .build(); } }; - private static HiveMetastoreExtension HIVE_METASTORE_EXTENSION; + + private static final org.apache.iceberg.Schema ICEBERG_SCHEMA = + IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA); + private static final SimpleFunction RECORD_FUNC = + new SimpleFunction() { + @Override + public Record apply(Row input) { + return IcebergUtils.beamRowToIcebergRecord(ICEBERG_SCHEMA, input); + } + }; + + private static HiveMetastoreExtension hiveMetastoreExtension; @Rule public TestPipeline writePipeline = TestPipeline.create(); @@ -108,12 +143,12 @@ public Row apply(Long num) { private static final String TEST_CATALOG = "test_catalog"; private static final String TEST_TABLE = "test_table"; private static HiveCatalog catalog; - private static final String TEST_DB = "test_db"; + private static final String TEST_DB = "test_db_" + System.nanoTime(); @BeforeClass public static void setUp() throws TException { String warehousePath = TestPipeline.testingPipelineOptions().getTempLocation(); - HIVE_METASTORE_EXTENSION = new HiveMetastoreExtension(warehousePath); + hiveMetastoreExtension = new HiveMetastoreExtension(warehousePath); catalog = (HiveCatalog) CatalogUtil.loadCatalog( @@ -122,52 +157,117 @@ public static void setUp() throws TException { ImmutableMap.of( CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, String.valueOf(TimeUnit.SECONDS.toMillis(10))), - HIVE_METASTORE_EXTENSION.hiveConf()); + hiveMetastoreExtension.hiveConf()); - String dbPath = HIVE_METASTORE_EXTENSION.metastore().getDatabasePath(TEST_DB); + String dbPath = hiveMetastoreExtension.metastore().getDatabasePath(TEST_DB); Database db = new Database(TEST_DB, "description", dbPath, Maps.newHashMap()); - HIVE_METASTORE_EXTENSION.metastoreClient().createDatabase(db); + hiveMetastoreExtension.metastoreClient().createDatabase(db); } @AfterClass public static void cleanup() throws Exception { - HIVE_METASTORE_EXTENSION.cleanup(); + hiveMetastoreExtension.cleanup(); } - @Test - public void testWriteReadWithHiveCatalog() { - TableIdentifier tableIdentifier = - TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE)); - catalog.createTable(tableIdentifier, IcebergUtils.beamSchemaToIcebergSchema(ROW_SCHEMA)); - - String metastoreUri = - HIVE_METASTORE_EXTENSION.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS); + private Map getManagedIcebergConfig(TableIdentifier table) { + String metastoreUri = hiveMetastoreExtension.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS); Map confProperties = ImmutableMap.builder() .put(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUri) .build(); - Map transformConfig = - ImmutableMap.builder() - .put("table", tableIdentifier.toString()) - .put("config_properties", confProperties) + return ImmutableMap.builder() + .put("table", table.toString()) + .put("config_properties", confProperties) + .build(); + } + + @Test + public void testReadWithHiveCatalog() throws IOException { + TableIdentifier tableIdentifier = + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE + "_read_test")); + Table table = catalog.createTable(tableIdentifier, ICEBERG_SCHEMA); + + List expectedRows = + LongStream.range(1, 1000).boxed().map(ROW_FUNC::apply).collect(Collectors.toList()); + List records = + expectedRows.stream().map(RECORD_FUNC::apply).collect(Collectors.toList()); + + // write iceberg records with hive catalog + String filepath = table.location() + "/" + UUID.randomUUID(); + DataWriter writer = + Parquet.writeData(table.io().newOutputFile(filepath)) + .schema(ICEBERG_SCHEMA) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .withSpec(table.spec()) .build(); + for (Record rec : records) { + writer.write(rec); + } + writer.close(); + AppendFiles appendFiles = table.newAppend(); + String manifestFilename = FileFormat.AVRO.addExtension(filepath + ".manifest"); + OutputFile outputFile = table.io().newOutputFile(manifestFilename); + ManifestWriter manifestWriter; + try (ManifestWriter openWriter = ManifestFiles.write(table.spec(), outputFile)) { + openWriter.add(writer.toDataFile()); + manifestWriter = openWriter; + } + appendFiles.appendManifest(manifestWriter.toManifestFile()); + appendFiles.commit(); + + // Run Managed Iceberg read + PCollection outputRows = + readPipeline + .apply( + Managed.read(Managed.ICEBERG).withConfig(getManagedIcebergConfig(tableIdentifier))) + .getSinglePCollection(); + PAssert.that(outputRows).containsInAnyOrder(expectedRows); + readPipeline.run().waitUntilFinish(); + } + + @Test + public void testWriteWithHiveCatalog() { + TableIdentifier tableIdentifier = + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE)); + catalog.createTable(tableIdentifier, IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA)); List inputRows = LongStream.range(1, 1000).mapToObj(ROW_FUNC::apply).collect(Collectors.toList()); + List expectedRecords = + inputRows.stream().map(RECORD_FUNC::apply).collect(Collectors.toList()); + // Run Managed Iceberg write writePipeline .apply(Create.of(inputRows)) - .setRowSchema(ROW_SCHEMA) - .apply(Managed.write(Managed.ICEBERG).withConfig(transformConfig)); + .setRowSchema(BEAM_SCHEMA) + .apply(Managed.write(Managed.ICEBERG).withConfig(getManagedIcebergConfig(tableIdentifier))); writePipeline.run().waitUntilFinish(); - PCollection outputRows = - readPipeline - .apply(Managed.read(Managed.ICEBERG).withConfig(transformConfig)) - .getSinglePCollection(); - PAssert.that(outputRows).containsInAnyOrder(inputRows); - readPipeline.run().waitUntilFinish(); + // read back the records and check everything's there + Table table = catalog.loadTable(tableIdentifier); + TableScan tableScan = table.newScan().project(ICEBERG_SCHEMA); + List writtenRecords = new ArrayList<>(); + for (CombinedScanTask task : tableScan.planTasks()) { + InputFilesDecryptor decryptor = new InputFilesDecryptor(task, table.io(), table.encryption()); + for (FileScanTask fileTask : task.files()) { + InputFile inputFile = decryptor.getInputFile(fileTask); + CloseableIterable iterable = + Parquet.read(inputFile) + .split(fileTask.start(), fileTask.length()) + .project(ICEBERG_SCHEMA) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(ICEBERG_SCHEMA, fileSchema)) + .filter(fileTask.residual()) + .build(); + + for (Record rec : iterable) { + writtenRecords.add(rec); + } + } + } + assertThat(expectedRecords, containsInAnyOrder(writtenRecords.toArray())); } } diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java index 6853d4e3230fb..adf941e00b4b6 100644 --- a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/testutils/ScriptRunner.java @@ -47,7 +47,7 @@ public class ScriptRunner { private final PrintWriter logWriter = new PrintWriter(System.out); private final PrintWriter errorLogWriter = new PrintWriter(System.err); - /** Default constructor */ + /** Default constructor. */ public ScriptRunner(Connection connection, boolean autoCommit, boolean stopOnError) { this.connection = connection; this.autoCommit = autoCommit; @@ -55,7 +55,7 @@ public ScriptRunner(Connection connection, boolean autoCommit, boolean stopOnErr } /** - * Runs an SQL script (read in using the Reader parameter) + * Runs an SQL script (read in using the Reader parameter). * * @param reader - the source of the script */ @@ -78,7 +78,7 @@ public void runScript(Reader reader) throws IOException, SQLException { } /** - * Runs an SQL script (read in using the Reader parameter) using the connection passed in + * Runs an SQL script (read in using the Reader parameter) using the connection passed in. * * @param conn - the connection to use for the script * @param reader - the source of the script From 394062430efee73189406734816e1e53613991bd Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 Aug 2024 17:12:26 -0400 Subject: [PATCH 3/8] add test documentation --- .../beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java index e7baae236bba4..79644003c5e91 100644 --- a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java @@ -70,6 +70,13 @@ import org.junit.Rule; import org.junit.Test; +/** + * Read and write test for {@link Managed} {@link org.apache.beam.sdk.io.iceberg.IcebergIO} using + * {@link HiveCatalog}. + * + *

Spins up a local Hive metastore to manage the Iceberg table. Warehouse path is set to a GCS + * bucket. + */ public class IcebergHiveCatalogIT { private static final Schema DOUBLY_NESTED_ROW_SCHEMA = Schema.builder() From bba7007366e3496f105b3dafcdc073d6b4e6142b Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 Aug 2024 17:32:19 -0400 Subject: [PATCH 4/8] extend new config_properties arg to translation tests --- .../io/iceberg/IcebergSchemaTransformTranslationTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java index 7863f7812a13f..86a5e0bcd432f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -71,6 +71,8 @@ public class IcebergSchemaTransformTranslationTest { .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .put("warehouse", "test_location") .build(); + private static final Map CONFIG_PROPERTIES = + ImmutableMap.builder().put("key", "value").put("key2", "value2").build(); @Test public void testReCreateWriteTransformFromRow() { @@ -79,6 +81,7 @@ public void testReCreateWriteTransformFromRow() { .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergWriteSchemaTransform writeTransform = (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); @@ -110,6 +113,7 @@ public void testWriteTransformProtoTranslation() .withFieldValue("table", "test_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergWriteSchemaTransform writeTransform = @@ -161,6 +165,7 @@ public void testReCreateReadTransformFromRow() { .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergReadSchemaTransform readTransform = @@ -192,6 +197,7 @@ public void testReadTransformProtoTranslation() .withFieldValue("table", identifier) .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", properties) + .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergReadSchemaTransform readTransform = From 3dbf16463b349f4797426722911e24fdc3ed507a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 6 Aug 2024 22:08:47 -0400 Subject: [PATCH 5/8] revert beam schema override --- .../io/iceberg/hive/IcebergHiveCatalogIT.java | 2 +- .../beam/sdk/io/iceberg/RecordWriter.java | 19 +++++-------------- .../io/iceberg/WriteGroupedRowsToFiles.java | 12 +++--------- .../io/iceberg/WriteUngroupedRowsToFiles.java | 16 ++++++---------- 4 files changed, 15 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java index 79644003c5e91..54a4998d37fba 100644 --- a/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java +++ b/sdks/java/io/iceberg/hive/src/test/java/org/apache/beam/sdk/io/iceberg/hive/IcebergHiveCatalogIT.java @@ -238,7 +238,7 @@ public void testReadWithHiveCatalog() throws IOException { @Test public void testWriteWithHiveCatalog() { TableIdentifier tableIdentifier = - TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE)); + TableIdentifier.parse(String.format("%s.%s", TEST_DB, TEST_TABLE + "_write_test")); catalog.createTable(tableIdentifier, IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA)); List inputRows = diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java index 66b441b4e641a..d7212783d1b6c 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamRowToIcebergRecord; import java.io.IOException; -import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; @@ -35,7 +34,6 @@ import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; -import org.checkerframework.checker.nullness.qual.Nullable; class RecordWriter { @@ -44,30 +42,23 @@ class RecordWriter { private final Table table; private final String absoluteFilename; - RecordWriter( - Catalog catalog, IcebergDestination destination, String filename, @Nullable Schema dataSchema) + RecordWriter(Catalog catalog, IcebergDestination destination, String filename) throws IOException { this( - catalog.loadTable(destination.getTableIdentifier()), - destination.getFileFormat(), - filename, - dataSchema); + catalog.loadTable(destination.getTableIdentifier()), destination.getFileFormat(), filename); } - RecordWriter(Table table, FileFormat fileFormat, String filename, @Nullable Schema dataSchema) - throws IOException { + RecordWriter(Table table, FileFormat fileFormat, String filename) throws IOException { this.table = table; this.absoluteFilename = table.location() + "/" + filename; OutputFile outputFile = table.io().newOutputFile(absoluteFilename); - org.apache.iceberg.Schema writerSchema = - dataSchema == null ? table.schema() : IcebergUtils.beamSchemaToIcebergSchema(dataSchema); switch (fileFormat) { case AVRO: icebergDataWriter = Avro.writeData(outputFile) .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) - .schema(writerSchema) + .schema(table.schema()) .withSpec(table.spec()) .overwrite() .build(); @@ -76,7 +67,7 @@ class RecordWriter { icebergDataWriter = Parquet.writeData(outputFile) .createWriterFunc(GenericParquetWriter::buildWriter) - .schema(writerSchema) + .schema(table.schema()) .withSpec(table.spec()) .overwrite() .build(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index d2d861bb6cbdb..c11263519442e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.UUID; -import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -27,10 +26,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.ShardedKey; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.iceberg.catalog.Catalog; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.checkerframework.checker.nullness.qual.Nullable; class WriteGroupedRowsToFiles extends PTransform< @@ -78,20 +75,17 @@ private org.apache.iceberg.catalog.Catalog getCatalog() { return catalog; } - private RecordWriter createWriter(IcebergDestination destination, @Nullable Schema dataSchema) - throws IOException { - return new RecordWriter(getCatalog(), destination, "-" + UUID.randomUUID(), dataSchema); + private RecordWriter createWriter(IcebergDestination destination) throws IOException { + return new RecordWriter(getCatalog(), destination, "-" + UUID.randomUUID()); } @ProcessElement public void processElement( ProcessContext c, @Element KV, Iterable> element) throws Exception { - @Nullable Row peekFirstRow = Iterables.getFirst(element.getValue(), null); - @Nullable Schema dataSchema = peekFirstRow != null ? peekFirstRow.getSchema() : null; Row destMetadata = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); - RecordWriter writer = createWriter(destination, dataSchema); + RecordWriter writer = createWriter(destination); for (Row e : element.getValue()) { writer.write(e); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index 110170d38c9ea..a00f3de4bb4e1 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -25,7 +25,6 @@ import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -214,12 +213,10 @@ private org.apache.iceberg.catalog.Catalog getCatalog() { return catalog; } - private RecordWriter createAndInsertWriter( - IcebergDestination destination, BoundedWindow window, Schema dataSchema) + private RecordWriter createAndInsertWriter(IcebergDestination destination, BoundedWindow window) throws IOException { RecordWriter writer = - new RecordWriter( - getCatalog(), destination, filename + "-" + UUID.randomUUID(), dataSchema); + new RecordWriter(getCatalog(), destination, filename + "-" + UUID.randomUUID()); getWindows().put(destination, window); getWriters().put(destination, writer); return writer; @@ -230,8 +227,7 @@ private RecordWriter createAndInsertWriter( * reached the maximum number of writers and should spill any records associated. */ @Nullable - RecordWriter getWriterIfPossible( - IcebergDestination destination, BoundedWindow window, Schema dataSchema) + RecordWriter getWriterIfPossible(IcebergDestination destination, BoundedWindow window) throws IOException { RecordWriter existingWriter = getWriters().get(destination); @@ -243,7 +239,7 @@ RecordWriter getWriterIfPossible( return null; } - return createAndInsertWriter(destination, window, dataSchema); + return createAndInsertWriter(destination, window); } @StartBundle @@ -259,7 +255,7 @@ public void processElement(@Element Row element, BoundedWindow window, MultiOutp IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); // Spill record if writer cannot be created - RecordWriter writer = getWriterIfPossible(destination, window, data.getSchema()); + RecordWriter writer = getWriterIfPossible(destination, window); if (writer == null) { out.get(SPILLED_ROWS_TAG).output(element); return; @@ -274,7 +270,7 @@ public void processElement(@Element Row element, BoundedWindow window, MultiOutp .setManifestFile(writer.getManifestFile()) .setTableIdentifier(destination.getTableIdentifier()) .build()); - writer = createAndInsertWriter(destination, window, data.getSchema()); + writer = createAndInsertWriter(destination, window); } // Actually write the data From 0483ffac2a5c5b77ce3e5d2701d22d4b157384a8 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 7 Aug 2024 07:09:37 -0400 Subject: [PATCH 6/8] actually run hive ITs --- .github/workflows/IO_Iceberg_Integration_Tests.yml | 4 +--- sdks/java/io/iceberg/build.gradle | 5 +++++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/.github/workflows/IO_Iceberg_Integration_Tests.yml b/.github/workflows/IO_Iceberg_Integration_Tests.yml index 025c622eccfa9..22b2b4f9287d5 100644 --- a/.github/workflows/IO_Iceberg_Integration_Tests.yml +++ b/.github/workflows/IO_Iceberg_Integration_Tests.yml @@ -75,6 +75,4 @@ jobs: - name: Run IcebergIO Integration Test uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: | - :sdks:java:io:iceberg:integrationTest \ - :sdks:java:io:iceberg:hive:integrationTest \ \ No newline at end of file + gradle-command: :sdks:java:io:iceberg:catalogTests \ No newline at end of file diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 7965cde86e7d9..3d653d6b276e9 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -115,6 +115,11 @@ task integrationTest(type: Test) { testClassesDirs = sourceSets.test.output.classesDirs } +tasks.register('catalogTests') { + dependsOn integrationTest + dependsOn ":sdks:java:io:iceberg:hive:integrationTest" +} + task loadTest(type: Test) { def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpTempLocation = project.findProperty('gcpTempLocation') ?: 'gs://temp-storage-for-end-to-end-tests/temp-lt' From e498f3739cf97b0d8ba85f59d545ed78bbf2b131 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 7 Aug 2024 15:36:23 -0400 Subject: [PATCH 7/8] trigger integration tests --- .github/trigger_files/IO_Iceberg_Integration_Tests.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index bbdc3a3910ef8..62ae7886c5731 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": 3 + "modification": 4 } From 713d74c70378bdb4854a550df394ccfb3873034a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 9 Aug 2024 10:38:18 -0400 Subject: [PATCH 8/8] cut down hive database source lines --- .../resources/hive-schema-3.1.0.derby.sql | 459 ------------------ 1 file changed, 459 deletions(-) diff --git a/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql index b1a4c158ebb9f..808c605857648 100644 --- a/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql +++ b/sdks/java/io/iceberg/hive/src/test/resources/hive-schema-3.1.0.derby.sql @@ -46,12 +46,8 @@ CREATE TABLE "APP"."DBS" ( "CTLG_NAME" VARCHAR(256) ); -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); @@ -60,84 +56,26 @@ CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(7 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"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT); - -CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); - -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); - 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" VARCHAR(4000)); - -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'); 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 -); - 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); @@ -150,83 +88,8 @@ CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_ 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 -); - 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 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)); - -CREATE TABLE "APP"."WM_RESOURCEPLAN" (RP_ID BIGINT NOT NULL, 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 -); - CREATE TABLE "APP"."CTLGS" ( "CTLG_ID" BIGINT NOT NULL, "NAME" VARCHAR(256) UNIQUE, @@ -237,64 +100,21 @@ CREATE TABLE "APP"."CTLGS" ( -- 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 UNIQUE INDEX "APP"."UNIQUEINDEX" ON "APP"."IDXS" ("INDEX_NAME", "ORIG_TBL_ID"); - -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 INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME"); - -CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_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" ("PART_NAME", "TBL_ID"); - -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" ("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"); @@ -303,40 +123,20 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); -- ---------------------------------------------- -- primary/unique -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_PK" PRIMARY KEY ("INDEX_ID"); - -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"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_PK" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); - 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"); @@ -353,18 +153,8 @@ ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KE 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"); @@ -377,54 +167,16 @@ ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LO 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"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("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"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK3" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -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"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_FK1" FOREIGN KEY ("INDEX_ID") REFERENCES "APP"."IDXS" ("INDEX_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; @@ -443,18 +195,8 @@ ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN K 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; @@ -469,96 +211,19 @@ ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY 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_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"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REBUILD IN ('Y','N')); - 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, - 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 -); - -CREATE TABLE TXN_COMPONENTS ( - TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), - TC_DATABASE varchar(128) NOT NULL, - TC_TABLE varchar(128), - 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 NEXT_TXN_ID ( - NTXN_NEXT bigint NOT NULL -); -INSERT INTO NEXT_TXN_ID VALUES(1); - CREATE TABLE HIVE_LOCKS ( HL_LOCK_EXT_ID bigint NOT NULL, HL_LOCK_INT_ID bigint NOT NULL, @@ -586,44 +251,6 @@ CREATE TABLE NEXT_LOCK_ID ( ); 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(128) 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_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) -); - -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(128) 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_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) -); - CREATE TABLE AUX_TABLE ( MT_KEY1 varchar(128) NOT NULL, MT_KEY2 bigint NOT NULL, @@ -633,92 +260,6 @@ CREATE TABLE AUX_TABLE ( --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(128) 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_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); -- ----------------------------------------------------------------- -- Record schema version. Should be the last step in the init script