diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index cab7fd1d4..d33877070 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -21,11 +21,11 @@ name: "Java CI" on: push: branches: - - 'master' - - '0.**' - tags: - - 'apache-iceberg-**' + - 'li-1.0.x' + tags-ignore: [v*] # release tags are autogenerated after a successful CI, no need to run CI against them pull_request: + branches: + - 'li-1.0.x' paths-ignore: - '.github/workflows/python-ci.yml' - '.github/workflows/spark-ci.yml' @@ -53,28 +53,28 @@ jobs: runs-on: ubuntu-20.04 strategy: matrix: - jvm: [8, 11] + jvm: [ 8, 11 ] env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v3 - - uses: actions/setup-java@v3 - with: - distribution: zulu - java-version: ${{ matrix.jvm }} - - uses: actions/cache@v3 - with: - path: ~/.gradle/caches - key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle') }} - restore-keys: ${{ runner.os }}-gradle - - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -Pquick=true -x javadoc - - uses: actions/upload-artifact@v3 - if: failure() - with: - name: test logs - path: | - **/build/testlogs + - uses: actions/checkout@v3 + - uses: actions/setup-java@v3 + with: + distribution: zulu + java-version: ${{ matrix.jvm }} + - uses: actions/cache@v3 + with: + path: ~/.gradle/caches + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle') }} + restore-keys: ${{ runner.os }}-gradle + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -Pquick=true -x javadoc + - uses: actions/upload-artifact@v3 + if: failure() + with: + name: test logs + path: | + **/build/testlogs build-checks: runs-on: ubuntu-20.04 @@ -84,7 +84,7 @@ jobs: with: distribution: zulu java-version: 8 - - run: ./gradlew -DflinkVersions=1.13,1.14,1.15 -DsparkVersions=2.4,3.0,3.1,3.2,3.3 -DhiveVersions=2,3 build -x test -x javadoc -x integrationTest + - run: ./gradlew -DflinkVersions=1.13,1.14,1.15 -DsparkVersions=2.4,3.1 -DhiveVersions=2,3 build -x test -x javadoc -x integrationTest build-javadoc: runs-on: ubuntu-20.04 @@ -94,4 +94,32 @@ jobs: with: distribution: zulu java-version: 8 - - run: ./gradlew -Pquick=true javadoc + - run: ./gradlew -P=true javadoc + + release: + if: ${{ github.event_name == 'push' }} + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v3 + with: + fetch-depth: '0' # https://github.com/shipkit/shipkit-changelog#fetch-depth-on-ci + - uses: actions/setup-java@v3 + with: + distribution: zulu + java-version: 8 + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: ./gradlew build -DflinkVersions=1.14 -DsparkVersions=2.4,3.1 -DhiveVersions= -Pquick=true build -x javadoc + - name: Perform release + # Release job, only for pushes to the main development branch + if: ${{ github.event_name == 'push' + && github.ref == 'refs/heads/li-1.0.x' + && github.repository == 'linkedin/iceberg' + && !contains(toJSON(github.event.commits.*.message), '[skip release]') }} + + run: ./gradlew -DflinkVersions=1.14 -DsparkVersions=2.4,3.1 -DhiveVersions= githubRelease publishToSonatype closeAndReleaseStagingRepository + env: + GITHUB_TOKEN: ${{secrets.GITHUB_TOKEN}} + SONATYPE_USER: ${{secrets.SONATYPE_USER}} + SONATYPE_PWD: ${{secrets.SONATYPE_PWD}} + PGP_KEY: ${{secrets.PGP_KEY}} + PGP_PWD: ${{secrets.PGP_PWD}} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 7a19669b0..e28e3757b 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -21,8 +21,7 @@ name: "Spark CI" on: push: branches: - - 'master' - - '0.**' + - 'li-1.0.x' tags: - 'apache-iceberg-**' pull_request: @@ -83,7 +82,7 @@ jobs: strategy: matrix: jvm: [8, 11] - spark: ['3.0', '3.1', '3.2', '3.3'] + spark: ['3.1'] env: SPARK_LOCAL_IP: localhost steps: @@ -107,6 +106,7 @@ jobs: **/build/testlogs spark-3x-scala-2-13-tests: + if: ${{ false }} runs-on: ubuntu-20.04 strategy: matrix: diff --git a/api/src/main/java/org/apache/iceberg/types/PruneColumns.java b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java index daf2e6bbc..96f50f3e0 100644 --- a/api/src/main/java/org/apache/iceberg/types/PruneColumns.java +++ b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java @@ -68,11 +68,19 @@ public Type struct(Types.StructType struct, List fieldResults) { if (field.isOptional()) { selectedFields.add( Types.NestedField.optional( - field.fieldId(), field.name(), projectedType, field.doc())); + field.fieldId(), + field.name(), + projectedType, + field.getDefaultValue(), + field.doc())); } else { selectedFields.add( Types.NestedField.required( - field.fieldId(), field.name(), projectedType, field.doc())); + field.fieldId(), + field.name(), + projectedType, + field.getDefaultValue(), + field.doc())); } } } diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 354899409..72c300a43 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -412,42 +412,124 @@ public int hashCode() { public static class NestedField implements Serializable { public static NestedField optional(int id, String name, Type type) { - return new NestedField(true, id, name, type, null); + return new NestedField(true, id, name, type, null, null); } public static NestedField optional(int id, String name, Type type, String doc) { - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, null, doc); + } + + public static NestedField optional( + int id, String name, Type type, Object defaultValue, String doc) { + return new NestedField(true, id, name, type, defaultValue, doc); } public static NestedField required(int id, String name, Type type) { - return new NestedField(false, id, name, type, null); + return new NestedField(false, id, name, type, null, null); } public static NestedField required(int id, String name, Type type, String doc) { - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, null, doc); + } + + public static NestedField required( + int id, String name, Type type, Object defaultValue, String doc) { + return new NestedField(false, id, name, type, defaultValue, doc); } public static NestedField of(int id, boolean isOptional, String name, Type type) { - return new NestedField(isOptional, id, name, type, null); + return new NestedField(isOptional, id, name, type, null, null); } public static NestedField of(int id, boolean isOptional, String name, Type type, String doc) { - return new NestedField(isOptional, id, name, type, doc); + return new NestedField(isOptional, id, name, type, null, doc); + } + + public static NestedField of( + int id, boolean isOptional, String name, Type type, Object defaultValue, String doc) { + return new NestedField(isOptional, id, name, type, defaultValue, doc); + } + + private static void validateDefaultValue(Object defaultValue, Type type) { + if (defaultValue == null) { + return; + } + switch (type.typeId()) { + case STRUCT: + Preconditions.checkArgument( + defaultValue instanceof Map, + "defaultValue should be a Map from fields names to values, for StructType"); + Map defaultStruct = (Map) defaultValue; + if (defaultStruct.isEmpty()) { + return; + } + for (NestedField field : type.asStructType().fields()) { + validateDefaultValue( + defaultStruct.getOrDefault(field.name(), field.getDefaultValue()), field.type()); + } + break; + + case LIST: + Preconditions.checkArgument( + defaultValue instanceof List, + "defaultValue should be an List of Objects, for ListType"); + List defaultList = (List) defaultValue; + if (defaultList.size() == 0) { + return; + } + defaultList.forEach( + dv -> NestedField.validateDefaultValue(dv, type.asListType().elementField.type)); + break; + + case MAP: + Preconditions.checkArgument( + defaultValue instanceof Map, "defaultValue should be an instance of Map for MapType"); + Map defaultMap = (Map) defaultValue; + if (defaultMap.isEmpty()) { + return; + } + for (Map.Entry e : defaultMap.entrySet()) { + NestedField.validateDefaultValue(e.getKey(), type.asMapType().keyField.type); + NestedField.validateDefaultValue(e.getValue(), type.asMapType().valueField.type); + } + break; + + case FIXED: + case BINARY: + Preconditions.checkArgument( + defaultValue instanceof byte[], + "defaultValue should be an instance of byte[] for TypeId.%s, but defaultValue.class = %s", + type.typeId().name(), + defaultValue.getClass().getCanonicalName()); + break; + + default: + Preconditions.checkArgument( + type.typeId().javaClass().isInstance(defaultValue), + "defaultValue should be and instance of %s for TypeId.%s, but defaultValue.class = %s", + type.typeId().javaClass(), + type.typeId().name(), + defaultValue.getClass().getCanonicalName()); + } } private final boolean isOptional; private final int id; private final String name; private final Type type; + private final Object defaultValue; private final String doc; - private NestedField(boolean isOptional, int id, String name, Type type, String doc) { + private NestedField( + boolean isOptional, int id, String name, Type type, Object defaultValue, String doc) { Preconditions.checkNotNull(name, "Name cannot be null"); Preconditions.checkNotNull(type, "Type cannot be null"); + validateDefaultValue(defaultValue, type); this.isOptional = isOptional; this.id = id; this.name = name; this.type = type; + this.defaultValue = defaultValue; this.doc = doc; } @@ -459,7 +541,7 @@ public NestedField asOptional() { if (isOptional) { return this; } - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, defaultValue, doc); } public boolean isRequired() { @@ -470,7 +552,15 @@ public NestedField asRequired() { if (!isOptional) { return this; } - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, defaultValue, doc); + } + + public boolean hasDefaultValue() { + return defaultValue != null; + } + + public Object getDefaultValue() { + return defaultValue; } public int fieldId() { @@ -492,6 +582,7 @@ public String doc() { @Override public String toString() { return String.format("%d: %s: %s %s", id, name, isOptional ? "optional" : "required", type) + + (hasDefaultValue() ? ", default value: " + defaultValue + ", " : "") + (doc != null ? " (" + doc + ")" : ""); } @@ -510,6 +601,9 @@ public boolean equals(Object o) { return false; } else if (!name.equals(that.name)) { return false; + } else if (!Objects.equals(defaultValue, that.defaultValue) + && !Arrays.equals((byte[]) defaultValue, (byte[]) that.defaultValue)) { + return false; } else if (!Objects.equals(doc, that.doc)) { return false; } @@ -518,7 +612,9 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(NestedField.class, id, isOptional, name, type); + return hasDefaultValue() + ? Objects.hash(NestedField.class, id, isOptional, name, type, defaultValue) + : Objects.hash(NestedField.class, id, isOptional, name, type); } } @@ -736,7 +832,6 @@ public boolean equals(Object o) { } else if (!(o instanceof ListType)) { return false; } - ListType listType = (ListType) o; return elementField.equals(listType.elementField); } diff --git a/api/src/test/java/org/apache/iceberg/types/TestDefaultValuesForContainerTypes.java b/api/src/test/java/org/apache/iceberg/types/TestDefaultValuesForContainerTypes.java new file mode 100644 index 000000000..2729e9eb5 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/types/TestDefaultValuesForContainerTypes.java @@ -0,0 +1,73 @@ +/* + * 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.iceberg.types; + +import static org.apache.iceberg.types.Types.NestedField; +import static org.apache.iceberg.types.Types.StructType; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestDefaultValuesForContainerTypes { + + static NestedField intFieldType; + static NestedField stringFieldType; + static StructType structType; + + @BeforeClass + public static void beforeClass() { + intFieldType = NestedField.optional(0, "optionalIntField", Types.IntegerType.get()); + stringFieldType = NestedField.required(1, "requiredStringField", Types.StringType.get()); + structType = StructType.of(Arrays.asList(intFieldType, stringFieldType)); + } + + @Test + public void testStructTypeDefault() { + Map structDefaultvalue = Maps.newHashMap(); + structDefaultvalue.put(intFieldType.name(), Integer.valueOf(1)); + structDefaultvalue.put(stringFieldType.name(), "two"); + NestedField structField = + NestedField.optional(2, "optionalStructField", structType, structDefaultvalue, "doc"); + Assert.assertTrue(structField.hasDefaultValue()); + Assert.assertEquals(structDefaultvalue, structField.getDefaultValue()); + } + + @Test(expected = IllegalArgumentException.class) + public void testStructTypeDefaultInvalidFieldsTypes() { + List structDefaultvalue = Lists.newArrayList(); + structDefaultvalue.add("one"); + structDefaultvalue.add("two"); + NestedField.optional(2, "optionalStructField", structType, structDefaultvalue, "doc"); + } + + @Test(expected = IllegalArgumentException.class) + public void testStructTypeDefaultInvalidNumberFields() { + List structDefaultvalue = Lists.newArrayList(); + structDefaultvalue.add(Integer.valueOf(1)); + structDefaultvalue.add("two"); + structDefaultvalue.add("three"); + NestedField.optional(2, "optionalStructField", structType, structDefaultvalue, "doc"); + } +} diff --git a/api/src/test/java/org/apache/iceberg/types/TestNestedFieldDefaultValues.java b/api/src/test/java/org/apache/iceberg/types/TestNestedFieldDefaultValues.java new file mode 100644 index 000000000..c94888264 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/types/TestNestedFieldDefaultValues.java @@ -0,0 +1,80 @@ +/* + * 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.iceberg.types; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import org.apache.iceberg.types.Types.NestedField; +import org.junit.Assert; +import org.junit.Test; + +public class TestNestedFieldDefaultValues { + + private final int id = 1; + private final String fieldName = "fieldName"; + private final Type fieldType = Types.IntegerType.get(); + private final String doc = "field doc"; + private final Integer defaultValue = 100; + + @Test + public void testConstructorsValidCases() { + // optional constructors + Assert.assertFalse(optional(id, fieldName, fieldType).hasDefaultValue()); + Assert.assertFalse(optional(id, fieldName, fieldType, doc).hasDefaultValue()); + NestedField nestedFieldWithDefault = optional(id, fieldName, fieldType, defaultValue, doc); + Assert.assertTrue(nestedFieldWithDefault.hasDefaultValue()); + Assert.assertEquals(defaultValue, nestedFieldWithDefault.getDefaultValue()); + nestedFieldWithDefault = optional(id, fieldName, fieldType, defaultValue, null); + Assert.assertTrue(nestedFieldWithDefault.hasDefaultValue()); + Assert.assertEquals(defaultValue, nestedFieldWithDefault.getDefaultValue()); + + // required constructors + Assert.assertFalse(required(id, fieldName, fieldType).hasDefaultValue()); + Assert.assertFalse(required(id, fieldName, fieldType, doc).hasDefaultValue()); + Assert.assertFalse(required(id, fieldName, fieldType, null, doc).hasDefaultValue()); + nestedFieldWithDefault = required(id, fieldName, fieldType, defaultValue, doc); + Assert.assertTrue(nestedFieldWithDefault.hasDefaultValue()); + Assert.assertEquals(defaultValue, nestedFieldWithDefault.getDefaultValue()); + nestedFieldWithDefault = required(id, fieldName, fieldType, defaultValue, null); + Assert.assertTrue(nestedFieldWithDefault.hasDefaultValue()); + Assert.assertEquals(defaultValue, nestedFieldWithDefault.getDefaultValue()); + + // of constructors + Assert.assertFalse(NestedField.of(id, true, fieldName, fieldType).hasDefaultValue()); + Assert.assertFalse(NestedField.of(id, true, fieldName, fieldType, doc).hasDefaultValue()); + nestedFieldWithDefault = NestedField.of(id, true, fieldName, fieldType, defaultValue, doc); + Assert.assertTrue(nestedFieldWithDefault.hasDefaultValue()); + Assert.assertEquals(defaultValue, nestedFieldWithDefault.getDefaultValue()); + } + + @Test(expected = IllegalArgumentException.class) + public void testOptionalWithInvalidDefaultValueClass() { + // class of default value does not match class of type + Long wrongClassDefaultValue = 100L; + optional(id, fieldName, fieldType, wrongClassDefaultValue, doc); + } + + @Test(expected = IllegalArgumentException.class) + public void testReqiredWithInvalidDefaultValueClass() { + // class of default value does not match class of type + Long wrongClassDefaultValue = 100L; + required(id, fieldName, fieldType, wrongClassDefaultValue, doc); + } +} diff --git a/build.gradle b/build.gradle index ab027e3d5..da2e31ee8 100644 --- a/build.gradle +++ b/build.gradle @@ -23,6 +23,8 @@ import java.util.regex.Pattern buildscript { repositories { + maven { url "https://plugins.gradle.org/m2/" } + jcenter() gradlePluginPortal() } dependencies { @@ -35,6 +37,9 @@ buildscript { classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.4.1" classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.1' + classpath 'io.github.gradle-nexus:publish-plugin:1.1.0' + classpath 'org.shipkit:shipkit-auto-version:1.2.0' + classpath 'org.shipkit:shipkit-changelog:1.1.10' } } @@ -86,8 +91,7 @@ dependencyRecommendations { } allprojects { - group = "org.apache.iceberg" - version = getProjectVersion() + group = "com.linkedin.iceberg" repositories { mavenCentral() mavenLocal() @@ -218,7 +222,7 @@ project(':iceberg-bundled-guava') { } project(':iceberg-api') { - apply plugin: 'com.palantir.revapi' +// apply plugin: 'com.palantir.revapi' dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') @@ -500,6 +504,81 @@ project(':iceberg-hive-metastore') { } } +project(':iceberg-hivelink-core') { + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + implementation project(':iceberg-core') + implementation project(':iceberg-hive-metastore') + implementation project(':iceberg-common') + + implementation "com.github.ben-manes.caffeine:caffeine" + implementation("org.apache.orc:orc-core::nohive") { + exclude group: 'org.apache.hadoop' + exclude group: 'commons-lang' + // These artifacts are shaded and included in the orc-core fat jar + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.hive', module: 'hive-storage-api' + } + + compileOnly "org.apache.avro:avro" + compileOnly("org.apache.hadoop:hadoop-client") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + } + + compileOnly("org.apache.hive:hive-metastore") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + } + + testImplementation project(':iceberg-data') + testImplementation project(':iceberg-orc') + + // By default, hive-exec is a fat/uber jar and it exports a guava library + // that's really old. We use the core classifier to be able to override our guava + // version. Luckily, hive-exec seems to work okay so far with this version of guava + // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. + testImplementation("org.apache.hive:hive-exec::core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + testImplementation("org.apache.hive:hive-metastore") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + } + } +} + project(':iceberg-orc') { dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') @@ -592,6 +671,8 @@ project(':iceberg-arrow') { } } + + project(':iceberg-pig') { dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') @@ -651,6 +732,70 @@ project(':iceberg-nessie') { } } +project(':iceberg-runtime') { + apply plugin: 'com.github.johnrengelman.shadow' + + tasks.jar.dependsOn tasks.shadowJar + + configurations { + compile { + exclude group: 'org.apache.spark' + // included in Spark + exclude group: 'org.slf4j' + exclude group: 'org.apache.commons' + exclude group: 'commons-pool' + exclude group: 'commons-codec' + exclude group: 'org.xerial.snappy' + exclude group: 'javax.xml.bind' + exclude group: 'javax.annotation' + } + } + + dependencies { + compile project(':iceberg-data') + compile project(':iceberg-hive-metastore') + compile project(':iceberg-orc') + } + + shadowJar { + configurations = [project.configurations.compile] + + zip64 true + + // include the LICENSE and NOTICE files for the shaded Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + // Relocate dependencies to avoid conflicts + relocate 'com.google', 'org.apache.iceberg.shaded.com.google' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' + relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded' + relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer' + relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' + relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + // relocate Avro's jackson dependency to share parquet-jackson locations + relocate 'org.codehaus.jackson', 'org.apache.iceberg.shaded.org.apache.parquet.shaded.org.codehaus.jackson' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + // relocate Arrow and related deps to shade Iceberg specific version + relocate 'io.netty.buffer', 'org.apache.iceberg.shaded.io.netty.buffer' + relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow' + relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' + relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' + + classifier null + } + + jar { + classifier = 'empty' + } +} + project(':iceberg-dell') { dependencies { implementation project(':iceberg-core') @@ -712,5 +857,33 @@ String getJavadocVersion() { apply from: 'jmh.gradle' apply from: 'baseline.gradle' apply from: 'deploy.gradle' -apply from: 'tasks.gradle' +apply from: 'shipkit.gradle' +project(':li-iceberg-depend-all') { + checkClassUniqueness { + enabled = false + } + + dependencies { + runtimeOnly project(':iceberg-api') + runtimeOnly project(':iceberg-arrow') + runtimeOnly project(':iceberg-common') + runtimeOnly project(':iceberg-core') + runtimeOnly project(':iceberg-data') + runtimeOnly project(':iceberg-hive-metastore') + runtimeOnly project(':iceberg-hivelink-core') + runtimeOnly project(':iceberg-orc') + runtimeOnly project(':iceberg-parquet') + + // engine runtimes + if (findProject(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') != null) { + runtimeOnly project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') + } + if (findProject(':iceberg-spark:iceberg-spark-runtime-2.4') != null) { + runtimeOnly project(':iceberg-spark:iceberg-spark-runtime-2.4') + } + if (findProject(':iceberg-flink:iceberg-flink-runtime-1.15') != null) { + runtimeOnly project(":iceberg-flink:iceberg-flink-runtime-1.15") + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index 1274728e5..368837d02 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -172,7 +172,7 @@ public FileScanTask next() { } } - private static final class SplitScanTask + public static final class SplitScanTask implements FileScanTask, MergeableScanTask { private final long len; private final long offset; @@ -234,5 +234,9 @@ public SplitScanTask merge(ScanTask other) { SplitScanTask that = (SplitScanTask) other; return new SplitScanTask(offset, len + that.length(), fileScanTask); } + + public FileScanTask underlyingFileScanTask() { + return fileScanTask; + } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 04ff3b7a3..e8ee7e5ae 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -86,7 +86,7 @@ private Table loadMetadataTable(TableIdentifier identifier) { } } - private boolean isValidMetadataIdentifier(TableIdentifier identifier) { + protected boolean isValidMetadataIdentifier(TableIdentifier identifier) { return MetadataTableType.from(identifier.name()) != null && isValidIdentifier(TableIdentifier.of(identifier.namespace().levels())); } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index d2a2a48ed..c217ff635 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -424,4 +424,12 @@ private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metada .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); } } + + protected void setCurrentMetadata(TableMetadata currentMetadata) { + this.currentMetadata = currentMetadata; + } + + protected void setShouldRefresh(boolean shouldRefresh) { + this.shouldRefresh = shouldRefresh; + } } diff --git a/core/src/main/java/org/apache/iceberg/SchemaParser.java b/core/src/main/java/org/apache/iceberg/SchemaParser.java index 0c60a9dec..85c10e149 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaParser.java +++ b/core/src/main/java/org/apache/iceberg/SchemaParser.java @@ -19,14 +19,18 @@ package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import java.io.IOException; import java.io.StringWriter; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; import java.util.Set; +import org.apache.avro.util.internal.JacksonUtils; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -57,6 +61,20 @@ private SchemaParser() {} private static final String REQUIRED = "required"; private static final String ELEMENT_REQUIRED = "element-required"; private static final String VALUE_REQUIRED = "value-required"; + private static final String DEFAULT = "default"; + + private static void writeDefaultValue(Object defaultValue, Type type, JsonGenerator generator) + throws IOException { + if (defaultValue == null) { + return; + } + generator.writeFieldName(DEFAULT); + if (isFixedOrBinary(type)) { + generator.writeRawValue(defaultValueToJsonString((byte[]) defaultValue)); + } else { + generator.writeRawValue(defaultValueToJsonString(defaultValue)); + } + } private static void toJson(Types.StructType struct, JsonGenerator generator) throws IOException { toJson(struct, null, null, generator); @@ -91,6 +109,7 @@ private static void toJson( generator.writeBooleanField(REQUIRED, field.isRequired()); generator.writeFieldName(TYPE); toJson(field.type(), generator); + writeDefaultValue(field.getDefaultValue(), field.type(), generator); if (field.doc() != null) { generator.writeStringField(DOC, field.doc()); } @@ -197,6 +216,27 @@ private static Type typeFromJson(JsonNode json) { throw new IllegalArgumentException("Cannot parse type from json: " + json); } + private static boolean isFixedOrBinary(Type type) { + return type.typeId() == Type.TypeID.FIXED || type.typeId() == Type.TypeID.BINARY; + } + + private static Object defaultValueFromJson(JsonNode field, Type type) { + if (!field.has(DEFAULT)) { + return null; + } + + if (isFixedOrBinary(type)) { + try { + return field.get(DEFAULT).binaryValue(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + return AvroSchemaUtil.convertToJavaDefaultValue( + JacksonUtils.toObject(field.get(DEFAULT), AvroSchemaUtil.convert(type))); + } + private static Types.StructType structFromJson(JsonNode json) { JsonNode fieldArray = JsonUtil.get(FIELDS, json); Preconditions.checkArgument( @@ -212,13 +252,13 @@ private static Types.StructType structFromJson(JsonNode json) { int id = JsonUtil.getInt(ID, field); String name = JsonUtil.getString(NAME, field); Type type = typeFromJson(JsonUtil.get(TYPE, field)); - + Object defaultValue = defaultValueFromJson(field, type); String doc = JsonUtil.getStringOrNull(DOC, field); boolean isRequired = JsonUtil.getBool(REQUIRED, field); if (isRequired) { - fields.add(Types.NestedField.required(id, name, type, doc)); + fields.add(Types.NestedField.required(id, name, type, defaultValue, doc)); } else { - fields.add(Types.NestedField.optional(id, name, type, doc)); + fields.add(Types.NestedField.optional(id, name, type, defaultValue, doc)); } } @@ -283,4 +323,20 @@ public static Schema fromJson(String json) { } }); } + + private static String defaultValueToJsonString(byte[] bytes) { + try { + return JsonUtil.mapper().writeValueAsString(ByteBuffer.wrap(bytes)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + private static String defaultValueToJsonString(Object value) { + try { + return JsonUtil.mapper().writeValueAsString(value); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 6f681c9c9..2f00544ea 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -246,7 +246,7 @@ public String toString() { private final List changes; @SuppressWarnings("checkstyle:CyclomaticComplexity") - TableMetadata( + public TableMetadata( String metadataFileLocation, int formatVersion, String uuid, diff --git a/core/src/main/java/org/apache/iceberg/TableScanContext.java b/core/src/main/java/org/apache/iceberg/TableScanContext.java index beca0db87..613aee81f 100644 --- a/core/src/main/java/org/apache/iceberg/TableScanContext.java +++ b/core/src/main/java/org/apache/iceberg/TableScanContext.java @@ -29,7 +29,7 @@ import org.apache.iceberg.util.ThreadPools; /** Context object with optional arguments for a TableScan. */ -final class TableScanContext { +public final class TableScanContext { private final Long snapshotId; private final Expression rowFilter; private final boolean ignoreResiduals; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index 46c17722f..88ea7e9bf 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.avro; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; @@ -158,17 +160,78 @@ public static boolean isOptionSchema(Schema schema) { return false; } - static Schema toOption(Schema schema) { + /** + * This method decides whether a schema represents a single type union, i.e., a union that + * contains only one option + * + * @param schema input schema + * @return true if schema is single type union + */ + public static boolean isSingleTypeUnion(Schema schema) { + return schema.getType() == UNION && schema.getTypes().size() == 1; + } + + /** + * This method decides whether a schema is of type union and is complex union and is optional + * + *

Complex union: the number of options in union not equals to 2 Optional: null is present in + * union + * + * @param schema input schema + * @return true if schema is complex union and it is optional + */ + public static boolean isOptionalComplexUnion(Schema schema) { + if (schema.getType() == UNION && schema.getTypes().size() != 2) { + for (Schema type : schema.getTypes()) { + if (type.getType() == Schema.Type.NULL) { + return true; + } + } + } + + return false; + } + + public static Schema discardNullFromUnionIfExist(Schema schema) { + Preconditions.checkArgument( + schema.getType() == UNION, "Expected union schema but was passed: %s", schema); + List result = Lists.newArrayList(); + for (Schema nested : schema.getTypes()) { + if (!(nested.getType() == Schema.Type.NULL)) { + result.add(nested); + } + } + return Schema.createUnion(result); + } + + public static boolean nullExistInUnion(Schema schema) { + Preconditions.checkArgument( + schema.getType() == UNION, "Expected union schema but was passed: %s", schema); + for (Schema nested : schema.getTypes()) { + if (nested.getType() == Schema.Type.NULL) { + return true; + } + } + return false; + } + + public static Schema toOption(Schema schema) { + return toOption(schema, false); + } + + public static Schema toOption(Schema schema, boolean nullIsSecondElement) { if (schema.getType() == UNION) { Preconditions.checkArgument( isOptionSchema(schema), "Union schemas are not supported: %s", schema); return schema; + } else if (nullIsSecondElement) { + return Schema.createUnion(schema, NULL); } else { return Schema.createUnion(NULL, schema); } } - static Schema fromOption(Schema schema) { + public static Schema fromOption(Schema schema) { Preconditions.checkArgument( schema.getType() == UNION, "Expected union schema but was passed: %s", schema); Preconditions.checkArgument( @@ -366,7 +429,7 @@ private static int toInt(Object value) { throw new UnsupportedOperationException("Cannot coerce value to int: " + value); } - static Schema copyRecord(Schema record, List newFields, String newName) { + public static Schema copyRecord(Schema record, List newFields, String newName) { Schema copy; if (newName != null) { copy = Schema.createRecord(newName, record.getDoc(), null, record.isError(), newFields); @@ -391,7 +454,7 @@ static Schema copyRecord(Schema record, List newFields, String new return copy; } - static Schema.Field copyField(Schema.Field field, Schema newSchema, String newName) { + public static Schema.Field copyField(Schema.Field field, Schema newSchema, String newName) { Schema.Field copy = new Schema.Field(newName, newSchema, field.doc(), field.defaultVal(), field.order()); @@ -477,4 +540,34 @@ private static String sanitize(char character) { } return "_x" + Integer.toHexString(character).toUpperCase(); } + + static boolean hasNonNullDefaultValue(Schema.Field field) { + // the schema should use JsonProperties.NULL_VALUE (i.e., null) as the null default + // value, but a user might also use "null" to indicate null while it is actually a String, so + // need to account for it. + return field.hasDefaultValue() + && field.defaultVal() != JsonProperties.NULL_VALUE + && !(field.defaultVal() instanceof String + && ((String) field.defaultVal()).equalsIgnoreCase("null")); + } + + public static Object convertToJavaDefaultValue(Object defaultValue) { + if (defaultValue instanceof List) { + return ((List) defaultValue) + .stream().map(AvroSchemaUtil::convertToJavaDefaultValue).collect(Collectors.toList()); + } else if (defaultValue instanceof Map) { + // can't seem to use the java8 stream api on map correctly because of setting null value in + // map + Map retMap = new LinkedHashMap<>(); + for (Map.Entry entry : ((Map) defaultValue).entrySet()) { + retMap.put(entry.getKey(), convertToJavaDefaultValue(entry.getValue())); + } + return retMap; + } else if (defaultValue == JsonProperties.NULL_VALUE) { + // convert the JsonProperties.NULL_VALUE whenever we see it + return null; + } + // don't touch any other primitive values + return defaultValue; + } } diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java index f22a3592a..279fe2b2f 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java @@ -51,8 +51,21 @@ public static T visit(Schema schema, AvroSchemaVisitor visitor) { case UNION: List types = schema.getTypes(); List options = Lists.newArrayListWithExpectedSize(types.size()); - for (Schema type : types) { - options.add(visit(type, visitor)); + if (AvroSchemaUtil.isOptionSchema(schema) || AvroSchemaUtil.isSingleTypeUnion(schema)) { + for (Schema type : types) { + options.add(visit(type, visitor)); + } + } else { + // complex union case + int idx = 0; + for (Schema type : types) { + if (type.getType() != Schema.Type.NULL) { + options.add(visitWithName("field" + idx, type, visitor)); + idx += 1; + } else { + options.add(visit(type, visitor)); + } + } } return visitor.union(schema, options); diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java index 85a8718ab..4e3c64658 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java @@ -82,12 +82,33 @@ private static T visitRecord( private static T visitUnion(Type type, Schema union, AvroSchemaWithTypeVisitor visitor) { List types = union.getTypes(); List options = Lists.newArrayListWithExpectedSize(types.size()); - for (Schema branch : types) { + + // simple union case + if (AvroSchemaUtil.isOptionSchema(union)) { + for (Schema branch : types) { + if (branch.getType() == Schema.Type.NULL) { + options.add(visit((Type) null, branch, visitor)); + } else { + options.add(visit(type, branch, visitor)); + } + } + } else if (AvroSchemaUtil.isSingleTypeUnion(union)) { // single type union case + Schema branch = types.get(0); if (branch.getType() == Schema.Type.NULL) { options.add(visit((Type) null, branch, visitor)); } else { options.add(visit(type, branch, visitor)); } + } else { // complex union case + int index = 1; + for (Schema branch : types) { + if (branch.getType() == Schema.Type.NULL) { + options.add(visit((Type) null, branch, visitor)); + } else { + options.add(visit(type.asStructType().fields().get(index).type(), branch, visitor)); + index += 1; + } + } } return visitor.union(type, union, options); } diff --git a/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java index 3f1a71a9e..5dc270bc0 100644 --- a/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java +++ b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java @@ -89,7 +89,6 @@ public Schema record(Schema record, List names, Iterable s List expectedFields = struct.fields(); for (int i = 0; i < expectedFields.size(); i += 1) { Types.NestedField field = expectedFields.get(i); - // detect reordering if (i < fields.size() && !field.name().equals(fields.get(i).name())) { hasChange = true; @@ -98,21 +97,51 @@ public Schema record(Schema record, List names, Iterable s Schema.Field avroField = updateMap.get(AvroSchemaUtil.makeCompatibleName(field.name())); if (avroField != null) { - updatedFields.add(avroField); - + // if the field has a defaultValue, but the avroField does not, we need to + // create a newField to copy over the non-null default value. + if (field.hasDefaultValue() && !AvroSchemaUtil.hasNonNullDefaultValue(avroField)) { + Schema newFiledSchema = + (field.isOptional()) + ? AvroSchemaUtil.toOption(avroField.schema(), true) + : avroField.schema(); + Schema.Field newField = + new Schema.Field( + avroField.name(), newFiledSchema, avroField.doc(), field.getDefaultValue()); + newField.addProp(AvroSchemaUtil.FIELD_ID_PROP, field.fieldId()); + updatedFields.add(newField); + hasChange = true; + } else { + // otherwise (i.e., expectedFiled has no default value, or it is null) we can use + // avroField as is + updatedFields.add(avroField); + } } else { + // here the field is missing from the file schema, so we verify it is either + // an optional field, a metadata column or one that has default value Preconditions.checkArgument( - field.isOptional() || MetadataColumns.metadataFieldIds().contains(field.fieldId()), - "Missing required field: %s", - field.name()); - // Create a field that will be defaulted to null. We assign a unique suffix to the field - // to make sure that even if records in the file have the field it is not projected. - Schema.Field newField = - new Schema.Field( - field.name() + "_r" + field.fieldId(), - AvroSchemaUtil.toOption(AvroSchemaUtil.convert(field.type())), - null, - JsonProperties.NULL_VALUE); + field.isOptional() + || MetadataColumns.metadataFieldIds().contains(field.fieldId()) + || field.hasDefaultValue(), + "Missing required field that has no default value: field: %s, avroField: null, record: %s", + field, + record); + + // Create a field that will be defaulted to the field's default value. If no default value, + // then default to null and assign a unique suffix to the field to make sure that even if + // records in the + // file have the field it is not projected. + String newFieldName = field.name(); + Schema newFiledSchema; + Object defaultValue; + if (field.hasDefaultValue()) { + newFiledSchema = AvroSchemaUtil.convert(field.type()); + defaultValue = field.getDefaultValue(); + } else { + newFieldName = newFieldName + "_r" + field.fieldId(); + newFiledSchema = AvroSchemaUtil.toOption(AvroSchemaUtil.convert(field.type())); + defaultValue = JsonProperties.NULL_VALUE; + } + Schema.Field newField = new Schema.Field(newFieldName, newFiledSchema, null, defaultValue); newField.addProp(AvroSchemaUtil.FIELD_ID_PROP, field.fieldId()); updatedFields.add(newField); hasChange = true; @@ -159,17 +188,15 @@ public Schema.Field field(Schema.Field field, Supplier fieldResult) { @Override public Schema union(Schema union, Iterable options) { - Preconditions.checkState( - AvroSchemaUtil.isOptionSchema(union), - "Invalid schema: non-option unions are not supported: %s", - union); - Schema nonNullOriginal = AvroSchemaUtil.fromOption(union); - Schema nonNullResult = AvroSchemaUtil.fromOptions(Lists.newArrayList(options)); - - if (!Objects.equals(nonNullOriginal, nonNullResult)) { - return AvroSchemaUtil.toOption(nonNullResult); - } + if (AvroSchemaUtil.isOptionSchema(union)) { + Schema nonNullOriginal = AvroSchemaUtil.fromOption(union); + Schema nonNullResult = AvroSchemaUtil.fromOptions(Lists.newArrayList(options)); + if (!Objects.equals(nonNullOriginal, nonNullResult)) { + boolean nullIsSecondOption = union.getTypes().get(1).getType() == Schema.Type.NULL; + return AvroSchemaUtil.toOption(nonNullResult, nullIsSecondOption); + } + } return union; } diff --git a/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java index 2de2c0fe0..55bced5b1 100644 --- a/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java +++ b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java @@ -23,7 +23,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import org.apache.avro.SchemaNormalization; import org.apache.iceberg.mapping.NameMapping; @@ -118,27 +117,7 @@ public Schema record(Schema record, List names, List fields) { @Override public Schema union(Schema union, List options) { - Preconditions.checkState( - AvroSchemaUtil.isOptionSchema(union), - "Invalid schema: non-option unions are not supported: %s", - union); - - // only unions with null are allowed, and a null schema results in null - Schema pruned = null; - if (options.get(0) != null) { - pruned = options.get(0); - } else if (options.get(1) != null) { - pruned = options.get(1); - } - - if (pruned != null) { - if (!Objects.equals(pruned, AvroSchemaUtil.fromOption(union))) { - return AvroSchemaUtil.toOption(pruned); - } - return union; - } - - return null; + return copyUnion(union, options); } @Override @@ -307,22 +286,9 @@ private static Schema makeEmptyCopy(Schema field) { } private static Schema.Field copyField(Schema.Field field, Schema newSchema, Integer fieldId) { - Schema newSchemaReordered; - // if the newSchema is an optional schema, make sure the NULL option is always the first - if (isOptionSchemaWithNonNullFirstOption(newSchema)) { - newSchemaReordered = AvroSchemaUtil.toOption(AvroSchemaUtil.fromOption(newSchema)); - } else { - newSchemaReordered = newSchema; - } - // do not copy over default values as the file is expected to have values for fields already in - // the file schema + // always copy over default values Schema.Field copy = - new Schema.Field( - field.name(), - newSchemaReordered, - field.doc(), - AvroSchemaUtil.isOptionSchema(newSchemaReordered) ? JsonProperties.NULL_VALUE : null, - field.order()); + new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal(), field.order()); for (Map.Entry prop : field.getObjectProps().entrySet()) { copy.addProp(prop.getKey(), prop.getValue()); @@ -345,4 +311,21 @@ private static boolean isOptionSchemaWithNonNullFirstOption(Schema schema) { return AvroSchemaUtil.isOptionSchema(schema) && schema.getTypes().get(0).getType() != Schema.Type.NULL; } + + // for primitive types, the visitResult will be null, we want to reuse the primitive types from + // the original + // schema, while for nested types, we want to use the visitResult because they have content from + // the previous + // recursive calls. + private static Schema copyUnion(Schema record, List visitResults) { + List alts = Lists.newArrayListWithExpectedSize(visitResults.size()); + for (int i = 0; i < visitResults.size(); i++) { + if (visitResults.get(i) == null) { + alts.add(record.getTypes().get(i)); + } else { + alts.add(visitResults.get(i)); + } + } + return Schema.createUnion(alts); + } } diff --git a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java index 174d63975..d0e2585ed 100644 --- a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java +++ b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java @@ -92,10 +92,20 @@ public Type record(Schema record, List names, List fieldTypes) { Type fieldType = fieldTypes.get(i); int fieldId = getId(field); - if (AvroSchemaUtil.isOptionSchema(field.schema())) { - newFields.add(Types.NestedField.optional(fieldId, field.name(), fieldType, field.doc())); + Object defaultValue = + AvroSchemaUtil.hasNonNullDefaultValue(field) + ? AvroSchemaUtil.convertToJavaDefaultValue(field.defaultVal()) + : null; + + if (AvroSchemaUtil.isOptionSchema(field.schema()) + || AvroSchemaUtil.isOptionalComplexUnion(field.schema())) { + newFields.add( + Types.NestedField.optional( + fieldId, field.name(), fieldType, defaultValue, field.doc())); } else { - newFields.add(Types.NestedField.required(fieldId, field.name(), fieldType, field.doc())); + newFields.add( + Types.NestedField.required( + fieldId, field.name(), fieldType, defaultValue, field.doc())); } } @@ -104,13 +114,30 @@ public Type record(Schema record, List names, List fieldTypes) { @Override public Type union(Schema union, List options) { - Preconditions.checkArgument( - AvroSchemaUtil.isOptionSchema(union), "Unsupported type: non-option union: %s", union); - // records, arrays, and maps will check nullability later - if (options.get(0) == null) { - return options.get(1); - } else { + if (AvroSchemaUtil.isOptionSchema(union)) { + // Optional simple union + // records, arrays, and maps will check nullability later + if (options.get(0) == null) { + return options.get(1); + } else { + return options.get(0); + } + } else if (AvroSchemaUtil.isSingleTypeUnion(union)) { + // Single type union return options.get(0); + } else { + // Complex union + List newFields = Lists.newArrayList(); + newFields.add(Types.NestedField.required(allocateId(), "tag", Types.IntegerType.get())); + + int tagIndex = 0; + for (Type type : options) { + if (type != null) { + newFields.add(Types.NestedField.optional(allocateId(), "field" + tagIndex++, type)); + } + } + + return Types.StructType.of(newFields); } } diff --git a/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java b/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java index bc2847e1b..76df444d1 100644 --- a/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java +++ b/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java @@ -100,12 +100,12 @@ public Schema struct(Types.StructType struct, List fieldSchemas) { String origFieldName = structField.name(); boolean isValidFieldName = AvroSchemaUtil.validAvroName(origFieldName); String fieldName = isValidFieldName ? origFieldName : AvroSchemaUtil.sanitize(origFieldName); + Object defaultValue = + structField.hasDefaultValue() + ? structField.getDefaultValue() + : (structField.isOptional() ? JsonProperties.NULL_VALUE : null); Schema.Field field = - new Schema.Field( - fieldName, - fieldSchemas.get(i), - structField.doc(), - structField.isOptional() ? JsonProperties.NULL_VALUE : null); + new Schema.Field(fieldName, fieldSchemas.get(i), structField.doc(), defaultValue); if (!isValidFieldName) { field.addProp(AvroSchemaUtil.ICEBERG_FIELD_NAME_PROP, origFieldName); } @@ -123,7 +123,7 @@ public Schema struct(Types.StructType struct, List fieldSchemas) { @Override public Schema field(Types.NestedField field, Schema fieldSchema) { if (field.isOptional()) { - return AvroSchemaUtil.toOption(fieldSchema); + return AvroSchemaUtil.toOption(fieldSchema, field.hasDefaultValue()); } else { return fieldSchema; } diff --git a/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java b/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java index de6ce2ad0..f86f5240a 100644 --- a/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java +++ b/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java @@ -262,8 +262,8 @@ private static S visit(MappedFields mapping, Visitor visitor) { return visitor.fields(mapping, fieldResults); } - private static class CreateMapping extends TypeUtil.SchemaVisitor { - private static final CreateMapping INSTANCE = new CreateMapping(); + public static class CreateMapping extends TypeUtil.SchemaVisitor { + public static final CreateMapping INSTANCE = new CreateMapping(); private CreateMapping() {} diff --git a/core/src/main/java/org/apache/iceberg/mapping/NameMapping.java b/core/src/main/java/org/apache/iceberg/mapping/NameMapping.java index aa5f5f3cd..07c293739 100644 --- a/core/src/main/java/org/apache/iceberg/mapping/NameMapping.java +++ b/core/src/main/java/org/apache/iceberg/mapping/NameMapping.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -41,13 +42,23 @@ public static NameMapping of(MappedFields fields) { } private final MappedFields mapping; + private final boolean caseSensitive; private transient Map fieldsById; private transient Map fieldsByName; + private transient Map fieldsByNameLowercase; NameMapping(MappedFields mapping) { + this(mapping, true); + } + + public NameMapping(MappedFields mapping, boolean caseSensitive) { this.mapping = mapping; + this.caseSensitive = caseSensitive; lazyFieldsById(); lazyFieldsByName(); + if (!caseSensitive) { + lazyFieldsByNameLowercase(); + } } public MappedField find(int id) { @@ -55,11 +66,23 @@ public MappedField find(int id) { } public MappedField find(String... names) { - return lazyFieldsByName().get(DOT.join(names)); + return find(DOT.join(names)); } public MappedField find(List names) { - return lazyFieldsByName().get(DOT.join(names)); + return find(DOT.join(names)); + } + + private MappedField find(String qualifiedName) { + MappedField field = lazyFieldsByName().get(qualifiedName); + if (field == null && !caseSensitive) { + field = lazyFieldsByNameLowercase().get(qualifiedName.toLowerCase()); + } + return field; + } + + public boolean isCaseSensitive() { + return caseSensitive; } public MappedFields asMappedFields() { @@ -80,6 +103,17 @@ private Map lazyFieldsByName() { return fieldsByName; } + private Map lazyFieldsByNameLowercase() { + if (fieldsByNameLowercase == null) { + this.fieldsByNameLowercase = + lazyFieldsByName().entrySet().stream() + .collect( + Collectors.toMap( + x -> x.getKey().toLowerCase(), Map.Entry::getValue, (u, v) -> u)); + } + return fieldsByNameLowercase; + } + @Override public String toString() { if (mapping.fields().isEmpty()) { diff --git a/core/src/main/java/org/apache/iceberg/mapping/NameMappingParser.java b/core/src/main/java/org/apache/iceberg/mapping/NameMappingParser.java index f5bce333f..54228adb5 100644 --- a/core/src/main/java/org/apache/iceberg/mapping/NameMappingParser.java +++ b/core/src/main/java/org/apache/iceberg/mapping/NameMappingParser.java @@ -41,6 +41,19 @@ * { "field-id": 5, "names": ["longitude", "long"] } * ] } ] * + * + * or + * + *

+ * { "case-sensitive": false,
+ *   "mapping": [ { "field-id": 1, "names": ["id", "record_id"] },
+ *       { "field-id": 2, "names": ["data"] },
+ *       { "field-id": 3, "names": ["location"], "fields": [
+ *           { "field-id": 4, "names": ["latitude", "lat"] },
+ *           { "field-id": 5, "names": ["longitude", "long"] }
+ *       ] } ]
+ * }
+ * 
*/ public class NameMappingParser { @@ -49,6 +62,8 @@ private NameMappingParser() {} private static final String FIELD_ID = "field-id"; private static final String NAMES = "names"; private static final String FIELDS = "fields"; + private static final String CASE_SENSITIVE = "case-sensitive"; + private static final String MAPPING = "mapping"; public static String toJson(NameMapping mapping) { try { @@ -64,7 +79,11 @@ public static String toJson(NameMapping mapping) { } static void toJson(NameMapping nameMapping, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeBooleanField(CASE_SENSITIVE, nameMapping.isCaseSensitive()); + generator.writeFieldName(MAPPING); toJson(nameMapping.asMappedFields(), generator); + generator.writeEndObject(); } private static void toJson(MappedFields mapping, JsonGenerator generator) throws IOException { @@ -106,7 +125,16 @@ public static NameMapping fromJson(String json) { } static NameMapping fromJson(JsonNode node) { - return new NameMapping(fieldsFromJson(node)); + Preconditions.checkArgument( + node.isObject() || node.isArray(), + "Cannot parse non-object or non-array name mapping: %s", + node); + if (node.isObject()) { + boolean caseSensitive = JsonUtil.getBool(CASE_SENSITIVE, node); + return new NameMapping(fieldsFromJson(node.get(MAPPING)), caseSensitive); + } else { + return new NameMapping(fieldsFromJson(node)); + } } private static MappedFields fieldsFromJson(JsonNode node) { diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaParserForDefaultValues.java b/core/src/test/java/org/apache/iceberg/TestSchemaParserForDefaultValues.java new file mode 100644 index 000000000..d25480a71 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSchemaParserForDefaultValues.java @@ -0,0 +1,284 @@ +/* + * 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.iceberg; + +import static org.apache.avro.Schema.Type.BOOLEAN; +import static org.apache.avro.Schema.Type.BYTES; +import static org.apache.avro.Schema.Type.DOUBLE; +import static org.apache.avro.Schema.Type.FLOAT; +import static org.apache.avro.Schema.Type.INT; +import static org.apache.avro.Schema.Type.LONG; +import static org.apache.avro.Schema.Type.NULL; +import static org.apache.avro.Schema.Type.STRING; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types.NestedField; +import org.junit.Assert; +import org.junit.Test; + +public class TestSchemaParserForDefaultValues { + + private void assertEqualStructs( + org.apache.iceberg.Schema expected, org.apache.iceberg.Schema actual) { + if (expected == null) { + Assert.assertNull(actual); + return; + } + Assert.assertNotNull(actual); + List expectedFields = expected.asStruct().fields(); + List actualFields = actual.asStruct().fields(); + + Assert.assertEquals(expectedFields.size(), actualFields.size()); + + for (int i = 0; i < expectedFields.size(); i++) { + NestedField expectedField = expectedFields.get(i); + NestedField actualField = actualFields.get(i); + Assert.assertEquals(expectedField.fieldId(), actualField.fieldId()); + Assert.assertEquals(expectedField.name(), actualField.name()); + Assert.assertEquals(expectedField.type(), actualField.type()); + Assert.assertEquals(expectedField.doc(), actualField.doc()); + if (expectedField.hasDefaultValue()) { + Assert.assertTrue(actualField.hasDefaultValue()); + switch (expectedField.type().typeId()) { + case BINARY: + case FIXED: + Assert.assertTrue( + Arrays.equals( + (byte[]) expectedField.getDefaultValue(), + (byte[]) actualField.getDefaultValue())); + break; + default: + Assert.assertEquals(expectedField.getDefaultValue(), actualField.getDefaultValue()); + } + } else { + Assert.assertFalse(actualField.hasDefaultValue()); + } + } + } + + private void testToFromJsonPreservingDefaultValues( + String[] fieldNames, Schema[] fieldsSchemas, Object[] defaults) { + List fields = Lists.newArrayList(); + IntStream.range(0, defaults.length) + .forEach( + i -> fields.add(new Schema.Field(fieldNames[i], fieldsSchemas[i], null, defaults[i]))); + + Schema schema = Schema.createRecord("root", null, null, false, fields); + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(schema); + String jsonString = SchemaParser.toJson(icebergSchema); + + Assert.assertTrue(jsonString.contains("default")); + + org.apache.iceberg.Schema icebergSchemaFromJson = SchemaParser.fromJson(jsonString); + + assertEqualStructs(icebergSchema, icebergSchemaFromJson); + } + + @Test + public void testPrimitiveTypes() { + Boolean defaultBoolean = true; + Integer defaultInt = 1; + Long defaultLong = -1L; + Double defaultDouble = 0.1; + Float defaultFloat = 0.1f; + String defaultString = "default string"; + String defaultBytes = "1111"; + int fixedSize = defaultBytes.getBytes().length; + + String[] fieldNames = { + "booleanField", + "intField", + "longField", + "doubleField", + "floatField", + "stringField", + "binaryField", + "fixedField" + }; + + Object[] defaults = { + defaultBoolean, + defaultInt, + defaultLong, + defaultDouble, + defaultFloat, + defaultString, + defaultBytes, + defaultBytes + }; + + Schema[] primitives = { + Schema.create(BOOLEAN), + Schema.create(INT), + Schema.create(LONG), + Schema.create(DOUBLE), + Schema.create(FLOAT), + Schema.create(STRING), + Schema.create(BYTES), + Schema.createFixed("md5", null, "namespace", fixedSize) + }; + + testToFromJsonPreservingDefaultValues(fieldNames, primitives, defaults); + } + + @Test + public void testLogicalTypes() { + Long longDefault = Long.valueOf(1234556789); + String[] fieldNames = {"dateField", "timeField", "timestampField", "uuidField", "decimalField"}; + + Object[] defaults = {Integer.valueOf(123446), longDefault, "randomUUID", longDefault}; + + Schema dateSchema = Schema.create(INT); + dateSchema.addProp("logicaltype", "date"); + Schema timestampSchema = Schema.create(LONG); + timestampSchema.addProp("logicaltype", "timestamp"); + Schema uuidSchema = Schema.create(STRING); + uuidSchema.addProp("logicaltype", "UUID"); + Schema bigDecimalSchema = Schema.create(LONG); + bigDecimalSchema.addProp("logicaltype", "decimal"); + + Schema[] logicals = {dateSchema, timestampSchema, uuidSchema, bigDecimalSchema}; + + testToFromJsonPreservingDefaultValues(fieldNames, logicals, defaults); + } + + @Test + public void testNestedTypes() { + String structStringFieldName = "stringFieldOfStruct"; + String structBooleanFieldName = "booleanFieldOfStruct"; + Map defaultStruct = + ImmutableMap.of( + structStringFieldName, "default string", structBooleanFieldName, Boolean.TRUE); + List defaultList = Arrays.asList(1, 2); + Map defaultMap = + ImmutableMap.of("key1", Long.valueOf(1L), "key2", Long.valueOf(2L)); + List structFields = + ImmutableList.of( + new Schema.Field(structStringFieldName, Schema.create(STRING), null), + new Schema.Field(structBooleanFieldName, Schema.create(BOOLEAN), null)); + + String[] fieldNames = {"structField", "listField", "mapField"}; + Object[] defaults = {defaultStruct, defaultList, defaultMap}; + Schema[] nested = { + Schema.createRecord("name", null, "namespace", false, structFields), + Schema.createArray(Schema.create(INT)), + Schema.createMap(Schema.create(LONG)) + }; + + testToFromJsonPreservingDefaultValues(fieldNames, nested, defaults); + } + + @Test + public void testOptionalWithDefault() { + Integer defaultInt = 1; + Map defaultMap = + ImmutableMap.of("key1", Long.valueOf(1L), "key2", Long.valueOf(2L)); + + String[] fieldNames = {"optionalPrimitive", "optionalNested"}; + Schema[] optionals = { + Schema.createUnion(Schema.create(INT), Schema.create(NULL)), + Schema.createUnion(Schema.createMap(Schema.create(LONG)), Schema.create(NULL)) + }; + Object[] defaults = {defaultInt, defaultMap}; + + testToFromJsonPreservingDefaultValues(fieldNames, optionals, defaults); + } + + @Test + public void testNestedOfNestedWithDefault() { + Integer defaultInt = 1; + Map defaultMap = + ImmutableMap.of("key1", Long.valueOf(1L), "key2", Long.valueOf(2L)); + + String structIntField = "intFieldOfStruct"; + String structMapFieldName = "mapFieldOfStruct"; + List structFields = + ImmutableList.of( + new Schema.Field(structIntField, Schema.create(INT), null, defaultInt), + new Schema.Field( + structMapFieldName, Schema.createMap(Schema.create(LONG)), null, defaultMap)); + + String[] fieldNames = {"intFieldNoDefault", "structFieldNoDefault"}; + Schema[] topLevelFields = { + Schema.create(INT), Schema.createRecord("name", null, "namespace", false, structFields) + }; + + List fields = Lists.newArrayList(); + IntStream.range(0, fieldNames.length) + .forEach(i -> fields.add(new Schema.Field(fieldNames[i], topLevelFields[i], null))); + + Schema schema = org.apache.avro.Schema.createRecord("root", null, null, false, fields); + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(schema); + String jsonString = SchemaParser.toJson(icebergSchema); + + Assert.assertTrue(jsonString.contains("default")); + + org.apache.iceberg.Schema fromJsonIcebergSchema = SchemaParser.fromJson(jsonString); + Assert.assertEquals(icebergSchema.toString(), fromJsonIcebergSchema.toString()); + } + + @Test + public void testDeepNestedWithDefault() { + Integer defaultInt = 1; + Map defaultMap = + ImmutableMap.of("key1", Long.valueOf(1L), "key2", Long.valueOf(2L)); + + String structIntField = "intFieldOfStruct"; + String structMapFieldName = "mapFieldOfStruct"; + List structFields = + ImmutableList.of( + new Schema.Field(structIntField, Schema.create(INT), null, defaultInt), + new Schema.Field( + structMapFieldName, Schema.createMap(Schema.create(LONG)), null, defaultMap)); + + Schema downLevelStruct = Schema.createRecord("name", null, "namespace0", false, structFields); + + List intermediateStructFields = + ImmutableList.of( + new Schema.Field("intermediateIntField", Schema.create(INT), null), + new Schema.Field("intermediateStructField", downLevelStruct, null)); + + Schema intermediateStruct = + Schema.createRecord("name", null, "namespace1", false, intermediateStructFields); + String[] fieldNames = {"topLevelLong", "topLevelString", "topLevelStruct"}; + Schema[] topLevelFields = {Schema.create(LONG), Schema.create(STRING), intermediateStruct}; + + List fields = Lists.newArrayList(); + IntStream.range(0, fieldNames.length) + .forEach(i -> fields.add(new Schema.Field(fieldNames[i], topLevelFields[i], null))); + + Schema schema = org.apache.avro.Schema.createRecord("root", null, null, false, fields); + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(schema); + String jsonString = SchemaParser.toJson(icebergSchema); + + Assert.assertTrue(jsonString.contains("default")); + + org.apache.iceberg.Schema fromJsonIcebergSchema = SchemaParser.fromJson(jsonString); + Assert.assertEquals(icebergSchema.toString(), fromJsonIcebergSchema.toString()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java index 26f8625ea..a351bf3f4 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java @@ -204,7 +204,6 @@ public void testMissingRequiredFields() { AssertHelpers.assertThrows( "Missing required field in nameMapping", IllegalArgumentException.class, - "Missing required field: x", // In this case, pruneColumns result is an empty record () -> writeAndRead(writeSchema, readSchema, record, nameMapping)); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroOptionsWithNonNullDefaults.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroOptionsWithNonNullDefaults.java index a6f0c393a..42e8de68f 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroOptionsWithNonNullDefaults.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroOptionsWithNonNullDefaults.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; @@ -39,6 +40,9 @@ public class TestAvroOptionsWithNonNullDefaults { + private static final String fieldWithDefaultName = "fieldWithDefault"; + private static final String noDefaultFiledName = "noDefaultField"; + @Rule public TemporaryFolder temp = new TemporaryFolder(); @Test @@ -145,4 +149,153 @@ public void writeAndValidateOptionWithNonNullDefaultsEvolution() throws IOExcept AvroTestHelpers.assertEquals(readIcebergSchema.asStruct(), expected.get(i), rows.get(i)); } } + + @Test + public void testDefaultValueUsedPrimitiveType() throws IOException { + Schema writeSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of(new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null))); + // evolved schema + Schema readSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of( + new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null), + new Schema.Field(fieldWithDefaultName, Schema.create(INT), null, -1))); + + GenericData.Record record1 = new GenericData.Record(writeSchema); + record1.put(noDefaultFiledName, 1); + GenericData.Record record2 = new GenericData.Record(writeSchema); + record2.put(noDefaultFiledName, 2); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(writeSchema, testFile); + writer.append(record1); + writer.append(record2); + } + + List expected = ImmutableList.of(record1, record2); + org.apache.iceberg.Schema readIcebergSchema = AvroSchemaUtil.toIceberg(readSchema); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)).project(readIcebergSchema).build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < expected.size(); i += 1) { + Assert.assertEquals( + expected.get(i).get(noDefaultFiledName), rows.get(i).get(noDefaultFiledName)); + // default should be used for records missing the field + Assert.assertEquals(-1, rows.get(i).get(fieldWithDefaultName)); + } + } + + @Test + public void testDefaultValueNotUsedWhenFiledHasValue() throws IOException { + Schema readSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of( + new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null), + new Schema.Field(fieldWithDefaultName, Schema.create(INT), null, -1))); + + GenericData.Record record1 = new GenericData.Record(readSchema); + record1.put(noDefaultFiledName, 3); + record1.put(fieldWithDefaultName, 3); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(readSchema, testFile); + writer.append(record1); + } + + List expected = ImmutableList.of(record1); + org.apache.iceberg.Schema readIcebergSchema = AvroSchemaUtil.toIceberg(readSchema); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)).project(readIcebergSchema).build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < expected.size(); i += 1) { + Assert.assertEquals( + expected.get(i).get(noDefaultFiledName), rows.get(i).get(noDefaultFiledName)); + // default value should NOT be used if field is populated + Assert.assertEquals( + expected.get(i).get(fieldWithDefaultName), rows.get(i).get(fieldWithDefaultName)); + } + } + + @Test + public void testDefaultValueUsedComplexType() throws IOException { + Schema writeSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of(new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null))); + // evolved schema + List defaultArray = Arrays.asList(-1, -2); + Schema readSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of( + new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null), + new Schema.Field( + fieldWithDefaultName, + Schema.createArray(Schema.create(INT)), + null, + defaultArray))); + + GenericData.Record record1 = new GenericData.Record(writeSchema); + record1.put(noDefaultFiledName, 1); + GenericData.Record record2 = new GenericData.Record(writeSchema); + record2.put(noDefaultFiledName, 2); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(writeSchema, testFile); + writer.append(record1); + writer.append(record2); + } + + List expected = ImmutableList.of(record1, record2); + org.apache.iceberg.Schema readIcebergSchema = AvroSchemaUtil.toIceberg(readSchema); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)).project(readIcebergSchema).build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < expected.size(); i += 1) { + Assert.assertEquals( + expected.get(i).get(noDefaultFiledName), rows.get(i).get(noDefaultFiledName)); + // default should be used for records missing the field + Assert.assertEquals(defaultArray, rows.get(i).get(fieldWithDefaultName)); + } + } } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestDefaultValuePreserving.java b/core/src/test/java/org/apache/iceberg/avro/TestDefaultValuePreserving.java new file mode 100644 index 000000000..b9e56496a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/avro/TestDefaultValuePreserving.java @@ -0,0 +1,96 @@ +/* + * 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.iceberg.avro; + +import static org.apache.avro.Schema.Type.INT; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +/** + * Testing the preserving of fields; default values in {@link SchemaToType} and {@link TypeToSchema} + */ +public class TestDefaultValuePreserving { + + String noDefaultFiledName = "fieldWithNoDefaultValue"; + String fieldWithDefaultName = "fieldWithDefaultValue"; + Integer defaultValue = -1; + + @Test + public void testSchemaToTypeRecord() { + Schema recordSchema = + Schema.createRecord( + "root", + null, + null, + false, + ImmutableList.of( + new Schema.Field(noDefaultFiledName, Schema.create(INT), null, null), + new Schema.Field(fieldWithDefaultName, Schema.create(INT), null, defaultValue))); + SchemaToType schemaToType = new SchemaToType(recordSchema); + List names = + recordSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + List types = ImmutableList.of(Types.IntegerType.get(), Types.IntegerType.get()); + + Type record = schemaToType.record(recordSchema, names, types); + + Assert.assertNotNull(record); + Assert.assertTrue(record.isStructType()); + Assert.assertEquals(names.size(), record.asStructType().fields().size()); + Assert.assertFalse(record.asStructType().field(noDefaultFiledName).hasDefaultValue()); + Assert.assertTrue(record.asStructType().field(fieldWithDefaultName).hasDefaultValue()); + Assert.assertEquals( + defaultValue, record.asStructType().field(fieldWithDefaultName).getDefaultValue()); + } + + @Test + public void testTypeToSchemaStruct() { + List nestedFields = + ImmutableList.of( + Types.NestedField.required(0, noDefaultFiledName, Types.IntegerType.get()), + Types.NestedField.required( + 1, fieldWithDefaultName, Types.IntegerType.get(), defaultValue, null)); + Types.StructType structType = Types.StructType.of(nestedFields); + Map names = ImmutableMap.of(structType, "tableName"); + TypeToSchema typeToSchema = new TypeToSchema(names); + List fieldSchemas = ImmutableList.of(Schema.create(INT), Schema.create(INT)); + + Schema structSchema = typeToSchema.struct(structType, fieldSchemas); + + Assert.assertNotNull(structSchema); + Assert.assertEquals(nestedFields.size(), structSchema.getFields().size()); + for (int i = 0; i < nestedFields.size(); i++) { + if (nestedFields.get(i).hasDefaultValue()) { + Assert.assertTrue(structSchema.getFields().get(i).hasDefaultValue()); + Assert.assertEquals( + nestedFields.get(i).getDefaultValue(), structSchema.getFields().get(i).defaultVal()); + } else { + Assert.assertFalse(structSchema.getFields().get(i).hasDefaultValue()); + } + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java index 8d57aac29..1fbf9fca4 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java @@ -28,9 +28,11 @@ import static org.apache.iceberg.types.Types.NestedField.required; import java.util.List; +import java.util.stream.IntStream; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.iceberg.SchemaParser; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; @@ -361,4 +363,201 @@ public void testFieldDocsArePreserved() { Lists.newArrayList(Iterables.transform(origSchema.columns(), Types.NestedField::doc)); Assert.assertEquals(origFieldDocs, fieldDocs); } + + @Test + public void testConversionOfRecordDefaultWithOptionalNestedField() { + String schemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"root\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"outer\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"outerRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"mapField\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": \"string\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"recordField\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"inner\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"innerString\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"mapField\": {}\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema schema = new Schema.Parser().parse(schemaString); + org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema); + Assert.assertEquals( + "table {\n" + + " 0: outer: required struct<4: mapField: required map, " + + "5: recordField: optional struct<3: innerString: required string>>, default value: {mapField={}}, \n" + + "}", + iSchema.toString()); + } + + @Test + public void testConversionOfRecordDefaultWithOptionalNestedField2() { + String schemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"root\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"outer\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"outerRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"mapField\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": \"string\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"recordField\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"inner\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"innerString\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"mapField\": {\n" + + " \"foo\": \"bar\",\n" + + " \"x\": \"y\"\n" + + " },\n" + + " \"recordField\": null\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema schema = new Schema.Parser().parse(schemaString); + org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema); + Assert.assertEquals( + "table {\n" + + " 0: outer: required struct<4: mapField: required map, " + + "5: recordField: optional struct<3: innerString: required string>>, " + + "default value: {mapField={foo=bar, x=y}, recordField=null}, \n" + + "}", + iSchema.toString()); + } + + @Test + public void testVariousTypesDefaultValues() { + String schemaString = + "{\n" + + " \"namespace\": \"com.razhang\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RAZHANG\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"f1\",\n" + + " \"type\": \"string\",\n" + + " \"default\": \"foo\"\n" + + " },\n" + + " {\n" + + " \"name\": \"f2\",\n" + + " \"type\": \"int\",\n" + + " \"default\": 1\n" + + " },\n" + + " {\n" + + " \"name\": \"f3\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\" : \"int\"\n" + + " },\n" + + " \"default\": {\"a\": 1}\n" + + " },\n" + + " {\n" + + " \"name\": \"f4\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " },\n" + + " \"default\": [1, 2, 3]\n" + + " },\n" + + " {\n" + + " \"name\": \"f5\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"F5\",\n" + + " \"fields\" : [\n" + + " {\"name\": \"ff1\", \"type\": \"long\"},\n" + + " {\"name\": \"ff2\", \"type\": [\"null\", \"string\"]}\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"ff1\": 999,\n" + + " \"ff2\": null\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"f6\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " }\n" + + " },\n" + + " \"default\": {\"key\": [1, 2, 3]}\n" + + " },\n" + + " {\n" + + " \"name\": \"f7\",\n" + + " \"type\": {\n" + + " \"type\": \"fixed\",\n" + + " \"name\": \"md5\",\n" + + " \"size\": 2\n" + + " },\n" + + " \"default\": \"fF\"\n" + + " }\n" + + " ]\n" + + "}"; + Schema schema = new Schema.Parser().parse(schemaString); + org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema); + + String schemaJson = SchemaParser.toJson(iSchema); + org.apache.iceberg.Schema roundTripiSchema = SchemaParser.fromJson(schemaJson); + + Assert.assertTrue( + IntStream.range(0, roundTripiSchema.columns().size()) + .allMatch(i -> roundTripiSchema.columns().get(i).equals(iSchema.columns().get(i)))); + } } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestUnionSchemaConversions.java b/core/src/test/java/org/apache/iceberg/avro/TestUnionSchemaConversions.java new file mode 100644 index 000000000..ff53fcc9b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/avro/TestUnionSchemaConversions.java @@ -0,0 +1,159 @@ +/* + * 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.iceberg.avro; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.Assert; +import org.junit.Test; + +public class TestUnionSchemaConversions { + + @Test + public void testRequiredComplexUnion() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = + "table {\n" + + " 0: unionCol: required struct<1: tag: required int, 2: field0: optional int, 3: field1: optional string>\n" + + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } + + @Test + public void testOptionalComplexUnion() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = + "table {\n" + + " 0: unionCol: optional struct<1: tag: required int, 2: field0: optional int, 3: field1: optional string>\n" + + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } + + @Test + public void testOptionalSingleUnionSchema() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("optionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .endUnion() + .nullDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = "table {\n" + " 0: optionCol: optional int\n" + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } + + @Test + public void testSingleTypeUnionSchema() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = "table {\n" + " 0: unionCol: required int\n" + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } + + @Test + public void testNestedSingleTypeUnionSchema() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = "table {\n" + " 0: col1: required list\n" + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } + + @Test + public void testSingleTypeUnionOfComplexTypeSchema() { + Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .array() + .items() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + org.apache.iceberg.Schema icebergSchema = AvroSchemaUtil.toIceberg(avroSchema); + String expectedIcebergSchema = "table {\n" + " 0: unionCol: required list\n" + "}"; + + Assert.assertEquals(expectedIcebergSchema, icebergSchema.toString()); + } +} diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowLevelFiltering.java b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowLevelFiltering.java new file mode 100644 index 000000000..46bdf5cea --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowLevelFiltering.java @@ -0,0 +1,186 @@ +/* + * 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.iceberg.data.orc; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTestHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcRowFilter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestOrcRowLevelFiltering { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private static final Schema SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), + required(101, "data1", Types.StringType.get()), + required(102, "data2", Types.StringType.get())); + + private static final List RECORDS = + LongStream.range(0, 100) + .mapToObj( + i -> { + Record record = GenericRecord.create(SCHEMA); + record.set(0, i); + record.set(1, "data1:" + i); + record.set(2, "data2:" + i); + return record; + }) + .collect(Collectors.toList()); + + @Test + public void testReadOrcWithRowFilterNoProjection() throws IOException { + testReadOrcWithRowFilter(SCHEMA, rowFilterId(), RECORDS.subList(75, 100)); + } + + @Test + public void testReadOrcWithRowFilterProjection() throws IOException { + Schema projectedSchema = new Schema(required(101, "data1", Types.StringType.get())); + + List expected = + RECORDS.subList(75, 100).stream() + .map( + r -> { + Record record = GenericRecord.create(projectedSchema); + record.set(0, r.get(1)); + return record; + }) + .collect(Collectors.toList()); + + testReadOrcWithRowFilter(projectedSchema, rowFilterId(), expected); + } + + @Test + public void testReadOrcWithRowFilterPartialFilterColumns() throws IOException { + Schema projectedSchema = + new Schema( + required(101, "data1", Types.StringType.get()), + required(102, "data2", Types.StringType.get())); + + List expected = + RECORDS.subList(25, 75).stream() + .map( + r -> { + Record record = GenericRecord.create(projectedSchema); + record.set(0, r.get(1)); + record.set(1, r.get(2)); + return record; + }) + .collect(Collectors.toList()); + + testReadOrcWithRowFilter(projectedSchema, rowFilterIdAndData1(), expected); + } + + @Test + public void testReadOrcWithRowFilterNonExistentColumn() throws IOException { + testReadOrcWithRowFilter(SCHEMA, rowFilterData3(), ImmutableList.of()); + } + + private void testReadOrcWithRowFilter( + Schema schema, OrcRowFilter rowFilter, List expected) throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + try (FileAppender writer = + ORC.write(Files.localOutput(testFile)) + .schema(SCHEMA) + .createWriterFunc(GenericOrcWriter::buildWriter) + .build()) { + for (Record rec : RECORDS) { + writer.add(rec); + } + } + + List rows; + try (CloseableIterable reader = + ORC.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(schema, fileSchema)) + .rowFilter(rowFilter) + .build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < expected.size(); i += 1) { + DataTestHelpers.assertEquals(schema.asStruct(), expected.get(i), rows.get(i)); + } + } + + private OrcRowFilter rowFilterId() { + return new OrcRowFilter() { + @Override + public Schema requiredSchema() { + return new Schema(required(100, "id", Types.LongType.get())); + } + + @Override + public boolean shouldKeep(Object[] values) { + return (Long) values[0] >= 75; + } + }; + } + + private OrcRowFilter rowFilterIdAndData1() { + return new OrcRowFilter() { + @Override + public Schema requiredSchema() { + return new Schema(SCHEMA.findField("id"), SCHEMA.findField("data1")); + } + + @Override + public boolean shouldKeep(Object[] values) { + return (Long) values[0] >= 25 && ((String) values[1]).compareTo("data1:75") < 0; + } + }; + } + + private OrcRowFilter rowFilterData3() { + return new OrcRowFilter() { + @Override + public Schema requiredSchema() { + return new Schema(optional(104, "data3", Types.LongType.get())); + } + + @Override + public boolean shouldKeep(Object[] values) { + return values[0] != null && (Long) values[0] >= 25; + } + }; + } +} diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestReadOrcFileWithoutIDs.java b/data/src/test/java/org/apache/iceberg/data/orc/TestReadOrcFileWithoutIDs.java new file mode 100644 index 000000000..9cca6b5b9 --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestReadOrcFileWithoutIDs.java @@ -0,0 +1,252 @@ +/* + * 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.iceberg.data.orc; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTestHelpers; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.ORCSchemaUtil; +import org.apache.iceberg.orc.OrcRowWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestReadOrcFileWithoutIDs { + + private static final Types.StructType SUPPORTED_PRIMITIVES = + Types.StructType.of( + required(100, "id", Types.LongType.get()), + optional(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + optional(103, "i", Types.IntegerType.get()), + required(104, "l", Types.LongType.get()), + optional(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + optional(107, "date", Types.DateType.get()), + required(108, "tsTz", Types.TimestampType.withZone()), + required(109, "ts", Types.TimestampType.withoutZone()), + required(110, "s", Types.StringType.get()), + optional(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), + required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + // Disabled some primitives because they cannot work without Iceberg's type + // attributes and hence won't be present + // in old data anyway + // required(112, "fixed", Types.FixedType.ofLength(7)) + // required(117, "time", Types.TimeType.get()) + ); + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void writeAndValidateORCFileWithoutIds() throws IOException { + Types.StructType structType = + Types.StructType.of( + required(0, "id", Types.LongType.get()), + optional( + 1, + "list_of_maps", + Types.ListType.ofOptional( + 2, + Types.MapType.ofOptional(3, 4, Types.StringType.get(), SUPPORTED_PRIMITIVES))), + optional( + 5, + "map_of_lists", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.ListType.ofOptional(8, SUPPORTED_PRIMITIVES))), + required( + 9, + "list_of_lists", + Types.ListType.ofOptional(10, Types.ListType.ofOptional(11, SUPPORTED_PRIMITIVES))), + required( + 12, + "map_of_maps", + Types.MapType.ofOptional( + 13, + 14, + Types.StringType.get(), + Types.MapType.ofOptional( + 15, 16, Types.StringType.get(), SUPPORTED_PRIMITIVES))), + required( + 17, + "list_of_struct_of_nested_types", + Types.ListType.ofOptional( + 19, + Types.StructType.of( + Types.NestedField.required( + 20, + "m1", + Types.MapType.ofOptional( + 21, 22, Types.StringType.get(), SUPPORTED_PRIMITIVES)), + Types.NestedField.optional( + 23, "l1", Types.ListType.ofRequired(24, SUPPORTED_PRIMITIVES)), + Types.NestedField.required( + 25, "l2", Types.ListType.ofRequired(26, SUPPORTED_PRIMITIVES)), + Types.NestedField.optional( + 27, + "m2", + Types.MapType.ofOptional( + 28, 29, Types.StringType.get(), SUPPORTED_PRIMITIVES)))))); + + Schema schema = + new Schema( + TypeUtil.assignFreshIds(structType, new AtomicInteger(0)::incrementAndGet) + .asStructType() + .fields()); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + List expected = RandomGenericData.generate(schema, 100, 0L); + + try (OrcWriter writer = new OrcWriter(schema, testFile)) { + for (Record record : expected) { + writer.write(record); + } + } + + Assert.assertEquals( + "Ensure written file does not have IDs in the file schema", + 0, + clearAttributes(orcFileSchema(testFile))); + + List rows; + try (CloseableIterable reader = + ORC.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(schema, fileSchema)) + .build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < expected.size(); i += 1) { + DataTestHelpers.assertEquals(schema.asStruct(), expected.get(i), rows.get(i)); + } + } + + private static TypeDescription orcFileSchema(File file) throws IOException { + return OrcFile.createReader( + new Path(file.getPath()), OrcFile.readerOptions(new Configuration())) + .getSchema(); + } + + /** + * Remove attributes from a given {@link TypeDescription} + * + * @param schema the {@link TypeDescription} to remove attributes from + * @return number of attributes removed + */ + public static int clearAttributes(TypeDescription schema) { + int result = 0; + for (String attribute : schema.getAttributeNames()) { + schema.removeAttribute(attribute); + result += 1; + } + List children = schema.getChildren(); + if (children != null) { + for (TypeDescription child : children) { + result += clearAttributes(child); + } + } + return result; + } + + private static class OrcWriter implements Closeable { + + private final VectorizedRowBatch batch; + private final Writer writer; + private final OrcRowWriter valueWriter; + private final File outputFile; + private boolean isClosed = false; + + private OrcWriter(Schema schema, File file) { + TypeDescription orcSchema = ORCSchemaUtil.convert(schema); + // clear attributes before writing schema to file so that file schema does not have IDs + TypeDescription orcSchemaWithoutAttributes = orcSchema.clone(); + clearAttributes(orcSchemaWithoutAttributes); + + this.outputFile = file; + this.batch = orcSchemaWithoutAttributes.createRowBatch(VectorizedRowBatch.DEFAULT_SIZE); + OrcFile.WriterOptions options = + OrcFile.writerOptions(new Configuration()).useUTCTimestamp(true); + options.setSchema(orcSchemaWithoutAttributes); + + final Path locPath = new Path(file.getPath()); + try { + this.writer = OrcFile.createWriter(locPath, options); + } catch (IOException e) { + throw new RuntimeException("Can't create file " + locPath, e); + } + this.valueWriter = GenericOrcWriter.buildWriter(schema, orcSchema); + } + + void write(Record record) { + try { + valueWriter.write(record, batch); + if (batch.size == VectorizedRowBatch.DEFAULT_SIZE) { + writer.addRowBatch(batch); + batch.reset(); + } + } catch (IOException e) { + throw new RuntimeException("Problem writing to ORC file " + outputFile.getPath(), e); + } + } + + @Override + public void close() throws IOException { + if (!isClosed) { + try { + if (batch.size > 0) { + writer.addRowBatch(batch); + batch.reset(); + } + } finally { + writer.close(); + this.isClosed = true; + } + } + } + } +} diff --git a/deploy.gradle b/deploy.gradle index 8e0d43fe0..3151999d8 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -64,8 +64,8 @@ subprojects { publishing { publications { - apache(MavenPublication) { - if (tasks.matching({task -> task.name == 'shadowJar'}).isEmpty()) { + javaLibrary(MavenPublication) { + if (tasks.matching({ task -> task.name == 'shadowJar' }).isEmpty()) { from components.java } else { project.shadow.component(it) @@ -81,53 +81,64 @@ subprojects { } } - groupId = 'org.apache.iceberg' - pom { - name = 'Apache Iceberg' - description = 'A table format for huge analytic datasets' - url = 'https://iceberg.apache.org' - licenses { - license { - name = 'The Apache Software License, Version 2.0' - url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' - } + groupId = 'com.linkedin.iceberg' + artifactId = project.archivesBaseName + + pom { + name = artifactId + description = 'A table format for huge analytic datasets' + url = 'https://github.com/linkedin/iceberg' + licenses { + license { + name = 'The Apache Software License, Version 2.0' + url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' } - mailingLists { - mailingList { - name = 'Dev Mailing List' - post = 'dev@iceberg.apache.org' - subscribe = 'dev-subscribe@iceberg.apache.org' - unsubscribe = 'dev-unsubscribe@iceberg.apache.org' + } + developers { + [ + 'wmoustafa:Walaa Eldin Moustafa', + 'funcheetah:Wenye Zhang', + 'shardulm94:Shardul Mahadik', + 'hotsushi:Sushant Raikar', + 'rzhang10:Raymond Zhang', + 'yiqiangin:Yiqiang Ding', + 'ljfgem:Jiefan Li' + ].each { devData -> + developer { + def devInfo = devData.split(':') + id = devInfo[0] + name = devInfo[1] + url = 'https://github.com/' + devInfo[0] + roles = ["Core developer"] } } - issueManagement { - system = 'GitHub' - url = 'https://github.com/apache/iceberg/issues' - } } - } - } - - repositories { - maven { - credentials { - username project.hasProperty('mavenUser') ? "$mavenUser" : "" - password project.hasProperty('mavenPassword') ? "$mavenPassword" : "" + scm { + url = 'https://github.com/linkedin/iceberg.git' + } + issueManagement { + system = 'GitHub' + url = 'https://github.com/linkedin/iceberg/issues' + } + ciManagement { + url = 'https://github.com/linkedin/iceberg/actions' + system = 'GitHub Actions' } - // upload to the releases repository using ./gradlew -Prelease publish - def apacheSnapshotsRepoUrl = 'https://repository.apache.org/content/repositories/snapshots' - def apacheReleasesRepoUrl = 'https://repository.apache.org/service/local/staging/deploy/maven2' - def snapshotsRepoUrl = project.hasProperty('mavenSnapshotsRepo') ? "$mavenSnapshotsRepo" : "$apacheSnapshotsRepoUrl" - def releasesRepoUrl = project.hasProperty('mavenReleasesRepo') ? "$mavenReleasesRepo" : "$apacheReleasesRepoUrl" - url = project.hasProperty('release') ? releasesRepoUrl : snapshotsRepoUrl } } } - if (project.hasProperty('release')) { - signing { - useGpgCmd() - sign publishing.publications.apache + //useful for testing - running "publish" will create artifacts/pom in a local dir + repositories { maven { url = "$rootProject.buildDir/repo" } } + } + + //fleshes out problems with Maven pom generation when building + tasks.build.dependsOn("publishJavaLibraryPublicationToMavenLocal") + + signing { + if (System.getenv("PGP_KEY")) { + useInMemoryPgpKeys(System.getenv("PGP_KEY"), System.getenv("PGP_PWD")) + sign publishing.publications.javaLibrary } } } diff --git a/format/spec.md b/format/spec.md index 1154cb744..335e24d12 100644 --- a/format/spec.md +++ b/format/spec.md @@ -157,6 +157,8 @@ A table's **schema** is a list of named columns. All data types are either primi For the representations of these types in Avro, ORC, and Parquet file formats, see Appendix A. +Default values for fields are supported, see Neted Types below. + #### Nested Types A **`struct`** is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. Fields may have an optional comment or doc string. Fields can have [default values](#default-values). @@ -165,6 +167,13 @@ A **`list`** is a collection of values with some element type. The element field A **`map`** is a collection of key-value pairs with a key type and a value type. Both the key field and value field each have an integer id that is unique in the table schema. Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. +Iceberg supports default-value semantics for fields of nested types (i.e., struct, list and map). Specifically, a field +of a nested type field can have a default value that will be returned upon reading this field, if it is not manifested. +The default value can be defined with both required and optional fields. Null default values are allowed with optional +fields only, and it's behavior is identical to optional fields with no default value, that is a Null is returned upon +reading this field when it is not manifested. + + #### Primitive Types | Primitive type | Description | Requirements | diff --git a/gradle.properties b/gradle.properties index 29fa24ecc..e70a31075 100644 --- a/gradle.properties +++ b/gradle.properties @@ -19,7 +19,7 @@ systemProp.defaultFlinkVersions=1.15 systemProp.knownFlinkVersions=1.13,1.14,1.15 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 -systemProp.defaultSparkVersions=3.3 +systemProp.defaultSparkVersions=3.1 systemProp.knownSparkVersions=2.4,3.0,3.1,3.2,3.3 systemProp.defaultScalaVersion=2.12 systemProp.knownScalaVersions=2.12,2.13 diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index eb07e9a8e..ea07c68f3 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -109,6 +109,14 @@ public void initialize(String inputName, Map properties) { this.clients = new CachedClientPool(conf, properties); } + protected ClientPool clientPool() { + return clients; + } + + protected Configuration conf() { + return conf; + } + @Override public List listTables(Namespace namespace) { Preconditions.checkArgument( diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 294605a55..0090f091d 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -363,7 +363,7 @@ void persistTable(Table hmsTable, boolean updateHiveTable) } } - private Table loadHmsTable() throws TException, InterruptedException { + protected Table loadHmsTable() throws TException, InterruptedException { try { return metaClients.run(client -> client.getTable(database, tableName)); } catch (NoSuchObjectException nte) { @@ -372,7 +372,7 @@ private Table loadHmsTable() throws TException, InterruptedException { } } - private Table newHmsTable() { + protected Table newHmsTable() { final long currentTimeMillis = System.currentTimeMillis(); Table newTable = @@ -396,7 +396,7 @@ private Table newHmsTable() { return newTable; } - private void setHmsTableParameters( + protected void setHmsTableParameters( String newMetadataLocation, Table tbl, TableMetadata metadata, @@ -538,7 +538,7 @@ private boolean exposeInHmsProperties() { return maxHiveTablePropertySize > 0; } - private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { + protected StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { final StorageDescriptor storageDescriptor = new StorageDescriptor(); storageDescriptor.setCols(HiveSchemaUtil.convert(metadata.schema())); @@ -559,8 +559,7 @@ private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hive } @SuppressWarnings("ReverseDnsLookup") - @VisibleForTesting - long acquireLock() throws UnknownHostException, TException, InterruptedException { + protected long acquireLock() throws UnknownHostException, TException, InterruptedException { final LockComponent lockComponent = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database); lockComponent.setTablename(tableName); @@ -638,7 +637,7 @@ long acquireLock() throws UnknownHostException, TException, InterruptedException return lockId; } - private void cleanupMetadataAndUnlock( + protected void cleanupMetadataAndUnlock( CommitStatus commitStatus, String metadataLocation, Optional lockId, diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/DirectoryInfo.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/DirectoryInfo.java new file mode 100644 index 000000000..203745d5c --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/DirectoryInfo.java @@ -0,0 +1,47 @@ +/* + * 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.iceberg.hivelink.core; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.StructLike; + +/** Metadata for a data directory referenced by either a Hive table or a partition */ +class DirectoryInfo { + private final String location; + private final FileFormat format; + private final StructLike partitionData; + + DirectoryInfo(String location, FileFormat format, StructLike partitionData) { + this.location = location; + this.format = format; + this.partitionData = partitionData; + } + + public String location() { + return location; + } + + public FileFormat format() { + return format; + } + + public StructLike partitionData() { + return partitionData; + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveExpressions.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveExpressions.java new file mode 100644 index 000000000..cec6f806d --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveExpressions.java @@ -0,0 +1,350 @@ +/* + * 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.iceberg.hivelink.core; + +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.expressions.UnboundTerm; +import org.apache.iceberg.types.Type; + +class HiveExpressions { + + private static final Expression REMOVED = (Expression) () -> null; + + private HiveExpressions() {} + + /** + * Simplifies the {@link Expression} so that it fits the restrictions of the expression that can + * be passed to the Hive metastore. For details about the simplification, please see {@link + * RemoveNonPartitionPredicates} and {@link RewriteUnsupportedOperators} + * + * @param expr The {@link Expression} to be simplified + * @param partitionColumnNames The set of partition column names + * @return TRUE if the simplified expression results in an always true expression or if there are + * no predicates on partition columns in the simplified expression, FALSE if the simplified + * expression results in an always false expression, otherwise returns the simplified + * expression + */ + static Expression simplifyPartitionFilter(Expression expr, Set partitionColumnNames) { + try { + // Pushing down NOTs is critical for the correctness of RemoveNonPartitionPredicates + // e.g. consider a predicate on a partition field (P) and a predicate on a non-partition field + // (NP) + // With simply ignoring NP, NOT(P and NP) will be written to NOT(P) + // However the correct behaviour is NOT(P and NP) => NOT(P) OR NOT(NP) => True + Expression notPushedDown = Expressions.rewriteNot(expr); + Expression partitionPredicatesOnly = + ExpressionVisitors.visit( + notPushedDown, new RemoveNonPartitionPredicates(partitionColumnNames)); + if (partitionPredicatesOnly == REMOVED) { + return Expressions.alwaysTrue(); + } else { + return ExpressionVisitors.visit(partitionPredicatesOnly, new RewriteUnsupportedOperators()); + } + } catch (Exception e) { + throw new RuntimeException("Error while processing expression: " + expr, e); + } + } + + /** + * Converts an {@link Expression} into a filter string which can be passed to the Hive metastore + * + *

It is expected that caller handles TRUE and FALSE expressions before calling this method. + * The given {@link Expressions} must also be passed through {@link + * #simplifyPartitionFilter(Expression, Set)} first to remove any unsupported predicates. + * + * @param expr The {@link Expression} to be converted into a filter string + * @return a filter string equivalent to the given {@link Expression} which can be passed to the + * Hive metastore + */ + static String toPartitionFilterString(Expression expr) { + return ExpressionVisitors.visit(expr, ExpressionToPartitionFilterString.get()); + } + + /** Removes any predicates on non-partition columns from the given {@link Expression} */ + private static class RemoveNonPartitionPredicates + extends ExpressionVisitors.ExpressionVisitor { + + private final Set partitionColumnNamesLowerCase; + + RemoveNonPartitionPredicates(Set partitionColumnNames) { + this.partitionColumnNamesLowerCase = + partitionColumnNames.stream().map(String::toLowerCase).collect(Collectors.toSet()); + } + + @Override + public Expression alwaysTrue() { + return Expressions.alwaysTrue(); + } + + @Override + public Expression alwaysFalse() { + return Expressions.alwaysFalse(); + } + + @Override + public Expression not(Expression result) { + return (result == REMOVED) ? REMOVED : Expressions.not(result); + } + + @Override + public Expression and(Expression leftResult, Expression rightResult) { + // if one of the children is a non partition predicate, we can ignore it as it will be applied + // as a post-scan + // filter + if (leftResult == REMOVED && rightResult == REMOVED) { + return REMOVED; + } else if (leftResult == REMOVED) { + return rightResult; + } else if (rightResult == REMOVED) { + return leftResult; + } else { + return Expressions.and(leftResult, rightResult); + } + } + + @Override + public Expression or(Expression leftResult, Expression rightResult) { + return (leftResult == REMOVED || rightResult == REMOVED) + ? REMOVED + : Expressions.or(leftResult, rightResult); + } + + @Override + public Expression predicate(BoundPredicate pred) { + throw new IllegalStateException("Bound predicate not expected: " + pred.getClass().getName()); + } + + @Override + public Expression predicate(UnboundPredicate pred) { + return (partitionColumnNamesLowerCase.contains(pred.ref().name().toLowerCase())) + ? pred + : REMOVED; + } + } + + /** + * Rewrites the {@link Expression} so that it fits the restrictions of the expression that can be + * passed to the Hive metastore. + * + *

This visitor assumes that all predicates are on partition columns. Predicates on + * non-partition columns should be removed using {@link RemoveNonPartitionPredicates} before + * calling this visitor. It performs the following changes: 1. Rewrites NOT operators by inverting + * binary operators, negating unary literals and using De Morgan's laws e.g. NOT(value > 0 AND + * TRUE) => value <= 0 OR FALSE NOT(value < 0 OR value > 10) => value >= 0 AND value <= 10 2. + * Removes IS NULL and IS NOT NULL predicates (Replaced with FALSE and TRUE respectively as + * partition column values are always non null for Hive) e.g. partitionColumn IS NULL => FALSE + * partitionColumn IS NOT NULL => TRUE 3. Expands IN and NOT IN operators into ORs of EQUAL + * operations and ANDs of NOT EQUAL operations respectively e.g. value IN (1, 2, 3) => value = 1 + * OR value = 2 OR value = 3 value NOT IN (1, 2, 3) => value != 1 AND value != 2 AND value != 3 4. + * Removes any children TRUE and FALSE expressions. The checks to remove these are happening + * inside {@link Expressions#and(Expression, Expression)} and {@link Expressions#or(Expression, + * Expression)} (Note that the rewritten expression still can be TRUE and FALSE at the root and + * will have to be handled appropriately by the caller) + * + *

For examples take a look at the tests in {@code TestHiveExpressions} + */ + private static class RewriteUnsupportedOperators + extends ExpressionVisitors.ExpressionVisitor { + + @Override + public Expression alwaysTrue() { + return Expressions.alwaysTrue(); + } + + @Override + public Expression alwaysFalse() { + return Expressions.alwaysFalse(); + } + + @Override + public Expression not(Expression result) { + return result.negate(); + } + + @Override + public Expression and(Expression leftResult, Expression rightResult) { + return Expressions.and(leftResult, rightResult); + } + + @Override + public Expression or(Expression leftResult, Expression rightResult) { + return Expressions.or(leftResult, rightResult); + } + + Expression in(UnboundTerm term, List> literals) { + Expression in = alwaysFalse(); + for (Literal literal : literals) { + in = Expressions.or(in, Expressions.equal(term, literal.value())); + } + return in; + } + + Expression notIn(UnboundTerm term, List> literals) { + Expression notIn = alwaysTrue(); + for (Literal literal : literals) { + notIn = Expressions.and(notIn, Expressions.notEqual(term, literal.value())); + } + return notIn; + } + + @Override + public Expression predicate(BoundPredicate pred) { + throw new IllegalStateException("Bound predicate not expected: " + pred.getClass().getName()); + } + + @Override + public Expression predicate(UnboundPredicate pred) { + switch (pred.op()) { + case LT: + case LT_EQ: + case GT: + case GT_EQ: + case EQ: + case NOT_EQ: + return pred; + case IS_NULL: + return Expressions.alwaysFalse(); + case NOT_NULL: + return Expressions.alwaysTrue(); + case IN: + return in(pred.term(), pred.literals()); + case NOT_IN: + return notIn(pred.term(), pred.literals()); + case STARTS_WITH: + throw new UnsupportedOperationException( + "STARTS_WITH predicate not supported in partition filter " + + "expression. Please use a combination of greater than AND less than predicates instead."); + default: + throw new IllegalStateException("Unexpected predicate: " + pred.op()); + } + } + } + + private static class ExpressionToPartitionFilterString + extends ExpressionVisitors.ExpressionVisitor { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final ExpressionToPartitionFilterString INSTANCE = + new ExpressionToPartitionFilterString(); + + private ExpressionToPartitionFilterString() {} + + static ExpressionToPartitionFilterString get() { + return INSTANCE; + } + + @Override + public String alwaysTrue() { + throw new IllegalStateException("TRUE literal not allowed in Hive partition filter string"); + } + + @Override + public String alwaysFalse() { + throw new IllegalStateException("FALSE literal not allowed in Hive partition filter string"); + } + + @Override + public String not(String result) { + throw new IllegalStateException("NOT operator not allowed in Hive partition filter string"); + } + + @Override + public String and(String leftResult, String rightResult) { + return String.format("((%s) AND (%s))", leftResult, rightResult); + } + + @Override + public String or(String leftResult, String rightResult) { + return String.format("((%s) OR (%s))", leftResult, rightResult); + } + + @Override + public String predicate(BoundPredicate pred) { + switch (pred.op()) { + case LT: + case LT_EQ: + case GT: + case GT_EQ: + case EQ: + case NOT_EQ: + return getBinaryExpressionString(pred); + default: + throw new IllegalStateException( + "Unexpected operator in Hive partition filter string: " + pred.op()); + } + } + + @Override + public String predicate(UnboundPredicate pred) { + throw new IllegalStateException( + "Unbound predicate not expected: " + pred.getClass().getName()); + } + + private String getBinaryExpressionString(BoundPredicate pred) { + String columnName = pred.ref().field().name(); + String opName = getOperationString(pred.op()); + String litValue = getLiteralValue(pred.asLiteralPredicate().literal(), pred.ref().type()); + return String.format("( %s %s %s )", columnName, opName, litValue); + } + + private String getOperationString(Expression.Operation op) { + switch (op) { + case LT: + return "<"; + case LT_EQ: + return "<="; + case GT: + return ">"; + case GT_EQ: + return ">="; + case EQ: + return "="; + case NOT_EQ: + return "!="; + default: + throw new IllegalStateException( + "Unexpected operator in Hive partition filter string: " + op); + } + } + + private String getLiteralValue(Literal lit, Type type) { + Object value = lit.value(); + if (type.typeId() == Type.TypeID.DATE) { + value = EPOCH.plus((Integer) value, ChronoUnit.DAYS).toLocalDate().toString(); + } + if (value instanceof String) { + String escapedString = ((String) value).replace("'", "\\'"); + return String.format("'%s'", escapedString); + } else { + return String.valueOf(value); + } + } + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingCatalog.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingCatalog.java new file mode 100644 index 000000000..a6f1c9585 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingCatalog.java @@ -0,0 +1,63 @@ +/* + * 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.iceberg.hivelink.core; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +/** + * A {@link HiveCatalog} which uses {@link HiveMetadataPreservingTableOperations} underneath. + * + *

This catalog should be only be used by metadata publishers wanting to publish/update Iceberg + * metadata to an existing Hive table while preserving the current Hive metadata + */ +public class HiveMetadataPreservingCatalog extends HiveCatalog { + + private static final String DEFAULT_NAME = "hive_meta_preserving"; + + public HiveMetadataPreservingCatalog() {} + + private static final Cache + HIVE_METADATA_PRESERVING_CATALOG_CACHE = Caffeine.newBuilder().build(); + + @Override + public TableOperations newTableOps(TableIdentifier tableIdentifier) { + String dbName = tableIdentifier.namespace().level(0); + String tableName = tableIdentifier.name(); + return new HiveMetadataPreservingTableOperations( + conf(), clientPool(), new HadoopFileIO(conf()), name(), dbName, tableName); + } + + public static Catalog loadHiveMetadataPreservingCatalog(Configuration conf) { + return loadHiveMetadataPreservingCatalog(DEFAULT_NAME, conf); + } + + public static Catalog loadHiveMetadataPreservingCatalog(String name, Configuration conf) { + return CatalogUtil.loadCatalog( + HiveMetadataPreservingCatalog.class.getName(), name, ImmutableMap.of(), conf); + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingTableOperations.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingTableOperations.java new file mode 100644 index 000000000..0c5b92bd4 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveMetadataPreservingTableOperations.java @@ -0,0 +1,420 @@ +/* + * 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.iceberg.hivelink.core; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.net.UnknownHostException; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator; +import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hive.HiveTableOperations; +import org.apache.iceberg.hive.MetastoreUtil; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link HiveTableOperations} that does not override any existing Hive metadata. TODO: This + * extension should be removed once dual-publish of iceberg+hive is stopped. + * + *

The behaviour of this class differs from {@link HiveTableOperations} in the following ways: 1. + * Does not modify serde information of existing Hive table, this means that if Iceberg schema is + * updated Hive schema will remain stale 2. If the Hive table already exists, no error is thrown. + * Instead Iceberg metadata is added to the table + * + *

This behaviour is useful if the Iceberg metadata is being generated/updated in response to + * Hive metadata being updated. + */ +public class HiveMetadataPreservingTableOperations extends HiveTableOperations { + private final ClientPool metaClients; + private final String database; + private final String tableName; + private final String fullName; + + private static final DynMethods.UnboundMethod ALTER_TABLE = + DynMethods.builder("alter_table") + .impl( + HiveMetaStoreClient.class, + "alter_table_with_environmentContext", + String.class, + String.class, + Table.class, + EnvironmentContext.class) + .impl( + HiveMetaStoreClient.class, + "alter_table", + String.class, + String.class, + Table.class, + EnvironmentContext.class) + .build(); + public static final String ORC_COLUMNS = "columns"; + public static final String ORC_COLUMNS_TYPES = "columns.types"; + private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS = + "iceberg.hive.table-level-lock-evict-ms"; + private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10); + private static final Logger LOG = + LoggerFactory.getLogger(HiveMetadataPreservingTableOperations.class); + + private static Cache commitLockCache; + + private static synchronized void initTableLevelLockCache(long evictionTimeout) { + if (commitLockCache == null) { + commitLockCache = + Caffeine.newBuilder().expireAfterAccess(evictionTimeout, TimeUnit.MILLISECONDS).build(); + } + } + + protected HiveMetadataPreservingTableOperations( + Configuration conf, + ClientPool metaClients, + FileIO fileIO, + String catalogName, + String database, + String table) { + super(conf, metaClients, fileIO, catalogName, database, table); + this.metaClients = metaClients; + this.database = database; + this.tableName = table; + this.fullName = catalogName + "." + database + "." + table; + long tableLevelLockCacheEvictionTimeout = + conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT); + initTableLevelLockCache(tableLevelLockCacheEvictionTimeout); + } + + @Override + protected void doRefresh() { + String metadataLocation = null; + try { + final Table table = metaClients.run(client -> client.getTable(database, tableName)); + String tableType = table.getParameters().get(TABLE_TYPE_PROP); + + if (tableType == null || !tableType.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE)) { + // [LINKEDIN] If table type is not Iceberg, that means there is no Iceberg metadata for the + // table yet. + // So do not throw an error, instead just continue, currentMetadata will continue to remain + // null + // which is what doRefresh would do if the table did not exist and further operations should + // work correctly + + // throw new IllegalArgumentException(String.format("Type of %s.%s is %s, not %s", + // database, tableName, + // tableType /* actual type */, ICEBERG_TABLE_TYPE_VALUE /* expected type */)); + } else { + metadataLocation = table.getParameters().get(METADATA_LOCATION_PROP); + if (metadataLocation == null) { + String errMsg = + String.format( + "%s.%s is missing %s property", database, tableName, METADATA_LOCATION_PROP); + throw new IllegalArgumentException(errMsg); + } + + if (!io().newInputFile(metadataLocation).exists()) { + String errMsg = + String.format( + "%s property for %s.%s points to a non-existent file %s", + METADATA_LOCATION_PROP, database, tableName, metadataLocation); + throw new IllegalArgumentException(errMsg); + } + } + } catch (NoSuchObjectException e) { + if (currentMetadataLocation() != null) { + throw new NoSuchTableException("No such table: %s.%s", database, tableName); + } + + } catch (TException e) { + String errMsg = + String.format("Failed to get table info from metastore %s.%s", database, tableName); + throw new RuntimeException(errMsg, e); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during refresh", e); + } + + refreshFromMetadataLocation(metadataLocation); + } + + @Override + protected void doCommit(TableMetadata base, TableMetadata metadata) { + String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1); + + CommitStatus commitStatus = CommitStatus.FAILURE; + Optional lockId = Optional.empty(); + // getting a process-level lock per table to avoid concurrent commit attempts to the same table + // from the same + // JVM process, which would result in unnecessary and costly HMS lock acquisition requests + ReentrantLock tableLevelMutex = commitLockCache.get(fullName, t -> new ReentrantLock(true)); + tableLevelMutex.lock(); + try { + lockId = Optional.of(acquireLock()); + // TODO add lock heart beating for cases where default lock timeout is too low. + Table tbl; + // [LINKEDIN] Instead of checking if base != null to check for table existence, we query + // metastore for existence + // base can be null if not Iceberg metadata exists, but Hive table exists, so we want to get + // the current table + // definition and not create a new definition + boolean tableExists = metaClients.run(client -> client.tableExists(database, tableName)); + if (tableExists) { + tbl = metaClients.run(client -> client.getTable(database, tableName)); + fixMismatchedSchema(tbl); + } else { + final long currentTimeMillis = System.currentTimeMillis(); + tbl = + new Table( + tableName, + database, + System.getProperty("user.name"), + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + storageDescriptor(metadata, false), + Collections.emptyList(), + Maps.newHashMap(), + null, + null, + TableType.EXTERNAL_TABLE.toString()); + tbl.getParameters() + .put("EXTERNAL", "TRUE"); // using the external table type also requires this + } + + // [LINKEDIN] Do not touch the Hive schema of the table, just modify Iceberg specific + // properties + // tbl.setSd(storageDescriptor(metadata)); // set to pickup any schema changes + final String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP); + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + if (!Objects.equals(baseMetadataLocation, metadataLocation)) { + throw new CommitFailedException( + "Base metadata location '%s' is not same as the current table metadata location '%s' for %s.%s", + baseMetadataLocation, metadataLocation, database, tableName); + } + + // [LINKEDIN] Only updates the metadata location property in HMS, instead of copying all + // iceberg properties to HMS + updateMetadataLocationInHms(newMetadataLocation, tbl); + + try { + persistTable(tbl, tableExists); + commitStatus = CommitStatus.SUCCESS; + } catch (Throwable persistFailure) { + LOG.error( + "Cannot tell if commit to {}.{} succeeded, attempting to reconnect and check.", + database, + tableName, + persistFailure); + commitStatus = checkCommitStatus(newMetadataLocation, metadata); + switch (commitStatus) { + case SUCCESS: + break; + case FAILURE: + throw persistFailure; + case UNKNOWN: + throw new CommitStateUnknownException(persistFailure); + } + } + } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) { + throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName); + + } catch (TException | UnknownHostException e) { + if (e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) { + throw new RuntimeException( + "Failed to acquire locks from metastore because 'HIVE_LOCKS' doesn't " + + "exist, this probably happened when using embedded metastore or doesn't create a " + + "transactional meta table. To fix this, use an alternative metastore", + e); + } + + throw new RuntimeException( + String.format("Metastore operation failed for %s.%s", database, tableName), e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during commit", e); + + } finally { + cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lockId, tableLevelMutex); + } + } + + /** + * [LINKEDIN] Due to an issue that the table read in is sometimes corrupted and has incorrect + * columns, compare the table columns to the avro.schema.literal property (if it exists) and fix + * the table columns if there is a mismatch + * + * @return true if the schema was mismatched and fixed + */ + static boolean fixMismatchedSchema(Table table) { + String avroSchemaLiteral = getAvroSchemaLiteral(table); + if (Strings.isNullOrEmpty(avroSchemaLiteral)) { + return false; + } + Schema schema = new Schema.Parser().parse(avroSchemaLiteral); + List hiveCols; + try { + hiveCols = getColsFromAvroSchema(schema); + } catch (SerDeException e) { + LOG.error("Failed to get get columns from avro schema when checking schema", e); + return false; + } + + boolean schemaMismatched; + if (table.getSd().getCols().size() != hiveCols.size()) { + schemaMismatched = true; + } else { + Map hiveFieldMap = + hiveCols.stream() + .collect( + Collectors.toMap( + field -> field.getName().toLowerCase(), + field -> field.getType().toLowerCase())); + Map tableFieldMap = + table.getSd().getCols().stream() + .collect( + Collectors.toMap( + field -> field.getName().toLowerCase(), + field -> field.getType().toLowerCase())); + schemaMismatched = !hiveFieldMap.equals(tableFieldMap); + } + + if (schemaMismatched) { + LOG.warn( + "Schema columns don't match avro.schema.literal, setting columns to avro.schema.literal. Schema " + + "columns: {}, avro.schema.literal columns: {}", + table.getSd().getCols().stream().map(Object::toString).collect(Collectors.joining(", ")), + hiveCols.stream().map(Object::toString).collect(Collectors.joining(", "))); + table.getSd().setCols(hiveCols); + if (!Strings.isNullOrEmpty(table.getSd().getInputFormat()) + && table.getSd().getInputFormat().contains("OrcInputFormat")) { + updateORCStorageDesc(hiveCols, table); + } + } + + return schemaMismatched; + } + + private static List getColsFromAvroSchema(Schema schema) throws SerDeException { + AvroObjectInspectorGenerator avroOI = new AvroObjectInspectorGenerator(schema); + List columnNames = avroOI.getColumnNames(); + List columnTypes = avroOI.getColumnTypes(); + if (columnNames.size() != columnTypes.size()) { + throw new IllegalStateException(); + } + + return IntStream.range(0, columnNames.size()) + .mapToObj(i -> new FieldSchema(columnNames.get(i), columnTypes.get(i).getTypeName(), "")) + .collect(Collectors.toList()); + } + + private static String getAvroSchemaLiteral(Table table) { + String schemaStr = + table.getParameters().get(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName()); + if (Strings.isNullOrEmpty(schemaStr)) { + schemaStr = + table + .getSd() + .getSerdeInfo() + .getParameters() + .get(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName()); + } + return schemaStr; + } + + private static void updateORCStorageDesc(List hiveCols, Table table) { + String columnsString = + hiveCols.stream().map(FieldSchema::getName).collect(Collectors.joining(",")); + String typesString = + hiveCols.stream().map(FieldSchema::getType).collect(Collectors.joining(",")); + + if (!table.getSd().isSetSerdeInfo()) { + table.getSd().setSerdeInfo(new SerDeInfo()); + } + if (!table.getSd().getSerdeInfo().isSetParameters()) { + table.getSd().getSerdeInfo().setParameters(Maps.newHashMap()); + } + + Map sdParams = table.getSd().getSerdeInfo().getParameters(); + sdParams.put(ORC_COLUMNS, columnsString); + sdParams.put(ORC_COLUMNS_TYPES, typesString); + } + + /** + * [LINKEDIN] a log-enhanced persistTable as a refactoring inspired by + * org.apache.iceberg.hive.HiveTableOperations#persistTable + */ + void persistTable(Table tbl, boolean tableExists) throws TException, InterruptedException { + if (tableExists) { + metaClients.run( + client -> { + MetastoreUtil.alterTable(client, database, tableName, tbl); + return null; + }); + } else { + metaClients.run( + client -> { + client.createTable(tbl); + return null; + }); + } + } + + private void updateMetadataLocationInHms(String newMetadataLocation, Table tbl) { + Map parameters = + Optional.ofNullable(tbl.getParameters()).orElseGet(Maps::newHashMap); + + parameters.put(TABLE_TYPE_PROP, ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH)); + parameters.put(METADATA_LOCATION_PROP, newMetadataLocation); + + if (currentMetadataLocation() != null && !currentMetadataLocation().isEmpty()) { + parameters.put(PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation()); + } + tbl.setParameters(parameters); + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveTypeToIcebergType.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveTypeToIcebergType.java new file mode 100644 index 000000000..cdcfbb258 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/HiveTypeToIcebergType.java @@ -0,0 +1,107 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.List; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; +import org.apache.iceberg.hivelink.core.utils.HiveTypeUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class HiveTypeToIcebergType extends HiveTypeUtil.HiveSchemaVisitor { + private static final String UNION_TO_STRUCT_CONVERSION_PREFIX = "field"; + private int nextId = 1; + + @Override + public Type struct(StructTypeInfo struct, List names, List fieldResults) { + List fields = Lists.newArrayListWithExpectedSize(fieldResults.size()); + for (int i = 0; i < names.size(); i++) { + fields.add(Types.NestedField.optional(allocateId(), names.get(i), fieldResults.get(i))); + } + return Types.StructType.of(fields); + } + + @Override + public Type map(MapTypeInfo map, Type keyResult, Type valueResult) { + return Types.MapType.ofOptional(allocateId(), allocateId(), keyResult, valueResult); + } + + @Override + public Type list(ListTypeInfo list, Type elementResult) { + return Types.ListType.ofOptional(allocateId(), elementResult); + } + + // Mimic the struct call behavior to construct a union converted struct type + @Override + public Type union(UnionTypeInfo union, List unionResults) { + List fields = Lists.newArrayListWithExpectedSize(unionResults.size() + 1); + fields.add(Types.NestedField.required(allocateId(), "tag", Types.IntegerType.get())); + for (int i = 0; i < unionResults.size(); i++) { + fields.add( + Types.NestedField.optional( + allocateId(), UNION_TO_STRUCT_CONVERSION_PREFIX + i, unionResults.get(i))); + } + return Types.StructType.of(fields); + } + + @Override + public Type primitive(PrimitiveTypeInfo primitive) { + switch (primitive.getPrimitiveCategory()) { + case FLOAT: + return Types.FloatType.get(); + case DOUBLE: + return Types.DoubleType.get(); + case BOOLEAN: + return Types.BooleanType.get(); + case BYTE: + case SHORT: + case INT: + return Types.IntegerType.get(); + case LONG: + return Types.LongType.get(); + case CHAR: + case VARCHAR: + case STRING: + return Types.StringType.get(); + case BINARY: + return Types.BinaryType.get(); + case DECIMAL: + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitive; + return Types.DecimalType.of(decimalTypeInfo.precision(), decimalTypeInfo.scale()); + case TIMESTAMP: + return Types.TimestampType.withoutZone(); + case DATE: + return Types.DateType.get(); + default: + throw new UnsupportedOperationException("Unsupported primitive type " + primitive); + } + } + + private int allocateId() { + int current = nextId; + nextId += 1; + return current; + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveCatalog.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveCatalog.java new file mode 100644 index 000000000..0b8c1aae4 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveCatalog.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link HiveCatalog} which uses Hive metadata to read tables. Features like time travel, + * snapshot isolation and incremental computation are not supported along with any WRITE operations + * to either the data or metadata. + */ +public class LegacyHiveCatalog extends HiveCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(LegacyHiveCatalog.class); + + private static final String DEFAULT_NAME = "hive_legacy"; + + public LegacyHiveCatalog() {} + + public static Catalog loadLegacyCatalog(Configuration conf) { + return loadLegacyCatalog(DEFAULT_NAME, conf); + } + + public static Catalog loadLegacyCatalog(String name, Configuration conf) { + return CatalogUtil.loadCatalog( + LegacyHiveCatalog.class.getName(), name, ImmutableMap.of(), conf); + } + + @Override + @SuppressWarnings("CatchBlockLogException") + public Table loadTable(TableIdentifier identifier) { + if (isValidIdentifier(identifier)) { + TableOperations ops = newTableOps(identifier); + if (ops.current() == null) { + throw new NoSuchTableException("Table does not exist: %s", identifier); + } + + return new LegacyHiveTable(ops, fullTableName(name(), identifier)); + } else if (isValidMetadataIdentifier(identifier)) { + throw new UnsupportedOperationException( + "Metadata views not supported for Hive tables without Iceberg metadata. Table: " + + identifier); + } else { + throw new NoSuchTableException("Invalid table identifier: %s", identifier); + } + } + + @Override + public TableOperations newTableOps(TableIdentifier tableIdentifier) { + String dbName = tableIdentifier.namespace().level(0); + String tableName = tableIdentifier.name(); + return new LegacyHiveTableOperations(conf(), clientPool(), dbName, tableName); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + "Dropping tables not supported through legacy Hive catalog. Table: " + identifier); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + "Renaming tables not supported through legacy Hive catalog. From: " + from + " To: " + to); + } + + @Override + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + String location, + Map properties) { + throw new UnsupportedOperationException( + "Creating tables not supported through legacy Hive catalog. Table: " + identifier); + } + + @Override + public Transaction newCreateTableTransaction( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + String location, + Map properties) { + throw new UnsupportedOperationException( + "Creating tables not supported through legacy Hive catalog. Table: " + identifier); + } + + @Override + public Transaction newReplaceTableTransaction( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + String location, + Map properties, + boolean orCreate) { + throw new UnsupportedOperationException( + "Replacing tables not supported through legacy Hive catalog. Table: " + identifier); + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTable.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTable.java new file mode 100644 index 000000000..ee5ada9ae --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTable.java @@ -0,0 +1,254 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.ExpireSnapshots; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.HistoryEntry; +import org.apache.iceberg.ManageSnapshots; +import org.apache.iceberg.OverwriteFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.ReplaceSortOrder; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.RewriteManifests; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateLocation; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +/** + * A {@link Table} which uses Hive table/partition metadata to perform scans using {@link + * LegacyHiveTableScan}. This table does not provide any time travel, snapshot isolation, + * incremental computation benefits. It also does not allow any WRITE operations to either the data + * or metadata. + */ +public class LegacyHiveTable implements Table, HasTableOperations { + private final TableOperations ops; + private final String name; + + protected LegacyHiveTable(TableOperations ops, String name) { + this.ops = ops; + this.name = name; + } + + @Override + public TableOperations operations() { + return ops; + } + + @Override + public void refresh() { + ops.refresh(); + } + + @Override + public TableScan newScan() { + return new LegacyHiveTableScan(ops, this); + } + + @Override + public Schema schema() { + return ops.current().schema(); + } + + @Override + public Map schemas() { + return ops.current().schemasById(); + } + + @Override + public PartitionSpec spec() { + return ops.current().spec(); + } + + @Override + public Map specs() { + return ImmutableMap.of(spec().specId(), spec()); + } + + @Override + public SortOrder sortOrder() { + return ops.current().sortOrder(); + } + + @Override + public Map sortOrders() { + return ops.current().sortOrdersById(); + } + + @Override + public Map properties() { + return ops.current().properties(); + } + + @Override + public String location() { + return ops.current().location(); + } + + @Override + public Snapshot currentSnapshot() { + return null; + } + + @Override + public Snapshot snapshot(long snapshotId) { + throw new UnsupportedOperationException( + "Snapshots not supported for Hive tables without Iceberg metadata"); + } + + @Override + public Iterable snapshots() { + return Collections.emptyList(); + } + + @Override + public List history() { + throw new UnsupportedOperationException( + "History not available for Hive tables without Iceberg metadata"); + } + + @Override + public UpdateSchema updateSchema() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public UpdatePartitionSpec updateSpec() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public UpdateProperties updateProperties() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public ReplaceSortOrder replaceSortOrder() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public UpdateLocation updateLocation() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public AppendFiles newAppend() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public RewriteFiles newRewrite() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public RewriteManifests rewriteManifests() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public OverwriteFiles newOverwrite() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public RowDelta newRowDelta() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public ReplacePartitions newReplacePartitions() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public DeleteFiles newDelete() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public ExpireSnapshots expireSnapshots() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public ManageSnapshots manageSnapshots() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public Transaction newTransaction() { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public FileIO io() { + return ops.io(); + } + + @Override + public EncryptionManager encryption() { + return ops.encryption(); + } + + @Override + public LocationProvider locationProvider() { + return ops.locationProvider(); + } + + @Override + public String toString() { + return name; + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableOperations.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableOperations.java new file mode 100644 index 000000000..8b8fa3758 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableOperations.java @@ -0,0 +1,363 @@ +/* + * 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.iceberg.hivelink.core; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.LocationProviders; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hivelink.core.utils.FileSystemUtils; +import org.apache.iceberg.hivelink.core.utils.MappingUtil; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LegacyHiveTableOperations extends BaseMetastoreTableOperations { + + private static final Logger LOG = LoggerFactory.getLogger(LegacyHiveTableOperations.class); + private static final long INITIAL_SEQUENCE_NUMBER = 0; + private static final int DEFAULT_TABLE_FORMAT_VERSION = 1; + private static final int INITIAL_SPEC_ID = 0; + private static final int INITIAL_SCHEMA_ID = 0; + private static final int PARTITION_DATA_ID_START = 1000; + + private final ClientPool metaClients; + private final String databaseName; + private final String tableName; + + private final String fullName; + private final Configuration conf; + + private FileIO fileIO; + + protected LegacyHiveTableOperations( + Configuration conf, ClientPool metaClients, String database, String table) { + this.conf = conf; + this.metaClients = metaClients; + this.databaseName = database; + this.tableName = table; + this.fullName = database + "." + table; + } + + @Override + protected String tableName() { + return fullName; + } + + @Override + public FileIO io() { + if (fileIO == null) { + fileIO = new HadoopFileIO(conf); + } + + return fileIO; + } + + @Override + protected void doRefresh() { + try { + org.apache.hadoop.hive.metastore.api.Table hiveTable = + metaClients.run(client -> client.getTable(databaseName, tableName)); + + Schema schema = LegacyHiveTableUtils.getSchema(hiveTable); + PartitionSpec spec = LegacyHiveTableUtils.getPartitionSpec(hiveTable, schema); + + Map tableProperties = + Maps.newHashMap(LegacyHiveTableUtils.getTableProperties(hiveTable)); + // Provide a case insensitive name mapping for Hive tables + tableProperties.put( + TableProperties.DEFAULT_NAME_MAPPING, + NameMappingParser.toJson(MappingUtil.create(schema, false))); + TableMetadata metadata = + newTableMetadataWithoutFreshIds( + schema, spec, hiveTable.getSd().getLocation(), tableProperties); + setCurrentMetadata(metadata); + } catch (TException e) { + String errMsg = + String.format("Failed to get table info from metastore %s.%s", databaseName, tableName); + throw new RuntimeException(errMsg, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during refresh", e); + } + setShouldRefresh(false); + } + + /** + * Returns an {@link Iterable} of {@link Iterable}s of {@link DataFile}s which belong to the + * current table and match the partition predicates from the given expression. + * + *

Each element in the outer {@link Iterable} maps to an {@link Iterable} of {@link DataFile}s + * originating from the same directory + */ + Iterable> getFilesByFilter(Expression expression) { + Iterable matchingDirectories; + if (current().spec().fields().isEmpty()) { + matchingDirectories = ImmutableList.of(getDirectoryInfo()); + } else { + matchingDirectories = getDirectoryInfosByFilter(expression); + } + + // Note that we return an Iterable of Iterables here so that the TableScan can process iterables + // of individual + // directories in parallel hence resulting in a parallel file listing + return Iterables.transform( + matchingDirectories, + directory -> { + List files; + if (FileSystemUtils.exists(directory.location(), conf)) { + files = FileSystemUtils.listFiles(directory.location(), conf); + } else { + LOG.warn("Cannot find directory: {}. Skipping.", directory.location()); + files = ImmutableList.of(); + } + return Iterables.transform( + files, + file -> + createDataFile( + file, current().spec(), directory.partitionData(), directory.format())); + }); + } + + private DirectoryInfo getDirectoryInfo() { + Preconditions.checkArgument( + current().spec().fields().isEmpty(), + "getDirectoryInfo only allowed for unpartitioned tables"); + try { + org.apache.hadoop.hive.metastore.api.Table hiveTable = + metaClients.run(client -> client.getTable(databaseName, tableName)); + + return LegacyHiveTableUtils.toDirectoryInfo(hiveTable); + } catch (TException e) { + String errMsg = + String.format( + "Failed to get table info for %s.%s from metastore", databaseName, tableName); + throw new RuntimeException(errMsg, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to getDirectoryInfo", e); + } + } + + private List getDirectoryInfosByFilter(Expression expression) { + Preconditions.checkArgument( + !current().spec().fields().isEmpty(), + "getDirectoryInfosByFilter only allowed for partitioned tables"); + try { + LOG.info( + "Fetching partitions for {}.{} with expression: {}", databaseName, tableName, expression); + Set partitionColumnNames = + current().spec().identitySourceIds().stream() + .map(id -> current().schema().findColumnName(id)) + .collect(Collectors.toSet()); + Expression simplified = + HiveExpressions.simplifyPartitionFilter(expression, partitionColumnNames); + Types.StructType partitionSchema = current().spec().partitionType(); + LOG.info("Simplified expression for {}.{} to {}", databaseName, tableName, simplified); + + List partitions; + Expression boundExpression; + if (simplified.equals(Expressions.alwaysFalse())) { + // If simplifyPartitionFilter returns FALSE, no partitions are going to match the filter + // expression + partitions = ImmutableList.of(); + } else if (simplified.equals(Expressions.alwaysTrue())) { + // If simplifyPartitionFilter returns TRUE, all partitions are going to match the filter + // expression + partitions = + metaClients.run( + client -> client.listPartitionsByFilter(databaseName, tableName, null, (short) -1)); + } else { + boundExpression = Binder.bind(partitionSchema, simplified, false); + Evaluator evaluator = new Evaluator(partitionSchema, simplified, false); + String partitionFilterString = HiveExpressions.toPartitionFilterString(boundExpression); + LOG.info( + "Listing partitions for {}.{} with filter string: {}", + databaseName, + tableName, + partitionFilterString); + try { + // We first try to use HMS API call to get the filtered partitions. + partitions = + metaClients.run( + client -> + client.listPartitionsByFilter( + databaseName, tableName, partitionFilterString, (short) -1)); + } catch (MetaException e) { + // If the above HMS call fails, we here try to do the partition filtering ourselves, + // by evaluating all the partitions we got back from HMS against the boundExpression, + // if the evaluation results in true, we include such partition, if false, we filter. + List allPartitions = + metaClients.run( + client -> + client.listPartitionsByFilter(databaseName, tableName, null, (short) -1)); + partitions = + allPartitions.stream() + .filter( + partition -> { + GenericRecord record = GenericRecord.create(partitionSchema); + for (int i = 0; i < record.size(); i++) { + String value = partition.getValues().get(i); + switch (partitionSchema.fields().get(i).type().typeId()) { + case DATE: + record.set(i, (int) LocalDate.parse(value).toEpochDay()); + break; + case TIMESTAMP: + // This format seems to be matching the hive timestamp column + // partition string literal value + record.set( + i, + LocalDateTime.parse( + value, + new DateTimeFormatterBuilder() + .parseLenient() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendFraction( + ChronoField.NANO_OF_SECOND, 0, 9, true) + .toFormatter()) + .toInstant(ZoneOffset.UTC) + .toEpochMilli() + * 1000); + break; + default: + record.set(i, partition.getValues().get(i)); + break; + } + } + return evaluator.eval(record); + }) + .collect(Collectors.toList()); + } + } + + return LegacyHiveTableUtils.toDirectoryInfos(partitions, current().spec()); + } catch (TException e) { + String errMsg = + String.format( + "Failed to get partition info for %s.%s + expression %s from metastore", + databaseName, tableName, expression); + throw new RuntimeException(errMsg, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to getPartitionsByFilter", e); + } + } + + private static DataFile createDataFile( + FileStatus fileStatus, + PartitionSpec partitionSpec, + StructLike partitionData, + FileFormat format) { + DataFiles.Builder builder = + DataFiles.builder(partitionSpec) + .withPath(fileStatus.getPath().toString()) + .withFormat(format) + .withFileSizeInBytes(fileStatus.getLen()) + .withMetrics(new Metrics(10000L, null, null, null, null)); + + if (partitionSpec.fields().isEmpty()) { + return builder.build(); + } else { + return builder.withPartition(partitionData).build(); + } + } + + @Override + public void commit(TableMetadata base, TableMetadata metadata) { + throw new UnsupportedOperationException( + "Writes not supported for Hive tables without Iceberg metadata"); + } + + @Override + public String metadataFileLocation(String filename) { + throw new UnsupportedOperationException( + "Metadata file location not available for Hive tables without Iceberg metadata"); + } + + @Override + public LocationProvider locationProvider() { + return LocationProviders.locationsFor(current().location(), current().properties()); + } + + private TableMetadata newTableMetadataWithoutFreshIds( + Schema schema, PartitionSpec spec, String location, Map properties) { + return new TableMetadata( + null, + DEFAULT_TABLE_FORMAT_VERSION, + UUID.randomUUID().toString(), + location, + INITIAL_SEQUENCE_NUMBER, + System.currentTimeMillis(), + -1, + INITIAL_SCHEMA_ID, + ImmutableList.of(schema), + INITIAL_SPEC_ID, + ImmutableList.of(spec), + PARTITION_DATA_ID_START, + SortOrder.unsorted().orderId(), + ImmutableList.of(SortOrder.unsorted()), + ImmutableMap.copyOf(properties), + -1, + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of(), + ImmutableMap.of(), + ImmutableList.of()); + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableScan.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableScan.java new file mode 100644 index 000000000..a97568ff6 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableScan.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.hivelink.core; + +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.DataTableScan; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.util.ParallelIterable; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link DataTableScan} which uses Hive table and partition metadata to read tables. This scan + * does not provide any time travel, snapshot isolation, incremental computation benefits. + */ +public class LegacyHiveTableScan extends DataTableScan { + private static final Logger LOG = LoggerFactory.getLogger(LegacyHiveTableScan.class); + + protected LegacyHiveTableScan(TableOperations ops, Table table) { + super(ops, table); + } + + protected LegacyHiveTableScan( + TableOperations ops, Table table, Schema schema, TableScanContext context) { + super(ops, table, schema, context); + } + + @Override + @SuppressWarnings("checkstyle:HiddenField") + protected TableScan newRefinedScan( + TableOperations ops, Table table, Schema schema, TableScanContext context) { + return new LegacyHiveTableScan(ops, table, schema, context); + } + + @Override + public CloseableIterable planFiles() { + LOG.info("Scanning table {} with filter {}", table().toString(), filter()); + + Listeners.notifyAll(new ScanEvent(table().toString(), -1, filter(), schema())); + + LegacyHiveTableOperations hiveOps = (LegacyHiveTableOperations) tableOps(); + PartitionSpec spec = hiveOps.current().spec(); + String schemaString = SchemaParser.toJson(spec.schema()); + String specString = PartitionSpecParser.toJson(spec); + ResidualEvaluator residuals = ResidualEvaluator.of(spec, filter(), isCaseSensitive()); + + Iterable> tasks = + Iterables.transform( + hiveOps.getFilesByFilter(filter()), + fileIterable -> + Iterables.transform( + fileIterable, + file -> + new BaseFileScanTask( + file, new DeleteFile[0], schemaString, specString, residuals))); + + return new ParallelIterable<>(tasks, ThreadPools.getWorkerPool()); + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableUtils.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableUtils.java new file mode 100644 index 000000000..eaac39171 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/LegacyHiveTableUtils.java @@ -0,0 +1,284 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.hivelink.core.schema.MergeHiveSchemaWithAvro; +import org.apache.iceberg.hivelink.core.utils.HiveTypeUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class LegacyHiveTableUtils { + + private LegacyHiveTableUtils() {} + + private static final Logger LOG = LoggerFactory.getLogger(LegacyHiveTableUtils.class); + + static Schema getSchema(org.apache.hadoop.hive.metastore.api.Table table) { + Map props = getTableProperties(table); + String schemaStr = props.get("avro.schema.literal"); + // Disable default value validation for backward compatibility with Avro 1.7 + org.apache.avro.Schema avroSchema = + schemaStr != null + ? new org.apache.avro.Schema.Parser().setValidateDefaults(false).parse(schemaStr) + : null; + Schema schema; + if (avroSchema != null) { + String serde = table.getSd().getSerdeInfo().getSerializationLib(); + org.apache.avro.Schema finalAvroSchema; + if (serde.equals("org.apache.hadoop.hive.serde2.avro.AvroSerDe") + || HasDuplicateLowercaseColumnNames.visit(avroSchema)) { + // Case 1: If serde == AVRO, early escape; Hive column info is not reliable and can be empty + // for these tables + // Hive itself uses avro.schema.literal as source of truth for these tables, so this + // should be fine + // Case 2: If avro.schema.literal has duplicate column names when lowercased, that means we + // cannot do reliable + // matching with Hive schema as multiple Avro fields can map to the same Hive field + finalAvroSchema = avroSchema; + } else { + finalAvroSchema = + MergeHiveSchemaWithAvro.visit( + structTypeInfoFromCols(table.getSd().getCols()), avroSchema); + } + schema = AvroSchemaUtil.toIceberg(finalAvroSchema); + } else { + // TODO: Do we need to support column and column.types properties for ORC tables? + LOG.info( + "Table {}.{} does not have an avro.schema.literal set; using Hive schema instead. " + + "The schema will not have case sensitivity and nullability information", + table.getDbName(), + table.getTableName()); + Type icebergType = HiveTypeUtil.convert(structTypeInfoFromCols(table.getSd().getCols())); + schema = new Schema(icebergType.asNestedType().asStructType().fields()); + } + Types.StructType dataStructType = schema.asStruct(); + List fields = Lists.newArrayList(dataStructType.fields()); + + String partitionColumnIdMappingString = props.get("partition.column.ids"); + Schema partitionSchema = + partitionSchema(table.getPartitionKeys(), schema, partitionColumnIdMappingString); + Types.StructType partitionStructType = partitionSchema.asStruct(); + fields.addAll(partitionStructType.fields()); + return new Schema(fields); + } + + static StructTypeInfo structTypeInfoFromCols(List cols) { + Preconditions.checkArgument(cols != null && cols.size() > 0, "No Hive schema present"); + List fieldNames = cols.stream().map(FieldSchema::getName).collect(Collectors.toList()); + List fieldTypeInfos = + cols.stream() + .map(f -> TypeInfoUtils.getTypeInfoFromTypeString(f.getType())) + .collect(Collectors.toList()); + return (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(fieldNames, fieldTypeInfos); + } + + private static Schema partitionSchema( + List partitionKeys, Schema dataSchema, String idMapping) { + Map nameToId = parsePartitionColId(idMapping); + AtomicInteger fieldId = new AtomicInteger(10000); + List partitionFields = Lists.newArrayList(); + partitionKeys.forEach( + f -> { + Types.NestedField field = dataSchema.findField(f.getName()); + if (field != null) { + throw new IllegalStateException( + String.format("Partition field %s also present in data", field.name())); + } + partitionFields.add( + Types.NestedField.optional( + nameToId.containsKey(f.getName()) + ? nameToId.get(f.getName()) + : fieldId.incrementAndGet(), + f.getName(), + primitiveIcebergType(f.getType()), + f.getComment())); + }); + return new Schema(partitionFields); + } + + /** + * @param idMapping A comma separated string representation of column name and its id, e.g. + * partitionCol1:10,partitionCol2:11, no whitespace is allowed in the middle + * @return The parsed in-mem Map representation of the name to id mapping + */ + private static Map parsePartitionColId(String idMapping) { + Map nameToId = Maps.newHashMap(); + if (idMapping != null) { + // parse idMapping string + Arrays.stream(idMapping.split(",")) + .forEach( + kv -> { + String[] split = kv.split(":"); + if (split.length != 2) { + throw new IllegalStateException( + String.format( + "partition.column.ids property is invalid format: %s", idMapping)); + } + String name = split[0]; + Integer id = Integer.parseInt(split[1]); + nameToId.put(name, id); + }); + } + return nameToId; + } + + private static Type primitiveIcebergType(String hiveTypeString) { + PrimitiveTypeInfo primitiveTypeInfo = TypeInfoFactory.getPrimitiveTypeInfo(hiveTypeString); + return HiveTypeUtil.convert(primitiveTypeInfo); + } + + static Map getTableProperties(org.apache.hadoop.hive.metastore.api.Table table) { + Map props = Maps.newHashMap(); + props.putAll(table.getSd().getParameters()); + props.putAll(table.getParameters()); + props.putAll(table.getSd().getSerdeInfo().getParameters()); + return props; + } + + static PartitionSpec getPartitionSpec( + org.apache.hadoop.hive.metastore.api.Table table, Schema schema) { + PartitionSpec.Builder builder = PartitionSpec.builderFor(schema); + table.getPartitionKeys().forEach(fieldSchema -> builder.identity(fieldSchema.getName())); + return builder.build(); + } + + static DirectoryInfo toDirectoryInfo(org.apache.hadoop.hive.metastore.api.Table table) { + return new DirectoryInfo( + table.getSd().getLocation(), + serdeToFileFormat(table.getSd().getSerdeInfo().getSerializationLib()), + null); + } + + static List toDirectoryInfos(List partitions, PartitionSpec spec) { + return partitions.stream() + .map( + p -> + new DirectoryInfo( + p.getSd().getLocation(), + serdeToFileFormat(p.getSd().getSerdeInfo().getSerializationLib()), + buildPartitionStructLike(p.getValues(), spec))) + .collect(Collectors.toList()); + } + + private static StructLike buildPartitionStructLike( + List partitionValues, PartitionSpec spec) { + List fields = spec.partitionType().fields(); + return new StructLike() { + @Override + public int size() { + return partitionValues.size(); + } + + @Override + public T get(int pos, Class javaClass) { + Type type = fields.get(pos).type(); + String partitionString = partitionValues.get(pos); + final Object partitionValue; + // Special handling of TIMESTAMP type since Iceberg Conversions.fromPartitionString + // does not support it + if (type.typeId() == Type.TypeID.TIMESTAMP) { + String isoFormatTs = partitionString.replaceFirst(" ", "T"); + partitionValue = Literal.of(isoFormatTs).to(Types.TimestampType.withoutZone()).value(); + } else { + partitionValue = Conversions.fromPartitionString(type, partitionString); + } + return javaClass.cast(partitionValue); + } + + @Override + public void set(int pos, T value) { + throw new IllegalStateException("Read-only"); + } + }; + } + + private static FileFormat serdeToFileFormat(String serde) { + switch (serde) { + case "org.apache.hadoop.hive.serde2.avro.AvroSerDe": + return FileFormat.AVRO; + case "org.apache.hadoop.hive.ql.io.orc.OrcSerde": + return FileFormat.ORC; + default: + throw new IllegalArgumentException("Unrecognized serde: " + serde); + } + } + + private static class HasDuplicateLowercaseColumnNames extends AvroSchemaVisitor { + + private static boolean visit(org.apache.avro.Schema schema) { + return AvroSchemaVisitor.visit(schema, new HasDuplicateLowercaseColumnNames()); + } + + @Override + public Boolean record( + org.apache.avro.Schema record, List names, List fieldResults) { + return fieldResults.stream().anyMatch(x -> x) + || names.stream() + .collect(Collectors.groupingBy(String::toLowerCase)) + .values() + .stream() + .anyMatch(x -> x.size() > 1); + } + + @Override + public Boolean union(org.apache.avro.Schema union, List optionResults) { + return optionResults.stream().anyMatch(x -> x); + } + + @Override + public Boolean array(org.apache.avro.Schema array, Boolean elementResult) { + return elementResult; + } + + @Override + public Boolean map(org.apache.avro.Schema map, Boolean valueResult) { + return valueResult; + } + + @Override + public Boolean primitive(org.apache.avro.Schema primitive) { + return false; + } + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/HiveSchemaWithPartnerVisitor.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/HiveSchemaWithPartnerVisitor.java new file mode 100644 index 000000000..9046b63ec --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/HiveSchemaWithPartnerVisitor.java @@ -0,0 +1,146 @@ +/* + * 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.iceberg.hivelink.core.schema; + +import java.util.List; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * A Hive {@link TypeInfo} visitor with an accompanying partner schema + * + *

This visitor traverses the Hive {@link TypeInfo} tree contiguously accessing the schema tree + * for the partner schema using {@link PartnerAccessor}. When visiting each type in the Hive tree, + * the implementation is also presented with the corresponding type from the partner schema, or else + * a {@code null} if no match was found. Matching behavior can be controlled by implementing the + * methods in {@link PartnerAccessor} + * + * @param

type of partner schema + * @param type of the field representation in the partner schema + * @param type of the resultant schema generated by the visitor + * @param type of the field representation in the resultant schema + */ +@SuppressWarnings("ClassTypeParameterName") +public abstract class HiveSchemaWithPartnerVisitor { + + /** + * Methods to access types in the partner schema corresponding to types in the Hive schema being + * traversed + * + * @param

type of partner schema + * @param type of the field representation in the partner schema + */ + public interface PartnerAccessor { + + FP fieldPartner(P partnerStruct, String fieldName); + + P fieldType(FP partnerField); + + P mapKeyPartner(P partnerMap); + + P mapValuePartner(P partnerMap); + + P listElementPartner(P partnerList); + + P unionObjectPartner(P partnerUnion, int ordinal); + } + + @SuppressWarnings("MethodTypeParameterName") + public static R visit( + TypeInfo typeInfo, + P partner, + HiveSchemaWithPartnerVisitor visitor, + PartnerAccessor accessor) { + switch (typeInfo.getCategory()) { + case STRUCT: + StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; + List names = structTypeInfo.getAllStructFieldNames(); + List results = Lists.newArrayListWithExpectedSize(names.size()); + for (String name : names) { + TypeInfo fieldTypeInfo = structTypeInfo.getStructFieldTypeInfo(name); + FP fieldPartner = partner != null ? accessor.fieldPartner(partner, name) : null; + P fieldPartnerType = fieldPartner != null ? accessor.fieldType(fieldPartner) : null; + R result = visit(fieldTypeInfo, fieldPartnerType, visitor, accessor); + results.add(visitor.field(name, fieldTypeInfo, fieldPartner, result)); + } + return visitor.struct(structTypeInfo, partner, results); + + case LIST: + ListTypeInfo listTypeInfo = (ListTypeInfo) typeInfo; + TypeInfo elementTypeInfo = listTypeInfo.getListElementTypeInfo(); + P elementPartner = partner != null ? accessor.listElementPartner(partner) : null; + R elementResult = visit(elementTypeInfo, elementPartner, visitor, accessor); + return visitor.list(listTypeInfo, partner, elementResult); + + case MAP: + MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; + P keyPartner = partner != null ? accessor.mapKeyPartner(partner) : null; + R keyResult = visit(mapTypeInfo.getMapKeyTypeInfo(), keyPartner, visitor, accessor); + P valuePartner = partner != null ? accessor.mapValuePartner(partner) : null; + R valueResult = visit(mapTypeInfo.getMapValueTypeInfo(), valuePartner, visitor, accessor); + return visitor.map(mapTypeInfo, partner, keyResult, valueResult); + + case PRIMITIVE: + return visitor.primitive((PrimitiveTypeInfo) typeInfo, partner); + + case UNION: + UnionTypeInfo unionTypeInfo = (UnionTypeInfo) typeInfo; + List allAlternatives = unionTypeInfo.getAllUnionObjectTypeInfos(); + List unionResults = Lists.newArrayListWithExpectedSize(allAlternatives.size()); + for (int i = 0; i < allAlternatives.size(); i++) { + P unionObjectPartner = partner != null ? accessor.unionObjectPartner(partner, i) : null; + R result = visit(allAlternatives.get(i), unionObjectPartner, visitor, accessor); + unionResults.add(result); + } + return visitor.union(unionTypeInfo, partner, unionResults); + + default: + throw new UnsupportedOperationException(typeInfo + " not supported"); + } + } + + public R struct(StructTypeInfo struct, P partner, List fieldResults) { + return null; + } + + public FR field(String name, TypeInfo field, FP partner, R fieldResult) { + return null; + } + + public R list(ListTypeInfo list, P partner, R elementResult) { + return null; + } + + public R map(MapTypeInfo map, P partner, R keyResult, R valueResult) { + return null; + } + + public R union(UnionTypeInfo union, P partner, List results) { + return null; + } + + public R primitive(PrimitiveTypeInfo primitive, P partner) { + return null; + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/MergeHiveSchemaWithAvro.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/MergeHiveSchemaWithAvro.java new file mode 100644 index 000000000..c2211c0eb --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/schema/MergeHiveSchemaWithAvro.java @@ -0,0 +1,309 @@ +/* + * 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.iceberg.hivelink.core.schema; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * A {@link HiveSchemaWithPartnerVisitor} which augments a Hive schema with extra metadata from a + * partner Avro schema and generates a resultant "merged" Avro schema + * + *

1. Fields are matched between Hive and Avro schemas using a case insensitive search by field + * name 2. Copies field names, nullability, default value, field props from the Avro schema 3. + * Copies field type from the Hive schema. TODO: We should also handle some cases of type promotion + * where the types in Avro are potentially more correct e.g.BINARY in Hive to FIXED in Avro, STRING + * in Hive to ENUM in Avro, etc 4. Retains fields found only in the Hive schema; Ignores fields + * found only in the Avro schema 5. Fields found only in Hive schema are represented as optional + * fields in the resultant Avro schema 6. For fields found only in Hive schema, field names are + * sanitized to make them compatible with Avro identifier spec + */ +public class MergeHiveSchemaWithAvro + extends HiveSchemaWithPartnerVisitor { + + public static Schema visit(StructTypeInfo typeInfo, Schema schema) { + return visit(typeInfo, schema, new MergeHiveSchemaWithAvro(), AvroPartnerAccessor.INSTANCE); + } + + private final AtomicInteger recordCounter = new AtomicInteger(0); + + @Override + public Schema struct(StructTypeInfo struct, Schema partner, List fieldResults) { + boolean shouldResultBeOptional = partner == null || AvroSchemaUtil.isOptionSchema(partner); + Schema result; + if (partner == null || extractIfOption(partner).getType() != Schema.Type.RECORD) { + // if there was no matching Avro struct, return a struct with new record/namespace + int recordNum = recordCounter.incrementAndGet(); + result = + Schema.createRecord( + "record" + recordNum, null, "namespace" + recordNum, false, fieldResults); + } else { + result = AvroSchemaUtil.copyRecord(extractIfOption(partner), fieldResults, null); + } + return shouldResultBeOptional ? AvroSchemaUtil.toOption(result) : result; + } + + @Override + public Schema.Field field(String name, TypeInfo field, Schema.Field partner, Schema fieldResult) { + // No need to infer `shouldResultBeOptional`. We expect other visitor methods to return optional + // schemas + // in their field results if required + if (partner == null) { + // if there was no matching Avro field, use name form the Hive schema and set a null default + return new Schema.Field( + AvroSchemaUtil.makeCompatibleName(name), + fieldResult, + null, + Schema.Field.NULL_DEFAULT_VALUE); + } else { + // TODO: How to ensure that field default value is compatible with new field type generated + // from Hive? + // Copy field type from the visitor result, copy everything else from the partner + // Avro requires the default value to match the first type in the option, reorder option if + // required + Schema reordered = reorderOptionIfRequired(fieldResult, partner.defaultVal()); + return AvroSchemaUtil.copyField(partner, reordered, partner.name()); + } + } + + /** + * Reorders an option schema so that the type of the provided default value is the first type in + * the option schema + * + *

e.g. If the schema is (NULL, INT) and the default value is 1, the returned schema is (INT, + * NULL) If the schema is not an option schema or if there is no default value, schema is returned + * as-is + */ + private Schema reorderOptionIfRequired(Schema schema, Object defaultValue) { + if (AvroSchemaUtil.isOptionSchema(schema) && defaultValue != null) { + boolean isNullFirstOption = schema.getTypes().get(0).getType() == Schema.Type.NULL; + if (isNullFirstOption && defaultValue.equals(JsonProperties.NULL_VALUE)) { + return schema; + } else { + return Schema.createUnion(schema.getTypes().get(1), schema.getTypes().get(0)); + } + } else { + return schema; + } + } + + @Override + public Schema list(ListTypeInfo list, Schema partner, Schema elementResult) { + // if there was no matching Avro list, or if matching Avro list was an option, return an + // optional list + boolean shouldResultBeOptional = partner == null || AvroSchemaUtil.isOptionSchema(partner); + Schema result = Schema.createArray(elementResult); + copySchemaProps(partner, result); + return shouldResultBeOptional ? AvroSchemaUtil.toOption(result) : result; + } + + @Override + public Schema map(MapTypeInfo map, Schema partner, Schema keyResult, Schema valueResult) { + Preconditions.checkArgument( + extractIfOption(keyResult).getType() == Schema.Type.STRING, + "Map keys should always be non-nullable strings. Found: %s", + keyResult); + // if there was no matching Avro map, or if matching Avro map was an option, return an optional + // map + boolean shouldResultBeOptional = partner == null || AvroSchemaUtil.isOptionSchema(partner); + Schema result = Schema.createMap(valueResult); + copySchemaProps(partner, result); + return shouldResultBeOptional ? AvroSchemaUtil.toOption(result) : result; + } + + @Override + public Schema union(UnionTypeInfo union, Schema partner, List results) { + if (AvroSchemaUtil.nullExistInUnion(partner)) { + List toAddNull = Lists.newArrayList(); + toAddNull.add(Schema.create(Schema.Type.NULL)); + toAddNull.addAll(results); + return Schema.createUnion(toAddNull); + } + return Schema.createUnion(results); + } + + @Override + public Schema primitive(PrimitiveTypeInfo primitive, Schema partner) { + boolean shouldResultBeOptional = partner == null || AvroSchemaUtil.isOptionSchema(partner); + Schema hivePrimitive = hivePrimitiveToAvro(primitive); + // if there was no matching Avro primitive, use the Hive primitive + Schema result = + partner == null ? hivePrimitive : checkCompatibilityAndPromote(hivePrimitive, partner); + return shouldResultBeOptional ? AvroSchemaUtil.toOption(result) : result; + } + + private Schema checkCompatibilityAndPromote(Schema schema, Schema partner) { + // TODO: Check if schema is compatible with partner + // Also do type promotion if required, schema = string & partner = enum, schema = bytes & + // partner = fixed, etc + return schema; + } + + private static void copySchemaProps(Schema from, Schema to) { + if (from != null) { + Schema sanitizedFrom; + if (AvroSchemaUtil.isOptionSchema(from)) { + // extract the actual type from the nullable union + sanitizedFrom = AvroSchemaUtil.fromOption(from); + } else { + sanitizedFrom = from; + } + for (Map.Entry prop : sanitizedFrom.getObjectProps().entrySet()) { + to.addProp(prop.getKey(), prop.getValue()); + } + } + } + + /** + * A {@link PartnerAccessor} which matches the requested field from a partner Avro struct by case + * insensitive field name match + */ + private static class AvroPartnerAccessor implements PartnerAccessor { + private static final AvroPartnerAccessor INSTANCE = new AvroPartnerAccessor(); + + private static final Schema MAP_KEY = Schema.create(Schema.Type.STRING); + + @Override + public Schema.Field fieldPartner(Schema partner, String fieldName) { + Schema schema = extractIfOption(partner); + return (schema.getType() == Schema.Type.RECORD) + ? findCaseInsensitive(schema, fieldName) + : null; + } + + @Override + public Schema fieldType(Schema.Field partnerField) { + return partnerField.schema(); + } + + @Override + public Schema mapKeyPartner(Schema partner) { + Schema schema = extractIfOption(partner); + return (schema.getType() == Schema.Type.MAP) ? MAP_KEY : null; + } + + @Override + public Schema mapValuePartner(Schema partner) { + Schema schema = extractIfOption(partner); + return (schema.getType() == Schema.Type.MAP) ? schema.getValueType() : null; + } + + @Override + public Schema listElementPartner(Schema partner) { + Schema schema = extractIfOption(partner); + return (schema.getType() == Schema.Type.ARRAY) ? schema.getElementType() : null; + } + + @Override + public Schema unionObjectPartner(Schema partner, int ordinal) { + if (partner.getType() != Schema.Type.UNION) { + return null; + } + Schema schema = AvroSchemaUtil.discardNullFromUnionIfExist(partner); + return schema.getTypes().get(ordinal); + } + + private Schema.Field findCaseInsensitive(Schema struct, String fieldName) { + Preconditions.checkArgument(struct.getType() == Schema.Type.RECORD); + // TODO: Optimize? This will be called for every struct field, we will run the for loop for + // every struct field + for (Schema.Field field : struct.getFields()) { + if (field.name().equalsIgnoreCase(fieldName)) { + return field; + } + } + return null; + } + } + + private static Schema extractIfOption(Schema schema) { + if (AvroSchemaUtil.isOptionSchema(schema)) { + return AvroSchemaUtil.fromOption(schema); + } else { + return schema; + } + } + + // Additional numeric type, similar to other logical type names in AvroSerde + private static final String SHORT_TYPE_NAME = "short"; + private static final String BYTE_TYPE_NAME = "byte"; + + // TODO: This should be refactored into a visitor if we ever require conversion of complex types + public Schema hivePrimitiveToAvro(PrimitiveTypeInfo primitive) { + switch (primitive.getPrimitiveCategory()) { + case INT: + case BYTE: + case SHORT: + return Schema.create(Schema.Type.INT); + + case LONG: + return Schema.create(Schema.Type.LONG); + + case FLOAT: + return Schema.create(Schema.Type.FLOAT); + + case DOUBLE: + return Schema.create(Schema.Type.DOUBLE); + + case BOOLEAN: + return Schema.create(Schema.Type.BOOLEAN); + + case CHAR: + case STRING: + case VARCHAR: + return Schema.create(Schema.Type.STRING); + + case BINARY: + return Schema.create(Schema.Type.BYTES); + + case VOID: + return Schema.create(Schema.Type.NULL); + + case DATE: + return LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + + case TIMESTAMP: + Schema schema = Schema.create(Schema.Type.LONG); + schema.addProp(AvroSchemaUtil.ADJUST_TO_UTC_PROP, false); + return LogicalTypes.timestampMillis().addToSchema(schema); + + case DECIMAL: + DecimalTypeInfo dti = (DecimalTypeInfo) primitive; + return LogicalTypes.decimal(dti.getPrecision(), dti.getScale()) + .addToSchema(Schema.create(Schema.Type.BYTES)); + + default: + throw new UnsupportedOperationException(primitive + " is not supported."); + } + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/FileSystemUtils.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/FileSystemUtils.java new file mode 100644 index 000000000..9a14d97d1 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/FileSystemUtils.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.iceberg.hivelink.core.utils; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +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.fs.PathFilter; +import org.apache.iceberg.exceptions.RuntimeIOException; + +public class FileSystemUtils { + + private FileSystemUtils() {} + + /** Lists all non-hidden files for the given directory */ + public static List listFiles(String directory, Configuration conf) { + + final Path directoryPath = new Path(directory); + final FileStatus[] files; + try { + FileSystem fs = directoryPath.getFileSystem(conf); + files = fs.listStatus(directoryPath, HiddenPathFilter.INSTANCE); + } catch (IOException e) { + throw new UncheckedIOException("Error listing files for directory: " + directory, e); + } + return Arrays.asList(files); + } + + public static boolean exists(String file, Configuration conf) { + final Path filePath = new Path(file); + try { + FileSystem fs = filePath.getFileSystem(conf); + return fs.exists(filePath); + } catch (IOException e) { + throw new RuntimeIOException(e, "Error determining if file or directory exists: %s", file); + } + } + + private enum HiddenPathFilter implements PathFilter { + INSTANCE; + + @Override + public boolean accept(Path path) { + return !path.getName().startsWith("_") && !path.getName().startsWith("."); + } + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/HiveTypeUtil.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/HiveTypeUtil.java new file mode 100644 index 000000000..5fdcfcf55 --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/HiveTypeUtil.java @@ -0,0 +1,97 @@ +/* + * 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.iceberg.hivelink.core.utils; + +import java.util.List; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; +import org.apache.iceberg.hivelink.core.HiveTypeToIcebergType; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; + +public class HiveTypeUtil { + private HiveTypeUtil() {} + + public static Type convert(TypeInfo typeInfo) { + return HiveTypeUtil.visit(typeInfo, new HiveTypeToIcebergType()); + } + + public static T visit(TypeInfo typeInfo, HiveSchemaVisitor visitor) { + switch (typeInfo.getCategory()) { + case STRUCT: + final StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; + List names = structTypeInfo.getAllStructFieldNames(); + List results = Lists.newArrayListWithExpectedSize(names.size()); + for (String name : names) { + results.add(visit(structTypeInfo.getStructFieldTypeInfo(name), visitor)); + } + return visitor.struct(structTypeInfo, names, results); + + case UNION: + final UnionTypeInfo unionTypeInfo = (UnionTypeInfo) typeInfo; + List unionResults = + Lists.newArrayListWithExpectedSize(unionTypeInfo.getAllUnionObjectTypeInfos().size()); + for (TypeInfo unionObjectTypeInfo : unionTypeInfo.getAllUnionObjectTypeInfos()) { + unionResults.add(visit(unionObjectTypeInfo, visitor)); + } + return visitor.union(unionTypeInfo, unionResults); + + case LIST: + ListTypeInfo listTypeInfo = (ListTypeInfo) typeInfo; + return visitor.list(listTypeInfo, visit(listTypeInfo.getListElementTypeInfo(), visitor)); + + case MAP: + final MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; + return visitor.map( + mapTypeInfo, + visit(mapTypeInfo.getMapKeyTypeInfo(), visitor), + visit(mapTypeInfo.getMapValueTypeInfo(), visitor)); + + default: + final PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo; + return visitor.primitive(primitiveTypeInfo); + } + } + + public static class HiveSchemaVisitor { + public T struct(StructTypeInfo struct, List names, List fieldResults) { + return null; + } + + public T list(ListTypeInfo list, T elementResult) { + return null; + } + + public T map(MapTypeInfo map, T keyResult, T valueResult) { + return null; + } + + public T union(UnionTypeInfo union, List unionResults) { + return null; + } + + public T primitive(PrimitiveTypeInfo primitive) { + return null; + } + } +} diff --git a/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/MappingUtil.java b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/MappingUtil.java new file mode 100644 index 000000000..68f9c506f --- /dev/null +++ b/hivelink-core/src/main/java/org/apache/iceberg/hivelink/core/utils/MappingUtil.java @@ -0,0 +1,44 @@ +/* + * 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.iceberg.hivelink.core.utils; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.mapping.MappedFields; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.types.TypeUtil; + +public class MappingUtil { + + private MappingUtil() {} + + /** + * Create a name-based mapping for a schema. + * + *

The mapping returned by this method will use the schema's name for each field. + * + * @param schema a {@link Schema} + * @param caseSensitive whether names should be matched case sensitively + * @return a {@link NameMapping} initialized with the schema's fields and names + */ + public static NameMapping create(Schema schema, boolean caseSensitive) { + final MappedFields mappedFields = + TypeUtil.visit(schema, org.apache.iceberg.mapping.MappingUtil.CreateMapping.INSTANCE); + return new NameMapping(mappedFields, caseSensitive); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/HiveMetastoreTest.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/HiveMetastoreTest.java new file mode 100644 index 000000000..492c134a0 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/HiveMetastoreTest.java @@ -0,0 +1,64 @@ +/* + * 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.iceberg.hivelink.core; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +/** hivelink refactoring: This class is copied from iceberg-hive-metastore module test code */ +public abstract class HiveMetastoreTest { + + private static final String CATALOG_NAME = "test_hive_catalog"; + protected static final String DB_NAME = "hivedb"; + + protected static HiveMetaStoreClient metastoreClient; + protected static Catalog catalog; + protected static HiveConf hiveConf; + protected static TestHiveMetastore metastore; + + @BeforeClass + public static void startMetastore() throws Exception { + HiveMetastoreTest.metastore = new TestHiveMetastore(); + metastore.start(); + HiveMetastoreTest.hiveConf = metastore.hiveConf(); + HiveMetastoreTest.metastoreClient = new HiveMetaStoreClient(hiveConf); + String dbPath = metastore.getDatabasePath(DB_NAME); + Database db = new Database(DB_NAME, "description", dbPath, Maps.newHashMap()); + metastoreClient.createDatabase(db); + catalog = CatalogUtil.buildIcebergCatalog(CATALOG_NAME, ImmutableMap.of(), hiveConf); + } + + @AfterClass + public static void stopMetastore() { + HiveMetastoreTest.catalog = null; + + metastoreClient.close(); + HiveMetastoreTest.metastoreClient = null; + + metastore.stop(); + HiveMetastoreTest.metastore = null; + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/ScriptRunner.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/ScriptRunner.java new file mode 100644 index 000000000..917ba18b3 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/ScriptRunner.java @@ -0,0 +1,231 @@ +/* + * 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.iceberg.hivelink.core; + +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; + +/** hivelink refactoring: This class is copied from iceberg-hive-metastore module test code */ +public class ScriptRunner { + + private static final String DEFAULT_DELIMITER = ";"; + + private Connection connection; + + private boolean stopOnError; + private boolean autoCommit; + + private PrintWriter logWriter = new PrintWriter(System.out); + private PrintWriter errorLogWriter = new PrintWriter(System.err); + + private String delimiter = DEFAULT_DELIMITER; + private boolean fullLineDelimiter = false; + + /** Default constructor */ + public ScriptRunner(Connection connection, boolean autoCommit, boolean stopOnError) { + this.connection = connection; + this.autoCommit = autoCommit; + this.stopOnError = stopOnError; + } + + public void setDelimiter(String newDelimiter, boolean newFullLineDelimiter) { + this.delimiter = newDelimiter; + this.fullLineDelimiter = newFullLineDelimiter; + } + + /** + * Setter for logWriter property + * + * @param logWriter - the new value of the logWriter property + */ + public void setLogWriter(PrintWriter logWriter) { + this.logWriter = logWriter; + } + + /** + * Setter for errorLogWriter property + * + * @param errorLogWriter - the new value of the errorLogWriter property + */ + public void setErrorLogWriter(PrintWriter errorLogWriter) { + this.errorLogWriter = errorLogWriter; + } + + /** + * 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 e) { + throw e; + } catch (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 = null; + while ((line = lineReader.readLine()) != null) { + if (command == null) { + command = new StringBuilder(); + } + String trimmedLine = line.trim(); + if (trimmedLine.startsWith("--")) { + println(trimmedLine); + } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("//")) { + // Do nothing + } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("--")) { + // Do nothing + } else if (!fullLineDelimiter && trimmedLine.endsWith(getDelimiter()) + || fullLineDelimiter && trimmedLine.equals(getDelimiter())) { + command.append(line.substring(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 (SQLException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + throw e; + } catch (IOException e) { + e.fillInStackTrace(); + printlnError("Error executing: " + command); + printlnError(e); + throw e; + } finally { + conn.rollback(); + flush(); + } + } + + private String getDelimiter() { + return 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/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveExpressions.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveExpressions.java new file mode 100644 index 000000000..e12b53526 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveExpressions.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.hivelink.core; + +import static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.in; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notIn; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.hivelink.core.HiveExpressions.simplifyPartitionFilter; + +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.Assert; +import org.junit.Test; + +public class TestHiveExpressions { + + @Test + public void testSimplifyRemoveNonPartitionColumns() { + Expression input = and(and(equal("pCol", 1), equal("nonpCol", 2)), isNull("nonpCol")); + Expression expected = equal("pCol", 1); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveNot() { + Expression input = not(and(equal("pCol", 1), equal("pCol", 2))); + Expression expected = or(notEqual("pCol", 1), notEqual("pCol", 2)); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveIsNull() { + Expression input = isNull("pcol"); + Expression expected = alwaysFalse(); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveNotNull() { + Expression input = notNull("pcol"); + Expression expected = alwaysTrue(); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyExpandIn() { + Expression input = in("pcol", 1, 2, 3); + Expression expected = or(or(equal("pcol", 1), equal("pcol", 2)), equal("pcol", 3)); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyExpandNotIn() { + Expression input = notIn("pcol", 1, 2, 3); + Expression expected = and(and(notEqual("pcol", 1), notEqual("pcol", 2)), notEqual("pcol", 3)); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveAlwaysTrueChildren() { + Expression input = and(alwaysTrue(), equal("pcol", 1)); + Expression expected = equal("pcol", 1); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + + input = or(alwaysTrue(), equal("pcol", 1)); + expected = alwaysTrue(); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveAlwaysFalseChildren() { + Expression input = and(alwaysFalse(), equal("pcol", 1)); + Expression expected = alwaysFalse(); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + + input = or(alwaysFalse(), equal("pcol", 1)); + expected = equal("pcol", 1); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveNonPartitionColumnsWithinNot1() { + Expression input = and(not(equal("nonpcol", "1")), equal("pcol", "1")); + Expression expected = equal("pcol", "1"); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } + + @Test + public void testSimplifyRemoveNonPartitionColumnsWithinNot2() { + Expression input = not(and(equal("nonpcol", "1"), equal("pcol", "1"))); + Expression expected = alwaysTrue(); + Assert.assertEquals( + expected.toString(), simplifyPartitionFilter(input, ImmutableSet.of("pcol")).toString()); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingCatalog.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingCatalog.java new file mode 100644 index 000000000..bf281f9e1 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingCatalog.java @@ -0,0 +1,147 @@ +/* + * 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.iceberg.hivelink.core; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.util.Collections; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.thrift.TException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestHiveMetadataPreservingCatalog extends HiveMetastoreTest { + + private static final String TABLE_NAME = "tbl"; + private static final TableIdentifier TABLE_IDENTIFIER = + TableIdentifier.of(HiveMetastoreTest.DB_NAME, TABLE_NAME); + private static final Schema SCHEMA = + new Schema( + required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get())); + private static final String CATALOG_NAME = "test_hive_preserve_catalog"; + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private String tableLocation; + + @BeforeClass + public static void setCustomHiveCatalog() { + HiveMetastoreTest.catalog = + HiveMetadataPreservingCatalog.loadHiveMetadataPreservingCatalog( + CATALOG_NAME, HiveMetastoreTest.hiveConf); + } + + @Before + public void createTableLocation() throws IOException { + tableLocation = temp.newFolder("hive-").getPath(); + } + + @After + public void cleanup() { + HiveMetastoreTest.catalog.dropTable(TABLE_IDENTIFIER); + } + + @Test + public void shouldNotThrowErrorIfTableExists() { + try { + HiveMetastoreTest.metastoreClient.createTable(hiveTable()); + } catch (Exception e) { + throw new RuntimeException(e); + } + + HiveMetastoreTest.catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + HiveMetastoreTest.catalog.loadTable(TABLE_IDENTIFIER); + } + + @Test + public void shouldNotOverrideExistingHiveMetadata() { + try { + HiveMetastoreTest.metastoreClient.createTable(hiveTable()); + } catch (Exception e) { + throw new RuntimeException(e); + } + + org.apache.iceberg.Table table = + HiveMetastoreTest.catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + Transaction txn = table.newTransaction(); + txn.updateSchema().addColumn("newCol", Types.IntegerType.get()).commit(); + txn.updateProperties().set("testProp", "dummy").commit(); + txn.commitTransaction(); + + try { + Table hiveTable = + HiveMetastoreTest.metastoreClient.getTable(HiveMetastoreTest.DB_NAME, TABLE_NAME); + Assert.assertEquals(2, hiveTable.getSd().getCols().size()); + Assert.assertFalse(hiveTable.getParameters().containsKey("testProp")); + Assert.assertEquals( + "org.apache.hadoop.hive.ql.io.orc.OrcSerde", + hiveTable.getSd().getSerdeInfo().getSerializationLib()); + } catch (TException e) { + throw new RuntimeException(e); + } + } + + private Table hiveTable() { + final long currentTimeMillis = System.currentTimeMillis(); + + final StorageDescriptor storageDescriptor = new StorageDescriptor(); + storageDescriptor.setCols( + ImmutableList.of(new FieldSchema("id", "int", ""), new FieldSchema("data", "string", ""))); + storageDescriptor.setLocation(tableLocation); + storageDescriptor.setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"); + storageDescriptor.setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"); + SerDeInfo serDeInfo = new SerDeInfo(); + serDeInfo.setSerializationLib("org.apache.hadoop.hive.ql.io.orc.OrcSerde"); + storageDescriptor.setSerdeInfo(serDeInfo); + + Table tbl = + new Table( + TABLE_NAME, + HiveMetastoreTest.DB_NAME, + System.getProperty("user.name"), + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + storageDescriptor, + Collections.emptyList(), + Maps.newHashMap(), + null, + null, + TableType.EXTERNAL_TABLE.toString()); + tbl.getParameters().put("EXTERNAL", "TRUE"); // using the external table type also requires this + + return tbl; + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingTableOperations.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingTableOperations.java new file mode 100644 index 000000000..1763bbab8 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetadataPreservingTableOperations.java @@ -0,0 +1,96 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.Collections; +import java.util.Map; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +public class TestHiveMetadataPreservingTableOperations { + + @Test + public void testFixMismatchedSchema() { + // Schema literal with 3 fields (name, id, nested) + String testSchemaLiteral = + "{\"name\":\"testSchema\",\"type\":\"record\",\"namespace\":\"com.linkedin.test\"," + + "\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"nested\"," + + "\"type\":{\"name\":\"nested\",\"type\":\"record\",\"fields\":[{\"name\":\"field1\",\"type\":\"string\"}," + + "{\"name\":\"field2\",\"type\":\"string\"}]}}]}"; + String testSchemaLiteralWithUppercase = + "{\"name\":\"testSchema\",\"type\":\"record\",\"namespace\":\"com" + + ".linkedin.test\", \"fields\":[{\"name\":\"Name\",\"type\":\"string\"},{\"name\":\"ID\",\"type\":\"int\"}" + + ",{\"name\":\"Nested\", \"type\":{\"name\":\"Nested\",\"type\":\"record\",\"fields\":[{\"name\":\"Field1\"," + + "\"type\":\"string\"}, {\"name\":\"Field2\",\"type\":\"string\"}]}}]}"; + + long currentTimeMillis = System.currentTimeMillis(); + StorageDescriptor storageDescriptor = new StorageDescriptor(); + FieldSchema field1 = new FieldSchema("name", "string", ""); + FieldSchema field2 = new FieldSchema("id", "int", ""); + FieldSchema field3 = new FieldSchema("nested", "struct", ""); + // Set cols with incorrect nested type + storageDescriptor.setCols( + ImmutableList.of( + field1, + field2, + new FieldSchema("nested", "struct", ""))); + storageDescriptor.setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"); + Map parameters = ImmutableMap.of("avro.schema.literal", testSchemaLiteral); + Table tbl = + new Table( + "tableName", + "dbName", + System.getProperty("user.name"), + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + storageDescriptor, + Collections.emptyList(), + parameters, + null, + null, + TableType.EXTERNAL_TABLE.toString()); + + Assert.assertTrue(HiveMetadataPreservingTableOperations.fixMismatchedSchema(tbl)); + Assert.assertEquals(3, tbl.getSd().getColsSize()); + Assert.assertEquals(field1, tbl.getSd().getCols().get(0)); + Assert.assertEquals(field2, tbl.getSd().getCols().get(1)); + Assert.assertEquals(field3, tbl.getSd().getCols().get(2)); + Assert.assertTrue( + storageDescriptor + .getSerdeInfo() + .getParameters() + .containsKey(HiveMetadataPreservingTableOperations.ORC_COLUMNS)); + Assert.assertTrue( + storageDescriptor + .getSerdeInfo() + .getParameters() + .containsKey(HiveMetadataPreservingTableOperations.ORC_COLUMNS_TYPES)); + + // Use same schema literal but containing uppercase and check no mismatch detected + tbl.setParameters(ImmutableMap.of("avro.schema.literal", testSchemaLiteralWithUppercase)); + Assert.assertFalse(HiveMetadataPreservingTableOperations.fixMismatchedSchema(tbl)); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetastore.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetastore.java new file mode 100644 index 000000000..c11978552 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveMetastore.java @@ -0,0 +1,257 @@ +/* + * 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.iceberg.hivelink.core; + +import static java.nio.file.Files.createTempDirectory; +import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; +import static java.nio.file.attribute.PosixFilePermissions.fromString; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +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; + +/** hivelink refactoring: This class is copied from iceberg-hive-metastore module test code */ +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(); + + private File hiveLocalDir; + private HiveConf hiveConf; + private ExecutorService executorService; + private TServer server; + private HiveMetaStore.HMSHandler baseHandler; + private HiveClientPool clientPool; + + /** + * Starts a TestHiveMetastore with the default connection pool size (5) and the default HiveConf. + */ + public void start() { + start(new HiveConf(new Configuration(), TestHiveMetastore.class), DEFAULT_POOL_SIZE); + } + + /** + * Starts a TestHiveMetastore with the default connection pool size (5) with the provided + * HiveConf. + * + * @param hiveConf 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 hiveConf The hive configuration to use + * @param poolSize The number of threads in the executor pool + */ + public void start(HiveConf conf, int poolSize) { + try { + this.hiveLocalDir = + createTempDirectory("hive", asFileAttribute(fromString("rwxrwxrwx"))).toFile(); + File derbyLogFile = new File(hiveLocalDir, "derby.log"); + System.setProperty("derby.stream.error.file", derbyLogFile.getAbsolutePath()); + setupMetastoreDB("jdbc:derby:" + getDerbyPath() + ";create=true"); + + 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()); + + // in Hive3, setting this as a system prop ensures that it will be picked up whenever a new + // HiveConf is created + System.setProperty( + HiveConf.ConfVars.METASTOREURIS.varname, + hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS)); + + this.clientPool = new HiveClientPool(1, hiveConf); + } catch (Exception e) { + throw new RuntimeException("Cannot start TestHiveMetastore", e); + } + } + + public void stop() { + if (clientPool != null) { + clientPool.close(); + } + if (server != null) { + server.stop(); + } + if (executorService != null) { + executorService.shutdown(); + } + if (hiveLocalDir != null) { + hiveLocalDir.delete(); + } + if (baseHandler != null) { + baseHandler.shutdown(); + } + METASTORE_THREADS_SHUTDOWN.invoke(); + } + + public HiveConf hiveConf() { + return hiveConf; + } + + public HiveClientPool clientPool() { + return clientPool; + } + + public String getDatabasePath(String dbName) { + File dbDir = new File(hiveLocalDir, dbName + ".db"); + return dbDir.getPath(); + } + + public void reset() throws Exception { + 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(hiveLocalDir.getAbsolutePath()); + 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:" + getDerbyPath() + ";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, "file:" + hiveLocalDir.getAbsolutePath()); + conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "true"); + conf.set(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES.varname, "false"); + conf.set("iceberg.hive.client-pool-size", "2"); + } + + private 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)) { + scriptRunner.runScript(reader); + } + } + + private String getDerbyPath() { + File metastoreDB = new File(hiveLocalDir, "metastore_db"); + return metastoreDB.getPath(); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveSchemaConversions.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveSchemaConversions.java new file mode 100644 index 000000000..cc970def0 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestHiveSchemaConversions.java @@ -0,0 +1,99 @@ +/* + * 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.iceberg.hivelink.core; + +import java.util.List; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.iceberg.hivelink.core.utils.HiveTypeUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +public class TestHiveSchemaConversions { + @Test + public void testPrimitiveTypes() { + List primitives = + Lists.newArrayList( + Types.BooleanType.get(), + Types.IntegerType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.DateType.get(), + Types.TimestampType.withoutZone(), + Types.StringType.get(), + Types.BinaryType.get(), + Types.DecimalType.of(9, 4)); + + List hivePrimitives = + Lists.newArrayList( + TypeInfoFactory.booleanTypeInfo, + TypeInfoFactory.intTypeInfo, + TypeInfoFactory.longTypeInfo, + TypeInfoFactory.floatTypeInfo, + TypeInfoFactory.doubleTypeInfo, + TypeInfoFactory.dateTypeInfo, + TypeInfoFactory.timestampTypeInfo, + TypeInfoFactory.stringTypeInfo, + TypeInfoFactory.binaryTypeInfo, + TypeInfoFactory.getDecimalTypeInfo(9, 4)); + + for (int i = 0; i < primitives.size(); i += 1) { + Type icebergType = primitives.get(i); + PrimitiveTypeInfo hiveType = hivePrimitives.get(i); + Assert.assertEquals( + "Hive schema to primitive: " + hiveType, icebergType, HiveTypeUtil.convert(hiveType)); + } + } + + @Test + public void testConversions() { + check("struct<1: a: optional int, 2: b: optional string>", "struct"); + check( + "struct<5: a: optional map>>", + "struct>>"); + check("struct<2: l1: optional list>", "struct>"); + check( + "struct<3: l1: optional list>>", + "struct>>"); + check( + "list>>>>", + "array>>>>"); + check( + "struct<" + + "6: length: optional int, 7: count: optional int, " + + "8: list: optional list>, " + + "9: wordcounts: optional map>", + "struct<" + + "length:int,count:int,list:array>," + + "wordcounts:map>"); + check( + "struct<1: tag: required int, 2: field0: optional int, 3: field1: optional string>", + "uniontype"); + } + + private static void check(String icebergTypeStr, String hiveTypeStr) { + Type icebergType = HiveTypeUtil.convert(TypeInfoUtils.getTypeInfoFromTypeString(hiveTypeStr)); + Assert.assertEquals(icebergTypeStr, icebergType.toString()); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestLegacyHiveTableScan.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestLegacyHiveTableScan.java new file mode 100644 index 000000000..b38717bee --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestLegacyHiveTableScan.java @@ -0,0 +1,428 @@ +/* + * 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.iceberg.hivelink.core; + +import static org.apache.iceberg.FileFormat.AVRO; +import static org.apache.iceberg.FileFormat.ORC; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hivelink.core.utils.HiveTypeUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestLegacyHiveTableScan extends HiveMetastoreTest { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + + private static final List DATA_COLUMNS = + ImmutableList.of( + new FieldSchema("strCol", "string", ""), new FieldSchema("intCol", "int", "")); + + private static final List PARTITION_COLUMNS = + ImmutableList.of( + new FieldSchema("pcol", "string", ""), new FieldSchema("pIntCol", "int", "")); + + private static final List PARTITION_COLUMNS_2 = + ImmutableList.of( + new FieldSchema("pcol", "string", ""), + new FieldSchema("pIntCol", "int", ""), + new FieldSchema("pDateCol", "date", "")); + + private static final List PARTITION_COLUMNS_3 = + ImmutableList.of( + new FieldSchema("pcol", "string", ""), new FieldSchema("pTsCol", "timestamp", "")); + + private static final List PARTITION_COLUMNS_4 = + ImmutableList.of( + new FieldSchema("pStringCol", "string", ""), + new FieldSchema("pIntCol", "int", ""), + new FieldSchema("pCharCol", "char(1)", ""), + new FieldSchema("pVarcharCol", "varchar(10)", ""), + new FieldSchema("pDateCol", "date", "")); + private static final String CATALOG_NAME = "test_legacy_hive_catalog"; + + private static Catalog legacyCatalog; + private static Path dbPath; + + @BeforeClass + public static void beforeClass() throws Exception { + legacyCatalog = LegacyHiveCatalog.loadLegacyCatalog(CATALOG_NAME, HiveMetastoreTest.hiveConf); + dbPath = Paths.get(URI.create(metastoreClient.getDatabase(DB_NAME).getLocationUri())); + } + + @AfterClass + public static void afterClass() { + TestLegacyHiveTableScan.legacyCatalog = null; + } + + @Test + public void testHiveScanUnpartitioned() throws Exception { + String tableName = "unpartitioned"; + Table table = createTable(tableName, DATA_COLUMNS, ImmutableList.of()); + addFiles(table, AVRO, "A", "B"); + filesMatch(ImmutableMap.of("A", AVRO, "B", AVRO), hiveScan(table)); + } + + @Test + public void testHiveScanSinglePartition() throws Exception { + String tableName = "single_partition"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A", "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=1/B", AVRO, "pcol=ds/pIntCol=1/A", AVRO), hiveScan(table)); + } + + @Test + public void testHiveScanMultiPartition() throws Exception { + String tableName = "multi_partition"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 2), AVRO, "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=2/B", AVRO, "pcol=ds/pIntCol=1/A", AVRO), hiveScan(table)); + } + + @Test + public void testHiveScanDanglingPartitions() throws Exception { + String tableName = "dangling_partition"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 2), AVRO, "B"); + addPartition(table, ImmutableList.of("ds", 3), AVRO, "C"); + makePartitionDangling(table, ImmutableList.of("ds", 3)); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=2/B", AVRO, "pcol=ds/pIntCol=1/A", AVRO), hiveScan(table)); + } + + @Test + public void testHiveScanNoAvroSchema() throws Exception { + String tableName = "hive_scan_no_avro_schema"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS, ORC); + addPartition(table, ImmutableList.of("ds", 1), ORC, "A"); + addPartition(table, ImmutableList.of("ds", 2), ORC, "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=2/B", ORC, "pcol=ds/pIntCol=1/A", ORC), hiveScan(table)); + } + + @Test + public void testHiveScanMultiPartitionWithFilter() throws Exception { + String tableName = "multi_partition_with_filter"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 2), AVRO, "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=1/A", AVRO, "pcol=ds/pIntCol=2/B", AVRO), + hiveScan(table, Expressions.equal("pcol", "ds"))); + } + + @Test + public void testHiveScanMultiPartitionWithFilterDate() throws Exception { + String tableName = "multi_partition_with_filter_date"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS_2); + addPartition(table, ImmutableList.of("ds", 1, LocalDate.of(2019, 4, 14)), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 1, LocalDate.of(2021, 6, 2)), AVRO, "B"); + // 18000 is the # of days since epoch for 2019-04-14, + // this representation matches how Iceberg internally store the value in DateLiteral. + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=1/pDateCol=2019-04-14/A", AVRO), + hiveScan(table, Expressions.equal("pDateCol", 18000))); + } + + @Test + public void testHiveScanMultiPartitionWithFilterTs() throws Exception { + LocalDateTime ldt = EPOCH.plus(1000000000111000L, ChronoUnit.MICROS).toLocalDateTime(); + + String tableName = "multi_partition_with_filter_ts"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS_3); + addPartition(table, ImmutableList.of("foo", ldt), AVRO, "A"); + addPartition(table, ImmutableList.of("bar", ldt), AVRO, "B"); + // 1000000000111000L microseconds since epoch correspond to 2001-09-09T01:46:40.111, + // this representation matches how Iceberg internally store the value in TimeStampLiteral. + filesMatch( + ImmutableMap.of("pcol=foo/pTsCol=2001-09-09T01:46:40.111/A", AVRO), + hiveScan( + table, + Expressions.and( + Expressions.equal("pCol", "foo"), Expressions.equal("pTsCol", 1000000000111000L)))); + } + + @Test + public void testHiveScanNonStringPartitionQuery() throws Exception { + String tableName = "multi_partition_with_filter_on_non_string_partition_cols"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + filesMatch( + ImmutableMap.of(), + hiveScan( + table, + Expressions.and(Expressions.equal("pcol", "ds"), Expressions.equal("pIntCol", 1)))); + } + + @Test + public void testHiveScanComplexNonStringPartitionQuery() throws Exception { + String tableName = "multi_partition_with_filter_on_complex_non_string_partition_cols"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS_4); + addPartition( + table, ImmutableList.of("foo", 0, "a", "xy", LocalDate.of(2019, 4, 14)), AVRO, "A"); + addPartition( + table, ImmutableList.of("foo", 1, "a", "xy", LocalDate.of(2019, 4, 14)), AVRO, "B"); + addPartition( + table, ImmutableList.of("foo", 1, "b", "xy", LocalDate.of(2019, 4, 14)), AVRO, "C"); + addPartition( + table, ImmutableList.of("foo", 1, "b", "xyz", LocalDate.of(2019, 4, 14)), AVRO, "D"); + addPartition( + table, ImmutableList.of("foo", 1, "b", "xyz", LocalDate.of(2020, 4, 14)), AVRO, "E"); + addPartition( + table, ImmutableList.of("bar", 0, "a", "xy", LocalDate.of(2020, 4, 14)), AVRO, "F"); + + filesMatch( + ImmutableMap.of( + "pStringCol=bar/pIntCol=0/pCharCol=a/pVarcharCol=xy/pDateCol=2020-04-14/F", AVRO), + hiveScan(table, Expressions.equal("pstringcol", "bar"))); + filesMatch( + ImmutableMap.of( + "pStringCol=foo/pIntCol=1/pCharCol=b/pVarcharCol=xyz/pDateCol=2019-04-14/D", + AVRO, + "pStringCol=foo/pIntCol=1/pCharCol=b/pVarcharCol=xyz/pDateCol=2020-04-14/E", + AVRO), + hiveScan( + table, + Expressions.and( + Expressions.equal("pcharcol", "b"), Expressions.equal("pvarcharcol", "xyz")))); + filesMatch( + ImmutableMap.of(), + hiveScan( + table, + Expressions.and( + Expressions.equal("pdatecol", "2020-04-14"), + Expressions.and( + Expressions.equal("pcharcol", "b"), Expressions.equal("pvarcharcol", "xy"))))); + } + + @Test + public void testHiveScanMultiPartitionWithNonPartitionFilter() throws Exception { + String tableName = "multi_partition_with_non_partition_filter"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 2), AVRO, "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=1/A", AVRO, "pcol=ds/pIntCol=2/B", AVRO), + hiveScan(table, Expressions.equal("intCol", 1))); + } + + @Test + public void testHiveScanHybridTable() throws Exception { + String tableName = "hybrid_table"; + Table table = createTable(tableName, DATA_COLUMNS, PARTITION_COLUMNS); + addPartition(table, ImmutableList.of("ds", 1), AVRO, "A"); + addPartition(table, ImmutableList.of("ds", 2), ORC, "B"); + filesMatch( + ImmutableMap.of("pcol=ds/pIntCol=1/A", AVRO, "pcol=ds/pIntCol=2/B", ORC), hiveScan(table)); + } + + private static Table createTable( + String tableName, List columns, List partitionColumns) + throws Exception { + return createTable(tableName, columns, partitionColumns, AVRO); + } + + private static Table createTable( + String tableName, + List columns, + List partitionColumns, + FileFormat format) + throws Exception { + long currentTimeMillis = System.currentTimeMillis(); + Path tableLocation = dbPath.resolve(tableName); + Files.createDirectories(tableLocation); + Table tbl = + new Table( + tableName, + DB_NAME, + System.getProperty("user.name"), + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + storageDescriptor(columns, tableLocation.toString(), format), + partitionColumns, + Maps.newHashMap(), + null, + null, + TableType.EXTERNAL_TABLE.toString()); + tbl.getParameters().put("EXTERNAL", "TRUE"); + metastoreClient.createTable(tbl); + return tbl; + } + + private static StorageDescriptor storageDescriptor( + List columns, String location, FileFormat format) { + final StorageDescriptor storageDescriptor = new StorageDescriptor(); + storageDescriptor.setCols(columns); + storageDescriptor.setLocation(location); + SerDeInfo serDeInfo = new SerDeInfo(); + switch (format) { + case AVRO: + storageDescriptor.setOutputFormat( + "org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"); + storageDescriptor.setInputFormat( + "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"); + serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.avro.AvroSerDe"); + storageDescriptor.setParameters( + ImmutableMap.of( + AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(), + schemaLiteral(columns))); + break; + case ORC: + storageDescriptor.setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"); + storageDescriptor.setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"); + serDeInfo.setSerializationLib("org.apache.hadoop.hive.ql.io.orc.OrcSerde"); + break; + default: + throw new UnsupportedOperationException("Unsupported file format: " + format); + } + storageDescriptor.setSerdeInfo(serDeInfo); + return storageDescriptor; + } + + private static String schemaLiteral(List columns) { + Type icebergType = HiveTypeUtil.convert(LegacyHiveTableUtils.structTypeInfoFromCols(columns)); + return AvroSchemaUtil.convert(icebergType).toString(); + } + + private static Path location(Table table) { + return Paths.get(table.getSd().getLocation()); + } + + private static Path location(Table table, List partitionValues) { + Path partitionLocation = location(table); + for (int i = 0; i < table.getPartitionKeysSize(); i++) { + partitionLocation = + partitionLocation.resolve( + table.getPartitionKeys().get(i).getName() + "=" + partitionValues.get(i)); + } + return partitionLocation; + } + + private void addFiles(Table table, FileFormat format, String... fileNames) throws IOException { + Path tableLocation = location(table); + for (String fileName : fileNames) { + Path filePath = tableLocation.resolve(format.addExtension(fileName)); + Files.createFile(filePath); + } + } + + private void addPartition( + Table table, List partitionValues, FileFormat format, String... fileNames) + throws Exception { + Path partitionLocation = location(table, partitionValues); + Files.createDirectories(partitionLocation); + long currentTimeMillis = System.currentTimeMillis(); + metastoreClient.add_partition( + new Partition( + Lists.transform(partitionValues, Object::toString), + table.getDbName(), + table.getTableName(), + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + storageDescriptor(table.getSd().getCols(), partitionLocation.toString(), format), + Maps.newHashMap())); + for (String fileName : fileNames) { + Path filePath = partitionLocation.resolve(format.addExtension(fileName)); + Files.createFile(filePath); + } + } + + private void makePartitionDangling(Table table, List partitionValues) throws Exception { + String partitionLocation = + metastoreClient + .getPartition( + table.getDbName(), + table.getTableName(), + Lists.transform(partitionValues, Object::toString)) + .getSd() + .getLocation(); + FileUtils.deleteDirectory(new File(new URI(partitionLocation))); + } + + private Map hiveScan(Table table) { + return hiveScan(table, Expressions.alwaysTrue()); + } + + private Map hiveScan(Table table, Expression filter) { + Path tableLocation = location(table); + CloseableIterable fileScanTasks = + legacyCatalog + .loadTable(TableIdentifier.of(table.getDbName(), table.getTableName())) + .newScan() + .filter(filter) + .planFiles(); + return StreamSupport.stream(fileScanTasks.spliterator(), false) + .collect( + Collectors.toMap( + f -> { + String fullPath = + tableLocation + .relativize(Paths.get(URI.create(f.file().path().toString()))) + .toString(); + int idx = fullPath.lastIndexOf("."); + return fullPath.substring(0, idx); + }, + f -> f.file().format())); + } + + private static void filesMatch(Map expected, Map actual) { + Assert.assertEquals(expected, actual); + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestMergeHiveSchemaWithAvro.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestMergeHiveSchemaWithAvro.java new file mode 100644 index 000000000..0a36e0228 --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestMergeHiveSchemaWithAvro.java @@ -0,0 +1,503 @@ +/* + * 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.iceberg.hivelink.core; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.util.internal.JacksonUtils; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.hivelink.core.schema.MergeHiveSchemaWithAvro; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class TestMergeHiveSchemaWithAvro { + + @Test + public void shouldUseFieldNamesFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + optional("fA", Schema.Type.INT), + optional("fB", struct("r2", optional("gA", Schema.Type.INT)))); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldUseNullabilityFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT), + required("fB", struct("r2", required("gA", Schema.Type.INT)))); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldUseTypesFromHive() { + String hive = "struct,fb:array,fc:map,fd:string>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT), + required("fB", Schema.Type.INT), + required("fC", Schema.Type.INT), + required("fD", Schema.Type.INT)); + + Schema expected = + struct( + "r1", + required("fA", struct("record1", null, "namespace1", optional("ga", Schema.Type.INT))), + required("fB", array(nullable(Schema.Type.INT))), + required("fC", map(nullable(Schema.Type.INT))), + required("fD", Schema.Type.STRING)); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldIgnoreExtraFieldsFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT), + required( + "fB", + struct("r2", required("gA", Schema.Type.INT), required("gB", Schema.Type.INT))), + required("fC", Schema.Type.INT)); + + Schema expected = + struct( + "r1", + required("fA", Schema.Type.INT), + required("fB", struct("r2", required("gA", Schema.Type.INT)))); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldRetainExtraFieldsFromHive() { + String hive = "struct,fc:int,fd:struct>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT), + required("fB", struct("r2", required("gA", Schema.Type.INT)))); + + Schema expected = + struct( + "r1", + required("fA", Schema.Type.INT), + required( + "fB", + struct( + "r2", + required("gA", Schema.Type.INT), + // Nested field missing in Avro + optional("gb", Schema.Type.INT))), + // Top level field missing in Avro + optional("fc", Schema.Type.INT), + // Top level struct missing in Avro + optional("fd", struct("record1", null, "namespace1", optional("ha", Schema.Type.INT)))); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldRetainDocStringsFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + "doc-r1", + "n1", + required("fA", Schema.Type.INT, "doc-fA", null, null), + required( + "fB", + struct("r2", "doc-r2", "n2", required("gA", Schema.Type.INT, "doc-gA", null, null)), + "doc-fB", + null, + null)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainDefaultValuesFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT, null, 1, null), + required( + "fB", + struct("r2", required("gA", Schema.Type.INT, null, 2, null)), + null, + fromJson("{\"gA\": 3}"), + null)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainFieldPropsFromAvro() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + required("fA", Schema.Type.INT, null, null, ImmutableMap.of("pfA", "vfA")), + required( + "fB", + struct( + "r2", + required("gA", Schema.Type.INT, null, null, ImmutableMap.of("pfB", "vfB"))), + null, + null, + ImmutableMap.of("pgA", "vgA"))); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldHandleLists() { + String hive = "struct,fb:array,fc:array>,fd:array>"; + Schema avro = + struct( + "r1", + required("fA", array(Schema.Type.INT)), + optional("fB", array(Schema.Type.INT)), + required("fC", array(struct("r2", required("gA", Schema.Type.INT))))); + + Schema expected = + struct( + "r1", + required("fA", array(Schema.Type.INT)), + optional("fB", array(Schema.Type.INT)), + required("fC", array(struct("r2", required("gA", Schema.Type.INT)))), + // Array element type is also nullable because it is generated from Hive + optional("fd", array(nullable(Schema.Type.INT)))); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldHandleMaps() { + String hive = + "struct,fb:map,fc:map>,fd:map>"; + Schema avro = + struct( + "r1", + required("fA", map(Schema.Type.INT)), + optional("fB", map(Schema.Type.INT)), + required("fC", map(struct("r2", required("gA", Schema.Type.INT))))); + + Schema expected = + struct( + "r1", + required("fA", map(Schema.Type.INT)), + optional("fB", map(Schema.Type.INT)), + required("fC", map(struct("r2", required("gA", Schema.Type.INT)))), + // Map value type is also nullable because it is generated from Hive + optional("fd", map(nullable(Schema.Type.INT)))); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldHandleUnions() { + String hive = + "struct,fb:uniontype,fc:uniontype>"; + Schema avro = + struct( + "r1", + required("fA", union(Schema.Type.NULL, Schema.Type.STRING, Schema.Type.INT)), + required("fB", union(Schema.Type.STRING, Schema.Type.INT)), + required("fC", union(Schema.Type.STRING, Schema.Type.INT, Schema.Type.NULL))); + + Schema expected = + struct( + "r1", + required("fA", union(Schema.Type.NULL, Schema.Type.STRING, Schema.Type.INT)), + required("fB", union(Schema.Type.STRING, Schema.Type.INT)), + // our merge logic always put the NULL alternative in the front + required("fC", union(Schema.Type.NULL, Schema.Type.STRING, Schema.Type.INT))); + + assertSchema(expected, merge(hive, avro)); + } + + @Test + public void shouldSanitizeIncompatibleFieldNames() { + StructTypeInfo typeInfo = + (StructTypeInfo) + TypeInfoFactory.getStructTypeInfo( + Lists.newArrayList("a.b.c", "$#@%!"), + Lists.newArrayList(TypeInfoFactory.intTypeInfo, TypeInfoFactory.intTypeInfo)); + Schema avro = struct("r1"); + + Schema expected = + struct( + "r1", + optional("a_x2Eb_x2Ec", Schema.Type.INT), + optional("_x24_x23_x40_x25_x21", Schema.Type.INT)); + assertSchema(expected, merge(typeInfo, avro)); + } + + @Test + public void shouldReorderOptionalSchemaToMatchDefaultValue() { + String hive = "struct>"; + Schema avro = + struct( + "r1", + field( + "fA", + Schema.createUnion(Schema.create(Schema.Type.INT), Schema.create(Schema.Type.NULL)), + null, + 1, + null), + field( + "fB", + Schema.createUnion( + struct("r2", required("gA", Schema.Type.INT, null, 2, null)), + Schema.create(Schema.Type.NULL)), + null, + fromJson("{\"gA\": 3}"), + null)); + + assertSchema(avro, merge(hive, avro)); + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void shouldFailForMapsWithNonStringKey() { + String hive = "struct>"; + Schema avro = struct("r1"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Map keys should always be non-nullable strings"); + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainMapProp() { + String hive = "struct>"; + Schema fa = map(Schema.Type.INT); + fa.addProp("key-id", 1); + fa.addProp("value-id", 2); + Schema avro = struct("r1", required("fa", fa)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainNullableMapProp() { + String hive = "struct>"; + Schema fa = map(Schema.Type.INT); + fa.addProp("key-id", 1); + fa.addProp("value-id", 2); + Schema avro = struct("r1", optional("fa", fa)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainListProp() { + String hive = "struct>"; + Schema fa = array(Schema.Type.INT); + fa.addProp("element-id", 1); + Schema avro = struct("r1", required("fA", fa)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRetainNullableListProp() { + String hive = "struct>"; + Schema fa = array(Schema.Type.INT); + fa.addProp("element-id", 1); + Schema avro = struct("r1", optional("fA", fa)); + + assertSchema(avro, merge(hive, avro)); + } + + @Test + public void shouldRecoverLogicalType() { + String hive = "struct"; + Schema avro = + struct( + "r1", + optional("fa", Schema.Type.INT), + optional("fb", Schema.Type.LONG), + optional("fc", Schema.Type.BYTES)); + Schema merged = merge(hive, avro); + + Schema expectedTimestampSchema = Schema.create(Schema.Type.LONG); + expectedTimestampSchema.addProp(AvroSchemaUtil.ADJUST_TO_UTC_PROP, false); + Schema expected = + struct( + "r1", + optional("fa", LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), + optional("fb", LogicalTypes.timestampMillis().addToSchema(expectedTimestampSchema)), + optional( + "fc", LogicalTypes.decimal(4, 2).addToSchema(Schema.create(Schema.Type.BYTES)))); + + assertSchema(expected, merged); + Assert.assertEquals( + "date", + AvroSchemaUtil.fromOption(merged.getField("fa").schema()).getLogicalType().getName()); + // This last line should not throw any exception. + AvroSchemaUtil.toIceberg(merged); + } + + // TODO: tests to retain schema props + // TODO: tests for explicit type compatibility check between hive and avro primitives, once we + // implement it + // TODO: tests for error case => default value in Avro does not match with type from hive + + /** Test Helpers */ + private void assertSchema(Schema expected, Schema actual) { + Assert.assertEquals(expected, actual); + Assert.assertEquals(expected.toString(true), actual.toString(true)); + } + + private Schema merge(StructTypeInfo typeInfo, Schema avro) { + return MergeHiveSchemaWithAvro.visit(typeInfo, avro); + } + + private Schema merge(String hive, Schema avro) { + StructTypeInfo typeInfo = (StructTypeInfo) TypeInfoUtils.getTypeInfoFromTypeString(hive); + return merge(typeInfo, avro); + } + + private Schema struct(String name, String doc, String namespace, Schema.Field... fields) { + return Schema.createRecord(name, doc, namespace, false, Arrays.asList(fields)); + } + + private Schema struct(String name, Schema.Field... fields) { + return struct(name, null, "n" + name, fields); + } + + private Schema array(Schema element) { + return Schema.createArray(element); + } + + private Schema array(Schema.Type elementType) { + return array(Schema.create(elementType)); + } + + private Schema map(Schema value) { + return Schema.createMap(value); + } + + private Schema map(Schema.Type valueType) { + return map(Schema.create(valueType)); + } + + private Schema union(Schema.Type... types) { + return Schema.createUnion( + Arrays.stream(types).map(Schema::create).collect(Collectors.toList())); + } + + private Schema.Field nullable(Schema.Field field) { + Preconditions.checkArgument(!AvroSchemaUtil.isOptionSchema(field.schema())); + return field( + field.name(), + nullable(field.schema()), + field.doc(), + Schema.Field.NULL_DEFAULT_VALUE, + field.getObjectProps()); + } + + private Schema nullable(Schema schema) { + return AvroSchemaUtil.toOption(schema); + } + + private Schema nullable(Schema.Type type) { + return nullable(Schema.create(type)); + } + + private Schema.Field field( + String name, Schema schema, String doc, Object defaultValue, Map props) { + Schema.Field field = new Schema.Field(name, schema, doc, defaultValue); + if (props != null) { + props.forEach(field::addProp); + } + return field; + } + + private Schema.Field required( + String name, Schema schema, String doc, Object defaultValue, Map props) { + return field(name, schema, doc, defaultValue, props); + } + + private Schema.Field required(String name, Schema schema) { + return required(name, schema, null, null, null); + } + + private Schema.Field required( + String name, Schema.Type type, String doc, Object defaultValue, Map props) { + return required(name, Schema.create(type), doc, defaultValue, props); + } + + private Schema.Field required(String name, Schema.Type type) { + return required(name, type, null, null, null); + } + + private Schema.Field optional(String name, Schema schema, String doc) { + return nullable(field(name, schema, doc, null, null)); + } + + private Schema.Field optional(String name, Schema schema) { + return optional(name, schema, null); + } + + private Schema.Field optional(String name, Schema.Type type, String doc) { + return optional(name, Schema.create(type), doc); + } + + private Schema.Field optional(String name, Schema.Type type) { + return optional(name, type, null); + } + + private Object fromJson(String json) { + ObjectMapper mapper = new ObjectMapper(); + try { + return JacksonUtils.toObject(mapper.readTree(json)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestReadFileWithCaseMismatch.java b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestReadFileWithCaseMismatch.java new file mode 100644 index 000000000..97707c01f --- /dev/null +++ b/hivelink-core/src/test/java/org/apache/iceberg/hivelink/core/TestReadFileWithCaseMismatch.java @@ -0,0 +1,377 @@ +/* + * 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.iceberg.hivelink.core; + +import static org.apache.iceberg.types.Types.NestedField.optional; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Locale; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.hivelink.core.utils.MappingUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestReadFileWithCaseMismatch { + + @Parameterized.Parameters(name = "format = {0}") + public static Object[] parameters() { + return new Object[] {"avro", "orc"}; + } + + private final FileFormat format; + + public TestReadFileWithCaseMismatch(String format) { + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void writeAndValidateFileWithLowercaseFields() throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + switch (format) { + case AVRO: + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("fieldone") + .type() + .optional() + .record("fieldone") + .fields() + .optionalString("innerfield") + .endRecord() + .optionalLong("fieldtwo") + .optionalLong("lowercasefield") + .endRecord(); + + org.apache.avro.Schema fieldoneSchema = + avroSchema.getField("fieldone").schema().getTypes().get(1); + GenericData.Record fieldoneRecord = new GenericData.Record(fieldoneSchema); + fieldoneRecord.put("innerfield", "1"); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("fieldone", fieldoneRecord); + record.put("fieldtwo", 2L); + record.put("lowercasefield", 3L); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(record); + } + return; + case ORC: + TypeDescription writerSchema = + TypeDescription.fromString( + "struct,fieldtwo:bigint,lowercasefield:bigint>"); + try (CloseableOrcWriterBatch writer = new CloseableOrcWriterBatch(testFile, writerSchema)) { + VectorizedRowBatch batch = writer.batch; + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[0]) + .setVal(0, "1".getBytes(StandardCharsets.UTF_8)); + ((LongColumnVector) batch.cols[1]).vector[0] = 2L; + ((LongColumnVector) batch.cols[2]).vector[0] = 3L; + } + return; + } + + Schema tableSchema = + new Schema( + optional( + 1, + "fieldOne", + Types.StructType.of( + optional(2, "innerField", Types.StringType.get()), + optional(3, "extraField", Types.StringType.get()))), + optional(4, "fieldTwo", Types.LongType.get()), + optional(5, "lowercasefield", Types.LongType.get())); + // Data should be readable using a case insensitive name mapping + List rows = readRows(testFile, tableSchema, MappingUtil.create(tableSchema, false)); + Assert.assertEquals("1", ((Record) rows.get(0).getField("fieldOne")).getField("innerField")); + Assert.assertNull(((Record) rows.get(0).getField("fieldOne")).getField("extraField")); + Assert.assertEquals(2L, rows.get(0).getField("fieldTwo")); + Assert.assertEquals(3L, rows.get(0).getField("lowercasefield")); + } + + @Test + public void writeAndValidateFileWithMultipleCandidatesInSchema() throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + switch (format) { + case AVRO: + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("outer") + .type() + .optional() + .record("outer") + .fields() + .optionalString("inner") + .endRecord() + .endRecord(); + + org.apache.avro.Schema outerSchema = + avroSchema.getField("outer").schema().getTypes().get(1); + GenericData.Record outerRecord = new GenericData.Record(outerSchema); + outerRecord.put("inner", "1"); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("outer", outerRecord); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(record); + } + return; + case ORC: + TypeDescription writerSchema = + TypeDescription.fromString("struct>"); + try (CloseableOrcWriterBatch writer = new CloseableOrcWriterBatch(testFile, writerSchema)) { + VectorizedRowBatch batch = writer.batch; + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[0]) + .setVal(0, "1".getBytes(StandardCharsets.UTF_8)); + } + return; + } + + Schema tableSchema = + new Schema( + optional( + 1, + "outer", + Types.StructType.of( + optional(2, "Inner", Types.StringType.get()), + optional(3, "INNER", Types.StringType.get())))); + // When the file has a field with multiple candidates fields in the table schema, the candidate + // which exactly + // matches the case is selected. If no such candidate is available, the selection of candidate + // is + // undefined. Based on the current implementation, the first candidate in the table schema by + // index will be picked + // e.g. Here "inner" has two candidates in table schema "Inner" and "INNER", it will map to + // "Inner" + List rows = readRows(testFile, tableSchema, MappingUtil.create(tableSchema, false)); + Assert.assertEquals("1", ((Record) rows.get(0).getField("outer")).getField("Inner")); + Assert.assertNull(((Record) rows.get(0).getField("outer")).getField("INNER")); + + tableSchema = + new Schema( + optional( + 1, + "outer", + Types.StructType.of( + optional(2, "Inner", Types.StringType.get()), + optional(3, "INNER", Types.StringType.get()), + optional(4, "inner", Types.StringType.get())))); + // If there is a candidate which exactly matches the case, then it should be selected + // e.g. Here "inner" has three candidates in table schema "Inner", "INNER", and "inner", it will + // map to "inner" + rows = readRows(testFile, tableSchema, MappingUtil.create(tableSchema, false)); + Assert.assertNull(((Record) rows.get(0).getField("outer")).getField("Inner")); + Assert.assertNull(((Record) rows.get(0).getField("outer")).getField("INNER")); + Assert.assertEquals("1", ((Record) rows.get(0).getField("outer")).getField("inner")); + + // Even if we our read schema (projection) contains only one of the candidate fields from the + // table schema + // the mapping should be done based on the table schema + // e.g. Here "inner" has three candidates in table schema "Inner", "INNER", and "inner", + // but the user requested projection is only "INNER", the field will still map to "inner" + rows = + readRows( + testFile, tableSchema.select("outer.INNER"), MappingUtil.create(tableSchema, false)); + Assert.assertNull(((Record) rows.get(0).getField("outer")).getField("INNER")); + } + + @Test + public void writeAndValidateMultipleCandidatesInFile() throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + switch (format) { + case AVRO: + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("outer") + .type() + .optional() + .record("outer") + .fields() + .optionalString("Inner") + .optionalString("INNER") + .endRecord() + .endRecord(); + + org.apache.avro.Schema outerSchema = + avroSchema.getField("outer").schema().getTypes().get(1); + GenericData.Record outerRecord = new GenericData.Record(outerSchema); + outerRecord.put("Inner", "1"); + outerRecord.put("INNER", "2"); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("outer", outerRecord); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(record); + } + return; + case ORC: + TypeDescription writerSchema = + TypeDescription.fromString("struct>"); + try (CloseableOrcWriterBatch writer = new CloseableOrcWriterBatch(testFile, writerSchema)) { + VectorizedRowBatch batch = writer.batch; + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[0]) + .setVal(0, "1".getBytes(StandardCharsets.UTF_8)); + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[1]) + .setVal(0, "2".getBytes(StandardCharsets.UTF_8)); + } + return; + } + + Schema tableSchema = + new Schema( + optional( + 1, "outer", Types.StructType.of(optional(2, "Inner", Types.StringType.get())))); + // When the file has two fields with the same name when lowercased, both can get mapped to the + // same field in the + // table schema, if there isn't an exact match for both. Which of the candidates eventually gets + // mapped is + // undefined. e.g. here two fields "Inner" and "INNER" can be mapped insensitively to "Inner" in + // table schema + // Based on the current implementation, the last candidate from the file gets picked + // so here "INNER" from file maps to "Inner" in table + List rows = readRows(testFile, tableSchema, MappingUtil.create(tableSchema, false)); + Assert.assertEquals("2", ((Record) rows.get(0).getField("outer")).getField("Inner")); + } + + @Test + public void writeAndValidateDuplicateLowercaseFieldsInFile() throws IOException { + // Duplicate field names are not possible in Avro + Assume.assumeTrue(format == FileFormat.ORC); + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + TypeDescription writerSchema = + TypeDescription.fromString("struct>"); + try (CloseableOrcWriterBatch writer = new CloseableOrcWriterBatch(testFile, writerSchema)) { + VectorizedRowBatch batch = writer.batch; + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[0]) + .setVal(0, "1".getBytes(StandardCharsets.UTF_8)); + ((BytesColumnVector) ((StructColumnVector) batch.cols[0]).fields[1]) + .setVal(0, "2".getBytes(StandardCharsets.UTF_8)); + } + + Schema tableSchema = + new Schema( + optional( + 1, "outer", Types.StructType.of(optional(2, "Inner", Types.StringType.get())))); + // When the file has two fields with the same name and case, both can get mapped to the same + // field in the table + // schema. Which of the candidates eventually gets mapped is undefined. Based on the current + // implementation, + // the last candidate in the file schema by index will be picked + List rows = readRows(testFile, tableSchema, MappingUtil.create(tableSchema, false)); + Assert.assertEquals("2", ((Record) rows.get(0).getField("outer")).getField("Inner")); + } + + private List readRows(File inputFile, Schema readSchema, NameMapping nameMapping) + throws IOException { + switch (format) { + case AVRO: + try (CloseableIterable reader = + Avro.read(Files.localInput(inputFile)) + .project(readSchema) + .createReaderFunc(fileSchema -> DataReader.create(readSchema, fileSchema)) + .withNameMapping(nameMapping) + .build()) { + return Lists.newArrayList(reader); + } + case ORC: + try (CloseableIterable reader = + ORC.read(Files.localInput(inputFile)) + .project(readSchema) + .createReaderFunc( + fileSchema -> GenericOrcReader.buildReader(readSchema, fileSchema)) + .withNameMapping(nameMapping) + .build()) { + return Lists.newArrayList(reader); + } + default: + throw new UnsupportedOperationException("File format: " + format + " not supported."); + } + } + + private static class CloseableOrcWriterBatch implements Closeable { + + private final Writer writer; + private final VectorizedRowBatch batch; + + CloseableOrcWriterBatch(File outputFile, TypeDescription schema) throws IOException { + this.writer = + OrcFile.createWriter( + new Path(outputFile.toString()), + OrcFile.writerOptions(new Configuration()).setSchema(schema)); + this.batch = schema.createRowBatch(); + batch.ensureSize(1); + batch.size = 1; + } + + @Override + public void close() throws IOException { + writer.addRowBatch(batch); + writer.close(); + } + } +} diff --git a/hivelink-core/src/test/resources/hive-schema-3.1.0.derby.sql b/hivelink-core/src/test/resources/hive-schema-3.1.0.derby.sql new file mode 100644 index 000000000..55097d663 --- /dev/null +++ b/hivelink-core/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'); diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergConfigUtil.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergConfigUtil.java new file mode 100644 index 000000000..def39e47e --- /dev/null +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergConfigUtil.java @@ -0,0 +1,75 @@ +/* + * 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.iceberg.mr.hive; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.mr.Catalogs; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.SerializationUtil; + +class HiveIcebergConfigUtil { + + private static final String TABLE_SCHEMA_MAP = "iceberg.mr.table.schema.map"; + + private HiveIcebergConfigUtil() {} + + /** + * Copies schema provided by schemaSupplier into the configuration. + * + *

While copying, the schema is added into a map(tablename -> schema). This map is serialized + * and set as the value for the key TABLE_SCHEMA_MAP. + */ + static void copySchemaToConf( + Supplier schemaSupplier, Configuration configuration, Properties tblProperties) { + String tableName = tblProperties.getProperty(Catalogs.NAME); + Map tableToSchema = + Optional.ofNullable(configuration.get(TABLE_SCHEMA_MAP)) + .map(x -> (HashMap) SerializationUtil.deserializeFromBase64(x)) + .orElseGet(() -> Maps.newHashMap()); + if (!tableToSchema.containsKey(tableName)) { + tableToSchema.put(tableName, SchemaParser.toJson(schemaSupplier.get())); + } + configuration.set(TABLE_SCHEMA_MAP, SerializationUtil.serializeToBase64(tableToSchema)); + } + + /** + * Gets schema from the configuration. + * + *

tblProperties is consulted to get the tablename. This tablename is looked up in the + * serialized map present in TABLE_SCHEMA_MAP configuration. The returned json schema is then + * parsed and returned. + */ + static Optional getSchemaFromConf( + @Nullable Configuration configuration, Properties tblProperties) { + String tableName = tblProperties.getProperty(Catalogs.NAME); + return Optional.ofNullable(configuration) + .map(c -> c.get(TABLE_SCHEMA_MAP)) + .map(x -> (HashMap) SerializationUtil.deserializeFromBase64(x)) + .map(map -> map.get(tableName)) + .map(SchemaParser::fromJson); + } +} diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java index 87d64066b..977c429b5 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java @@ -37,8 +37,8 @@ import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree; import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; -import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynFields; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -121,8 +121,16 @@ private static Expression translateLeaf(PredicateLeaf leaf) { // converts values to // Timestamp using Date#getTime. This conversion discards microseconds, so this is a necessary to // avoid it. + private static final Class PREDICATE_LEAF_IMPL_CLASS = + DynClasses.builder() + .impl("org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl$PredicateLeafImpl") + .build(); + private static final DynFields.UnboundField LITERAL_FIELD = - DynFields.builder().hiddenImpl(SearchArgumentImpl.PredicateLeafImpl.class, "literal").build(); + DynFields.builder().hiddenImpl(PREDICATE_LEAF_IMPL_CLASS, "literal").build(); + + private static final DynFields.UnboundField> LITERAL_LIST_FIELD = + DynFields.builder().hiddenImpl(PREDICATE_LEAF_IMPL_CLASS, "literalList").build(); private static Object leafToLiteral(PredicateLeaf leaf) { switch (leaf.getType()) { @@ -130,16 +138,16 @@ private static Object leafToLiteral(PredicateLeaf leaf) { case BOOLEAN: case STRING: case FLOAT: - return leaf.getLiteral(); + return LITERAL_FIELD.get(leaf); case DATE: - if (leaf.getLiteral() instanceof Date) { - return daysFromDate((Date) leaf.getLiteral()); + if (LITERAL_FIELD.get(leaf) instanceof Date) { + return daysFromDate((Date) LITERAL_FIELD.get(leaf)); } - return daysFromTimestamp((Timestamp) leaf.getLiteral()); + return daysFromTimestamp((Timestamp) LITERAL_FIELD.get(leaf)); case TIMESTAMP: return microsFromTimestamp((Timestamp) LITERAL_FIELD.get(leaf)); case DECIMAL: - return hiveDecimalToBigDecimal((HiveDecimalWritable) leaf.getLiteral()); + return hiveDecimalToBigDecimal((HiveDecimalWritable) LITERAL_FIELD.get(leaf)); default: throw new UnsupportedOperationException("Unknown type: " + leaf.getType()); @@ -152,17 +160,17 @@ private static List leafToLiteralList(PredicateLeaf leaf) { case BOOLEAN: case FLOAT: case STRING: - return leaf.getLiteralList(); + return LITERAL_LIST_FIELD.get(leaf); case DATE: - return leaf.getLiteralList().stream() + return LITERAL_LIST_FIELD.get(leaf).stream() .map(value -> daysFromDate((Date) value)) .collect(Collectors.toList()); case DECIMAL: - return leaf.getLiteralList().stream() + return LITERAL_LIST_FIELD.get(leaf).stream() .map(value -> hiveDecimalToBigDecimal((HiveDecimalWritable) value)) .collect(Collectors.toList()); case TIMESTAMP: - return leaf.getLiteralList().stream() + return LITERAL_LIST_FIELD.get(leaf).stream() .map(value -> microsFromTimestamp((Timestamp) value)) .collect(Collectors.toList()); default: diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java index add9e07c5..6d5b0807f 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java @@ -23,12 +23,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface; import org.apache.hadoop.hive.ql.exec.vector.VectorizedSupport; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -38,6 +36,7 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.data.Record; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.hive.MetastoreUtil; @@ -78,14 +77,36 @@ public class HiveIcebergInputFormat extends MapredIcebergInputFormat } } + private static final DynMethods.StaticMethod DESERIALIZE_OBJECT = + DynMethods.builder("deserializeObject") + .impl("org.apache.hadoop.hive.ql.exec.SerializationUtilities", String.class, Class.class) + .impl("org.apache.hadoop.hive.ql.exec.Utilities", String.class, Class.class) + .buildStatic(); + private static final DynMethods.StaticMethod CONSTRUCT_SARG_HIVE_1 = + DynMethods.builder("create") + .impl( + "org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory", + ExprNodeGenericFuncDesc.class) + .orNoop() + .buildStatic(); + private static final DynMethods.StaticMethod CONSTRUCT_SARG_HIVE_2 = + DynMethods.builder("create") + .impl( + "org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg", + Configuration.class, + ExprNodeGenericFuncDesc.class) + .orNoop() + .buildStatic(); + static final String SPLIT_LOCATION = "iceberg.hive.split.location"; + @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { // Convert Hive filter to Iceberg filter String hiveFilter = job.get(TableScanDesc.FILTER_EXPR_CONF_STR); if (hiveFilter != null) { ExprNodeGenericFuncDesc exprNodeDesc = - SerializationUtilities.deserializeObject(hiveFilter, ExprNodeGenericFuncDesc.class); - SearchArgument sarg = ConvertAstToSearchArg.create(job, exprNodeDesc); + DESERIALIZE_OBJECT.invoke(hiveFilter, ExprNodeGenericFuncDesc.class); + SearchArgument sarg = constructSearchArgument(job, exprNodeDesc); try { Expression filter = HiveIcebergFilterFactory.generateFilterExpression(sarg); job.set(InputFormatConfig.FILTER_EXPRESSION, SerializationUtil.serializeToBase64(filter)); @@ -99,7 +120,7 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { String[] selectedColumns = ColumnProjectionUtils.getReadColumnNames(job); job.setStrings(InputFormatConfig.SELECTED_COLUMNS, selectedColumns); - String location = job.get(InputFormatConfig.TABLE_LOCATION); + String location = job.get(SPLIT_LOCATION); return Arrays.stream(super.getSplits(job, numSplits)) .map(split -> new HiveIcebergSplit((IcebergSplit) split, location)) .toArray(InputSplit[]::new); @@ -140,4 +161,13 @@ public boolean shouldSkipCombine(Path path, Configuration conf) { public VectorizedSupport.Support[] getSupportedFeatures() { return new VectorizedSupport.Support[0]; } + + private SearchArgument constructSearchArgument( + JobConf job, ExprNodeGenericFuncDesc exprNodeDesc) { + SearchArgument searchArgument = CONSTRUCT_SARG_HIVE_2.invoke(job, exprNodeDesc); + if (searchArgument == null) { + searchArgument = CONSTRUCT_SARG_HIVE_1.invoke(exprNodeDesc); + } + return searchArgument; + } } diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index f206bba47..96b033b58 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import org.apache.hadoop.conf.Configuration; @@ -75,7 +76,14 @@ public void initialize(@Nullable Configuration configuration, Properties serDePr // HiveIcebergStorageHandler.configureInputJobProperties() and // the resulting properties are serialized and distributed to the executors - if (serDeProperties.get(InputFormatConfig.TABLE_SCHEMA) != null) { + // LinkedIn's Hive doesn't call configureInputJobProperties() before initializing SerDe. This is + // a workaround + // to appropriately capture configs from configureJobConf() + Optional configSchema = + HiveIcebergConfigUtil.getSchemaFromConf(configuration, serDeProperties); + if (configSchema.isPresent()) { + this.tableSchema = configSchema.get(); + } else if (serDeProperties.get(InputFormatConfig.TABLE_SCHEMA) != null) { this.tableSchema = SchemaParser.fromJson((String) serDeProperties.get(InputFormatConfig.TABLE_SCHEMA)); } else { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index da40f4c73..c2cbb037d 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -133,6 +133,10 @@ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { if (catalogName != null) { jobConf.set(InputFormatConfig.TABLE_CATALOG_PREFIX + tableName, catalogName); } + HiveIcebergConfigUtil.copySchemaToConf( + () -> Catalogs.loadTable(conf, tableDesc.getProperties()).schema(), + jobConf, + tableDesc.getProperties()); } } @@ -301,6 +305,8 @@ static void overlayTableProperties( // this is an exception to the interface documentation, but it's a safe operation to add this // property props.put(InputFormatConfig.TABLE_SCHEMA, schemaJson); + + map.put(HiveIcebergInputFormat.SPLIT_LOCATION, props.getProperty(Catalogs.LOCATION)); } private static class NonSerializingConfig implements Serializable { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java b/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java index b5204068c..f87cb3aed 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java @@ -138,7 +138,7 @@ private static class IcebergRecordStructField implements StructField { @Override public String getFieldName() { - return field.name(); + return field.name().toLowerCase(); } @Override diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java index 4b74ad4fc..9e8dbdd3f 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -118,12 +118,16 @@ public static Collection parameters() { String javaVersion = System.getProperty("java.specification.version"); // Run tests with every FileFormat for a single Catalog (HiveCatalog) - for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { + for (FileFormat fileFormat : new FileFormat[] {FileFormat.AVRO}) { for (String engine : EXECUTION_ENGINES) { // include Tez tests only for Java 8 if (javaVersion.equals("1.8") || "mr".equals(engine)) { + // testParams.add(new Object[] {fileFormat, engine, + // TestTables.TestTableType.HIVE_CATALOG, false}); testParams.add( - new Object[] {fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, false}); + new Object[] { + fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG_UNQUALIFIED_URI, false + }); // test for vectorization=ON in case of ORC format and Tez engine if ((fileFormat == FileFormat.PARQUET || fileFormat == FileFormat.ORC) && "tez".equals(engine) @@ -137,11 +141,11 @@ public static Collection parameters() { // Run tests for every Catalog for a single FileFormat (PARQUET) and execution engine (mr) // skip HiveCatalog tests as they are added before - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { - if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { - testParams.add(new Object[] {FileFormat.PARQUET, "mr", testTableType, false}); - } - } + // for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + // if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { + // testParams.add(new Object[] {FileFormat.PARQUET, "mr", testTableType, false}); + // } + // } return testParams; } @@ -1233,6 +1237,9 @@ private boolean isUnsupportedVectorizedTypeForHive(Type type) { } private void testComplexTypeWrite(Schema schema, List records) throws IOException { + Assume.assumeTrue( + "Table writes not supported on Hive catalog tables with unqualified URIs", + testTableType != TestTables.TestTableType.HIVE_CATALOG_UNQUALIFIED_URI); String tableName = "complex_table"; Table table = testTables.createTable(shell, "complex_table", schema, fileFormat, ImmutableList.of()); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java index ec166b6dc..c1262e559 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java @@ -21,6 +21,8 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.OffsetDateTime; @@ -546,6 +548,45 @@ public String createHiveTableSQL(TableIdentifier identifier, Map } } + static class HiveTestTablesUnqualifiedURI extends TestTables { + + private static class HiveCatalogUnqualifiedURI extends HiveCatalog { + + HiveCatalogUnqualifiedURI(String catalogName, Configuration conf) { + initialize(catalogName, ImmutableMap.of()); + this.setConf(conf); + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { + try { + return new URI(super.defaultWarehouseLocation(tableIdentifier)).getPath(); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + } + + HiveTestTablesUnqualifiedURI(Configuration conf, TemporaryFolder temp, String catalogName) { + super(new HiveCatalogUnqualifiedURI(catalogName, conf), temp, catalogName); + } + + @Override + public Map properties() { + return ImmutableMap.of(InputFormatConfig.CATALOG, "hive"); + } + + @Override + public String locationForCreateTableSQL(TableIdentifier identifier) { + return ""; + } + + @Override + public String createHiveTableSQL(TableIdentifier identifier, Map tblProps) { + return null; + } + } + private static String tablePath(TableIdentifier identifier) { return "/" + Joiner.on("/").join(identifier.namespace().levels()) + "/" + identifier.name(); } @@ -596,6 +637,12 @@ public TestTables instance( Configuration conf, TemporaryFolder temporaryFolder, String catalogName) { return new HiveTestTables(conf, temporaryFolder, catalogName); } + }, + HIVE_CATALOG_UNQUALIFIED_URI { + public TestTables instance( + Configuration conf, TemporaryFolder temporaryFolder, String catalogName) { + return new HiveTestTablesUnqualifiedURI(conf, temporaryFolder, catalogName); + } }; public abstract TestTables instance( diff --git a/orc/src/main/java/org/apache/iceberg/orc/ApplyNameMapping.java b/orc/src/main/java/org/apache/iceberg/orc/ApplyNameMapping.java index 9c78556d3..b39dbf326 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ApplyNameMapping.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ApplyNameMapping.java @@ -70,6 +70,21 @@ public TypeDescription record( return setId(structType, field); } + @Override + public TypeDescription union(TypeDescription union, List options) { + Preconditions.checkArgument(options.size() >= 1, "Union type must have options"); + MappedField field = nameMapping.find(currentPath()); + TypeDescription unionType = TypeDescription.createUnion(); + + for (TypeDescription option : options) { + if (option != null) { + unionType.addUnionChild(option); + } + } + + return setId(unionType, field); + } + @Override public TypeDescription list(TypeDescription array, TypeDescription element) { Preconditions.checkArgument(element != null, "List type must have element type"); diff --git a/orc/src/main/java/org/apache/iceberg/orc/HasIds.java b/orc/src/main/java/org/apache/iceberg/orc/HasIds.java index 849f24a3a..a783d52a2 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/HasIds.java +++ b/orc/src/main/java/org/apache/iceberg/orc/HasIds.java @@ -30,6 +30,12 @@ public Boolean record(TypeDescription record, List names, List || fields.stream().anyMatch(Predicate.isEqual(true)); } + @Override + public Boolean union(TypeDescription union, List options) { + return ORCSchemaUtil.icebergID(union).isPresent() + || options.stream().anyMatch(Predicate.isEqual(true)); + } + @Override public Boolean list(TypeDescription array, Boolean element) { return ORCSchemaUtil.icebergID(array).isPresent() || element; diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 79c83aec9..8d350b519 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -631,6 +631,7 @@ public static class ReadBuilder { private Expression filter = null; private boolean caseSensitive = true; private NameMapping nameMapping = null; + private OrcRowFilter rowFilter = null; private Function> readerFunc; private Function> batchedReaderFunc; @@ -711,6 +712,11 @@ public ReadBuilder withNameMapping(NameMapping newNameMapping) { return this; } + public ReadBuilder rowFilter(OrcRowFilter newRowFilter) { + this.rowFilter = newRowFilter; + return this; + } + public CloseableIterable build() { Preconditions.checkNotNull(schema, "Schema is required"); return new OrcIterable<>( @@ -724,7 +730,8 @@ public CloseableIterable build() { caseSensitive, filter, batchedReaderFunc, - recordsPerBatch); + recordsPerBatch, + rowFilter); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java index fae1a76c3..fa9da6030 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java @@ -268,60 +268,40 @@ public static TypeDescription buildOrcProjection( private static TypeDescription buildOrcProjection( Integer fieldId, Type type, boolean isRequired, Map mapping) { final TypeDescription orcType; + final OrcField orcField = mapping.getOrDefault(fieldId, null); switch (type.typeId()) { case STRUCT: - orcType = TypeDescription.createStruct(); - for (Types.NestedField nestedField : type.asStructType().fields()) { - // Using suffix _r to avoid potential underlying issues in ORC reader - // with reused column names between ORC and Iceberg; - // e.g. renaming column c -> d and adding new column d - String name = - Optional.ofNullable(mapping.get(nestedField.fieldId())) - .map(OrcField::name) - .orElseGet(() -> nestedField.name() + "_r" + nestedField.fieldId()); - TypeDescription childType = - buildOrcProjection( - nestedField.fieldId(), - nestedField.type(), - isRequired && nestedField.isRequired(), - mapping); - orcType.addField(name, childType); - } + orcType = buildOrcProjectForStructType(fieldId, type, isRequired, mapping); break; case LIST: - Types.ListType list = (Types.ListType) type; - TypeDescription elementType = - buildOrcProjection( - list.elementId(), - list.elementType(), - isRequired && list.isElementRequired(), - mapping); - orcType = TypeDescription.createList(elementType); + orcType = + buildOrcProjectionForListType((Types.ListType) type, isRequired, mapping, orcField); break; case MAP: - Types.MapType map = (Types.MapType) type; - TypeDescription keyType = - buildOrcProjection(map.keyId(), map.keyType(), isRequired, mapping); - TypeDescription valueType = - buildOrcProjection( - map.valueId(), map.valueType(), isRequired && map.isValueRequired(), mapping); - orcType = TypeDescription.createMap(keyType, valueType); + orcType = buildOrcProjectionForMapType((Types.MapType) type, isRequired, mapping, orcField); break; default: if (mapping.containsKey(fieldId)) { TypeDescription originalType = mapping.get(fieldId).type(); - Optional promotedType = getPromotedType(type, originalType); - - if (promotedType.isPresent()) { - orcType = promotedType.get(); - } else { - Preconditions.checkArgument( - isSameType(originalType, type), - "Can not promote %s type to %s", - originalType.getCategory(), - type.typeId().name()); + if (originalType != null + && originalType.getCategory().equals(TypeDescription.Category.UNION)) { + Preconditions.checkState( + originalType.getChildren().size() == 1, "Expect single type union for orc schema."); orcType = originalType.clone(); + } else { + Optional promotedType = getPromotedType(type, originalType); + + if (promotedType.isPresent()) { + orcType = promotedType.get(); + } else { + Preconditions.checkArgument( + isSameType(originalType, type), + "Can not promote %s type to %s", + originalType.getCategory(), + type.typeId().name()); + orcType = originalType.clone(); + } } } else { if (isRequired) { @@ -338,6 +318,134 @@ private static TypeDescription buildOrcProjection( return orcType; } + private static TypeDescription buildOrcProjectionForMapType( + Types.MapType type, boolean isRequired, Map mapping, OrcField orcField) { + final TypeDescription orcType; + if (orcField != null && orcField.type.getCategory().equals(TypeDescription.Category.UNION)) { + Preconditions.checkState( + orcField.type.getChildren().size() == 1, "Expect single type union for orc schema."); + + orcType = TypeDescription.createUnion(); + Types.MapType map = type; + TypeDescription keyType = buildOrcProjection(map.keyId(), map.keyType(), isRequired, mapping); + TypeDescription valueType = + buildOrcProjection( + map.valueId(), map.valueType(), isRequired && map.isValueRequired(), mapping); + orcType.addUnionChild(TypeDescription.createMap(keyType, valueType)); + } else { + Types.MapType map = type; + TypeDescription keyType = buildOrcProjection(map.keyId(), map.keyType(), isRequired, mapping); + TypeDescription valueType = + buildOrcProjection( + map.valueId(), map.valueType(), isRequired && map.isValueRequired(), mapping); + orcType = TypeDescription.createMap(keyType, valueType); + } + return orcType; + } + + private static TypeDescription buildOrcProjectionForListType( + Types.ListType type, boolean isRequired, Map mapping, OrcField orcField) { + final TypeDescription orcType; + if (orcField != null && orcField.type.getCategory().equals(TypeDescription.Category.UNION)) { + Preconditions.checkState( + orcField.type.getChildren().size() == 1, "Expect single type union for orc schema."); + + orcType = TypeDescription.createUnion(); + Types.ListType list = type; + TypeDescription elementType = + buildOrcProjection( + list.elementId(), + list.elementType(), + isRequired && list.isElementRequired(), + mapping); + orcType.addUnionChild(TypeDescription.createList(elementType)); + } else { + Types.ListType list = type; + TypeDescription elementType = + buildOrcProjection( + list.elementId(), + list.elementType(), + isRequired && list.isElementRequired(), + mapping); + orcType = TypeDescription.createList(elementType); + } + return orcType; + } + + private static TypeDescription buildOrcProjectForStructType( + Integer fieldId, Type type, boolean isRequired, Map mapping) { + TypeDescription orcType; + OrcField orcField = mapping.getOrDefault(fieldId, null); + + if (orcField != null && orcField.type.getCategory().equals(TypeDescription.Category.UNION)) { + // this branch means the iceberg struct schema actually correspond to an underlying union + orcType = getOrcSchemaForUnionType(type, isRequired, mapping, orcField); + } else { + orcType = TypeDescription.createStruct(); + for (Types.NestedField nestedField : type.asStructType().fields()) { + // Using suffix _r to avoid potential underlying issues in ORC reader + // with reused column names between ORC and Iceberg; + // e.g. renaming column c -> d and adding new column d + if (mapping.get(nestedField.fieldId()) == null && nestedField.hasDefaultValue()) { + continue; + } + String name = + Optional.ofNullable(mapping.get(nestedField.fieldId())) + .map(OrcField::name) + .orElseGet(() -> nestedField.name() + "_r" + nestedField.fieldId()); + TypeDescription childType = + buildOrcProjection( + nestedField.fieldId(), + nestedField.type(), + isRequired && nestedField.isRequired(), + mapping); + orcType.addField(name, childType); + } + } + return orcType; + } + + private static TypeDescription getOrcSchemaForUnionType( + Type type, boolean isRequired, Map mapping, OrcField orcField) { + TypeDescription orcType; + if (orcField.type.getChildren().size() == 1) { // single type union + orcType = TypeDescription.createUnion(); + + TypeDescription childOrcStructType = TypeDescription.createStruct(); + for (Types.NestedField nestedField : type.asStructType().fields()) { + if (mapping.get(nestedField.fieldId()) == null && nestedField.hasDefaultValue()) { + continue; + } + String name = + Optional.ofNullable(mapping.get(nestedField.fieldId())) + .map(OrcField::name) + .orElseGet(() -> nestedField.name()); + TypeDescription childType = + buildOrcProjection( + nestedField.fieldId(), + nestedField.type(), + isRequired && nestedField.isRequired(), + mapping); + childOrcStructType.addField(name, childType); + } + + orcType.addUnionChild(childOrcStructType); + } else { // complex union + orcType = TypeDescription.createUnion(); + List nestedFields = type.asStructType().fields(); + for (Types.NestedField nestedField : nestedFields.subList(1, nestedFields.size())) { + TypeDescription childType = + buildOrcProjection( + nestedField.fieldId(), + nestedField.type(), + isRequired && nestedField.isRequired(), + mapping); + orcType.addUnionChild(childType); + } + } + return orcType; + } + private static Map icebergToOrcMapping(String name, TypeDescription orcType) { Map icebergToOrc = Maps.newHashMap(); switch (orcType.getCategory()) { @@ -348,6 +456,13 @@ private static Map icebergToOrcMapping(String name, TypeDescr icebergToOrc.putAll(icebergToOrcMapping(childrenNames.get(i), children.get(i))); } break; + case UNION: + // This is part of building orc read schema in file level. orcType has union type inside it. + List options = orcType.getChildren(); + for (int i = 0; i < options.size(); i++) { + icebergToOrc.putAll(icebergToOrcMapping("option" + i, options.get(i))); + } + break; case LIST: icebergToOrc.putAll(icebergToOrcMapping("element", orcType.getChildren().get(0))); break; diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java index 58cf5d1f9..03dfca01e 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.orc; import java.io.IOException; +import java.util.Set; import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Schema; @@ -32,6 +33,9 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.Pair; import org.apache.orc.Reader; import org.apache.orc.TypeDescription; @@ -51,6 +55,7 @@ class OrcIterable extends CloseableGroup implements CloseableIterable { private final Function> batchReaderFunction; private final int recordsPerBatch; private NameMapping nameMapping; + private final OrcRowFilter rowFilter; OrcIterable( InputFile file, @@ -63,7 +68,8 @@ class OrcIterable extends CloseableGroup implements CloseableIterable { boolean caseSensitive, Expression filter, Function> batchReaderFunction, - int recordsPerBatch) { + int recordsPerBatch, + OrcRowFilter rowFilter) { this.schema = schema; this.readerFunction = readerFunction; this.file = file; @@ -75,6 +81,7 @@ class OrcIterable extends CloseableGroup implements CloseableIterable { this.filter = (filter == Expressions.alwaysTrue()) ? null : filter; this.batchReaderFunction = batchReaderFunction; this.recordsPerBatch = recordsPerBatch; + this.rowFilter = rowFilter; } @SuppressWarnings("unchecked") @@ -85,35 +92,72 @@ public CloseableIterator iterator() { TypeDescription fileSchema = orcFileReader.getSchema(); final TypeDescription readOrcSchema; + final TypeDescription fileSchemaWithIds; if (ORCSchemaUtil.hasIds(fileSchema)) { - readOrcSchema = ORCSchemaUtil.buildOrcProjection(schema, fileSchema); + fileSchemaWithIds = fileSchema; } else { if (nameMapping == null) { nameMapping = MappingUtil.create(schema); } - TypeDescription typeWithIds = ORCSchemaUtil.applyNameMapping(fileSchema, nameMapping); - readOrcSchema = ORCSchemaUtil.buildOrcProjection(schema, typeWithIds); + fileSchemaWithIds = ORCSchemaUtil.applyNameMapping(fileSchema, nameMapping); } + readOrcSchema = ORCSchemaUtil.buildOrcProjection(schema, fileSchemaWithIds); + // If the projected ORC schema is an empty struct, it means we are only projecting columns + // with default values that aren't existent in previously written files, and thus we won't need + // to push down filters to ORC's SearchArgument, since we are not reading anything from files at + // all + boolean isEmptyStruct = readOrcSchema.getChildren().size() == 0; + + // If the projected ORC schema has union type, we can't reliably build a pushdown filter for the + // underlying ORC files, since the Iceberg schema has a different representation than the ORC + // schema. + boolean containsUnion = readOrcSchema.toString().contains("uniontype"); SearchArgument sarg = null; - if (filter != null) { + if (filter != null && !isEmptyStruct && !containsUnion) { Expression boundFilter = Binder.bind(schema.asStruct(), filter, caseSensitive); sarg = ExpressionToSearchArgument.convert(boundFilter, readOrcSchema); } - VectorizedRowBatchIterator rowBatchIterator = - newOrcIterator(file, readOrcSchema, start, length, orcFileReader, sarg, recordsPerBatch); - if (batchReaderFunction != null) { - OrcBatchReader batchReader = (OrcBatchReader) batchReaderFunction.apply(readOrcSchema); - return CloseableIterator.transform( - rowBatchIterator, - pair -> { - batchReader.setBatchContext(pair.second()); - return batchReader.read(pair.first()); - }); + if (rowFilter == null) { + VectorizedRowBatchIterator rowBatchIterator = + newOrcIterator(file, readOrcSchema, start, length, orcFileReader, sarg, recordsPerBatch); + if (batchReaderFunction != null) { + OrcBatchReader batchReader = + (OrcBatchReader) batchReaderFunction.apply(readOrcSchema); + return CloseableIterator.transform( + rowBatchIterator, + pair -> { + batchReader.setBatchContext(pair.second()); + return batchReader.read(pair.first()); + }); + } else { + return new OrcRowIterator<>( + rowBatchIterator, (OrcRowReader) readerFunction.apply(readOrcSchema), null, null); + } } else { + Preconditions.checkArgument( + batchReaderFunction == null, "Row-level filtering not supported by vectorized reader"); + Set filterColumnIds = TypeUtil.getProjectedIds(rowFilter.requiredSchema()); + Set filterColumnIdsNotInReadSchema = + Sets.difference(filterColumnIds, TypeUtil.getProjectedIds(schema)); + Schema extraFilterColumns = + TypeUtil.select(rowFilter.requiredSchema(), filterColumnIdsNotInReadSchema); + Schema finalReadSchema = TypeUtil.join(schema, extraFilterColumns); + + TypeDescription finalReadOrcSchema = + ORCSchemaUtil.buildOrcProjection(finalReadSchema, fileSchemaWithIds); + TypeDescription rowFilterOrcSchema = + ORCSchemaUtil.buildOrcProjection(rowFilter.requiredSchema(), fileSchemaWithIds); + RowFilterValueReader filterReader = + new RowFilterValueReader(finalReadOrcSchema, rowFilterOrcSchema); + return new OrcRowIterator<>( - rowBatchIterator, (OrcRowReader) readerFunction.apply(readOrcSchema)); + newOrcIterator( + file, finalReadOrcSchema, start, length, orcFileReader, sarg, recordsPerBatch), + (OrcRowReader) readerFunction.apply(readOrcSchema), + rowFilter, + filterReader); } } @@ -142,37 +186,70 @@ private static VectorizedRowBatchIterator newOrcIterator( private static class OrcRowIterator implements CloseableIterator { - private int nextRow; - private VectorizedRowBatch current; - private int currentBatchSize; + private int currentRow; + private VectorizedRowBatch currentBatch; + private boolean advanced = false; private final VectorizedRowBatchIterator batchIter; private final OrcRowReader reader; + private final OrcRowFilter filter; + private final RowFilterValueReader filterReader; - OrcRowIterator(VectorizedRowBatchIterator batchIter, OrcRowReader reader) { + OrcRowIterator( + VectorizedRowBatchIterator batchIter, + OrcRowReader reader, + OrcRowFilter filter, + RowFilterValueReader filterReader) { this.batchIter = batchIter; this.reader = reader; - current = null; - nextRow = 0; - currentBatchSize = 0; + this.filter = filter; + this.filterReader = filterReader; + currentBatch = null; + currentRow = 0; + } + + private void advance() { + if (!advanced) { + while (true) { + currentRow++; + // if batch has been consumed, move to next batch + if (currentBatch == null || currentRow >= currentBatch.size) { + if (batchIter.hasNext()) { + Pair nextBatch = batchIter.next(); + currentBatch = nextBatch.first(); + currentRow = 0; + reader.setBatchContext(nextBatch.second()); + if (filterReader != null) { + filterReader.setBatchContext(nextBatch.second()); + } + } else { + // no more batches left to process + currentBatch = null; + currentRow = -1; + break; + } + } + if (filter == null || filter.shouldKeep(filterReader.read(currentBatch, currentRow))) { + // we have found our row + break; + } + } + advanced = true; + } } @Override public boolean hasNext() { - return (current != null && nextRow < currentBatchSize) || batchIter.hasNext(); + advance(); + return currentBatch != null; } @Override public T next() { - if (current == null || nextRow >= currentBatchSize) { - Pair nextBatch = batchIter.next(); - current = nextBatch.first(); - currentBatchSize = current.size; - nextRow = 0; - this.reader.setBatchContext(nextBatch.second()); - } - - return this.reader.read(current, nextRow++); + advance(); + // mark current row as used + advanced = false; + return this.reader.read(currentBatch, currentRow); } @Override diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilter.java b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilter.java new file mode 100644 index 000000000..46aff5fde --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilter.java @@ -0,0 +1,47 @@ +/* + * 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.iceberg.orc; + +import java.io.Serializable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; + +/** + * [LINKEDIN ONLY] Represents a filter which can be applied to every row + * + *

Currently Row filter evaluation is only supported in Spark on unpartitioned tables for top + * level primitives + */ +public interface OrcRowFilter extends Serializable { + + /** + * Returns a {@link Schema} which includes all fields required from the table {@link Schema} to + * evaluate this filter. + */ + Schema requiredSchema(); + + /** + * Decides whether the row described by the provided field values should be kept or not. + * + *

The order of field values inside the values array is the same as the order of field ids + * returned by {@link #requiredSchema()}. The java class of the field value is dictated by {@link + * Type.TypeID#javaClass()}. + */ + boolean shouldKeep(Object[] values); +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterUtils.java b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterUtils.java new file mode 100644 index 000000000..0a7f2a799 --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterUtils.java @@ -0,0 +1,40 @@ +/* + * 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.iceberg.orc; + +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.FileScanTask; + +public class OrcRowFilterUtils { + + private OrcRowFilterUtils() {} + + public static OrcRowFilter rowFilterFromTask(FileScanTask task) { + FileScanTask fileScanTask; + if (task instanceof BaseFileScanTask.SplitScanTask) { + fileScanTask = ((BaseFileScanTask.SplitScanTask) task).underlyingFileScanTask(); + } else { + fileScanTask = task; + } + if (fileScanTask instanceof OrcRowFilterableFileScanTask) { + return ((OrcRowFilterableFileScanTask) fileScanTask).orcRowFilter(); + } + return null; + } +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterableFileScanTask.java b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterableFileScanTask.java new file mode 100644 index 000000000..cb6dcba87 --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcRowFilterableFileScanTask.java @@ -0,0 +1,27 @@ +/* + * 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.iceberg.orc; + +import org.apache.iceberg.FileScanTask; + +public interface OrcRowFilterableFileScanTask extends FileScanTask { + + /** [LINKEDIN ONLY] Returns a row filter to be applied to rows in a file scan */ + OrcRowFilter orcRowFilter(); +} diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaVisitor.java b/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaVisitor.java index 455aca15f..53a7837c5 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaVisitor.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaVisitor.java @@ -44,7 +44,23 @@ public static T visit(TypeDescription schema, OrcSchemaVisitor visitor) { return visitRecord(schema, visitor); case UNION: - throw new UnsupportedOperationException("Cannot handle " + schema); + List types = schema.getChildren(); + List options = Lists.newArrayListWithExpectedSize(types.size()); + + if (types.size() == 1) { + options.add(visit(types.get(0), visitor)); + } else { + for (int i = 0; i < types.size(); i++) { + visitor.beforeUnionOption(types.get(i), i); + try { + options.add(visit(types.get(i), visitor)); + } finally { + visitor.afterUnionOption(types.get(i), i); + } + } + } + + return visitor.union(schema, options); case LIST: final T elementResult; @@ -110,6 +126,10 @@ private static T visitRecord(TypeDescription record, OrcSchemaVisitor vis return visitor.record(record, names, visitFields(fields, names, visitor)); } + public String optionName(int ordinal) { + return "field" + ordinal; + } + public String elementName() { return "_elem"; } @@ -134,6 +154,14 @@ public void afterField(String name, TypeDescription type) { fieldNames.pop(); } + public void beforeUnionOption(TypeDescription option, int ordinal) { + beforeField(optionName(ordinal), option); + } + + public void afterUnionOption(TypeDescription option, int ordinal) { + afterField(optionName(ordinal), option); + } + public void beforeElementField(TypeDescription element) { beforeField(elementName(), element); } @@ -162,6 +190,10 @@ public T record(TypeDescription record, List names, List fields) { return null; } + public T union(TypeDescription union, List options) { + return null; + } + public T list(TypeDescription array, T element) { return null; } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaWithTypeVisitor.java b/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaWithTypeVisitor.java index fd37283a8..edd251892 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaWithTypeVisitor.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcSchemaWithTypeVisitor.java @@ -36,10 +36,10 @@ public static T visit( Type iType, TypeDescription schema, OrcSchemaWithTypeVisitor visitor) { switch (schema.getCategory()) { case STRUCT: - return visitRecord(iType != null ? iType.asStructType() : null, schema, visitor); + return visitor.visitRecord(iType != null ? iType.asStructType() : null, schema, visitor); case UNION: - throw new UnsupportedOperationException("Cannot handle " + schema); + return visitor.visitUnion(iType, schema, visitor); case LIST: Types.ListType list = iType != null ? iType.asListType() : null; @@ -61,7 +61,7 @@ public static T visit( } } - private static T visitRecord( + protected T visitRecord( Types.StructType struct, TypeDescription record, OrcSchemaWithTypeVisitor visitor) { List fields = record.getChildren(); List names = record.getFieldNames(); @@ -74,11 +74,30 @@ private static T visitRecord( return visitor.record(struct, record, names, results); } + protected T visitUnion(Type type, TypeDescription union, OrcSchemaWithTypeVisitor visitor) { + List types = union.getChildren(); + List options = Lists.newArrayListWithCapacity(types.size()); + + if (types.size() == 1) { // single type union + options.add(visit(type, types.get(0), visitor)); + } else { // complex union + for (int i = 0; i < types.size(); i += 1) { + options.add(visit(type.asStructType().fields().get(i + 1).type(), types.get(i), visitor)); + } + } + + return visitor.union(type, union, options); + } + public T record( Types.StructType iStruct, TypeDescription record, List names, List fields) { return null; } + public T union(Type iUnion, TypeDescription union, List options) { + return null; + } + public T list(Types.ListType iList, TypeDescription array, T element) { return null; } diff --git a/orc/src/main/java/org/apache/iceberg/orc/RowFilterValueReader.java b/orc/src/main/java/org/apache/iceberg/orc/RowFilterValueReader.java new file mode 100644 index 000000000..0eaa77c60 --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/RowFilterValueReader.java @@ -0,0 +1,144 @@ +/* + * 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.iceberg.orc; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +class RowFilterValueReader implements OrcRowReader { + + private final Converter[] converters; + private final List columns; + private final int[] filterSchemaToReadSchemaColumnIndex; + + RowFilterValueReader(TypeDescription readSchema, TypeDescription filterSchema) { + columns = filterSchema.getChildren(); + converters = buildConverters(); + filterSchemaToReadSchemaColumnIndex = + buildFilterSchemaToReadSchemaColumnIndex(readSchema, filterSchema); + } + + private int[] buildFilterSchemaToReadSchemaColumnIndex( + TypeDescription readSchema, TypeDescription filterSchema) { + int[] index = new int[filterSchema.getChildren().size()]; + List filterFieldNames = filterSchema.getFieldNames(); + List readSchemaFieldNames = readSchema.getFieldNames(); + Map readSchemaFieldNameToIndex = Maps.newHashMap(); + for (int i = 0; i < readSchemaFieldNames.size(); i++) { + readSchemaFieldNameToIndex.put(readSchemaFieldNames.get(i), i); + } + for (int i = 0; i < filterFieldNames.size(); i++) { + index[i] = readSchemaFieldNameToIndex.get(filterFieldNames.get(i)); + } + return index; + } + + @Override + public Object[] read(VectorizedRowBatch batch, int row) { + Object[] rowFields = new Object[converters.length]; + for (int c = 0; c < converters.length; ++c) { + rowFields[c] = converters[c].convert(batch.cols[filterSchemaToReadSchemaColumnIndex[c]], row); + } + return rowFields; + } + + @Override + public void setBatchContext(long batchOffsetInFile) {} + + interface Converter { + default T convert(ColumnVector vector, int row) { + int rowIndex = vector.isRepeating ? 0 : row; + if (!vector.noNulls && vector.isNull[rowIndex]) { + return null; + } else { + return convertNonNullValue(vector, rowIndex); + } + } + + T convertNonNullValue(ColumnVector vector, int row); + } + + private Converter[] buildConverters() { + Converter[] newConverters = new Converter[columns.size()]; + for (int c = 0; c < newConverters.length; ++c) { + newConverters[c] = buildConverter(columns.get(c)); + } + return newConverters; + } + + private static Converter buildConverter(final TypeDescription schema) { + switch (schema.getCategory()) { + case INT: + return new IntConverter(); + case LONG: + String longAttributeValue = + schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE); + ORCSchemaUtil.LongType longType = + longAttributeValue == null + ? ORCSchemaUtil.LongType.LONG + : ORCSchemaUtil.LongType.valueOf(longAttributeValue); + switch (longType) { + case LONG: + return new LongConverter(); + default: + throw new IllegalStateException( + "Unhandled Long type found in ORC type attribute: " + longType); + } + case STRING: + case CHAR: + case VARCHAR: + return new StringConverter(); + default: + throw new IllegalArgumentException("Unhandled type " + schema); + } + } + + private static class IntConverter implements Converter { + @Override + public Integer convertNonNullValue(ColumnVector vector, int row) { + return (int) ((LongColumnVector) vector).vector[row]; + } + } + + private static class LongConverter implements Converter { + @Override + public Long convertNonNullValue(ColumnVector vector, int row) { + return ((LongColumnVector) vector).vector[row]; + } + } + + private static class StringConverter implements Converter { + @Override + public String convertNonNullValue(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + return new String( + bytesVector.vector[row], + bytesVector.start[row], + bytesVector.length[row], + StandardCharsets.UTF_8); + } + } +} diff --git a/settings.gradle b/settings.gradle index 3f6e2cf03..4c022c6c6 100644 --- a/settings.gradle +++ b/settings.gradle @@ -31,9 +31,12 @@ include 'bundled-guava' include 'spark' include 'pig' include 'hive-metastore' +include 'hivelink-core' include 'nessie' include 'gcp' include 'dell' +include 'runtime' +include 'depend-all' project(':api').name = 'iceberg-api' project(':common').name = 'iceberg-common' @@ -48,9 +51,12 @@ project(':bundled-guava').name = 'iceberg-bundled-guava' project(':spark').name = 'iceberg-spark' project(':pig').name = 'iceberg-pig' project(':hive-metastore').name = 'iceberg-hive-metastore' +project(':hivelink-core').name = 'iceberg-hivelink-core' project(':nessie').name = 'iceberg-nessie' project(':gcp').name = 'iceberg-gcp' project(':dell').name = 'iceberg-dell' +project(':runtime').name = 'iceberg-runtime' +project(':depend-all').name = 'li-iceberg-depend-all' List knownFlinkVersions = System.getProperty("knownFlinkVersions").split(",") String flinkVersionsString = System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions") diff --git a/shipkit.gradle b/shipkit.gradle new file mode 100644 index 000000000..bcf3e5fb2 --- /dev/null +++ b/shipkit.gradle @@ -0,0 +1,55 @@ +/* + * 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. + */ + +//Plugin jars are added to the buildscript classpath in the root build.gradle file +apply plugin: "org.shipkit.shipkit-auto-version" //https://github.com/shipkit/shipkit-auto-version + +apply plugin: "org.shipkit.shipkit-changelog" //https://github.com/shipkit/shipkit-changelog +tasks.named("generateChangelog") { + previousRevision = project.ext.'shipkit-auto-version.previous-tag' + githubToken = System.getenv("GITHUB_TOKEN") + repository = "linkedin/iceberg" +} + +apply plugin: "org.shipkit.shipkit-github-release" //https://github.com/shipkit/shipkit-changelog +tasks.named("githubRelease") { + def genTask = tasks.named("generateChangelog").get() + dependsOn genTask + repository = genTask.repository + changelog = genTask.outputFile + githubToken = System.getenv("GITHUB_TOKEN") + newTagRevision = System.getenv("GITHUB_SHA") +} + +apply plugin: "io.github.gradle-nexus.publish-plugin" //https://github.com/gradle-nexus/publish-plugin/ +nexusPublishing { + repositories { + if (System.getenv("SONATYPE_PWD")) { + sonatype { + username = System.getenv("SONATYPE_USER") + password = System.getenv("SONATYPE_PWD") + nexusUrl.set(uri("https://s01.oss.sonatype.org/service/local/")) + snapshotRepositoryUrl.set(uri("https://s01.oss.sonatype.org/content/repositories/snapshots/")) + } + } + } + transitionCheckOptions { + maxRetries.set(100) + } +} diff --git a/spark/v2.4/build.gradle b/spark/v2.4/build.gradle index 4cc20baea..8ad6c2fa2 100644 --- a/spark/v2.4/build.gradle +++ b/spark/v2.4/build.gradle @@ -60,6 +60,7 @@ project(':iceberg-spark:iceberg-spark-2.4') { implementation project(':iceberg-orc') implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') + implementation project(':iceberg-hivelink-core') implementation "com.github.ben-manes.caffeine:caffeine" compileOnly "com.google.errorprone:error_prone_annotations" @@ -165,6 +166,7 @@ project(':iceberg-spark:iceberg-spark-runtime-2.4') { relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' // relocate Arrow and related deps to shade Iceberg specific version relocate 'io.netty.buffer', 'org.apache.iceberg.shaded.io.netty.buffer' + relocate 'io.netty.util', 'org.apache.iceberg.shaded.io.netty.util' relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow' relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java new file mode 100644 index 000000000..4521ee80f --- /dev/null +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java @@ -0,0 +1,121 @@ +/* + * 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.iceberg.spark; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.orc.ORCSchemaUtil; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.BaseDataReader; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; + +public abstract class OrcSchemaWithTypeVisitorSpark extends OrcSchemaWithTypeVisitor { + + private final Map idToConstant; + + public Map getIdToConstant() { + return idToConstant; + } + + protected OrcSchemaWithTypeVisitorSpark(Map idToConstant) { + this.idToConstant = Maps.newHashMap(); + this.idToConstant.putAll(idToConstant); + } + + @Override + protected T visitRecord( + Types.StructType struct, TypeDescription record, OrcSchemaWithTypeVisitor visitor) { + Preconditions.checkState( + icebergFiledIdsContainOrcFieldIdsInOrder(struct, record), + "Iceberg schema and ORC schema doesn't align, please call ORCSchemaUtil.buildOrcProjection" + + "to get an aligned ORC schema first!"); + List iFields = struct.fields(); + List fields = record.getChildren(); + List names = record.getFieldNames(); + List results = Lists.newArrayListWithExpectedSize(fields.size()); + + for (int i = 0, j = 0; i < iFields.size(); i++) { + Types.NestedField iField = iFields.get(i); + TypeDescription field = j < fields.size() ? fields.get(j) : null; + if (field == null || (iField.fieldId() != ORCSchemaUtil.fieldId(field))) { + // there are 3 cases where we need to use idToConstant for an iField + // 1. The field is MetadataColumns.ROW_POSITION, we build a RowPositionReader + // 2. The field is a partition column, we build a ConstantReader + // 3. The field should be read using the default value, where we build a ConstantReader + // Here we should only need to update idToConstant when it's the 3rd case, + // because the first 2 cases have been handled by logic in PartitionUtil.constantsMap + if (MetadataColumns.nonMetadataColumn(iField.name()) + && !idToConstant.containsKey(iField.fieldId())) { + idToConstant.put( + iField.fieldId(), + BaseDataReader.convertConstant(iField.type(), iField.getDefaultValue())); + } + } else { + results.add(visit(iField.type(), field, visitor)); + j++; + } + } + return visitor.record(struct, record, names, results); + } + + private static boolean icebergFiledIdsContainOrcFieldIdsInOrder( + Types.StructType struct, TypeDescription record) { + List icebergIDList = + struct.fields().stream().map(Types.NestedField::fieldId).collect(Collectors.toList()); + List orcIDList = + record.getChildren().stream().map(ORCSchemaUtil::fieldId).collect(Collectors.toList()); + + return containsInOrder(icebergIDList, orcIDList); + } + + /** + * Checks whether the first list contains all the integers in the same order as regarding to the + * second list, the first list can contain extra integers that the second list doesn't, but the + * ones that exist in the second list should occur in the same relative order in the first list. + * + * @param list1 the first list + * @param list2 the second list + * @return the condition check result + */ + private static boolean containsInOrder(List list1, List list2) { + if (list1.size() < list2.size()) { + return false; + } + + for (int i = 0, j = 0; j < list2.size(); j++) { + if (i >= list1.size()) { + return false; + } + while (!list1.get(i).equals(list2.get(j))) { + i++; + if (i >= list1.size()) { + return false; + } + } + i++; + } + return true; + } +} diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java index 4622d2928..2c8494787 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java @@ -27,6 +27,7 @@ import org.apache.avro.Schema; import org.apache.avro.io.DatumReader; import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; import org.apache.iceberg.avro.SupportsRowPosition; import org.apache.iceberg.avro.ValueReader; @@ -88,7 +89,11 @@ public ValueReader record( @Override public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); + if (AvroSchemaUtil.isOptionSchema(union) || AvroSchemaUtil.isSingleTypeUnion(union)) { + return ValueReaders.union(options); + } else { + return SparkValueReaders.union(union, options); + } } @Override diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 78db13705..6cee60ce0 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -25,6 +25,7 @@ import org.apache.iceberg.orc.OrcValueReader; import org.apache.iceberg.orc.OrcValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.OrcSchemaWithTypeVisitorSpark; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; @@ -64,11 +65,10 @@ public void setBatchContext(long batchOffsetInFile) { reader.setBatchContext(batchOffsetInFile); } - private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { - private final Map idToConstant; + public static class ReadBuilder extends OrcSchemaWithTypeVisitorSpark> { private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; + super(idToConstant); } @Override @@ -77,7 +77,7 @@ public OrcValueReader record( TypeDescription record, List names, List> fields) { - return SparkOrcValueReaders.struct(fields, expected, idToConstant); + return SparkOrcValueReaders.struct(fields, expected, getIdToConstant()); } @Override @@ -95,6 +95,12 @@ public OrcValueReader map( return SparkOrcValueReaders.map(keyReader, valueReader); } + @Override + public OrcValueReader union( + Type expected, TypeDescription union, List> options) { + return SparkOrcValueReaders.union(options); + } + @Override public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { switch (primitive.getCategory()) { diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java index 9e9b3e53b..d2f60eb97 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java @@ -32,6 +32,7 @@ import org.apache.orc.storage.ql.exec.vector.ListColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.serde2.io.HiveDecimalWritable; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -76,6 +77,10 @@ static OrcValueReader map(OrcValueReader keyReader, OrcValueReader valu return new MapReader(keyReader, valueReader); } + static OrcValueReader union(List> readers) { + return new UnionReader(readers); + } + private static class ArrayReader implements OrcValueReader { private final OrcValueReader elementReader; @@ -157,6 +162,37 @@ protected void set(InternalRow struct, int pos, Object value) { } } + static class UnionReader implements OrcValueReader { + private final OrcValueReader[] readers; + + private UnionReader(List> readers) { + this.readers = new OrcValueReader[readers.size()]; + for (int i = 0; i < this.readers.length; i += 1) { + this.readers[i] = readers.get(i); + } + } + + @Override + public Object nonNullRead(ColumnVector vector, int row) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + int fieldIndex = unionColumnVector.tags[row]; + Object value = this.readers[fieldIndex].read(unionColumnVector.fields[fieldIndex], row); + + if (readers.length == 1) { + return value; + } else { + InternalRow struct = new GenericInternalRow(readers.length + 1); + for (int i = 0; i < readers.length; i += 1) { + struct.setNullAt(i + 1); + } + struct.update(0, fieldIndex); + struct.update(fieldIndex + 1, value); + + return struct; + } + } + } + private static class StringReader implements OrcValueReader { private static final StringReader INSTANCE = new StringReader(); diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java index 11655c72d..92776a405 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -26,6 +26,8 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.Objects; +import org.apache.avro.Schema; import org.apache.avro.io.Decoder; import org.apache.avro.util.Utf8; import org.apache.iceberg.avro.ValueReader; @@ -40,6 +42,8 @@ import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SparkValueReaders { @@ -79,6 +83,10 @@ static ValueReader struct( return new StructReader(readers, struct, idToConstant); } + static ValueReader union(Schema schema, List> readers) { + return new UnionReader(schema, readers); + } + private static class StringReader implements ValueReader { private static final StringReader INSTANCE = new StringReader(); @@ -102,6 +110,7 @@ public UTF8String read(Decoder decoder, Object reuse) throws IOException { } private static class EnumReader implements ValueReader { + private static final Logger LOG = LoggerFactory.getLogger(EnumReader.class); private final UTF8String[] symbols; private EnumReader(List symbols) { @@ -114,7 +123,14 @@ private EnumReader(List symbols) { @Override public UTF8String read(Decoder decoder, Object ignore) throws IOException { int index = decoder.readEnum(); - return symbols[index]; + if (index >= symbols.length) { + LOG.error( + "Unable to read the symbol in the given enum as the deserialized index {} is out of bound", + index); + throw new ArrayIndexOutOfBoundsException(); + } else { + return symbols[index]; + } } } @@ -285,4 +301,59 @@ protected void set(InternalRow struct, int pos, Object value) { } } } + + private static class UnionReader implements ValueReader { + private final Schema schema; + private final ValueReader[] readers; + + private UnionReader(Schema schema, List> readers) { + this.schema = schema; + this.readers = new ValueReader[readers.size()]; + for (int i = 0; i < this.readers.length; i += 1) { + this.readers[i] = readers.get(i); + } + } + + @Override + public InternalRow read(Decoder decoder, Object reuse) throws IOException { + // first we need to filter out NULL alternative if it exists in the union schema + int nullIndex = -1; + List alts = schema.getTypes(); + for (int i = 0; i < alts.size(); i++) { + Schema alt = alts.get(i); + if (Objects.equals(alt.getType(), Schema.Type.NULL)) { + nullIndex = i; + break; + } + } + + int index = decoder.readIndex(); + if (index == nullIndex) { + // if it is a null data, directly return null as the whole union result + // we know for sure it is a null so the casting will always work. + return (InternalRow) readers[nullIndex].read(decoder, reuse); + } + + // otherwise, we need to return an InternalRow as a struct data + InternalRow struct = new GenericInternalRow(nullIndex >= 0 ? alts.size() : alts.size() + 1); + for (int i = 0; i < struct.numFields(); i += 1) { + struct.setNullAt(i); + } + + Object value = readers[index].read(decoder, reuse); + + if (nullIndex < 0) { + struct.update(index + 1, value); + struct.setInt(0, index); + } else if (index < nullIndex) { + struct.update(index + 1, value); + struct.setInt(0, index); + } else { + struct.update(index, value); + struct.setInt(0, index - 1); + } + + return struct; + } + } } diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java new file mode 100644 index 000000000..2208594bb --- /dev/null +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.spark.data.vectorized; + +import java.util.Arrays; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class ConstantArrayColumnVector extends ConstantColumnVector { + + private final Object[] constantArray; + + public ConstantArrayColumnVector(DataType type, int batchSize, Object[] constantArray) { + super(type, batchSize, constantArray); + this.constantArray = constantArray; + } + + @Override + public boolean getBoolean(int rowId) { + return (boolean) constantArray[rowId]; + } + + @Override + public byte getByte(int rowId) { + return (byte) constantArray[rowId]; + } + + @Override + public short getShort(int rowId) { + return (short) constantArray[rowId]; + } + + @Override + public int getInt(int rowId) { + return (int) constantArray[rowId]; + } + + @Override + public long getLong(int rowId) { + return (long) constantArray[rowId]; + } + + @Override + public float getFloat(int rowId) { + return (float) constantArray[rowId]; + } + + @Override + public double getDouble(int rowId) { + return (double) constantArray[rowId]; + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + return (Decimal) constantArray[rowId]; + } + + @Override + public UTF8String getUTF8String(int rowId) { + return (UTF8String) constantArray[rowId]; + } + + @Override + public byte[] getBinary(int rowId) { + return (byte[]) constantArray[rowId]; + } + + @Override + public ColumnarArray getArray(int rowId) { + return new ColumnarArray( + new ConstantArrayColumnVector( + ((ArrayType) type).elementType(), + getBatchSize(), + ((ArrayData) constantArray[rowId]).array()), + 0, + ((ArrayData) constantArray[rowId]).numElements()); + } + + @Override + public ColumnarMap getMap(int rowId) { + ColumnVector keys = + new ConstantArrayColumnVector( + ((MapType) type).keyType(), + getBatchSize(), + ((MapData) constantArray[rowId]).keyArray().array()); + ColumnVector values = + new ConstantArrayColumnVector( + ((MapType) type).valueType(), + getBatchSize(), + ((MapData) constantArray[rowId]).valueArray().array()); + return new ColumnarMap(keys, values, 0, ((MapData) constantArray[rowId]).numElements()); + } + + @Override + protected ColumnVector getChild(int ordinal) { + DataType fieldType = ((StructType) type).fields()[ordinal].dataType(); + return new ConstantArrayColumnVector( + fieldType, + getBatchSize(), + Arrays.stream(constantArray) + .map(row -> ((InternalRow) row).get(ordinal, fieldType)) + .toArray()); + } +} diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java index 8a0b329eb..d411ee454 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java @@ -20,7 +20,14 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarArray; import org.apache.spark.sql.vectorized.ColumnarMap; @@ -37,6 +44,16 @@ class ConstantColumnVector extends ColumnVector { this.batchSize = batchSize; } + ConstantColumnVector(DataType type, int batchSize, Object constant) { + super(type); + this.constant = constant; + this.batchSize = batchSize; + } + + protected int getBatchSize() { + return batchSize; + } + @Override public void close() {} @@ -92,12 +109,22 @@ public double getDouble(int rowId) { @Override public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + return new ColumnarArray( + new ConstantArrayColumnVector( + ((ArrayType) type).elementType(), batchSize, ((ArrayData) constant).array()), + 0, + ((ArrayData) constant).numElements()); } @Override public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + ColumnVector keys = + new ConstantArrayColumnVector( + ((MapType) type).keyType(), batchSize, ((MapData) constant).keyArray().array()); + ColumnVector values = + new ConstantArrayColumnVector( + ((MapType) type).valueType(), batchSize, ((MapData) constant).valueArray().array()); + return new ColumnarMap(keys, values, 0, ((MapData) constant).numElements()); } @Override @@ -117,6 +144,8 @@ public byte[] getBinary(int rowId) { @Override protected ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + DataType fieldType = ((StructType) type).fields()[ordinal].dataType(); + return new ConstantColumnVector( + fieldType, batchSize, ((InternalRow) constant).get(ordinal, fieldType)); } } diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 7c3b825a6..d2fe70824 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.data.vectorized; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.IntStream; @@ -28,14 +29,17 @@ import org.apache.iceberg.orc.OrcValueReader; import org.apache.iceberg.orc.OrcValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.OrcSchemaWithTypeVisitorSpark; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.SparkOrcValueReaders; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.vectorized.ColumnVector; @@ -85,11 +89,10 @@ ColumnVector convert( long batchOffsetInFile); } - private static class ReadBuilder extends OrcSchemaWithTypeVisitor { - private final Map idToConstant; + private static class ReadBuilder extends OrcSchemaWithTypeVisitorSpark { private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; + super(idToConstant); } @Override @@ -98,7 +101,12 @@ public Converter record( TypeDescription record, List names, List fields) { - return new StructConverter(iStruct, fields, idToConstant); + return new StructConverter(iStruct, fields, getIdToConstant()); + } + + @Override + public Converter union(Type iType, TypeDescription union, List options) { + return new UnionConverter(iType, options); } @Override @@ -456,4 +464,56 @@ public ColumnVector getChild(int ordinal) { }; } } + + private static class UnionConverter implements Converter { + private final Type type; + private final List optionConverters; + + private UnionConverter(Type type, List optionConverters) { + this.type = type; + this.optionConverters = optionConverters; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + if (optionConverters.size() > 1) { + // the case of complex union with multiple types + List fields = type.asStructType().fields(); + List fieldVectors = Lists.newArrayListWithExpectedSize(fields.size()); + + LongColumnVector longColumnVector = new LongColumnVector(); + longColumnVector.vector = Arrays.stream(unionColumnVector.tags).asLongStream().toArray(); + + fieldVectors.add( + new PrimitiveOrcColumnVector( + Types.IntegerType.get(), + batchSize, + longColumnVector, + OrcValueReaders.ints(), + batchOffsetInFile)); + for (int i = 0; i < fields.size() - 1; i += 1) { + fieldVectors.add( + optionConverters + .get(i) + .convert(unionColumnVector.fields[i], batchSize, batchOffsetInFile)); + } + + return new BaseOrcColumnVector(type.asStructType(), batchSize, vector) { + @Override + public ColumnVector getChild(int ordinal) { + return fieldVectors.get(ordinal); + } + }; + } else { + // the case of single type union + return optionConverters + .get(0) + .convert(unionColumnVector.fields[0], batchSize, batchOffsetInFile); + } + } + } } diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java index 2cab8ee23..3c83837ba 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java @@ -22,9 +22,11 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericData; import org.apache.avro.util.Utf8; @@ -41,6 +43,7 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types.NestedField; @@ -48,18 +51,23 @@ import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.PartitionUtil; import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; /** * Base class of Spark readers. * * @param is the Java class returned by this reader whose objects contain one or more rows. */ -abstract class BaseDataReader implements Closeable { +public abstract class BaseDataReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(BaseDataReader.class); private final Table table; @@ -156,12 +164,50 @@ protected InputFile getInputFile(String location) { } } - protected static Object convertConstant(Type type, Object value) { + public static Object convertConstant(Type type, Object value) { if (value == null) { return null; } switch (type.typeId()) { + case STRUCT: + StructType structType = type.asStructType(); + + if (structType.fields().isEmpty()) { + return new GenericInternalRow(); + } + + InternalRow ret = new GenericInternalRow(structType.fields().size()); + for (int i = 0; i < structType.fields().size(); i++) { + NestedField field = structType.fields().get(i); + Type fieldType = field.type(); + if (value instanceof Map) { + ret.update(i, convertConstant(field.type(), ((Map) value).get(field.name()))); + } else { + ret.update( + i, + convertConstant( + fieldType, ((StructLike) value).get(i, fieldType.typeId().javaClass()))); + } + } + return ret; + case LIST: + List javaList = + ((Collection) value) + .stream() + .map(e -> convertConstant(type.asListType().elementType(), e)) + .collect(Collectors.toList()); + return ArrayData.toArrayData( + JavaConverters.collectionAsScalaIterableConverter(javaList).asScala().toSeq()); + case MAP: + List keyList = Lists.newArrayList(); + List valueList = Lists.newArrayList(); + for (Map.Entry entry : ((Map) value).entrySet()) { + keyList.add(convertConstant(type.asMapType().keyType(), entry.getKey())); + valueList.add(convertConstant(type.asMapType().valueType(), entry.getValue())); + } + return new ArrayBasedMapData( + new GenericArrayData(keyList.toArray()), new GenericArrayData(valueList.toArray())); case DECIMAL: return Decimal.apply((BigDecimal) value); case STRING: @@ -179,25 +225,6 @@ protected static Object convertConstant(Type type, Object value) { return ByteBuffers.toByteArray((ByteBuffer) value); case BINARY: return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - StructType structType = (StructType) type; - - if (structType.fields().isEmpty()) { - return new GenericInternalRow(); - } - - List fields = structType.fields(); - Object[] values = new Object[fields.size()]; - StructLike struct = (StructLike) value; - - for (int index = 0; index < fields.size(); index++) { - NestedField field = fields.get(index); - Type fieldType = field.type(); - values[index] = - convertConstant(fieldType, struct.get(index, fieldType.typeId().javaClass())); - } - - return new GenericInternalRow(values); default: } return value; diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 94396d218..126a841a0 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.OptionalLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -43,7 +44,9 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.hivelink.core.LegacyHiveTable; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -355,6 +358,11 @@ public void pruneColumns(StructType newRequestedSchema) { @Override public Statistics estimateStatistics() { + if (table instanceof LegacyHiveTable) { + // We currently don't have reliable stats for Hive tables + return EMPTY_STATS; + } + // its a fresh table, no data if (table.currentSnapshot() == null) { return new Stats(0L, 0L); @@ -385,6 +393,19 @@ public Statistics estimateStatistics() { return new Stats(sizeInBytes, numRows); } + private static final Statistics EMPTY_STATS = + new Statistics() { + @Override + public OptionalLong sizeInBytes() { + return OptionalLong.empty(); + } + + @Override + public OptionalLong numRows() { + return OptionalLong.empty(); + } + }; + @Override public boolean enableBatchRead() { if (readUsingBatch == null) { @@ -408,6 +429,16 @@ public boolean enableBatchRead() { fileScanTask -> fileScanTask.file().format().equals(FileFormat.ORC))); + boolean hasNoRowFilters = + tasks().stream() + .allMatch( + combinedScanTask -> + !combinedScanTask.isDataTask() + && combinedScanTask.files().stream() + .allMatch( + fileScanTask -> + OrcRowFilterUtils.rowFilterFromTask(fileScanTask) == null)); + boolean atLeastOneColumn = lazySchema().columns().size() > 0; boolean onlyPrimitives = @@ -420,7 +451,7 @@ public boolean enableBatchRead() { this.readUsingBatch = batchReadsEnabled && hasNoDeleteFiles - && (allOrcFileScanTasks + && (allOrcFileScanTasks && hasNoRowFilters || (allParquetFileScanTasks && atLeastOneColumn && onlyPrimitives)); if (readUsingBatch) { diff --git a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index f206149da..78da82df8 100644 --- a/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v2.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -22,6 +22,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataTask; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; @@ -35,6 +36,8 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcRowFilter; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -43,6 +46,7 @@ import org.apache.iceberg.spark.data.SparkOrcReader; import org.apache.iceberg.spark.data.SparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.rdd.InputFileBlockHolder; import org.apache.spark.sql.catalyst.InternalRow; @@ -149,6 +153,10 @@ private CloseableIterable newParquetIterable( private CloseableIterable newOrcIterable( InputFile location, FileScanTask task, Schema readSchema, Map idToConstant) { + OrcRowFilter orcRowFilter = OrcRowFilterUtils.rowFilterFromTask(task); + if (orcRowFilter != null) { + validateRowFilterRequirements(task, orcRowFilter); + } Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot( readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); @@ -160,7 +168,8 @@ private CloseableIterable newOrcIterable( .createReaderFunc( readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema, idToConstant)) .filter(task.residual()) - .caseSensitive(caseSensitive); + .caseSensitive(caseSensitive) + .rowFilter(orcRowFilter); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); @@ -194,4 +203,22 @@ protected InputFile getInputFile(String location) { return RowDataReader.this.getInputFile(location); } } + + private void validateRowFilterRequirements(FileScanTask task, OrcRowFilter filter) { + Preconditions.checkArgument( + task.file().format() == FileFormat.ORC, "Row filter can only be applied to ORC files"); + Preconditions.checkArgument( + task.spec().fields().size() == 0, "Row filter can only be applied to unpartitioned tables"); + for (Types.NestedField column : filter.requiredSchema().columns()) { + Preconditions.checkArgument( + tableSchema.findField(column.name()) != null, + "Row filter can only be applied to top level fields. %s is not a top level field", + column.name()); + Preconditions.checkArgument( + column.type().isPrimitiveType(), + "Row filter can only be applied to primitive fields. %s is of type %s", + column.name(), + column.type()); + } + } } diff --git a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java new file mode 100644 index 000000000..9a1d62323 --- /dev/null +++ b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java @@ -0,0 +1,202 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkAvroReaderForFieldsWithDefaultValue { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testAvroDefaultValues() throws IOException { + + String writeSchemaString = + "{\n" + + " \"namespace\": \"com.n1\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"n1\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"f0\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + "}"; + org.apache.avro.Schema writeSchema = + new org.apache.avro.Schema.Parser().parse(writeSchemaString); + org.apache.iceberg.Schema icebergWriteSchema = AvroSchemaUtil.toIceberg(writeSchema); + List expected = RandomData.generateList(icebergWriteSchema, 2, 0L); + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + // write records with initial writeSchema + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)).schema(icebergWriteSchema).named("test").build()) { + for (GenericData.Record rec : expected) { + writer.add(rec); + } + } + + // evolve schema by adding a required field with default value + String evolvedSchemaString = + "{\n" + + " \"namespace\": \"com.n1\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"n1\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"f0\",\n" + + " \"type\": \"string\"\n" + + " },\n" + + " {\n" + + " \"name\": \"f1\",\n" + + " \"type\": \"string\",\n" + + " \"default\": \"foo\"\n" + + " },\n" + + " {\n" + + " \"name\": \"f2\",\n" + + " \"type\": \"int\",\n" + + " \"default\": 1\n" + + " },\n" + + " {\n" + + " \"name\": \"f3\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\" : \"int\"\n" + + " },\n" + + " \"default\": {\"a\": 1}\n" + + " },\n" + + " {\n" + + " \"name\": \"f4\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " },\n" + + " \"default\": [1, 2, 3]\n" + + " },\n" + + " {\n" + + " \"name\": \"f5\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"F5\",\n" + + " \"fields\" : [\n" + + " {\"name\": \"ff1\", \"type\": \"long\"},\n" + + " {\"name\": \"ff2\", \"type\": \"string\"}\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"ff1\": 999,\n" + + " \"ff2\": \"foo\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"f6\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " }\n" + + " },\n" + + " \"default\": {\"key\": [1, 2, 3]}\n" + + " },\n" + + " {\n" + + " \"name\": \"f7\",\n" + + " \"type\": {\n" + + " \"type\": \"fixed\",\n" + + " \"name\": \"md5\",\n" + + " \"size\": 2\n" + + " },\n" + + " \"default\": \"FF\"\n" + + " }\n" + + " ]\n" + + "}"; + org.apache.avro.Schema evolvedSchema = + new org.apache.avro.Schema.Parser().parse(evolvedSchemaString); + + List rows; + Schema icebergReadSchema = AvroSchemaUtil.toIceberg(evolvedSchema); + // read written rows with evolved schema + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(icebergReadSchema) + .build()) { + rows = Lists.newArrayList(reader); + } + + Assert.assertNotNull(rows); + Assert.assertEquals(expected.size(), rows.size()); + for (int row = 0; row < expected.size(); row++) { + InternalRow actualRow = rows.get(row); + final InternalRow expectedRow = new GenericInternalRow(8); + expectedRow.update(0, UTF8String.fromString((String) expected.get(row).get(0))); + expectedRow.update(1, UTF8String.fromString("foo")); + expectedRow.update(2, 1); + expectedRow.update( + 3, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("a"))), + new GenericArrayData(Arrays.asList(1)))); + expectedRow.update(4, new GenericArrayData(ImmutableList.of(1, 2, 3).toArray())); + + final InternalRow nestedStructData = new GenericInternalRow(2); + nestedStructData.update(0, 999L); + nestedStructData.update(1, UTF8String.fromString("foo")); + expectedRow.update(5, nestedStructData); + + List listOfLists = new ArrayList(1); + listOfLists.add(new GenericArrayData(ImmutableList.of(1, 2, 3).toArray())); + expectedRow.update( + 6, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("key"))), + new GenericArrayData(listOfLists.toArray()))); + + byte[] objGUIDByteArr = "FF".getBytes("UTF-8"); + expectedRow.update(7, objGUIDByteArr); + assertEquals(icebergReadSchema, actualRow, expectedRow); + } + } +} diff --git a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java new file mode 100644 index 000000000..324aec17b --- /dev/null +++ b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java @@ -0,0 +1,622 @@ +/* + * 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.iceberg.spark.data; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkAvroUnions { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void writeAndValidateRequiredComplexUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", "foo"); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(3, rows.get(0).getStruct(0, 3).numFields()); + Assert.assertEquals(1, rows.get(0).getStruct(0, 3).getInt(0)); + Assert.assertTrue(rows.get(0).getStruct(0, 3).isNullAt(1)); + Assert.assertEquals("foo", rows.get(0).getStruct(0, 3).getString(2)); + + Assert.assertEquals(3, rows.get(1).getStruct(0, 3).numFields()); + Assert.assertEquals(0, rows.get(1).getStruct(0, 3).getInt(0)); + Assert.assertEquals(1, rows.get(1).getStruct(0, 3).getInt(1)); + Assert.assertTrue(rows.get(1).getStruct(0, 3).isNullAt(2)); + } + } + + @Test + public void writeAndValidateOptionalComplexUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .and() + .stringType() + .endUnion() + .nullDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", "foo"); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + GenericData.Record unionRecord3 = new GenericData.Record(avroSchema); + unionRecord3.put("unionCol", null); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + writer.append(unionRecord3); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getStruct(0, 3).getString(2)); + Assert.assertEquals(1, rows.get(1).getStruct(0, 3).getInt(1)); + Assert.assertTrue(rows.get(2).isNullAt(0)); + } + } + + @Test + public void writeAndValidateSingleTypeUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", 0); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(0, rows.get(0).getInt(0)); + Assert.assertEquals(1, rows.get(1).getInt(0)); + } + } + + @Test + public void writeAndValidateNestedSingleTypeUnion1() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("col1", Arrays.asList("foo")); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("col1", Arrays.asList("bar")); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getArray(0).getUTF8String(0).toString()); + Assert.assertEquals("bar", rows.get(1).getArray(0).getUTF8String(0).toString()); + } + } + + @Test + public void writeAndValidateNestedSingleTypeUnion2() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("outerUnion") + .type() + .unionOf() + .record("r") + .fields() + .name("innerUnion") + .type() + .unionOf() + .stringType() + .endUnion() + .noDefault() + .endRecord() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + GenericData.Record innerRecord1 = + new GenericData.Record(avroSchema.getFields().get(0).schema().getTypes().get(0)); + innerRecord1.put("innerUnion", "foo"); + unionRecord1.put("outerUnion", innerRecord1); + + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + GenericData.Record innerRecord2 = + new GenericData.Record(avroSchema.getFields().get(0).schema().getTypes().get(0)); + innerRecord2.put("innerUnion", "bar"); + unionRecord2.put("outerUnion", innerRecord2); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getStruct(0, 1).getUTF8String(0).toString()); + Assert.assertEquals("bar", rows.get(1).getStruct(0, 1).getUTF8String(0).toString()); + } + } + + @Test + public void writeAndValidateSingleTypeUnionOfComplexType() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .array() + .items() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", Arrays.asList(1)); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", Arrays.asList(2)); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(1, rows.get(0).getArray(0).getInt(0)); + Assert.assertEquals(2, rows.get(1).getArray(0).getInt(0)); + } + } + + @Test + public void writeAndValidateOptionalSingleUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .endUnion() + .nullDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", 0); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(0, rows.get(0).getInt(0)); + Assert.assertEquals(1, rows.get(1).getInt(0)); + } + } + + @Test + public void testDeeplyNestedUnionSchema1() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .nullType() + .and() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("col1", Arrays.asList("foo", 1)); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("col1", Arrays.asList(2, "bar")); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + // making sure it reads the correctly nested structured data, based on the transformation from + // union to struct + Assert.assertEquals("foo", rows.get(0).getArray(0).getStruct(0, 3).getString(2)); + } + } + + @Test + public void testDeeplyNestedUnionSchema2() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .record("r1") + .fields() + .name("id") + .type() + .intType() + .noDefault() + .endRecord() + .and() + .record("r2") + .fields() + .name("id") + .type() + .intType() + .noDefault() + .endRecord() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record outer = new GenericData.Record(avroSchema); + GenericData.Record inner = + new GenericData.Record( + avroSchema.getFields().get(0).schema().getElementType().getTypes().get(0)); + + inner.put("id", 1); + outer.put("col1", Arrays.asList(inner)); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(outer); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + // making sure it reads the correctly nested structured data, based on the transformation from + // union to struct + Assert.assertEquals(1, rows.get(0).getArray(0).getStruct(0, 3).getStruct(1, 1).getInt(0)); + } + } + + @Test + public void testDeeplyNestedUnionSchema3() throws IOException { + /* + * the printed write schema: + * { + "type": "record", + "name": "root", + "fields": [ + { + "name": "value", + "type": [ + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "ff1", + "type": "long" + }, + { + "name": "ff2", + "type": { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "fff1", + "type": [ + "null", + "string", + "int" + ], + "default": null + } + ] + } + }, + { + "name": "ff3", + "type": { + "type": "array", + "items": "string" + }, + "default": [] + } + ] + }, + "null" + ] + } + ] + } + * */ + org.apache.avro.Schema writeSchema = + SchemaBuilder.record("root") + .fields() + .name("value") + .type() + .unionOf() + .record("r1") + .fields() + .name("ff1") + .type() + .longType() + .noDefault() + .name("ff2") + .type() + .record("r2") + .fields() + .name("fff1") + .type() + .unionOf() + .nullType() + .and() + .stringType() + .and() + .intType() + .endUnion() + .nullDefault() + .endRecord() + .noDefault() + .name("ff3") + .type() + .array() + .items() + .stringType() + .arrayDefault(ImmutableList.of()) + .endRecord() + .and() + .nullType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record record1 = new GenericData.Record(writeSchema); + GenericData.Record record11 = + new GenericData.Record(writeSchema.getField("value").schema().getTypes().get(0)); + GenericData.Record record111 = + new GenericData.Record( + writeSchema.getField("value").schema().getTypes().get(0).getField("ff2").schema()); + // record111.put("fff1", 1); + record11.put("ff1", 99); + record11.put("ff2", record111); + record11.put("ff3", ImmutableList.of()); + record1.put("value", record11); + + GenericData.Record record2 = new GenericData.Record(writeSchema); + GenericData.Record record22 = + new GenericData.Record(writeSchema.getField("value").schema().getTypes().get(0)); + GenericData.Record record222 = + new GenericData.Record( + writeSchema.getField("value").schema().getTypes().get(0).getField("ff2").schema()); + record222.put("fff1", 1); + record22.put("ff1", 99); + record22.put("ff2", record222); + record22.put("ff3", ImmutableList.of("foo")); + record2.put("value", record22); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(writeSchema, testFile); + writer.append(record1); + writer.append(record2); + } + + List expected = ImmutableList.of(record1, record2); + + org.apache.iceberg.Schema readIcebergSchema = AvroSchemaUtil.toIceberg(writeSchema); + // read written rows with evolved schema + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(readIcebergSchema) + .build()) { + rows = Lists.newArrayList(reader); + } + + // making sure the rows can be read successfully + Assert.assertEquals(2, rows.size()); + } +} diff --git a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java new file mode 100644 index 000000000..980c63eac --- /dev/null +++ b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java @@ -0,0 +1,254 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkOrcReaderForFieldsWithDefaultValue { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testOrcDefaultValues() throws IOException { + final int numRows = 10; + + final InternalRow expectedFirstRow = new GenericInternalRow(6); + expectedFirstRow.update(0, 0); + expectedFirstRow.update(1, UTF8String.fromString("foo")); + expectedFirstRow.update(2, new GenericArrayData(ImmutableList.of(1, 2).toArray())); + expectedFirstRow.update( + 3, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("foo"))), + new GenericArrayData(Arrays.asList(1)))); + + final InternalRow nestedStructData = new GenericInternalRow(1); + nestedStructData.update(0, 1); + expectedFirstRow.update(4, nestedStructData); + + // test deeply nested data: array of structs + final InternalRow deepNestedStructData1 = new GenericInternalRow(1); + deepNestedStructData1.update(0, 1); + final InternalRow deepNestedStructData2 = new GenericInternalRow(1); + deepNestedStructData2.update(0, 2); + expectedFirstRow.update( + 5, + new GenericArrayData( + ImmutableList.of(deepNestedStructData1, deepNestedStructData2).toArray())); + + TypeDescription orcSchema = TypeDescription.fromString("struct"); + + Schema readSchema = + new Schema( + Types.NestedField.required(1, "col1", Types.IntegerType.get()), + Types.NestedField.required(2, "col2", Types.StringType.get(), "foo", null), + Types.NestedField.required( + 3, + "col3", + Types.ListType.ofRequired(10, Types.IntegerType.get()), + ImmutableList.of(1, 2), + null), + Types.NestedField.required( + 4, + "col4", + Types.MapType.ofRequired(11, 12, Types.StringType.get(), Types.IntegerType.get()), + ImmutableMap.of("foo", 1), + null), + Types.NestedField.required( + 5, + "col5", + Types.StructType.of( + Types.NestedField.required(13, "nested_col1", Types.IntegerType.get())), + ImmutableMap.of("nested_col1", 1), + null), + Types.NestedField.required( + 6, + "col6", + Types.ListType.ofRequired( + 14, + Types.StructType.of( + Types.NestedField.required(15, "nested_col2", Types.IntegerType.get()))), + ImmutableList.of( + ImmutableMap.of("nested_col2", 1), ImmutableMap.of("nested_col2", 2)), + null)); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector firstCol = (LongColumnVector) batch.cols[0]; + for (int r = 0; r < numRows; ++r) { + int row = batch.size++; + firstCol.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // try to read the data using the readSchema, which is an evolved + // schema that contains a new column with default value + + // non-vectorized read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + + // vectorized-read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + readSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRows = batchesToRows(reader.iterator()); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + } + + @Test + public void testSelectionAndFilterWithDefaultValueColumnOnly() throws IOException { + final int numRows = 10; + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + // expectedFirstRow.update(0, 0); + expectedFirstRow.update(0, UTF8String.fromString("foo")); + + TypeDescription orcSchema = TypeDescription.fromString("struct"); + + Schema readSchema = + new Schema(Types.NestedField.required(2, "col2", Types.StringType.get(), "foo", null)); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector firstCol = (LongColumnVector) batch.cols[0]; + for (int r = 0; r < numRows; ++r) { + int row = batch.size++; + firstCol.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // try to read the data using the readSchema, which is an evolved + // schema that contains a new column with default value + + // non-vectorized read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .filter(Expressions.equal("col2", "foo")) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + + // vectorized-read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .filter(Expressions.equal("col2", "foo")) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + readSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRows = batchesToRows(reader.iterator()); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } +} diff --git a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java new file mode 100644 index 000000000..6f9435697 --- /dev/null +++ b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java @@ -0,0 +1,495 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkOrcUnions { + private static final int NUM_OF_ROWS = 50; + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testComplexUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of( + Types.NestedField.optional(3, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional(2, "field1", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow field1 = new GenericInternalRow(3); + field1.update(0, 0); + field1.update(1, 0); + field1.update(2, null); + expectedFirstRow.update(0, field1); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow field2 = new GenericInternalRow(3); + field2.update(0, 1); + field2.update(1, null); + field2.update(2, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, field2); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector longColumnVector = new LongColumnVector(NUM_OF_ROWS); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = + new UnionColumnVector(NUM_OF_ROWS, longColumnVector, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = i % 2; + longColumnVector.vector[i] = i; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeeplyNestedUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "c1", + Types.StructType.of( + Types.NestedField.optional(100, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 2, + "field1", + Types.StructType.of( + Types.NestedField.optional(3, "c2", Types.StringType.get()), + Types.NestedField.optional( + 4, + "c3", + Types.StructType.of( + Types.NestedField.optional(101, "tag", Types.IntegerType.get()), + Types.NestedField.optional( + 5, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 6, "field1", Types.StringType.get())))))))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow inner1 = new GenericInternalRow(3); + inner1.update(0, 1); + inner1.update(1, null); + final InternalRow inner2 = new GenericInternalRow(2); + inner2.update(0, UTF8String.fromString("foo0")); + final InternalRow inner3 = new GenericInternalRow(3); + inner3.update(0, 0); + inner3.update(1, 0); + inner3.update(2, null); + inner2.update(1, inner3); + inner1.update(2, inner2); + expectedFirstRow.update(0, inner1); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector innerUnion1 = (UnionColumnVector) batch.cols[0]; + LongColumnVector innerInt1 = (LongColumnVector) innerUnion1.fields[0]; + innerInt1.fillWithNulls(); + StructColumnVector innerStruct2 = (StructColumnVector) innerUnion1.fields[1]; + BytesColumnVector innerString2 = (BytesColumnVector) innerStruct2.fields[0]; + UnionColumnVector innerUnion3 = (UnionColumnVector) innerStruct2.fields[1]; + LongColumnVector innerInt3 = (LongColumnVector) innerUnion3.fields[0]; + BytesColumnVector innerString3 = (BytesColumnVector) innerUnion3.fields[1]; + innerString3.fillWithNulls(); + + for (int r = 0; r < NUM_OF_ROWS; ++r) { + int row = batch.size++; + innerUnion1.tags[row] = 1; + innerString2.setVal(row, ("foo" + row).getBytes(StandardCharsets.UTF_8)); + innerUnion3.tags[row] = 0; + innerInt3.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + innerInt1.fillWithNulls(); + innerString3.fillWithNulls(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // test non-vectorized reader + List results = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(results::add); + final InternalRow actualFirstRow = results.get(0); + + Assert.assertEquals(results.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualFirstRow); + } + + // test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnion() throws IOException { + TypeDescription orcSchema = TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema(Types.NestedField.optional(0, "unionCol", Types.StringType.get())); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + expectedFirstRow.update(0, UTF8String.fromString("foo-0")); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + expectedSecondRow.update(0, UTF8String.fromString("foo-1")); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = new UnionColumnVector(NUM_OF_ROWS, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnionOfStruct() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of(Types.NestedField.optional(1, "c", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector complexUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) complexUnion.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) structColumnVector.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeepNestedSingleTypeUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "outerUnion", + Types.StructType.of( + Types.NestedField.optional(1, "innerUnion", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector outerUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) outerUnion.fields[0]; + UnionColumnVector innerUnion = (UnionColumnVector) structColumnVector.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) innerUnion.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + outerUnion.tags[i] = 0; + innerUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } +} diff --git a/spark/v3.0/build.gradle b/spark/v3.0/build.gradle index 102e8e05d..6bc798087 100644 --- a/spark/v3.0/build.gradle +++ b/spark/v3.0/build.gradle @@ -60,6 +60,7 @@ project(':iceberg-spark:iceberg-spark-3.0_2.12') { implementation project(':iceberg-orc') implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') + implementation project(':iceberg-hivelink-core') compileOnly "com.google.errorprone:error_prone_annotations" compileOnly "org.apache.avro:avro" diff --git a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 78db13705..66710b476 100644 --- a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -80,6 +80,12 @@ public OrcValueReader record( return SparkOrcValueReaders.struct(fields, expected, idToConstant); } + @Override + public OrcValueReader union( + Type expected, TypeDescription union, List> options) { + return SparkOrcValueReaders.union(options); + } + @Override public OrcValueReader list( Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { diff --git a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java index 9e9b3e53b..ffbdc7dc5 100644 --- a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java +++ b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java @@ -32,6 +32,7 @@ import org.apache.orc.storage.ql.exec.vector.ListColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.serde2.io.HiveDecimalWritable; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -68,6 +69,10 @@ static OrcValueReader struct( return new StructReader(readers, struct, idToConstant); } + static OrcValueReader union(List> readers) { + return new UnionReader(readers); + } + static OrcValueReader array(OrcValueReader elementReader) { return new ArrayReader(elementReader); } @@ -157,6 +162,37 @@ protected void set(InternalRow struct, int pos, Object value) { } } + static class UnionReader implements OrcValueReader { + private final OrcValueReader[] readers; + + private UnionReader(List> readers) { + this.readers = new OrcValueReader[readers.size()]; + for (int i = 0; i < this.readers.length; i += 1) { + this.readers[i] = readers.get(i); + } + } + + @Override + public Object nonNullRead(ColumnVector vector, int row) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + int fieldIndex = unionColumnVector.tags[row]; + Object value = this.readers[fieldIndex].read(unionColumnVector.fields[fieldIndex], row); + + if (readers.length == 1) { + return value; + } else { + InternalRow struct = new GenericInternalRow(readers.length + 1); + for (int i = 0; i < readers.length; i += 1) { + struct.setNullAt(i + 1); + } + struct.update(0, fieldIndex); + struct.update(fieldIndex + 1, value); + + return struct; + } + } + } + private static class StringReader implements OrcValueReader { private static final StringReader INSTANCE = new StringReader(); diff --git a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 7c3b825a6..aae0b459e 100644 --- a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.data.vectorized; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.IntStream; @@ -34,8 +35,10 @@ import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.vectorized.ColumnVector; @@ -111,6 +114,11 @@ public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Con return new MapConverter(iMap, key, value); } + @Override + public Converter union(Type iType, TypeDescription union, List options) { + return new UnionConverter(iType, options); + } + @Override public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { final OrcValueReader primitiveValueReader; @@ -456,4 +464,56 @@ public ColumnVector getChild(int ordinal) { }; } } + + private static class UnionConverter implements Converter { + private final Type type; + private final List optionConverters; + + private UnionConverter(Type type, List optionConverters) { + this.type = type; + this.optionConverters = optionConverters; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + if (optionConverters.size() > 1) { + // the case of complex union with multiple types + List fields = type.asStructType().fields(); + List fieldVectors = Lists.newArrayListWithExpectedSize(fields.size()); + + LongColumnVector longColumnVector = new LongColumnVector(); + longColumnVector.vector = Arrays.stream(unionColumnVector.tags).asLongStream().toArray(); + + fieldVectors.add( + new PrimitiveOrcColumnVector( + Types.IntegerType.get(), + batchSize, + longColumnVector, + OrcValueReaders.ints(), + batchOffsetInFile)); + for (int i = 0; i < fields.size() - 1; i += 1) { + fieldVectors.add( + optionConverters + .get(i) + .convert(unionColumnVector.fields[i], batchSize, batchOffsetInFile)); + } + + return new BaseOrcColumnVector(type.asStructType(), batchSize, vector) { + @Override + public ColumnVector getChild(int ordinal) { + return fieldVectors.get(ordinal); + } + }; + } else { + // the case of single type union + return optionConverters + .get(0) + .convert(unionColumnVector.fields[0], batchSize, batchOffsetInFile); + } + } + } } diff --git a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java index 3b3d62d96..c16146e48 100644 --- a/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java +++ b/spark/v3.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.OptionalLong; import java.util.stream.Collectors; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileFormat; @@ -33,6 +34,8 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.hivelink.core.LegacyHiveTable; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; @@ -184,6 +187,16 @@ public PartitionReaderFactory createReaderFactory() { fileScanTask -> fileScanTask.file().format().equals(FileFormat.ORC))); + boolean hasNoRowFilters = + tasks().stream() + .allMatch( + combinedScanTask -> + !combinedScanTask.isDataTask() + && combinedScanTask.files().stream() + .allMatch( + fileScanTask -> + OrcRowFilterUtils.rowFilterFromTask(fileScanTask) == null)); + boolean atLeastOneColumn = expectedSchema.columns().size() > 0; boolean onlyPrimitives = @@ -196,7 +209,7 @@ public PartitionReaderFactory createReaderFactory() { boolean readUsingBatch = batchReadsEnabled && hasNoDeleteFiles - && (allOrcFileScanTasks + && (allOrcFileScanTasks && hasNoRowFilters || (allParquetFileScanTasks && atLeastOneColumn && onlyPrimitives)); int batchSize = readUsingBatch ? batchSize(allParquetFileScanTasks, allOrcFileScanTasks) : 0; @@ -226,6 +239,11 @@ private int batchSize(boolean isParquetOnly, boolean isOrcOnly) { @Override public Statistics estimateStatistics() { + if (table instanceof LegacyHiveTable) { + // We currently don't have reliable stats for Hive tables + return EMPTY_STATS; + } + // its a fresh table, no data if (table.currentSnapshot() == null) { return new Stats(0L, 0L); @@ -257,6 +275,19 @@ public Statistics estimateStatistics() { return new Stats(sizeInBytes, numRows); } + private static final Statistics EMPTY_STATS = + new Statistics() { + @Override + public OptionalLong sizeInBytes() { + return OptionalLong.empty(); + } + + @Override + public OptionalLong numRows() { + return OptionalLong.empty(); + } + }; + @Override public String description() { String filters = diff --git a/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java b/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java new file mode 100644 index 000000000..6f9435697 --- /dev/null +++ b/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java @@ -0,0 +1,495 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkOrcUnions { + private static final int NUM_OF_ROWS = 50; + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testComplexUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of( + Types.NestedField.optional(3, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional(2, "field1", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow field1 = new GenericInternalRow(3); + field1.update(0, 0); + field1.update(1, 0); + field1.update(2, null); + expectedFirstRow.update(0, field1); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow field2 = new GenericInternalRow(3); + field2.update(0, 1); + field2.update(1, null); + field2.update(2, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, field2); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector longColumnVector = new LongColumnVector(NUM_OF_ROWS); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = + new UnionColumnVector(NUM_OF_ROWS, longColumnVector, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = i % 2; + longColumnVector.vector[i] = i; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeeplyNestedUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "c1", + Types.StructType.of( + Types.NestedField.optional(100, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 2, + "field1", + Types.StructType.of( + Types.NestedField.optional(3, "c2", Types.StringType.get()), + Types.NestedField.optional( + 4, + "c3", + Types.StructType.of( + Types.NestedField.optional(101, "tag", Types.IntegerType.get()), + Types.NestedField.optional( + 5, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 6, "field1", Types.StringType.get())))))))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow inner1 = new GenericInternalRow(3); + inner1.update(0, 1); + inner1.update(1, null); + final InternalRow inner2 = new GenericInternalRow(2); + inner2.update(0, UTF8String.fromString("foo0")); + final InternalRow inner3 = new GenericInternalRow(3); + inner3.update(0, 0); + inner3.update(1, 0); + inner3.update(2, null); + inner2.update(1, inner3); + inner1.update(2, inner2); + expectedFirstRow.update(0, inner1); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector innerUnion1 = (UnionColumnVector) batch.cols[0]; + LongColumnVector innerInt1 = (LongColumnVector) innerUnion1.fields[0]; + innerInt1.fillWithNulls(); + StructColumnVector innerStruct2 = (StructColumnVector) innerUnion1.fields[1]; + BytesColumnVector innerString2 = (BytesColumnVector) innerStruct2.fields[0]; + UnionColumnVector innerUnion3 = (UnionColumnVector) innerStruct2.fields[1]; + LongColumnVector innerInt3 = (LongColumnVector) innerUnion3.fields[0]; + BytesColumnVector innerString3 = (BytesColumnVector) innerUnion3.fields[1]; + innerString3.fillWithNulls(); + + for (int r = 0; r < NUM_OF_ROWS; ++r) { + int row = batch.size++; + innerUnion1.tags[row] = 1; + innerString2.setVal(row, ("foo" + row).getBytes(StandardCharsets.UTF_8)); + innerUnion3.tags[row] = 0; + innerInt3.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + innerInt1.fillWithNulls(); + innerString3.fillWithNulls(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // test non-vectorized reader + List results = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(results::add); + final InternalRow actualFirstRow = results.get(0); + + Assert.assertEquals(results.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualFirstRow); + } + + // test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnion() throws IOException { + TypeDescription orcSchema = TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema(Types.NestedField.optional(0, "unionCol", Types.StringType.get())); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + expectedFirstRow.update(0, UTF8String.fromString("foo-0")); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + expectedSecondRow.update(0, UTF8String.fromString("foo-1")); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = new UnionColumnVector(NUM_OF_ROWS, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnionOfStruct() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of(Types.NestedField.optional(1, "c", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector complexUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) complexUnion.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) structColumnVector.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeepNestedSingleTypeUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "outerUnion", + Types.StructType.of( + Types.NestedField.optional(1, "innerUnion", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector outerUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) outerUnion.fields[0]; + UnionColumnVector innerUnion = (UnionColumnVector) structColumnVector.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) innerUnion.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + outerUnion.tags[i] = 0; + innerUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } +} diff --git a/spark/v3.1/build.gradle b/spark/v3.1/build.gradle index 592d1b51b..e5f484558 100644 --- a/spark/v3.1/build.gradle +++ b/spark/v3.1/build.gradle @@ -60,6 +60,7 @@ project(':iceberg-spark:iceberg-spark-3.1_2.12') { implementation project(':iceberg-orc') implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') + implementation project(':iceberg-hivelink-core') compileOnly "com.google.errorprone:error_prone_annotations" compileOnly "org.apache.avro:avro" diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java new file mode 100644 index 000000000..4521ee80f --- /dev/null +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/OrcSchemaWithTypeVisitorSpark.java @@ -0,0 +1,121 @@ +/* + * 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.iceberg.spark; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.orc.ORCSchemaUtil; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.BaseDataReader; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; + +public abstract class OrcSchemaWithTypeVisitorSpark extends OrcSchemaWithTypeVisitor { + + private final Map idToConstant; + + public Map getIdToConstant() { + return idToConstant; + } + + protected OrcSchemaWithTypeVisitorSpark(Map idToConstant) { + this.idToConstant = Maps.newHashMap(); + this.idToConstant.putAll(idToConstant); + } + + @Override + protected T visitRecord( + Types.StructType struct, TypeDescription record, OrcSchemaWithTypeVisitor visitor) { + Preconditions.checkState( + icebergFiledIdsContainOrcFieldIdsInOrder(struct, record), + "Iceberg schema and ORC schema doesn't align, please call ORCSchemaUtil.buildOrcProjection" + + "to get an aligned ORC schema first!"); + List iFields = struct.fields(); + List fields = record.getChildren(); + List names = record.getFieldNames(); + List results = Lists.newArrayListWithExpectedSize(fields.size()); + + for (int i = 0, j = 0; i < iFields.size(); i++) { + Types.NestedField iField = iFields.get(i); + TypeDescription field = j < fields.size() ? fields.get(j) : null; + if (field == null || (iField.fieldId() != ORCSchemaUtil.fieldId(field))) { + // there are 3 cases where we need to use idToConstant for an iField + // 1. The field is MetadataColumns.ROW_POSITION, we build a RowPositionReader + // 2. The field is a partition column, we build a ConstantReader + // 3. The field should be read using the default value, where we build a ConstantReader + // Here we should only need to update idToConstant when it's the 3rd case, + // because the first 2 cases have been handled by logic in PartitionUtil.constantsMap + if (MetadataColumns.nonMetadataColumn(iField.name()) + && !idToConstant.containsKey(iField.fieldId())) { + idToConstant.put( + iField.fieldId(), + BaseDataReader.convertConstant(iField.type(), iField.getDefaultValue())); + } + } else { + results.add(visit(iField.type(), field, visitor)); + j++; + } + } + return visitor.record(struct, record, names, results); + } + + private static boolean icebergFiledIdsContainOrcFieldIdsInOrder( + Types.StructType struct, TypeDescription record) { + List icebergIDList = + struct.fields().stream().map(Types.NestedField::fieldId).collect(Collectors.toList()); + List orcIDList = + record.getChildren().stream().map(ORCSchemaUtil::fieldId).collect(Collectors.toList()); + + return containsInOrder(icebergIDList, orcIDList); + } + + /** + * Checks whether the first list contains all the integers in the same order as regarding to the + * second list, the first list can contain extra integers that the second list doesn't, but the + * ones that exist in the second list should occur in the same relative order in the first list. + * + * @param list1 the first list + * @param list2 the second list + * @return the condition check result + */ + private static boolean containsInOrder(List list1, List list2) { + if (list1.size() < list2.size()) { + return false; + } + + for (int i = 0, j = 0; j < list2.size(); j++) { + if (i >= list1.size()) { + return false; + } + while (!list1.get(i).equals(list2.get(j))) { + i++; + if (i >= list1.size()) { + return false; + } + } + i++; + } + return true; + } +} diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java index 4622d2928..2c8494787 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java @@ -27,6 +27,7 @@ import org.apache.avro.Schema; import org.apache.avro.io.DatumReader; import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; import org.apache.iceberg.avro.SupportsRowPosition; import org.apache.iceberg.avro.ValueReader; @@ -88,7 +89,11 @@ public ValueReader record( @Override public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); + if (AvroSchemaUtil.isOptionSchema(union) || AvroSchemaUtil.isSingleTypeUnion(union)) { + return ValueReaders.union(options); + } else { + return SparkValueReaders.union(union, options); + } } @Override diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 78db13705..6cee60ce0 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -25,6 +25,7 @@ import org.apache.iceberg.orc.OrcValueReader; import org.apache.iceberg.orc.OrcValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.OrcSchemaWithTypeVisitorSpark; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; @@ -64,11 +65,10 @@ public void setBatchContext(long batchOffsetInFile) { reader.setBatchContext(batchOffsetInFile); } - private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { - private final Map idToConstant; + public static class ReadBuilder extends OrcSchemaWithTypeVisitorSpark> { private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; + super(idToConstant); } @Override @@ -77,7 +77,7 @@ public OrcValueReader record( TypeDescription record, List names, List> fields) { - return SparkOrcValueReaders.struct(fields, expected, idToConstant); + return SparkOrcValueReaders.struct(fields, expected, getIdToConstant()); } @Override @@ -95,6 +95,12 @@ public OrcValueReader map( return SparkOrcValueReaders.map(keyReader, valueReader); } + @Override + public OrcValueReader union( + Type expected, TypeDescription union, List> options) { + return SparkOrcValueReaders.union(options); + } + @Override public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { switch (primitive.getCategory()) { diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java index 9e9b3e53b..d2f60eb97 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java @@ -32,6 +32,7 @@ import org.apache.orc.storage.ql.exec.vector.ListColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.serde2.io.HiveDecimalWritable; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -76,6 +77,10 @@ static OrcValueReader map(OrcValueReader keyReader, OrcValueReader valu return new MapReader(keyReader, valueReader); } + static OrcValueReader union(List> readers) { + return new UnionReader(readers); + } + private static class ArrayReader implements OrcValueReader { private final OrcValueReader elementReader; @@ -157,6 +162,37 @@ protected void set(InternalRow struct, int pos, Object value) { } } + static class UnionReader implements OrcValueReader { + private final OrcValueReader[] readers; + + private UnionReader(List> readers) { + this.readers = new OrcValueReader[readers.size()]; + for (int i = 0; i < this.readers.length; i += 1) { + this.readers[i] = readers.get(i); + } + } + + @Override + public Object nonNullRead(ColumnVector vector, int row) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + int fieldIndex = unionColumnVector.tags[row]; + Object value = this.readers[fieldIndex].read(unionColumnVector.fields[fieldIndex], row); + + if (readers.length == 1) { + return value; + } else { + InternalRow struct = new GenericInternalRow(readers.length + 1); + for (int i = 0; i < readers.length; i += 1) { + struct.setNullAt(i + 1); + } + struct.update(0, fieldIndex); + struct.update(fieldIndex + 1, value); + + return struct; + } + } + } + private static class StringReader implements OrcValueReader { private static final StringReader INSTANCE = new StringReader(); diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java index 11655c72d..8237f9d04 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -26,6 +26,8 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.Objects; +import org.apache.avro.Schema; import org.apache.avro.io.Decoder; import org.apache.avro.util.Utf8; import org.apache.iceberg.avro.ValueReader; @@ -79,6 +81,10 @@ static ValueReader struct( return new StructReader(readers, struct, idToConstant); } + static ValueReader union(Schema schema, List> readers) { + return new UnionReader(schema, readers); + } + private static class StringReader implements ValueReader { private static final StringReader INSTANCE = new StringReader(); @@ -285,4 +291,59 @@ protected void set(InternalRow struct, int pos, Object value) { } } } + + private static class UnionReader implements ValueReader { + private final Schema schema; + private final ValueReader[] readers; + + private UnionReader(Schema schema, List> readers) { + this.schema = schema; + this.readers = new ValueReader[readers.size()]; + for (int i = 0; i < this.readers.length; i += 1) { + this.readers[i] = readers.get(i); + } + } + + @Override + public InternalRow read(Decoder decoder, Object reuse) throws IOException { + // first we need to filter out NULL alternative if it exists in the union schema + int nullIndex = -1; + List alts = schema.getTypes(); + for (int i = 0; i < alts.size(); i++) { + Schema alt = alts.get(i); + if (Objects.equals(alt.getType(), Schema.Type.NULL)) { + nullIndex = i; + break; + } + } + + int index = decoder.readIndex(); + if (index == nullIndex) { + // if it is a null data, directly return null as the whole union result + // we know for sure it is a null so the casting will always work. + return (InternalRow) readers[nullIndex].read(decoder, reuse); + } + + // otherwise, we need to return an InternalRow as a struct data + InternalRow struct = new GenericInternalRow(nullIndex >= 0 ? alts.size() : alts.size() + 1); + for (int i = 0; i < struct.numFields(); i += 1) { + struct.setNullAt(i); + } + + Object value = readers[index].read(decoder, reuse); + + if (nullIndex < 0) { + struct.update(index + 1, value); + struct.setInt(0, index); + } else if (index < nullIndex) { + struct.update(index + 1, value); + struct.setInt(0, index); + } else { + struct.update(index, value); + struct.setInt(0, index - 1); + } + + return struct; + } + } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java new file mode 100644 index 000000000..e76e172ef --- /dev/null +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantArrayColumnVector.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.spark.data.vectorized; + +import java.util.Arrays; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class ConstantArrayColumnVector extends ConstantColumnVector { + + private final Object[] constantArray; + + public ConstantArrayColumnVector(DataType type, int batchSize, Object[] constantArray) { + super(type, batchSize, constantArray); + this.constantArray = constantArray; + } + + @Override + public boolean getBoolean(int rowId) { + return (boolean) constantArray[rowId]; + } + + @Override + public byte getByte(int rowId) { + return (byte) constantArray[rowId]; + } + + @Override + public short getShort(int rowId) { + return (short) constantArray[rowId]; + } + + @Override + public int getInt(int rowId) { + return (int) constantArray[rowId]; + } + + @Override + public long getLong(int rowId) { + return (long) constantArray[rowId]; + } + + @Override + public float getFloat(int rowId) { + return (float) constantArray[rowId]; + } + + @Override + public double getDouble(int rowId) { + return (double) constantArray[rowId]; + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + return (Decimal) constantArray[rowId]; + } + + @Override + public UTF8String getUTF8String(int rowId) { + return (UTF8String) constantArray[rowId]; + } + + @Override + public byte[] getBinary(int rowId) { + return (byte[]) constantArray[rowId]; + } + + @Override + public ColumnarArray getArray(int rowId) { + return new ColumnarArray( + new ConstantArrayColumnVector( + ((ArrayType) type).elementType(), + getBatchSize(), + ((ArrayData) constantArray[rowId]).array()), + 0, + ((ArrayData) constantArray[rowId]).numElements()); + } + + @Override + public ColumnarMap getMap(int rowId) { + ColumnVector keys = + new ConstantArrayColumnVector( + ((MapType) type).keyType(), + getBatchSize(), + ((MapData) constantArray[rowId]).keyArray().array()); + ColumnVector values = + new ConstantArrayColumnVector( + ((MapType) type).valueType(), + getBatchSize(), + ((MapData) constantArray[rowId]).valueArray().array()); + return new ColumnarMap(keys, values, 0, ((MapData) constantArray[rowId]).numElements()); + } + + @Override + public ColumnVector getChild(int ordinal) { + DataType fieldType = ((StructType) type).fields()[ordinal].dataType(); + return new ConstantArrayColumnVector( + fieldType, + getBatchSize(), + Arrays.stream(constantArray) + .map(row -> ((InternalRow) row).get(ordinal, fieldType)) + .toArray()); + } +} diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java index 42683ffa9..fa5103b50 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java @@ -20,7 +20,14 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarArray; import org.apache.spark.sql.vectorized.ColumnarMap; @@ -37,6 +44,16 @@ class ConstantColumnVector extends ColumnVector { this.batchSize = batchSize; } + ConstantColumnVector(DataType type, int batchSize, Object constant) { + super(type); + this.constant = constant; + this.batchSize = batchSize; + } + + protected int getBatchSize() { + return batchSize; + } + @Override public void close() {} @@ -92,12 +109,22 @@ public double getDouble(int rowId) { @Override public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + return new ColumnarArray( + new ConstantArrayColumnVector( + ((ArrayType) type).elementType(), batchSize, ((ArrayData) constant).array()), + 0, + ((ArrayData) constant).numElements()); } @Override public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + ColumnVector keys = + new ConstantArrayColumnVector( + ((MapType) type).keyType(), batchSize, ((MapData) constant).keyArray().array()); + ColumnVector values = + new ConstantArrayColumnVector( + ((MapType) type).valueType(), batchSize, ((MapData) constant).valueArray().array()); + return new ColumnarMap(keys, values, 0, ((MapData) constant).numElements()); } @Override @@ -117,6 +144,8 @@ public byte[] getBinary(int rowId) { @Override public ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException("ConstantColumnVector only supports primitives"); + DataType fieldType = ((StructType) type).fields()[ordinal].dataType(); + return new ConstantColumnVector( + fieldType, batchSize, ((InternalRow) constant).get(ordinal, fieldType)); } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 7c3b825a6..d2fe70824 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.data.vectorized; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.IntStream; @@ -28,14 +29,17 @@ import org.apache.iceberg.orc.OrcValueReader; import org.apache.iceberg.orc.OrcValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.OrcSchemaWithTypeVisitorSpark; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.SparkOrcValueReaders; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; import org.apache.orc.storage.ql.exec.vector.MapColumnVector; import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.vectorized.ColumnVector; @@ -85,11 +89,10 @@ ColumnVector convert( long batchOffsetInFile); } - private static class ReadBuilder extends OrcSchemaWithTypeVisitor { - private final Map idToConstant; + private static class ReadBuilder extends OrcSchemaWithTypeVisitorSpark { private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; + super(idToConstant); } @Override @@ -98,7 +101,12 @@ public Converter record( TypeDescription record, List names, List fields) { - return new StructConverter(iStruct, fields, idToConstant); + return new StructConverter(iStruct, fields, getIdToConstant()); + } + + @Override + public Converter union(Type iType, TypeDescription union, List options) { + return new UnionConverter(iType, options); } @Override @@ -456,4 +464,56 @@ public ColumnVector getChild(int ordinal) { }; } } + + private static class UnionConverter implements Converter { + private final Type type; + private final List optionConverters; + + private UnionConverter(Type type, List optionConverters) { + this.type = type; + this.optionConverters = optionConverters; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile) { + UnionColumnVector unionColumnVector = (UnionColumnVector) vector; + if (optionConverters.size() > 1) { + // the case of complex union with multiple types + List fields = type.asStructType().fields(); + List fieldVectors = Lists.newArrayListWithExpectedSize(fields.size()); + + LongColumnVector longColumnVector = new LongColumnVector(); + longColumnVector.vector = Arrays.stream(unionColumnVector.tags).asLongStream().toArray(); + + fieldVectors.add( + new PrimitiveOrcColumnVector( + Types.IntegerType.get(), + batchSize, + longColumnVector, + OrcValueReaders.ints(), + batchOffsetInFile)); + for (int i = 0; i < fields.size() - 1; i += 1) { + fieldVectors.add( + optionConverters + .get(i) + .convert(unionColumnVector.fields[i], batchSize, batchOffsetInFile)); + } + + return new BaseOrcColumnVector(type.asStructType(), batchSize, vector) { + @Override + public ColumnVector getChild(int ordinal) { + return fieldVectors.get(ordinal); + } + }; + } else { + // the case of single type union + return optionConverters + .get(0) + .convert(unionColumnVector.fields[0], batchSize, batchOffsetInFile); + } + } + } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java index 2cab8ee23..3c83837ba 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java @@ -22,9 +22,11 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericData; import org.apache.avro.util.Utf8; @@ -41,6 +43,7 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types.NestedField; @@ -48,18 +51,23 @@ import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.PartitionUtil; import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; /** * Base class of Spark readers. * * @param is the Java class returned by this reader whose objects contain one or more rows. */ -abstract class BaseDataReader implements Closeable { +public abstract class BaseDataReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(BaseDataReader.class); private final Table table; @@ -156,12 +164,50 @@ protected InputFile getInputFile(String location) { } } - protected static Object convertConstant(Type type, Object value) { + public static Object convertConstant(Type type, Object value) { if (value == null) { return null; } switch (type.typeId()) { + case STRUCT: + StructType structType = type.asStructType(); + + if (structType.fields().isEmpty()) { + return new GenericInternalRow(); + } + + InternalRow ret = new GenericInternalRow(structType.fields().size()); + for (int i = 0; i < structType.fields().size(); i++) { + NestedField field = structType.fields().get(i); + Type fieldType = field.type(); + if (value instanceof Map) { + ret.update(i, convertConstant(field.type(), ((Map) value).get(field.name()))); + } else { + ret.update( + i, + convertConstant( + fieldType, ((StructLike) value).get(i, fieldType.typeId().javaClass()))); + } + } + return ret; + case LIST: + List javaList = + ((Collection) value) + .stream() + .map(e -> convertConstant(type.asListType().elementType(), e)) + .collect(Collectors.toList()); + return ArrayData.toArrayData( + JavaConverters.collectionAsScalaIterableConverter(javaList).asScala().toSeq()); + case MAP: + List keyList = Lists.newArrayList(); + List valueList = Lists.newArrayList(); + for (Map.Entry entry : ((Map) value).entrySet()) { + keyList.add(convertConstant(type.asMapType().keyType(), entry.getKey())); + valueList.add(convertConstant(type.asMapType().valueType(), entry.getValue())); + } + return new ArrayBasedMapData( + new GenericArrayData(keyList.toArray()), new GenericArrayData(valueList.toArray())); case DECIMAL: return Decimal.apply((BigDecimal) value); case STRING: @@ -179,25 +225,6 @@ protected static Object convertConstant(Type type, Object value) { return ByteBuffers.toByteArray((ByteBuffer) value); case BINARY: return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - StructType structType = (StructType) type; - - if (structType.fields().isEmpty()) { - return new GenericInternalRow(); - } - - List fields = structType.fields(); - Object[] values = new Object[fields.size()]; - StructLike struct = (StructLike) value; - - for (int index = 0; index < fields.size(); index++) { - NestedField field = fields.get(index); - Type fieldType = field.type(); - values[index] = - convertConstant(fieldType, struct.get(index, fieldType.typeId().javaClass())); - } - - return new GenericInternalRow(values); default: } return value; diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index f206149da..fe745fdfe 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -22,6 +22,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataTask; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; @@ -35,6 +36,8 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcRowFilter; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -43,6 +46,7 @@ import org.apache.iceberg.spark.data.SparkOrcReader; import org.apache.iceberg.spark.data.SparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.rdd.InputFileBlockHolder; import org.apache.spark.sql.catalyst.InternalRow; @@ -149,6 +153,11 @@ private CloseableIterable newParquetIterable( private CloseableIterable newOrcIterable( InputFile location, FileScanTask task, Schema readSchema, Map idToConstant) { + OrcRowFilter orcRowFilter = OrcRowFilterUtils.rowFilterFromTask(task); + if (orcRowFilter != null) { + validateRowFilterRequirements(task, orcRowFilter); + } + Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot( readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); @@ -160,7 +169,8 @@ private CloseableIterable newOrcIterable( .createReaderFunc( readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema, idToConstant)) .filter(task.residual()) - .caseSensitive(caseSensitive); + .caseSensitive(caseSensitive) + .rowFilter(orcRowFilter); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); @@ -194,4 +204,22 @@ protected InputFile getInputFile(String location) { return RowDataReader.this.getInputFile(location); } } + + private void validateRowFilterRequirements(FileScanTask task, OrcRowFilter filter) { + Preconditions.checkArgument( + task.file().format() == FileFormat.ORC, "Row filter can only be applied to ORC files"); + Preconditions.checkArgument( + task.spec().fields().size() == 0, "Row filter can only be applied to unpartitioned tables"); + for (Types.NestedField column : filter.requiredSchema().columns()) { + Preconditions.checkArgument( + tableSchema.findField(column.name()) != null, + "Row filter can only be applied to top level fields. %s is not a top level field", + column.name()); + Preconditions.checkArgument( + column.type().isPrimitiveType(), + "Row filter can only be applied to primitive fields. %s is of type %s", + column.name(), + column.type()); + } + } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java index 3b3d62d96..c16146e48 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.OptionalLong; import java.util.stream.Collectors; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileFormat; @@ -33,6 +34,8 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.hivelink.core.LegacyHiveTable; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; @@ -184,6 +187,16 @@ public PartitionReaderFactory createReaderFactory() { fileScanTask -> fileScanTask.file().format().equals(FileFormat.ORC))); + boolean hasNoRowFilters = + tasks().stream() + .allMatch( + combinedScanTask -> + !combinedScanTask.isDataTask() + && combinedScanTask.files().stream() + .allMatch( + fileScanTask -> + OrcRowFilterUtils.rowFilterFromTask(fileScanTask) == null)); + boolean atLeastOneColumn = expectedSchema.columns().size() > 0; boolean onlyPrimitives = @@ -196,7 +209,7 @@ public PartitionReaderFactory createReaderFactory() { boolean readUsingBatch = batchReadsEnabled && hasNoDeleteFiles - && (allOrcFileScanTasks + && (allOrcFileScanTasks && hasNoRowFilters || (allParquetFileScanTasks && atLeastOneColumn && onlyPrimitives)); int batchSize = readUsingBatch ? batchSize(allParquetFileScanTasks, allOrcFileScanTasks) : 0; @@ -226,6 +239,11 @@ private int batchSize(boolean isParquetOnly, boolean isOrcOnly) { @Override public Statistics estimateStatistics() { + if (table instanceof LegacyHiveTable) { + // We currently don't have reliable stats for Hive tables + return EMPTY_STATS; + } + // its a fresh table, no data if (table.currentSnapshot() == null) { return new Stats(0L, 0L); @@ -257,6 +275,19 @@ public Statistics estimateStatistics() { return new Stats(sizeInBytes, numRows); } + private static final Statistics EMPTY_STATS = + new Statistics() { + @Override + public OptionalLong sizeInBytes() { + return OptionalLong.empty(); + } + + @Override + public OptionalLong numRows() { + return OptionalLong.empty(); + } + }; + @Override public String description() { String filters = diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java new file mode 100644 index 000000000..9a1d62323 --- /dev/null +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReaderForFieldsWithDefaultValue.java @@ -0,0 +1,202 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkAvroReaderForFieldsWithDefaultValue { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testAvroDefaultValues() throws IOException { + + String writeSchemaString = + "{\n" + + " \"namespace\": \"com.n1\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"n1\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"f0\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + "}"; + org.apache.avro.Schema writeSchema = + new org.apache.avro.Schema.Parser().parse(writeSchemaString); + org.apache.iceberg.Schema icebergWriteSchema = AvroSchemaUtil.toIceberg(writeSchema); + List expected = RandomData.generateList(icebergWriteSchema, 2, 0L); + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + // write records with initial writeSchema + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)).schema(icebergWriteSchema).named("test").build()) { + for (GenericData.Record rec : expected) { + writer.add(rec); + } + } + + // evolve schema by adding a required field with default value + String evolvedSchemaString = + "{\n" + + " \"namespace\": \"com.n1\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"n1\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"f0\",\n" + + " \"type\": \"string\"\n" + + " },\n" + + " {\n" + + " \"name\": \"f1\",\n" + + " \"type\": \"string\",\n" + + " \"default\": \"foo\"\n" + + " },\n" + + " {\n" + + " \"name\": \"f2\",\n" + + " \"type\": \"int\",\n" + + " \"default\": 1\n" + + " },\n" + + " {\n" + + " \"name\": \"f3\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\" : \"int\"\n" + + " },\n" + + " \"default\": {\"a\": 1}\n" + + " },\n" + + " {\n" + + " \"name\": \"f4\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " },\n" + + " \"default\": [1, 2, 3]\n" + + " },\n" + + " {\n" + + " \"name\": \"f5\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"F5\",\n" + + " \"fields\" : [\n" + + " {\"name\": \"ff1\", \"type\": \"long\"},\n" + + " {\"name\": \"ff2\", \"type\": \"string\"}\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"ff1\": 999,\n" + + " \"ff2\": \"foo\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"f6\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": {\n" + + " \"type\": \"array\",\n" + + " \"items\" : \"int\"\n" + + " }\n" + + " },\n" + + " \"default\": {\"key\": [1, 2, 3]}\n" + + " },\n" + + " {\n" + + " \"name\": \"f7\",\n" + + " \"type\": {\n" + + " \"type\": \"fixed\",\n" + + " \"name\": \"md5\",\n" + + " \"size\": 2\n" + + " },\n" + + " \"default\": \"FF\"\n" + + " }\n" + + " ]\n" + + "}"; + org.apache.avro.Schema evolvedSchema = + new org.apache.avro.Schema.Parser().parse(evolvedSchemaString); + + List rows; + Schema icebergReadSchema = AvroSchemaUtil.toIceberg(evolvedSchema); + // read written rows with evolved schema + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(icebergReadSchema) + .build()) { + rows = Lists.newArrayList(reader); + } + + Assert.assertNotNull(rows); + Assert.assertEquals(expected.size(), rows.size()); + for (int row = 0; row < expected.size(); row++) { + InternalRow actualRow = rows.get(row); + final InternalRow expectedRow = new GenericInternalRow(8); + expectedRow.update(0, UTF8String.fromString((String) expected.get(row).get(0))); + expectedRow.update(1, UTF8String.fromString("foo")); + expectedRow.update(2, 1); + expectedRow.update( + 3, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("a"))), + new GenericArrayData(Arrays.asList(1)))); + expectedRow.update(4, new GenericArrayData(ImmutableList.of(1, 2, 3).toArray())); + + final InternalRow nestedStructData = new GenericInternalRow(2); + nestedStructData.update(0, 999L); + nestedStructData.update(1, UTF8String.fromString("foo")); + expectedRow.update(5, nestedStructData); + + List listOfLists = new ArrayList(1); + listOfLists.add(new GenericArrayData(ImmutableList.of(1, 2, 3).toArray())); + expectedRow.update( + 6, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("key"))), + new GenericArrayData(listOfLists.toArray()))); + + byte[] objGUIDByteArr = "FF".getBytes("UTF-8"); + expectedRow.update(7, objGUIDByteArr); + assertEquals(icebergReadSchema, actualRow, expectedRow); + } + } +} diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java new file mode 100644 index 000000000..324aec17b --- /dev/null +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java @@ -0,0 +1,622 @@ +/* + * 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.iceberg.spark.data; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkAvroUnions { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void writeAndValidateRequiredComplexUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", "foo"); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(3, rows.get(0).getStruct(0, 3).numFields()); + Assert.assertEquals(1, rows.get(0).getStruct(0, 3).getInt(0)); + Assert.assertTrue(rows.get(0).getStruct(0, 3).isNullAt(1)); + Assert.assertEquals("foo", rows.get(0).getStruct(0, 3).getString(2)); + + Assert.assertEquals(3, rows.get(1).getStruct(0, 3).numFields()); + Assert.assertEquals(0, rows.get(1).getStruct(0, 3).getInt(0)); + Assert.assertEquals(1, rows.get(1).getStruct(0, 3).getInt(1)); + Assert.assertTrue(rows.get(1).getStruct(0, 3).isNullAt(2)); + } + } + + @Test + public void writeAndValidateOptionalComplexUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .and() + .stringType() + .endUnion() + .nullDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", "foo"); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + GenericData.Record unionRecord3 = new GenericData.Record(avroSchema); + unionRecord3.put("unionCol", null); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + writer.append(unionRecord3); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getStruct(0, 3).getString(2)); + Assert.assertEquals(1, rows.get(1).getStruct(0, 3).getInt(1)); + Assert.assertTrue(rows.get(2).isNullAt(0)); + } + } + + @Test + public void writeAndValidateSingleTypeUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", 0); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(0, rows.get(0).getInt(0)); + Assert.assertEquals(1, rows.get(1).getInt(0)); + } + } + + @Test + public void writeAndValidateNestedSingleTypeUnion1() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("col1", Arrays.asList("foo")); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("col1", Arrays.asList("bar")); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getArray(0).getUTF8String(0).toString()); + Assert.assertEquals("bar", rows.get(1).getArray(0).getUTF8String(0).toString()); + } + } + + @Test + public void writeAndValidateNestedSingleTypeUnion2() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("outerUnion") + .type() + .unionOf() + .record("r") + .fields() + .name("innerUnion") + .type() + .unionOf() + .stringType() + .endUnion() + .noDefault() + .endRecord() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + GenericData.Record innerRecord1 = + new GenericData.Record(avroSchema.getFields().get(0).schema().getTypes().get(0)); + innerRecord1.put("innerUnion", "foo"); + unionRecord1.put("outerUnion", innerRecord1); + + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + GenericData.Record innerRecord2 = + new GenericData.Record(avroSchema.getFields().get(0).schema().getTypes().get(0)); + innerRecord2.put("innerUnion", "bar"); + unionRecord2.put("outerUnion", innerRecord2); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals("foo", rows.get(0).getStruct(0, 1).getUTF8String(0).toString()); + Assert.assertEquals("bar", rows.get(1).getStruct(0, 1).getUTF8String(0).toString()); + } + } + + @Test + public void writeAndValidateSingleTypeUnionOfComplexType() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .array() + .items() + .intType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", Arrays.asList(1)); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", Arrays.asList(2)); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(1, rows.get(0).getArray(0).getInt(0)); + Assert.assertEquals(2, rows.get(1).getArray(0).getInt(0)); + } + } + + @Test + public void writeAndValidateOptionalSingleUnion() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("unionCol") + .type() + .unionOf() + .nullType() + .and() + .intType() + .endUnion() + .nullDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("unionCol", 0); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("unionCol", 1); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + Assert.assertEquals(0, rows.get(0).getInt(0)); + Assert.assertEquals(1, rows.get(1).getInt(0)); + } + } + + @Test + public void testDeeplyNestedUnionSchema1() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .nullType() + .and() + .intType() + .and() + .stringType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record unionRecord1 = new GenericData.Record(avroSchema); + unionRecord1.put("col1", Arrays.asList("foo", 1)); + GenericData.Record unionRecord2 = new GenericData.Record(avroSchema); + unionRecord2.put("col1", Arrays.asList(2, "bar")); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(unionRecord1); + writer.append(unionRecord2); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + // making sure it reads the correctly nested structured data, based on the transformation from + // union to struct + Assert.assertEquals("foo", rows.get(0).getArray(0).getStruct(0, 3).getString(2)); + } + } + + @Test + public void testDeeplyNestedUnionSchema2() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("col1") + .type() + .array() + .items() + .unionOf() + .record("r1") + .fields() + .name("id") + .type() + .intType() + .noDefault() + .endRecord() + .and() + .record("r2") + .fields() + .name("id") + .type() + .intType() + .noDefault() + .endRecord() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record outer = new GenericData.Record(avroSchema); + GenericData.Record inner = + new GenericData.Record( + avroSchema.getFields().get(0).schema().getElementType().getTypes().get(0)); + + inner.put("id", 1); + outer.put("col1", Arrays.asList(inner)); + + File testFile = temp.newFile(); + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(outer); + } + + Schema expectedSchema = AvroSchemaUtil.toIceberg(avroSchema); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + + // making sure it reads the correctly nested structured data, based on the transformation from + // union to struct + Assert.assertEquals(1, rows.get(0).getArray(0).getStruct(0, 3).getStruct(1, 1).getInt(0)); + } + } + + @Test + public void testDeeplyNestedUnionSchema3() throws IOException { + /* + * the printed write schema: + * { + "type": "record", + "name": "root", + "fields": [ + { + "name": "value", + "type": [ + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "ff1", + "type": "long" + }, + { + "name": "ff2", + "type": { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "fff1", + "type": [ + "null", + "string", + "int" + ], + "default": null + } + ] + } + }, + { + "name": "ff3", + "type": { + "type": "array", + "items": "string" + }, + "default": [] + } + ] + }, + "null" + ] + } + ] + } + * */ + org.apache.avro.Schema writeSchema = + SchemaBuilder.record("root") + .fields() + .name("value") + .type() + .unionOf() + .record("r1") + .fields() + .name("ff1") + .type() + .longType() + .noDefault() + .name("ff2") + .type() + .record("r2") + .fields() + .name("fff1") + .type() + .unionOf() + .nullType() + .and() + .stringType() + .and() + .intType() + .endUnion() + .nullDefault() + .endRecord() + .noDefault() + .name("ff3") + .type() + .array() + .items() + .stringType() + .arrayDefault(ImmutableList.of()) + .endRecord() + .and() + .nullType() + .endUnion() + .noDefault() + .endRecord(); + + GenericData.Record record1 = new GenericData.Record(writeSchema); + GenericData.Record record11 = + new GenericData.Record(writeSchema.getField("value").schema().getTypes().get(0)); + GenericData.Record record111 = + new GenericData.Record( + writeSchema.getField("value").schema().getTypes().get(0).getField("ff2").schema()); + // record111.put("fff1", 1); + record11.put("ff1", 99); + record11.put("ff2", record111); + record11.put("ff3", ImmutableList.of()); + record1.put("value", record11); + + GenericData.Record record2 = new GenericData.Record(writeSchema); + GenericData.Record record22 = + new GenericData.Record(writeSchema.getField("value").schema().getTypes().get(0)); + GenericData.Record record222 = + new GenericData.Record( + writeSchema.getField("value").schema().getTypes().get(0).getField("ff2").schema()); + record222.put("fff1", 1); + record22.put("ff1", 99); + record22.put("ff2", record222); + record22.put("ff3", ImmutableList.of("foo")); + record2.put("value", record22); + + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (DataFileWriter writer = + new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(writeSchema, testFile); + writer.append(record1); + writer.append(record2); + } + + List expected = ImmutableList.of(record1, record2); + + org.apache.iceberg.Schema readIcebergSchema = AvroSchemaUtil.toIceberg(writeSchema); + // read written rows with evolved schema + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createReaderFunc(SparkAvroReader::new) + .project(readIcebergSchema) + .build()) { + rows = Lists.newArrayList(reader); + } + + // making sure the rows can be read successfully + Assert.assertEquals(2, rows.size()); + } +} diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java new file mode 100644 index 000000000..980c63eac --- /dev/null +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReaderForFieldsWithDefaultValue.java @@ -0,0 +1,254 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkOrcReaderForFieldsWithDefaultValue { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testOrcDefaultValues() throws IOException { + final int numRows = 10; + + final InternalRow expectedFirstRow = new GenericInternalRow(6); + expectedFirstRow.update(0, 0); + expectedFirstRow.update(1, UTF8String.fromString("foo")); + expectedFirstRow.update(2, new GenericArrayData(ImmutableList.of(1, 2).toArray())); + expectedFirstRow.update( + 3, + new ArrayBasedMapData( + new GenericArrayData(Arrays.asList(UTF8String.fromString("foo"))), + new GenericArrayData(Arrays.asList(1)))); + + final InternalRow nestedStructData = new GenericInternalRow(1); + nestedStructData.update(0, 1); + expectedFirstRow.update(4, nestedStructData); + + // test deeply nested data: array of structs + final InternalRow deepNestedStructData1 = new GenericInternalRow(1); + deepNestedStructData1.update(0, 1); + final InternalRow deepNestedStructData2 = new GenericInternalRow(1); + deepNestedStructData2.update(0, 2); + expectedFirstRow.update( + 5, + new GenericArrayData( + ImmutableList.of(deepNestedStructData1, deepNestedStructData2).toArray())); + + TypeDescription orcSchema = TypeDescription.fromString("struct"); + + Schema readSchema = + new Schema( + Types.NestedField.required(1, "col1", Types.IntegerType.get()), + Types.NestedField.required(2, "col2", Types.StringType.get(), "foo", null), + Types.NestedField.required( + 3, + "col3", + Types.ListType.ofRequired(10, Types.IntegerType.get()), + ImmutableList.of(1, 2), + null), + Types.NestedField.required( + 4, + "col4", + Types.MapType.ofRequired(11, 12, Types.StringType.get(), Types.IntegerType.get()), + ImmutableMap.of("foo", 1), + null), + Types.NestedField.required( + 5, + "col5", + Types.StructType.of( + Types.NestedField.required(13, "nested_col1", Types.IntegerType.get())), + ImmutableMap.of("nested_col1", 1), + null), + Types.NestedField.required( + 6, + "col6", + Types.ListType.ofRequired( + 14, + Types.StructType.of( + Types.NestedField.required(15, "nested_col2", Types.IntegerType.get()))), + ImmutableList.of( + ImmutableMap.of("nested_col2", 1), ImmutableMap.of("nested_col2", 2)), + null)); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector firstCol = (LongColumnVector) batch.cols[0]; + for (int r = 0; r < numRows; ++r) { + int row = batch.size++; + firstCol.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // try to read the data using the readSchema, which is an evolved + // schema that contains a new column with default value + + // non-vectorized read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + + // vectorized-read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + readSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRows = batchesToRows(reader.iterator()); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + } + + @Test + public void testSelectionAndFilterWithDefaultValueColumnOnly() throws IOException { + final int numRows = 10; + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + // expectedFirstRow.update(0, 0); + expectedFirstRow.update(0, UTF8String.fromString("foo")); + + TypeDescription orcSchema = TypeDescription.fromString("struct"); + + Schema readSchema = + new Schema(Types.NestedField.required(2, "col2", Types.StringType.get(), "foo", null)); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector firstCol = (LongColumnVector) batch.cols[0]; + for (int r = 0; r < numRows; ++r) { + int row = batch.size++; + firstCol.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // try to read the data using the readSchema, which is an evolved + // schema that contains a new column with default value + + // non-vectorized read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .filter(Expressions.equal("col2", "foo")) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + + // vectorized-read + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(readSchema) + .filter(Expressions.equal("col2", "foo")) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + readSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRows = batchesToRows(reader.iterator()); + final InternalRow actualFirstRow = actualRows.next(); + + assertEquals(readSchema, expectedFirstRow, actualFirstRow); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } +} diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java new file mode 100644 index 000000000..6f9435697 --- /dev/null +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcUnions.java @@ -0,0 +1,495 @@ +/* + * 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.iceberg.spark.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.UnionColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestSparkOrcUnions { + private static final int NUM_OF_ROWS = 50; + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testComplexUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of( + Types.NestedField.optional(3, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional(2, "field1", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow field1 = new GenericInternalRow(3); + field1.update(0, 0); + field1.update(1, 0); + field1.update(2, null); + expectedFirstRow.update(0, field1); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow field2 = new GenericInternalRow(3); + field2.update(0, 1); + field2.update(1, null); + field2.update(2, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, field2); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + LongColumnVector longColumnVector = new LongColumnVector(NUM_OF_ROWS); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = + new UnionColumnVector(NUM_OF_ROWS, longColumnVector, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = i % 2; + longColumnVector.vector[i] = i; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeeplyNestedUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "c1", + Types.StructType.of( + Types.NestedField.optional(100, "tag", Types.IntegerType.get()), + Types.NestedField.optional(1, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 2, + "field1", + Types.StructType.of( + Types.NestedField.optional(3, "c2", Types.StringType.get()), + Types.NestedField.optional( + 4, + "c3", + Types.StructType.of( + Types.NestedField.optional(101, "tag", Types.IntegerType.get()), + Types.NestedField.optional( + 5, "field0", Types.IntegerType.get()), + Types.NestedField.optional( + 6, "field1", Types.StringType.get())))))))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow inner1 = new GenericInternalRow(3); + inner1.update(0, 1); + inner1.update(1, null); + final InternalRow inner2 = new GenericInternalRow(2); + inner2.update(0, UTF8String.fromString("foo0")); + final InternalRow inner3 = new GenericInternalRow(3); + inner3.update(0, 0); + inner3.update(1, 0); + inner3.update(2, null); + inner2.update(1, inner3); + inner1.update(2, inner2); + expectedFirstRow.update(0, inner1); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector innerUnion1 = (UnionColumnVector) batch.cols[0]; + LongColumnVector innerInt1 = (LongColumnVector) innerUnion1.fields[0]; + innerInt1.fillWithNulls(); + StructColumnVector innerStruct2 = (StructColumnVector) innerUnion1.fields[1]; + BytesColumnVector innerString2 = (BytesColumnVector) innerStruct2.fields[0]; + UnionColumnVector innerUnion3 = (UnionColumnVector) innerStruct2.fields[1]; + LongColumnVector innerInt3 = (LongColumnVector) innerUnion3.fields[0]; + BytesColumnVector innerString3 = (BytesColumnVector) innerUnion3.fields[1]; + innerString3.fillWithNulls(); + + for (int r = 0; r < NUM_OF_ROWS; ++r) { + int row = batch.size++; + innerUnion1.tags[row] = 1; + innerString2.setVal(row, ("foo" + row).getBytes(StandardCharsets.UTF_8)); + innerUnion3.tags[row] = 0; + innerInt3.vector[row] = r; + // If the batch is full, write it out and start over. + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + innerInt1.fillWithNulls(); + innerString3.fillWithNulls(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + writer.close(); + + // test non-vectorized reader + List results = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(results::add); + final InternalRow actualFirstRow = results.get(0); + + Assert.assertEquals(results.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualFirstRow); + } + + // test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnion() throws IOException { + TypeDescription orcSchema = TypeDescription.fromString("struct>"); + + Schema expectedSchema = + new Schema(Types.NestedField.optional(0, "unionCol", Types.StringType.get())); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + expectedFirstRow.update(0, UTF8String.fromString("foo-0")); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + expectedSecondRow.update(0, UTF8String.fromString("foo-1")); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + BytesColumnVector bytesColumnVector = new BytesColumnVector(NUM_OF_ROWS); + UnionColumnVector complexUnion = new UnionColumnVector(NUM_OF_ROWS, bytesColumnVector); + + complexUnion.init(); + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + batch.cols[0] = complexUnion; + + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testSingleTypeUnionOfStruct() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString("struct>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "unionCol", + Types.StructType.of(Types.NestedField.optional(1, "c", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector complexUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) complexUnion.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) structColumnVector.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + complexUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + @Test + public void testDeepNestedSingleTypeUnion() throws IOException { + TypeDescription orcSchema = + TypeDescription.fromString( + "struct>>>"); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional( + 0, + "outerUnion", + Types.StructType.of( + Types.NestedField.optional(1, "innerUnion", Types.StringType.get())))); + + final InternalRow expectedFirstRow = new GenericInternalRow(1); + final InternalRow innerExpectedFirstRow = new GenericInternalRow(1); + innerExpectedFirstRow.update(0, UTF8String.fromString("foo-0")); + expectedFirstRow.update(0, innerExpectedFirstRow); + + final InternalRow expectedSecondRow = new GenericInternalRow(1); + final InternalRow innerExpectedSecondRow = new GenericInternalRow(1); + innerExpectedSecondRow.update(0, UTF8String.fromString("foo-1")); + expectedSecondRow.update(0, innerExpectedSecondRow); + + Configuration conf = new Configuration(); + + File orcFile = temp.newFile(); + Path orcFilePath = new Path(orcFile.getPath()); + + Writer writer = + OrcFile.createWriter( + orcFilePath, OrcFile.writerOptions(conf).setSchema(orcSchema).overwrite(true)); + + VectorizedRowBatch batch = orcSchema.createRowBatch(); + UnionColumnVector outerUnion = (UnionColumnVector) batch.cols[0]; + StructColumnVector structColumnVector = (StructColumnVector) outerUnion.fields[0]; + UnionColumnVector innerUnion = (UnionColumnVector) structColumnVector.fields[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) innerUnion.fields[0]; + + for (int i = 0; i < NUM_OF_ROWS; i += 1) { + outerUnion.tags[i] = 0; + innerUnion.tags[i] = 0; + String stringValue = "foo-" + i; + bytesColumnVector.setVal(i, stringValue.getBytes(StandardCharsets.UTF_8)); + } + + batch.size = NUM_OF_ROWS; + writer.addRowBatch(batch); + batch.reset(); + writer.close(); + + // Test non-vectorized reader + List actualRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(expectedSchema, readOrcSchema)) + .build()) { + reader.forEach(actualRows::add); + + Assert.assertEquals(actualRows.size(), NUM_OF_ROWS); + assertEquals(expectedSchema, expectedFirstRow, actualRows.get(0)); + assertEquals(expectedSchema, expectedSecondRow, actualRows.get(1)); + } + + // Test vectorized reader + try (CloseableIterable reader = + ORC.read(Files.localInput(orcFile)) + .project(expectedSchema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + expectedSchema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRowsIt = batchesToRows(reader.iterator()); + + assertEquals(expectedSchema, expectedFirstRow, actualRowsIt.next()); + assertEquals(expectedSchema, expectedSecondRow, actualRowsIt.next()); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } +} diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 4bd73bf8f..7edef957e 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -22,6 +22,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataTask; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; @@ -35,6 +36,8 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.orc.OrcRowFilter; +import org.apache.iceberg.orc.OrcRowFilterUtils; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -149,6 +152,11 @@ private CloseableIterable newParquetIterable( private CloseableIterable newOrcIterable( InputFile location, FileScanTask task, Schema readSchema, Map idToConstant) { + OrcRowFilter orcRowFilter = OrcRowFilterUtils.rowFilterFromTask(task); + if (orcRowFilter != null) { + validateRowFilterRequirements(task, orcRowFilter); + } + Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot( readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); @@ -160,7 +168,8 @@ private CloseableIterable newOrcIterable( .createReaderFunc( readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema, idToConstant)) .filter(task.residual()) - .caseSensitive(caseSensitive); + .caseSensitive(caseSensitive) + .rowFilter(orcRowFilter); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); @@ -199,4 +208,22 @@ protected void markRowDeleted(InternalRow row) { row.setBoolean(columnIsDeletedPosition(), true); } } + + private void validateRowFilterRequirements(FileScanTask task, OrcRowFilter filter) { + Preconditions.checkArgument( + task.file().format() == FileFormat.ORC, "Row filter can only be applied to ORC files"); + Preconditions.checkArgument( + task.spec().fields().size() == 0, "Row filter can only be applied to unpartitioned tables"); + for (Types.NestedField column : filter.requiredSchema().columns()) { + Preconditions.checkArgument( + tableSchema.findField(column.name()) != null, + "Row filter can only be applied to top level fields. %s is not a top level field", + column.name()); + Preconditions.checkArgument( + column.type().isPrimitiveType(), + "Row filter can only be applied to primitive fields. %s is of type %s", + column.name(), + column.type()); + } + } } diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 53a06c968..76cb355d6 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -190,6 +190,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio exclude group: 'org.antlr' exclude group: 'org.scala-lang' exclude group: 'org.scala-lang.modules' + exclude group: 'io.netty', module: 'netty-common' } } diff --git a/version.properties b/version.properties new file mode 100644 index 000000000..5e944a1b4 --- /dev/null +++ b/version.properties @@ -0,0 +1,22 @@ +# +# 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. +# + +# Version of the produced binaries. +# The version is inferred by shipkit-auto-version Gradle plugin (https://github.com/shipkit/shipkit-auto-version) +version=1.0.0.* \ No newline at end of file