From df0e467d29d2cbaed9ec01c7e4267923896e58cb Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Mon, 31 Jan 2022 15:36:17 -0600 Subject: [PATCH 01/19] SNS Implementation for Event Notification --- .../apache/iceberg/aws/glue/GlueTestBase.java | 13 +++ .../aws/glue/TestGlueCatalogNotification.java | 96 +++++++++++++++++ .../aws/glue/TestGlueCatalogTable.java | 3 + .../aws/AssumeRoleAwsClientFactory.java | 6 ++ .../iceberg/aws/AwsClientFactories.java | 7 ++ .../apache/iceberg/aws/AwsClientFactory.java | 3 + .../apache/iceberg/aws/sns/SNSListener.java | 41 +++++++ .../iceberg/aws/TestAwsClientFactories.java | 6 ++ build.gradle | 1 + .../org/apache/iceberg/util/EventParser.java | 101 ++++++++++++++++++ 10 files changed, 277 insertions(+) create mode 100644 aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java create mode 100644 aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java create mode 100644 core/src/main/java/org/apache/iceberg/util/EventParser.java diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index c47673ab7fa2..32050ed4965e 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.TableProperties; @@ -42,6 +44,7 @@ import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sns.SnsClient; @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) public class GlueTestBase { @@ -59,6 +62,10 @@ public class GlueTestBase { static final AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); static final GlueClient glue = clientFactory.glue(); static final S3Client s3 = clientFactory.s3(); + static final SnsClient sns = clientFactory.sns(); + + static final String testARN = "arn:aws:sns:us-east-1:420609218074:IcebergKunal"; + // iceberg static GlueCatalog glueCatalog; @@ -73,6 +80,12 @@ public class GlueTestBase { TableProperties.WRITE_METADATA_LOCATION, "s3://" + testBucketName + "/writeMetaDataLoc", TableProperties.WRITE_FOLDER_STORAGE_LOCATION, "s3://" + testBucketName + "/writeFolderStorageLoc"); + static final DataFile testDataFile = DataFiles.builder(partitionSpec) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + @BeforeClass public static void beforeClass() { String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java new file mode 100644 index 000000000000..0fa88029f5f9 --- /dev/null +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java @@ -0,0 +1,96 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.aws.glue; + +import com.google.common.collect.Iterables; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.aws.sns.SNSListener; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.events.IncrementalScanEvent; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.junit.Test; + +public class TestGlueCatalogNotification extends GlueTestBase{ + @Test + public void testNotifyOnCreateSnapshotEvent() { + Listeners.register(new SNSListener(testARN, sns), CreateSnapshotEvent.class); + + String namespace = createNamespace(); + String tableName = getRandomName(); + createTable(namespace, tableName); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + + table.newAppend().appendFile(testDataFile).commit(); + } + + @Test + public void testNotifyOnScanEvent() { + Listeners.register(new SNSListener(testARN, sns), ScanEvent.class); + + String namespace = createNamespace(); + String tableName = getRandomName(); + createTable(namespace, tableName); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + + table.newAppend().appendFile(testDataFile).commit(); + table.refresh(); + + table.newScan().planFiles(); + } + + @Test + public void testNotifyOnIncrementalScan() { + Listeners.register(new SNSListener(testARN, sns), IncrementalScanEvent.class); + + String namespace = createNamespace(); + String tableName = getRandomName(); + createTable(namespace, tableName); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + + table.newAppend().appendFile(testDataFile).commit(); + table.newAppend().appendFile(testDataFile).commit(); + table.refresh(); + + Iterable snapshots = table.snapshots(); + table.newScan().appendsBetween(Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()).planFiles(); + } + + @Test + public void testNotifyOnAllEvents() { + SNSListener snsListener = new SNSListener(testARN, sns); + Listeners.register(snsListener, CreateSnapshotEvent.class); + Listeners.register(snsListener, ScanEvent.class); + Listeners.register(snsListener, IncrementalScanEvent.class); + + String namespace = createNamespace(); + String tableName = getRandomName(); + createTable(namespace, tableName); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + + table.newAppend().appendFile(testDataFile).commit(); + table.newScan().planFiles(); + + table.newAppend().appendFile(testDataFile).commit(); + table.refresh(); + + Iterable snapshots = table.snapshots(); + table.newScan().appendsBetween(Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()).planFiles(); + } +} diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 0cdeab6347d2..cb8bc4c4e700 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -31,8 +31,11 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.aws.sns.SNSListener; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.events.Listeners; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.ValidationException; diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java index 1d3a53dce627..cda5042c5f49 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -31,6 +31,7 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sns.SnsClient; import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; @@ -66,6 +67,11 @@ public DynamoDbClient dynamo() { return DynamoDbClient.builder().applyMutation(this::configure).build(); } + @Override + public SnsClient sns() { + return SnsClient.builder().applyMutation(this::configure).build(); + } + @Override public void initialize(Map properties) { roleArn = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_ARN); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index fdbc258c2413..2ca496cdd513 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -31,10 +31,12 @@ import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.core.client.builder.SdkClientBuilder; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sns.SnsClient; public class AwsClientFactories { @@ -108,6 +110,11 @@ public DynamoDbClient dynamo() { return DynamoDbClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); } + @Override + public SnsClient sns() { + return SnsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build(); + } + @Override public void initialize(Map properties) { this.s3Endpoint = properties.get(AwsProperties.S3FILEIO_ENDPOINT); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java index 1ed5e6f5a9c0..6293c748445f 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java @@ -25,6 +25,7 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sns.SnsClient; /** * Interface to customize AWS clients used by Iceberg. @@ -56,6 +57,8 @@ public interface AwsClientFactory extends Serializable { */ DynamoDbClient dynamo(); + SnsClient sns(); + /** * Initialize AWS client factory from catalog properties. * @param properties catalog properties diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java new file mode 100644 index 000000000000..3d7541abdb86 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -0,0 +1,41 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.aws.sns; + +import org.apache.iceberg.events.Listener; +import org.apache.iceberg.util.EventParser; +import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sns.model.PublishRequest; + +public class SNSListener implements Listener { + private String topicArn; + // private AwsClientFactory awsClientFactory; // to be used later + private SnsClient sns; + + public SNSListener(String topicArn, SnsClient sns) { + this.sns = sns; + this.topicArn = topicArn; + } + + @Override + public void notify(Object event) { + String msg = EventParser.toJson(event); + PublishRequest request = PublishRequest.builder() + .message(msg) + .topicArn(topicArn) + .build(); + sns.publish(request); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index 86a10e491ae3..cb68c784d0f3 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -33,6 +33,7 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sns.SnsClient; public class TestAwsClientFactories { @@ -108,6 +109,11 @@ public DynamoDbClient dynamo() { return null; } + @Override + public SnsClient sns() { + return null; + } + @Override public void initialize(Map properties) { diff --git a/build.gradle b/build.gradle index 77b15e8eb08c..7ff3b82f138c 100644 --- a/build.gradle +++ b/build.gradle @@ -317,6 +317,7 @@ project(':iceberg-aws') { compileOnly 'software.amazon.awssdk:glue' compileOnly 'software.amazon.awssdk:sts' compileOnly 'software.amazon.awssdk:dynamodb' + compileOnly 'software.amazon.awssdk:sns' compileOnly("org.apache.hadoop:hadoop-common") { exclude group: 'org.apache.avro', module: 'avro' diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java new file mode 100644 index 000000000000..ad6033da8b67 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -0,0 +1,101 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.util; + +import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; +import java.io.StringWriter; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.events.IncrementalScanEvent; + +public class EventParser { + private static final String TABLE_NAME = "table-name"; + private static final String SNAPSHOT_ID = "snapshot-id"; + private static final String PROJECTION = "projection"; + private static final String OPERATION = "operation"; + private static final String SEQUENCE_NUMBER = "sequence-number"; + private static final String SUMMARY = "summary"; + private static final String FROM_SNAPSHOT_ID = "from-snapshot-id"; + private static final String TO_SNAPSHOT_ID = "to-snapshot-id"; + + private EventParser() { + } + + public static String toJson(Object event) { + try { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + if (event instanceof ScanEvent) { + toJson((ScanEvent) event, generator); + } else if (event instanceof CreateSnapshotEvent) { + toJson((CreateSnapshotEvent) event, generator); + } else if (event instanceof IncrementalScanEvent) { + toJson((IncrementalScanEvent) event, generator); + } + + generator.flush(); + return writer.toString(); + } catch (IOException e) { + throw new UncheckedIOException(String.format("Failed to write json"), e); + } + } + + public static void toJson(ScanEvent event, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeFieldName(TABLE_NAME); + generator.writeString(event.tableName()); + generator.writeFieldName(SNAPSHOT_ID); + generator.writeNumber(event.snapshotId()); + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(event.projection(), generator); + generator.writeEndObject(); + } + + public static void toJson(CreateSnapshotEvent event, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeFieldName(TABLE_NAME); + generator.writeString(event.tableName()); + generator.writeFieldName(OPERATION); + generator.writeString(event.operation()); + generator.writeFieldName(SNAPSHOT_ID); + generator.writeNumber(event.snapshotId()); + generator.writeFieldName(SEQUENCE_NUMBER); + generator.writeNumber(event.sequenceNumber()); + generator.writeObjectFieldStart(SUMMARY); + for (Map.Entry keyValue : event.summary().entrySet()) { + generator.writeStringField(keyValue.getKey(), keyValue.getValue()); + } + + generator.writeEndObject(); + generator.writeEndObject(); + } + + public static void toJson(IncrementalScanEvent event, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeFieldName(TABLE_NAME); + generator.writeString(event.tableName()); + generator.writeFieldName(FROM_SNAPSHOT_ID); + generator.writeNumber(event.fromSnapshotId()); + generator.writeFieldName(TO_SNAPSHOT_ID); + generator.writeNumber(event.toSnapshotId()); + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(event.projection(), generator); + generator.writeEndObject(); + } +} From 1edc3ecbab021f0bab8afcec30d25e3aa1cde9a2 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Fri, 4 Feb 2022 14:52:12 -0600 Subject: [PATCH 02/19] Expression Parser (toJson) (#3) --- .../apache/iceberg/expressions/Literals.java | 6 +- .../aws/glue/TestGlueCatalogNotification.java | 39 ++- .../aws/glue/TestGlueCatalogTable.java | 3 - .../iceberg/aws/AwsClientFactories.java | 1 - .../apache/iceberg/aws/sns/SNSListener.java | 23 +- .../iceberg/aws/TestAwsClientFactories.java | 2 +- .../iceberg/expressions/ExpressionParser.java | 228 +++++++++++++++ .../org/apache/iceberg/util/EventParser.java | 25 +- .../expressions/TestExpressionParser.java | 265 ++++++++++++++++++ 9 files changed, 550 insertions(+), 42 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java create mode 100644 core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 483c2a1a7a79..19722e912aca 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -98,7 +98,7 @@ static BelowMin belowMin() { return BelowMin.INSTANCE; } - private abstract static class BaseLiteral implements Literal { + protected abstract static class BaseLiteral implements Literal { private final T value; private transient volatile ByteBuffer byteBuffer = null; @@ -171,7 +171,7 @@ public Comparator comparator() { static class AboveMax implements Literal { private static final AboveMax INSTANCE = new AboveMax(); - private AboveMax() { + protected AboveMax() { } @Override @@ -198,7 +198,7 @@ public String toString() { static class BelowMin implements Literal { private static final BelowMin INSTANCE = new BelowMin(); - private BelowMin() { + protected BelowMin() { } @Override diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java index 0fa88029f5f9..a53b7996eef7 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java @@ -1,22 +1,24 @@ /* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 + * 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. + * 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.aws.glue; -import com.google.common.collect.Iterables; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.aws.sns.SNSListener; @@ -25,9 +27,10 @@ import org.apache.iceberg.events.IncrementalScanEvent; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.junit.Test; -public class TestGlueCatalogNotification extends GlueTestBase{ +public class TestGlueCatalogNotification extends GlueTestBase { @Test public void testNotifyOnCreateSnapshotEvent() { Listeners.register(new SNSListener(testARN, sns), CreateSnapshotEvent.class); @@ -69,7 +72,10 @@ public void testNotifyOnIncrementalScan() { table.refresh(); Iterable snapshots = table.snapshots(); - table.newScan().appendsBetween(Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()).planFiles(); + table.newScan().appendsBetween( + Iterables.get(snapshots, 0).snapshotId(), + Iterables.get(snapshots, 1).snapshotId()) + .planFiles(); } @Test @@ -91,6 +97,9 @@ public void testNotifyOnAllEvents() { table.refresh(); Iterable snapshots = table.snapshots(); - table.newScan().appendsBetween(Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()).planFiles(); + table.newScan().appendsBetween( + Iterables.get(snapshots, 0).snapshotId(), + Iterables.get(snapshots, 1).snapshotId()) + .planFiles(); } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index cb8bc4c4e700..0cdeab6347d2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -31,11 +31,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; -import org.apache.iceberg.aws.sns.SNSListener; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.events.Listeners; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.ValidationException; diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 2ca496cdd513..5e11b378fc05 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -31,7 +31,6 @@ import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.core.client.builder.SdkClientBuilder; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; -import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index 3d7541abdb86..dc84728a0f6e 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -1,15 +1,20 @@ /* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 + * 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. + * 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.aws.sns; diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index cb68c784d0f3..bb6863077216 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -111,7 +111,7 @@ public DynamoDbClient dynamo() { @Override public SnsClient sns() { - return null; + return null; } @Override diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java new file mode 100644 index 000000000000..e9fe6c9901b5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -0,0 +1,228 @@ +/* + * 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.expressions; + +import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; +import java.io.StringWriter; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.JsonUtil; + +public class ExpressionParser { + + private static final String TYPE = "type"; + private static final String VALUE = "value"; + private static final String OPERATION = "operation"; + private static final String LITERALS = "literals"; + private static final String TERM = "term"; + private static final String LEFT_OPERAND = "left-operand"; + private static final String RIGHT_OPERAND = "right-operand"; + private static final String OPERAND = "operand"; + + private static final String AND = "and"; + private static final String OR = "or"; + private static final String NOT = "not"; + private static final String TRUE = "true"; + private static final String FALSE = "false"; + private static final String UNBOUNDED_PREDICATE = "unbounded-predicate"; + private static final String BOUNDED_LITERAL_PREDICATE = "bounded-literal-predicate"; + private static final String BOUNDED_SET_PREDICATE = "bounded-set-predicate"; + private static final String BOUNDED_UNARY_PREDICATE = "bounded-unary-predicate"; + + private static final String NAMED_REFERENCE = "named-reference"; + private static final String BOUND_REFERENCE = "bound-reference"; + + private static final String ABOVE_MAX = "above-max"; + private static final String BELOW_MIN = "below-min"; + + private static final Set oneInputs = ImmutableSet.of( + Expression.Operation.IS_NULL, + Expression.Operation.NOT_NULL, + Expression.Operation.IS_NAN, + Expression.Operation.NOT_NAN); + + + private ExpressionParser() { + } + + public static String toJson(Expression expression, boolean pretty) { + try { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + if (pretty) { + generator.useDefaultPrettyPrinter(); + } + toJson(expression, generator); + generator.flush(); + return writer.toString(); + + } catch (IOException e) { + throw new UncheckedIOException(String.format("Failed to write json"), e); + } + } + + public static void toJson(Expression expression, JsonGenerator generator) throws IOException { + if (expression instanceof And) { + toJson((And) expression, generator); + } else if (expression instanceof Or) { + toJson((Or) expression, generator); + } else if (expression instanceof Not) { + toJson((Not) expression, generator); + } else if (expression instanceof True) { + toJson((True) expression, generator); + } else if (expression instanceof False) { + toJson((False) expression, generator); + } else if (expression instanceof Predicate) { + toJson((Predicate) expression, generator); + } else { + throw new IllegalArgumentException("Invalid Operation Type"); + } + } + + public static void toJson(And expression, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, AND); + generator.writeFieldName(LEFT_OPERAND); + toJson(expression.left(), generator); + generator.writeFieldName(RIGHT_OPERAND); + toJson(expression.right(), generator); + generator.writeEndObject(); + } + + public static void toJson(Or expression, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, OR); + generator.writeFieldName(LEFT_OPERAND); + toJson(expression.left(), generator); + generator.writeFieldName(RIGHT_OPERAND); + toJson(expression.right(), generator); + generator.writeEndObject(); + } + + public static void toJson(Not expression, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, NOT); + generator.writeFieldName(OPERAND); + toJson(expression.child(), generator); + generator.writeEndObject(); + } + + public static void toJson(True expression, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, TRUE); + generator.writeEndObject(); + } + + public static void toJson(False expression, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, FALSE); + generator.writeEndObject(); + } + + public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { + if (predicate instanceof UnboundPredicate) { + toJson((UnboundPredicate) predicate, generator); + } else if (predicate instanceof BoundLiteralPredicate) { + toJson((BoundLiteralPredicate) predicate, generator); + } else if (predicate instanceof BoundSetPredicate) { + toJson((BoundSetPredicate) predicate, generator); + } else if (predicate instanceof BoundUnaryPredicate) { + toJson((BoundUnaryPredicate) predicate, generator); + } else { + throw new IllegalArgumentException("Cannot find valid Predicate Type for " + predicate + "."); + } + } + + public static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, UNBOUNDED_PREDICATE); + generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); + generator.writeFieldName(TERM); + toJson(predicate.term(), generator); + if (!(oneInputs.contains(predicate.op()))) { + generator.writeFieldName(LITERALS); + toJson(predicate.literals(), generator); + } + + generator.writeEndObject(); + } + + public static void toJson(BoundLiteralPredicate predicate, JsonGenerator generator) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundLiteralPredicate is not currently supported."); + } + + public static void toJson(BoundSetPredicate predicate, JsonGenerator generator) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundSetPredicate is not currently supported."); + } + + public static void toJson(BoundUnaryPredicate predicate, JsonGenerator generator) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundUnaryPredicate is not currently supported."); + } + + public static void toJson(Term term, JsonGenerator generator) throws IOException { + if (term instanceof NamedReference) { + toJson((NamedReference) term, generator); + } else if (term instanceof BoundReference) { + toJson((BoundReference) term, generator); // Need to Implement + } else { + throw new IllegalArgumentException("Cannot find valid Term Type for " + term + "."); + } + } + + public static void toJson(NamedReference term, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(TYPE, NAMED_REFERENCE); + generator.writeStringField(VALUE, term.name()); + generator.writeEndObject(); + } + + public static void toJson(BoundReference term, JsonGenerator generator) { + throw new UnsupportedOperationException("Serialization of Term type BoundReference is not currently supported"); + } + + public static void toJson(List literals, JsonGenerator generator) throws IOException { + generator.writeStartArray(); + for (int i = 0; i < literals.size(); i++) { + toJson(literals.get(i), generator); + } + generator.writeEndArray(); + } + + public static void toJson(Literal literal, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + if (literal instanceof Literals.AboveMax) { + generator.writeStringField(TYPE, ABOVE_MAX); + } else if (literal instanceof Literals.BelowMin) { + generator.writeStringField(TYPE, BELOW_MIN); + } else { + generator.writeStringField(TYPE, ((Literals.BaseLiteral) literal).typeId().toString().toLowerCase()); + generator.writeStringField(VALUE, StandardCharsets.UTF_8.decode(literal.toByteBuffer()).toString()); + } + + generator.writeEndObject(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java index ad6033da8b67..a21d52406b78 100644 --- a/core/src/main/java/org/apache/iceberg/util/EventParser.java +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -1,15 +1,20 @@ /* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 + * 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. + * 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.util; @@ -20,9 +25,9 @@ import java.io.UncheckedIOException; import java.util.Map; import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.IncrementalScanEvent; +import org.apache.iceberg.events.ScanEvent; public class EventParser { private static final String TABLE_NAME = "table-name"; diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java new file mode 100644 index 000000000000..bc57825a7114 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -0,0 +1,265 @@ +/* + * 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.expressions; + +import org.junit.Assert; +import org.junit.Test; + +public class TestExpressionParser { + @Test + public void testPredicate() { + String expected = "{\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"in\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"integer\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + "}"; + + UnboundPredicate inPredicate = new UnboundPredicate( + Expression.Operation.IN, + new NamedReference("Column-Name"), + Literal.of(50)); + + String actual = ExpressionParser.toJson(inPredicate, true); + Assert.assertEquals(expected, actual); + } + @Test + public void testAnd() { + String expected = "{\n" + + " \"type\" : \"and\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"gt_eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"integer\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + " },\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"in\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column2-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"string\",\n" + + " \"value\" : \"Check\"\n" + + " } ]\n" + + " }\n" + + "}"; + + UnboundPredicate gtEqPredicate = new UnboundPredicate( + Expression.Operation.GT_EQ, + new NamedReference("Column1-Name"), + Literal.of(50)); + + UnboundPredicate inPredicate = new UnboundPredicate( + Expression.Operation.IN, + new NamedReference("Column2-Name"), + Literal.of("Check")); + + And andExpression = new And(gtEqPredicate, inPredicate); + String actual = ExpressionParser.toJson(andExpression, true); + Assert.assertEquals(expected, actual); + } + + @Test + public void testOr() { + String expected = "{\n" + + " \"type\" : \"or\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"lt\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"integer\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + " },\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"not_null\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column2-Name\"\n" + + " }\n" + + " }\n" + + "}"; + + UnboundPredicate ltPredicate = new UnboundPredicate( + Expression.Operation.LT, + new NamedReference("Column1-Name"), + Literal.of(50)); + + UnboundPredicate notNullPredicate = new UnboundPredicate( + Expression.Operation.NOT_NULL, + new NamedReference("Column2-Name")); + + Or orExpression = new Or(ltPredicate, notNullPredicate); + String actual = ExpressionParser.toJson(orExpression, true); + Assert.assertEquals(expected, actual); + } + + @Test + public void testNot() { + String expected = "{\n" + + " \"type\" : \"not\",\n" + + " \"operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"lt\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"integer\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + " }\n" + + "}"; + + UnboundPredicate ltPredicate = new UnboundPredicate( + Expression.Operation.LT, + new NamedReference("Column1-Name"), + Literal.of(50)); + + Not notExpression = new Not(ltPredicate); + String actual = ExpressionParser.toJson(notExpression, true); + Assert.assertEquals(expected, actual); + } + + @Test + public void testNestedExpression() { + String expected = "{\n" + + " \"type\" : \"or\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"and\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"in\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"integer\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + " },\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column2-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"string\",\n" + + " \"value\" : \"Test\"\n" + + " } ]\n" + + " }\n" + + " },\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"is_nan\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column3-Name\"\n" + + " }\n" + + " }\n" + + "}"; + + UnboundPredicate inPredicate = new UnboundPredicate( + Expression.Operation.IN, + new NamedReference("Column1-Name"), + Literal.of(50)); + + UnboundPredicate eqPredicate = new UnboundPredicate( + Expression.Operation.EQ, + new NamedReference("Column2-Name"), + Literal.of("Test")); + + UnboundPredicate isNanPredicate = new UnboundPredicate( + Expression.Operation.IS_NAN, + new NamedReference("Column3-Name")); + + And andExpression = new And(inPredicate, eqPredicate); + Or orNestedExpression = new Or(andExpression, isNanPredicate); + String actual = ExpressionParser.toJson(orNestedExpression, true); + Assert.assertEquals(expected, actual); + } + + @Test + public void aboveMaxAndBelowMinTest() { + String expected = "{\n" + + " \"type\" : \"and\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"lt\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"above-max\"\n" + + " } ]\n" + + " },\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"gt_eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column2-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"below-min\"\n" + + " } ]\n" + + " }\n" + + "}"; + + UnboundPredicate aboveMaxPredicate = new UnboundPredicate( + Expression.Operation.LT, + new NamedReference("Column1-Name"), + new Literals.AboveMax()); + + UnboundPredicate belowMinPredicate = new UnboundPredicate( + Expression.Operation.GT_EQ, + new NamedReference("Column2-Name"), + new Literals.BelowMin<>()); + + And andExpression = new And(aboveMaxPredicate, belowMinPredicate); + String actual = ExpressionParser.toJson(andExpression, true); + Assert.assertEquals(expected, actual); + } +} From 8bc441ec46c6567f311f86104b7e19997e37a8b8 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Mon, 7 Feb 2022 12:51:57 -0600 Subject: [PATCH 03/19] Include ExpressionParser in EventParser (#4) * Event Parser and Logging Added Expression parser in event parser and adding error logging for notify. * Removed unused imports * Update scan test logic and SNS error message Combine calls for scan event logic in test. Add topic name in error message for SNS topic. * Remove unused import Did not remove Tablescan import after changing scan to one inline operation * Fix error throwing --- .../aws/glue/TestGlueCatalogNotification.java | 5 +++- .../apache/iceberg/aws/sns/SNSListener.java | 24 ++++++++++++++----- .../org/apache/iceberg/util/EventParser.java | 6 +++++ 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java index a53b7996eef7..f31be86df5be 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java @@ -27,6 +27,8 @@ import org.apache.iceberg.events.IncrementalScanEvent; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.junit.Test; @@ -55,7 +57,8 @@ public void testNotifyOnScanEvent() { table.newAppend().appendFile(testDataFile).commit(); table.refresh(); - table.newScan().planFiles(); + Expression andExpression = Expressions.and(Expressions.equal("c1", "First"), Expressions.equal("c1", "Second")); + table.newScan().filter(andExpression).planFiles(); } @Test diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index dc84728a0f6e..c71709266c8f 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -21,10 +21,15 @@ import org.apache.iceberg.events.Listener; import org.apache.iceberg.util.EventParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.sns.SnsClient; import software.amazon.awssdk.services.sns.model.PublishRequest; +import software.amazon.awssdk.services.sns.model.SnsException; public class SNSListener implements Listener { + private static final Logger LOG = LoggerFactory.getLogger(SNSListener.class); + private String topicArn; // private AwsClientFactory awsClientFactory; // to be used later private SnsClient sns; @@ -36,11 +41,18 @@ public SNSListener(String topicArn, SnsClient sns) { @Override public void notify(Object event) { - String msg = EventParser.toJson(event); - PublishRequest request = PublishRequest.builder() - .message(msg) - .topicArn(topicArn) - .build(); - sns.publish(request); + try { + String msg = EventParser.toJson(event); + PublishRequest request = PublishRequest.builder() + .message(msg) + .topicArn(topicArn) + .build(); + sns.publish(request); + } catch (SnsException e) { + LOG.error("Failed to send notification event to SNS topic", e); + } catch (RuntimeException e) { + LOG.error("Failed to notify subscriber", e); + } } } + diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java index a21d52406b78..ecf018a1a8e8 100644 --- a/core/src/main/java/org/apache/iceberg/util/EventParser.java +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -28,11 +28,13 @@ import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.IncrementalScanEvent; import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.ExpressionParser; public class EventParser { private static final String TABLE_NAME = "table-name"; private static final String SNAPSHOT_ID = "snapshot-id"; private static final String PROJECTION = "projection"; + private static final String EXPRESSION = "expression"; private static final String OPERATION = "operation"; private static final String SEQUENCE_NUMBER = "sequence-number"; private static final String SUMMARY = "summary"; @@ -67,6 +69,8 @@ public static void toJson(ScanEvent event, JsonGenerator generator) throws IOExc generator.writeString(event.tableName()); generator.writeFieldName(SNAPSHOT_ID); generator.writeNumber(event.snapshotId()); + generator.writeFieldName(EXPRESSION); + ExpressionParser.toJson(event.filter(), generator); generator.writeFieldName(PROJECTION); SchemaParser.toJson(event.projection(), generator); generator.writeEndObject(); @@ -99,6 +103,8 @@ public static void toJson(IncrementalScanEvent event, JsonGenerator generator) t generator.writeNumber(event.fromSnapshotId()); generator.writeFieldName(TO_SNAPSHOT_ID); generator.writeNumber(event.toSnapshotId()); + generator.writeFieldName(EXPRESSION); + ExpressionParser.toJson(event.filter(), generator); generator.writeFieldName(PROJECTION); SchemaParser.toJson(event.projection(), generator); generator.writeEndObject(); From 0c6a0bc8f042f8a2650ee476856b5a79057a7323 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Tue, 8 Feb 2022 18:01:22 -0600 Subject: [PATCH 04/19] Add SQSListener for automatic integration test (#6) --- .../apache/iceberg/aws/glue/GlueTestBase.java | 5 +- .../aws/glue/TestGlueCatalogNotification.java | 200 ++++++++++++++++-- .../aws/AssumeRoleAwsClientFactory.java | 6 + .../iceberg/aws/AwsClientFactories.java | 6 + .../apache/iceberg/aws/AwsClientFactory.java | 3 + .../apache/iceberg/aws/sns/SNSListener.java | 6 +- .../apache/iceberg/aws/sqs/SQSListener.java | 58 +++++ .../iceberg/aws/TestAwsClientFactories.java | 6 + build.gradle | 1 + 9 files changed, 273 insertions(+), 18 deletions(-) create mode 100644 aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index 32050ed4965e..38505c1569e5 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -45,6 +45,7 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sqs.SqsClient; @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) public class GlueTestBase { @@ -63,9 +64,7 @@ public class GlueTestBase { static final GlueClient glue = clientFactory.glue(); static final S3Client s3 = clientFactory.s3(); static final SnsClient sns = clientFactory.sns(); - - static final String testARN = "arn:aws:sns:us-east-1:420609218074:IcebergKunal"; - + static final SqsClient sqs = clientFactory.sqs(); // iceberg static GlueCatalog glueCatalog; diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java index f31be86df5be..3e607b72e8ae 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java @@ -19,9 +19,14 @@ package org.apache.iceberg.aws.glue; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.util.List; +import java.util.Set; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.aws.sns.SNSListener; +import org.apache.iceberg.aws.sqs.SQSListener; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.IncrementalScanEvent; @@ -30,12 +35,26 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.Assert; import org.junit.Test; +import software.amazon.awssdk.services.sqs.model.CreateQueueRequest; +import software.amazon.awssdk.services.sqs.model.DeleteQueueRequest; +import software.amazon.awssdk.services.sqs.model.GetQueueUrlRequest; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; public class TestGlueCatalogNotification extends GlueTestBase { @Test - public void testNotifyOnCreateSnapshotEvent() { - Listeners.register(new SNSListener(testARN, sns), CreateSnapshotEvent.class); + public void testNotifyOnCreateSnapshotEvent() throws IOException { + String queueUrl = createSqsQueue("CreateSnapshotEvent"); + + List messages = getMessages(queueUrl); + messages = getMessages(queueUrl); + Assert.assertEquals(0, messages.size()); + + Listeners.register(new SQSListener(queueUrl, sqs), CreateSnapshotEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -43,11 +62,36 @@ public void testNotifyOnCreateSnapshotEvent() { Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); table.newAppend().appendFile(testDataFile).commit(); + + messages = getMessages(queueUrl); + Assert.assertEquals(1, messages.size()); + + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); + + String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"operation\":\"append\",\"snapshot-id\":" + table.currentSnapshot().snapshotId() + "," + + "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\"," + + "\"added-records\":\"1\",\"added-files-size\":\"10\"," + + "\"changed-partition-count\":\"1\",\"total-records\":\"1\"," + + "\"total-files-size\":\"10\",\"total-data-files\":\"1\"," + + "\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\"," + + "\"total-equality-deletes\":\"0\"}}"; + + Assert.assertEquals(expectedMessage, bodyNode.toString()); + + destroySqsQueue(queueUrl); } @Test - public void testNotifyOnScanEvent() { - Listeners.register(new SNSListener(testARN, sns), ScanEvent.class); + public void testNotifyOnScanEvent() throws IOException { + String queueUrl = createSqsQueue("ScanEvent"); + + List messages = getMessages(queueUrl); + messages = getMessages(queueUrl); + Assert.assertEquals(0, messages.size()); + + Listeners.register(new SQSListener(queueUrl, sqs), ScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -59,11 +103,39 @@ public void testNotifyOnScanEvent() { Expression andExpression = Expressions.and(Expressions.equal("c1", "First"), Expressions.equal("c1", "Second")); table.newScan().filter(andExpression).planFiles(); + + messages = getMessages(queueUrl); + Assert.assertEquals(1, messages.size()); + + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); + + String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"snapshot-id\":" + table.currentSnapshot().snapshotId() + "," + + "\"expression\":{\"type\":\"and\"," + + "\"left-operand\":{\"type\":\"unbounded-predicate\"," + + "\"operation\":\"eq\",\"term\":{\"type\":\"named-reference\",\"value\":\"c1\"}," + + "\"literals\":[{\"type\":\"string\",\"value\":\"First\"}]}," + + "\"right-operand\":{\"type\":\"unbounded-predicate\"," + + "\"operation\":\"eq\",\"term\":{\"type\":\"named-reference\",\"value\":\"c1\"}," + + "\"literals\":[{\"type\":\"string\",\"value\":\"Second\"}]}}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; + + Assert.assertEquals(expectedMessage, bodyNode.toString()); + + destroySqsQueue(queueUrl); } @Test - public void testNotifyOnIncrementalScan() { - Listeners.register(new SNSListener(testARN, sns), IncrementalScanEvent.class); + public void testNotifyOnIncrementalScan() throws IOException { + String queueUrl = createSqsQueue("IncrementalScan"); + + List messages = getMessages(queueUrl); + messages = getMessages(queueUrl); + Assert.assertEquals(0, messages.size()); + + Listeners.register(new SQSListener(queueUrl, sqs), IncrementalScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -79,14 +151,41 @@ public void testNotifyOnIncrementalScan() { Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()) .planFiles(); + + messages = getMessages(queueUrl); + Assert.assertEquals(1, messages.size()); + + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); + + String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"from-snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + + "\"to-snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + + "\"expression\":{\"type\":\"true\"}," + + "\"projection\":{\"type\":\"struct\"," + + "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; + + Assert.assertEquals(expectedMessage, bodyNode.toString()); + + destroySqsQueue(queueUrl); } @Test - public void testNotifyOnAllEvents() { - SNSListener snsListener = new SNSListener(testARN, sns); - Listeners.register(snsListener, CreateSnapshotEvent.class); - Listeners.register(snsListener, ScanEvent.class); - Listeners.register(snsListener, IncrementalScanEvent.class); + public void testNotifyOnAllEvents() throws IOException { + String queueUrl = createSqsQueue("AllEvents"); + + List messages = getMessages(queueUrl); + + messages = getMessages(queueUrl); + Assert.assertEquals(0, messages.size()); + + SQSListener createSnapshotListener = new SQSListener(queueUrl, sqs); + SQSListener scanListener = new SQSListener(queueUrl, sqs); + SQSListener incrementalScanListener = new SQSListener(queueUrl, sqs); + Listeners.register(createSnapshotListener, CreateSnapshotEvent.class); + Listeners.register(scanListener, ScanEvent.class); + Listeners.register(incrementalScanListener, IncrementalScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -104,5 +203,82 @@ public void testNotifyOnAllEvents() { Iterables.get(snapshots, 0).snapshotId(), Iterables.get(snapshots, 1).snapshotId()) .planFiles(); + + messages = getMessages(queueUrl); + Assert.assertEquals(4, messages.size()); + + ObjectMapper objectMapper = new ObjectMapper(); + List actualBodyNodesMessages = Lists.newArrayList(); + + for (int i = 0; i < 4; i++) { + actualBodyNodesMessages.add(objectMapper.readTree(messages.get(i).body()).toString()); + + } + + Set expectedBodyNodesMessages = Sets.newHashSet(); + expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + + "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + + "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"); + expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + + "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + + "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"2\"," + + "\"total-files-size\":\"20\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\"," + + "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}"); + expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"from-snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + + "\"to-snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + + "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + + "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"); + expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + + "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + + "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"1\"," + + "\"total-files-size\":\"10\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\"," + + "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}"); + + Assert.assertEquals(4, expectedBodyNodesMessages.size()); + for (String message : actualBodyNodesMessages) { + expectedBodyNodesMessages.remove(message); + } + + Assert.assertEquals(0, expectedBodyNodesMessages.size()); + + destroySqsQueue(queueUrl); + } + + public static List getMessages(String sqsUrl) { + ReceiveMessageRequest receiveMessageRequest = ReceiveMessageRequest.builder() + .queueUrl(sqsUrl) + .visibilityTimeout(100) + .waitTimeSeconds(2) + .maxNumberOfMessages(10) + .build(); + + List messages = Lists.newArrayList(); + messages.addAll(sqs.receiveMessage(receiveMessageRequest).messages()); + int prevCounter = -1; + while (prevCounter != messages.size()) { + prevCounter = messages.size(); + messages.addAll(sqs.receiveMessage(receiveMessageRequest).messages()); + } + + return messages; } + + public static String createSqsQueue(String queueName) { + CreateQueueRequest createRequest = CreateQueueRequest.builder().queueName(queueName).build(); + sqs.createQueue(createRequest); + GetQueueUrlRequest urlRequest = GetQueueUrlRequest.builder().queueName(queueName).build(); + return sqs.getQueueUrl(urlRequest).queueUrl(); + } + + public static void destroySqsQueue(String queueUrl) { + DeleteQueueRequest deleteRequest = DeleteQueueRequest.builder().queueUrl(queueUrl).build(); + sqs.deleteQueue(deleteRequest); + } + } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java index cda5042c5f49..1114d077e74f 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -32,6 +32,7 @@ import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; @@ -72,6 +73,11 @@ public SnsClient sns() { return SnsClient.builder().applyMutation(this::configure).build(); } + @Override + public SqsClient sqs() { + return SqsClient.builder().applyMutation(this::configure).build(); + } + @Override public void initialize(Map properties) { roleArn = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_ARN); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 5e11b378fc05..400ef0a2bd14 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -36,6 +36,7 @@ import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sqs.SqsClient; public class AwsClientFactories { @@ -114,6 +115,11 @@ public SnsClient sns() { return SnsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build(); } + @Override + public SqsClient sqs() { + return SqsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build(); + } + @Override public void initialize(Map properties) { this.s3Endpoint = properties.get(AwsProperties.S3FILEIO_ENDPOINT); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java index 6293c748445f..3b24c851b729 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java @@ -26,6 +26,7 @@ import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sqs.SqsClient; /** * Interface to customize AWS clients used by Iceberg. @@ -59,6 +60,8 @@ public interface AwsClientFactory extends Serializable { SnsClient sns(); + SqsClient sqs(); + /** * Initialize AWS client factory from catalog properties. * @param properties catalog properties diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index c71709266c8f..c51571c7c70a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -27,12 +27,12 @@ import software.amazon.awssdk.services.sns.model.PublishRequest; import software.amazon.awssdk.services.sns.model.SnsException; -public class SNSListener implements Listener { +public class SNSListener implements Listener { private static final Logger LOG = LoggerFactory.getLogger(SNSListener.class); - private String topicArn; + private final String topicArn; // private AwsClientFactory awsClientFactory; // to be used later - private SnsClient sns; + private final SnsClient sns; public SNSListener(String topicArn, SnsClient sns) { this.sns = sns; diff --git a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java new file mode 100644 index 000000000000..8ea1b9350013 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.sqs; + +import org.apache.iceberg.events.Listener; +import org.apache.iceberg.util.EventParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.SendMessageRequest; +import software.amazon.awssdk.services.sqs.model.SqsException; + +public class SQSListener implements Listener { + private static final Logger LOG = LoggerFactory.getLogger(SQSListener.class); + + private final String queueURL; + // private AwsClientFactory awsClientFactory; // to be used later + private final SqsClient sqs; + + public SQSListener(String queueURL, SqsClient sqs) { + this.sqs = sqs; + this.queueURL = queueURL; + } + + @Override + public void notify(Object event) { + try { + String msg = EventParser.toJson(event); + SendMessageRequest request = SendMessageRequest.builder() + .queueUrl(queueURL) + .messageBody(msg) + .build(); + sqs.sendMessage(request); + } catch (SqsException e) { + LOG.error("Failed to send notification event to SQS", e); + } catch (RuntimeException e) { + LOG.error("Failed to add to queue", e); + } + } +} + diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index bb6863077216..ad0ece3e5631 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -34,6 +34,7 @@ import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sqs.SqsClient; public class TestAwsClientFactories { @@ -114,6 +115,11 @@ public SnsClient sns() { return null; } + @Override + public SqsClient sqs() { + return null; + } + @Override public void initialize(Map properties) { diff --git a/build.gradle b/build.gradle index 7ff3b82f138c..edce810177bc 100644 --- a/build.gradle +++ b/build.gradle @@ -318,6 +318,7 @@ project(':iceberg-aws') { compileOnly 'software.amazon.awssdk:sts' compileOnly 'software.amazon.awssdk:dynamodb' compileOnly 'software.amazon.awssdk:sns' + compileOnly 'software.amazon.awssdk:sqs' compileOnly("org.apache.hadoop:hadoop-common") { exclude group: 'org.apache.avro', module: 'avro' From 29aa16e4f59fdad89b81630d3ee9a781a9357a3f Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Tue, 8 Feb 2022 17:05:49 -0800 Subject: [PATCH 05/19] Fix integration test and cleanup codebase --- .../apache/iceberg/aws/glue/GlueTestBase.java | 14 +- ...va => TestGlueCatalogSqsNotification.java} | 158 +++++++++--------- .../apache/iceberg/aws/sns/SNSListener.java | 15 +- .../apache/iceberg/aws/sqs/SQSListener.java | 21 ++- .../iceberg/expressions/ExpressionParser.java | 5 +- 5 files changed, 104 insertions(+), 109 deletions(-) rename aws/src/integration/java/org/apache/iceberg/aws/glue/{TestGlueCatalogNotification.java => TestGlueCatalogSqsNotification.java} (73%) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index 38505c1569e5..fc37de5ad097 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -22,8 +22,6 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.TableProperties; @@ -44,8 +42,6 @@ import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sqs.SqsClient; @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) public class GlueTestBase { @@ -63,8 +59,6 @@ public class GlueTestBase { static final AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); static final GlueClient glue = clientFactory.glue(); static final S3Client s3 = clientFactory.s3(); - static final SnsClient sns = clientFactory.sns(); - static final SqsClient sqs = clientFactory.sqs(); // iceberg static GlueCatalog glueCatalog; @@ -79,13 +73,7 @@ public class GlueTestBase { TableProperties.WRITE_METADATA_LOCATION, "s3://" + testBucketName + "/writeMetaDataLoc", TableProperties.WRITE_FOLDER_STORAGE_LOCATION, "s3://" + testBucketName + "/writeFolderStorageLoc"); - static final DataFile testDataFile = DataFiles.builder(partitionSpec) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); - - @BeforeClass +@BeforeClass public static void beforeClass() { String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; S3FileIO fileIO = new S3FileIO(clientFactory::s3); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java similarity index 73% rename from aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java rename to aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java index 3e607b72e8ae..1cbe3f9ae9bf 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNotification.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java @@ -22,14 +22,19 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.aws.sqs.SQSListener; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.IncrementalScanEvent; +import org.apache.iceberg.events.Listener; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expression; @@ -37,24 +42,52 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.JsonUtil; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sqs.model.CreateQueueRequest; import software.amazon.awssdk.services.sqs.model.DeleteQueueRequest; import software.amazon.awssdk.services.sqs.model.GetQueueUrlRequest; import software.amazon.awssdk.services.sqs.model.Message; import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; -public class TestGlueCatalogNotification extends GlueTestBase { - @Test - public void testNotifyOnCreateSnapshotEvent() throws IOException { - String queueUrl = createSqsQueue("CreateSnapshotEvent"); +public class TestGlueCatalogSqsNotification extends GlueTestBase { + + private static final SqsClient sqs = clientFactory.sqs(); + private static final DataFile testDataFile = DataFiles.builder(partitionSpec) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + private static final int RETRY = 3; + private static final long RETRY_INTERVAL_MS = 1000; + + private String queueUrl; + + @Before + public void before() { + String queueName = getRandomName(); + sqs.createQueue(CreateQueueRequest.builder() + .queueName(queueName) + .build()); + this.queueUrl = sqs.getQueueUrl(GetQueueUrlRequest.builder() + .queueName(queueName) + .build()) + .queueUrl(); + } - List messages = getMessages(queueUrl); - messages = getMessages(queueUrl); - Assert.assertEquals(0, messages.size()); + @After + public void after() { + DeleteQueueRequest deleteRequest = DeleteQueueRequest.builder().queueUrl(queueUrl).build(); + sqs.deleteQueue(deleteRequest); + } - Listeners.register(new SQSListener(queueUrl, sqs), CreateSnapshotEvent.class); + @Test + public void testNotifyOnCreateSnapshotEvent() throws IOException { + Listeners.register(listener(), CreateSnapshotEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -63,7 +96,7 @@ public void testNotifyOnCreateSnapshotEvent() throws IOException { table.newAppend().appendFile(testDataFile).commit(); - messages = getMessages(queueUrl); + List messages = receiveAllMessages(queueUrl); Assert.assertEquals(1, messages.size()); ObjectMapper objectMapper = new ObjectMapper(); @@ -79,19 +112,11 @@ public void testNotifyOnCreateSnapshotEvent() throws IOException { "\"total-equality-deletes\":\"0\"}}"; Assert.assertEquals(expectedMessage, bodyNode.toString()); - - destroySqsQueue(queueUrl); } @Test public void testNotifyOnScanEvent() throws IOException { - String queueUrl = createSqsQueue("ScanEvent"); - - List messages = getMessages(queueUrl); - messages = getMessages(queueUrl); - Assert.assertEquals(0, messages.size()); - - Listeners.register(new SQSListener(queueUrl, sqs), ScanEvent.class); + Listeners.register(listener(), ScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -104,7 +129,7 @@ public void testNotifyOnScanEvent() throws IOException { Expression andExpression = Expressions.and(Expressions.equal("c1", "First"), Expressions.equal("c1", "Second")); table.newScan().filter(andExpression).planFiles(); - messages = getMessages(queueUrl); + List messages = receiveAllMessages(queueUrl); Assert.assertEquals(1, messages.size()); ObjectMapper objectMapper = new ObjectMapper(); @@ -123,19 +148,11 @@ public void testNotifyOnScanEvent() throws IOException { "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; Assert.assertEquals(expectedMessage, bodyNode.toString()); - - destroySqsQueue(queueUrl); } @Test public void testNotifyOnIncrementalScan() throws IOException { - String queueUrl = createSqsQueue("IncrementalScan"); - - List messages = getMessages(queueUrl); - messages = getMessages(queueUrl); - Assert.assertEquals(0, messages.size()); - - Listeners.register(new SQSListener(queueUrl, sqs), IncrementalScanEvent.class); + Listeners.register(listener(), IncrementalScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -152,7 +169,7 @@ public void testNotifyOnIncrementalScan() throws IOException { Iterables.get(snapshots, 1).snapshotId()) .planFiles(); - messages = getMessages(queueUrl); + List messages = receiveAllMessages(queueUrl); Assert.assertEquals(1, messages.size()); ObjectMapper objectMapper = new ObjectMapper(); @@ -167,25 +184,13 @@ public void testNotifyOnIncrementalScan() throws IOException { "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; Assert.assertEquals(expectedMessage, bodyNode.toString()); - - destroySqsQueue(queueUrl); } @Test - public void testNotifyOnAllEvents() throws IOException { - String queueUrl = createSqsQueue("AllEvents"); - - List messages = getMessages(queueUrl); - - messages = getMessages(queueUrl); - Assert.assertEquals(0, messages.size()); - - SQSListener createSnapshotListener = new SQSListener(queueUrl, sqs); - SQSListener scanListener = new SQSListener(queueUrl, sqs); - SQSListener incrementalScanListener = new SQSListener(queueUrl, sqs); - Listeners.register(createSnapshotListener, CreateSnapshotEvent.class); - Listeners.register(scanListener, ScanEvent.class); - Listeners.register(incrementalScanListener, IncrementalScanEvent.class); + public void testNotifyOnAllEvents() { + Listeners.register(listener(), CreateSnapshotEvent.class); + Listeners.register(listener(), ScanEvent.class); + Listeners.register(listener(), IncrementalScanEvent.class); String namespace = createNamespace(); String tableName = getRandomName(); @@ -204,53 +209,49 @@ public void testNotifyOnAllEvents() throws IOException { Iterables.get(snapshots, 1).snapshotId()) .planFiles(); - messages = getMessages(queueUrl); + List messages = receiveAllMessages(queueUrl); Assert.assertEquals(4, messages.size()); - ObjectMapper objectMapper = new ObjectMapper(); - List actualBodyNodesMessages = Lists.newArrayList(); - - for (int i = 0; i < 4; i++) { - actualBodyNodesMessages.add(objectMapper.readTree(messages.get(i).body()).toString()); - - } - - Set expectedBodyNodesMessages = Sets.newHashSet(); - expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + Set actualBodyNodesMessages = messages.stream() + .map(m -> { + try { + return JsonUtil.mapper().readTree(m.body()).toString(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }) + .collect(Collectors.toSet()); + + Set expectedBodyNodesMessages = Sets.newHashSet( + "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + "\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"); - expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}", + "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"2\"," + "\"total-files-size\":\"20\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\"," + - "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}"); - expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}", + "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + "\"from-snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + "\"to-snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"); - expectedBodyNodesMessages.add("{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + + "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}", + "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"1\"," + "\"total-files-size\":\"10\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\"," + - "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}"); + "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}" + ); - Assert.assertEquals(4, expectedBodyNodesMessages.size()); - for (String message : actualBodyNodesMessages) { - expectedBodyNodesMessages.remove(message); - } - - Assert.assertEquals(0, expectedBodyNodesMessages.size()); - - destroySqsQueue(queueUrl); + Assert.assertEquals(expectedBodyNodesMessages, actualBodyNodesMessages); } - public static List getMessages(String sqsUrl) { + public static List receiveAllMessages(String sqsUrl) { ReceiveMessageRequest receiveMessageRequest = ReceiveMessageRequest.builder() .queueUrl(sqsUrl) .visibilityTimeout(100) @@ -269,16 +270,7 @@ public static List getMessages(String sqsUrl) { return messages; } - public static String createSqsQueue(String queueName) { - CreateQueueRequest createRequest = CreateQueueRequest.builder().queueName(queueName).build(); - sqs.createQueue(createRequest); - GetQueueUrlRequest urlRequest = GetQueueUrlRequest.builder().queueName(queueName).build(); - return sqs.getQueueUrl(urlRequest).queueUrl(); + private Listener listener() { + return new SQSListener<>(queueUrl, sqs, RETRY, RETRY_INTERVAL_MS); } - - public static void destroySqsQueue(String queueUrl) { - DeleteQueueRequest deleteRequest = DeleteQueueRequest.builder().queueUrl(queueUrl).build(); - sqs.deleteQueue(deleteRequest); - } - } diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index c51571c7c70a..2585c754dabc 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -21,9 +21,11 @@ import org.apache.iceberg.events.Listener; import org.apache.iceberg.util.EventParser; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.sns.SnsClient; +import software.amazon.awssdk.services.sns.model.NotFoundException; import software.amazon.awssdk.services.sns.model.PublishRequest; import software.amazon.awssdk.services.sns.model.SnsException; @@ -31,12 +33,15 @@ public class SNSListener implements Listener { private static final Logger LOG = LoggerFactory.getLogger(SNSListener.class); private final String topicArn; - // private AwsClientFactory awsClientFactory; // to be used later private final SnsClient sns; + private final int retry; + private final long retryIntervalMs; - public SNSListener(String topicArn, SnsClient sns) { + public SNSListener(String topicArn, SnsClient sns, int retry, long retryIntervalMs) { this.sns = sns; this.topicArn = topicArn; + this.retry = retry; + this.retryIntervalMs = retryIntervalMs; } @Override @@ -47,7 +52,11 @@ public void notify(Object event) { .message(msg) .topicArn(topicArn) .build(); - sns.publish(request); + Tasks.foreach(request) + .exponentialBackoff(retryIntervalMs, retryIntervalMs, retryIntervalMs, 1 /* scale factor */) + .retry(retry) + .onlyRetryOn(NotFoundException.class) + .run(sns::publish); } catch (SnsException e) { LOG.error("Failed to send notification event to SNS topic", e); } catch (RuntimeException e) { diff --git a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java index 8ea1b9350013..71ead546c7e4 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java @@ -21,22 +21,27 @@ import org.apache.iceberg.events.Listener; import org.apache.iceberg.util.EventParser; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.QueueDoesNotExistException; import software.amazon.awssdk.services.sqs.model.SendMessageRequest; import software.amazon.awssdk.services.sqs.model.SqsException; public class SQSListener implements Listener { private static final Logger LOG = LoggerFactory.getLogger(SQSListener.class); - private final String queueURL; - // private AwsClientFactory awsClientFactory; // to be used later + private final String queueUrl; private final SqsClient sqs; + private final int retry; + private final long retryIntervalMs; - public SQSListener(String queueURL, SqsClient sqs) { + public SQSListener(String queueUrl, SqsClient sqs, int retry, long retryIntervalMs) { this.sqs = sqs; - this.queueURL = queueURL; + this.queueUrl = queueUrl; + this.retry = retry; + this.retryIntervalMs = retryIntervalMs; } @Override @@ -44,10 +49,14 @@ public void notify(Object event) { try { String msg = EventParser.toJson(event); SendMessageRequest request = SendMessageRequest.builder() - .queueUrl(queueURL) + .queueUrl(queueUrl) .messageBody(msg) .build(); - sqs.sendMessage(request); + Tasks.foreach(request) + .exponentialBackoff(retryIntervalMs, retryIntervalMs, retryIntervalMs, 1 /* scale factor */) + .retry(retry) + .onlyRetryOn(QueueDoesNotExistException.class) + .run(sqs::sendMessage); } catch (SqsException e) { LOG.error("Failed to send notification event to SQS", e); } catch (RuntimeException e) { diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index e9fe6c9901b5..1abf6a754829 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -39,7 +39,6 @@ public class ExpressionParser { private static final String LEFT_OPERAND = "left-operand"; private static final String RIGHT_OPERAND = "right-operand"; private static final String OPERAND = "operand"; - private static final String AND = "and"; private static final String OR = "or"; private static final String NOT = "not"; @@ -49,10 +48,8 @@ public class ExpressionParser { private static final String BOUNDED_LITERAL_PREDICATE = "bounded-literal-predicate"; private static final String BOUNDED_SET_PREDICATE = "bounded-set-predicate"; private static final String BOUNDED_UNARY_PREDICATE = "bounded-unary-predicate"; - private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; - private static final String ABOVE_MAX = "above-max"; private static final String BELOW_MIN = "below-min"; @@ -78,7 +75,7 @@ public static String toJson(Expression expression, boolean pretty) { return writer.toString(); } catch (IOException e) { - throw new UncheckedIOException(String.format("Failed to write json"), e); + throw new UncheckedIOException("Failed to write json", e); } } From c57789f51aa2c86ecb9b53689fd99e611f8d3970 Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Thu, 24 Feb 2022 21:02:45 -0800 Subject: [PATCH 06/19] Catalog Listener (#7) (#9) Co-authored-by: Kunal Bhargava --- .../org/apache/iceberg/events/Listener.java | 5 + .../org/apache/iceberg/aws/AwsProperties.java | 12 ++ .../iceberg/aws/dynamodb/DynamoDbCatalog.java | 1 + .../apache/iceberg/aws/glue/GlueCatalog.java | 1 + .../apache/iceberg/aws/sns/SNSListener.java | 38 ++++++- .../apache/iceberg/aws/sqs/SQSListener.java | 38 ++++++- .../apache/iceberg/BaseMetastoreCatalog.java | 38 +++++++ .../org/apache/iceberg/CatalogProperties.java | 1 + .../java/org/apache/iceberg/CatalogUtil.java | 46 ++++++++ .../apache/iceberg/hadoop/HadoopCatalog.java | 1 + .../org/apache/iceberg/jdbc/JdbcCatalog.java | 1 + .../org/apache/iceberg/TestCatalogUtil.java | 103 +++++++++++++++++ .../org/apache/iceberg/hive/HiveCatalog.java | 1 + .../apache/iceberg/nessie/NessieCatalog.java | 1 + .../spark/TestSparkCatalogEndToEnd.java | 107 ++++++++++++++++++ 15 files changed, 386 insertions(+), 8 deletions(-) create mode 100644 spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java diff --git a/api/src/main/java/org/apache/iceberg/events/Listener.java b/api/src/main/java/org/apache/iceberg/events/Listener.java index 843a1de91402..0011cd414e12 100644 --- a/api/src/main/java/org/apache/iceberg/events/Listener.java +++ b/api/src/main/java/org/apache/iceberg/events/Listener.java @@ -19,9 +19,14 @@ package org.apache.iceberg.events; +import java.util.Map; + /** * A listener interface that can receive notifications. */ public interface Listener { void notify(E event); + + default void initialize(String listenerName, Map properties) { + } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index fef845a0a005..76cd173dc2b4 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -221,6 +221,18 @@ public class AwsProperties implements Serializable { @Deprecated public static final boolean CLIENT_ENABLE_ETAG_CHECK_DEFAULT = false; + public static final String SQS_QUEUE_URL = "sqs.queue-url"; + public static final String SQS_RETRY = "sqs.retry"; + public static final String SQS_RETRY_INTERVAL_MS = "sqs.retryIntervalMs"; + public static final Integer SQS_RETRY_DEFAULT = 3; + public static final Integer SQS_RETRY_INTERVAL_MS_DEFAULT = 1000; + + public static final String SNS_TOPIC_ARN = "sns.topic-arn"; + public static final String SNS_RETRY = "sns.retry"; + public static final String SNS_RETRY_INTERVAL_MS = "sns.retryIntervalMs"; + public static final Integer SNS_RETRY_DEFAULT = 3; + public static final Integer SNS_RETRY_INTERVAL_MS_DEFAULT = 1000; + private String s3FileIoSseType; private String s3FileIoSseKey; private String s3FileIoSseMd5; diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index b9443e4a8164..01e6a75009a7 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -117,6 +117,7 @@ public DynamoDbCatalog() { @Override public void initialize(String name, Map properties) { + super.initialize(name, properties); initialize( name, properties.get(CatalogProperties.WAREHOUSE_LOCATION), diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index 4dafcb9b4325..e9cc0f346014 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -104,6 +104,7 @@ public GlueCatalog() { @Override public void initialize(String name, Map properties) { + super.initialize(name, properties); initialize( name, properties.get(CatalogProperties.WAREHOUSE_LOCATION), diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index 2585c754dabc..2a1c9ad10b49 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -19,8 +19,13 @@ package org.apache.iceberg.aws.sns; +import java.util.Map; +import org.apache.iceberg.aws.AwsClientFactories; +import org.apache.iceberg.aws.AwsClientFactory; +import org.apache.iceberg.aws.AwsProperties; import org.apache.iceberg.events.Listener; import org.apache.iceberg.util.EventParser; +import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,10 +37,13 @@ public class SNSListener implements Listener { private static final Logger LOG = LoggerFactory.getLogger(SNSListener.class); - private final String topicArn; - private final SnsClient sns; - private final int retry; - private final long retryIntervalMs; + private String topicArn; + private SnsClient sns; + private int retry; + private long retryIntervalMs; + + public SNSListener() { + } public SNSListener(String topicArn, SnsClient sns, int retry, long retryIntervalMs) { this.sns = sns; @@ -63,5 +71,27 @@ public void notify(Object event) { LOG.error("Failed to notify subscriber", e); } } + + @Override + public void initialize(String listenerName, Map properties) { + AwsClientFactory factory = AwsClientFactories.from(properties); + this.sns = factory.sns(); + + if (listenerName == null) { + throw new NullPointerException("Listener Name cannot be null"); + } + + if (properties.get(AwsProperties.SNS_TOPIC_ARN) == null) { + throw new NullPointerException("SNS queue url cannot be null"); + } + + this.topicArn = properties.get(AwsProperties.SNS_TOPIC_ARN); + + this.retry = PropertyUtil.propertyAsInt( + properties, AwsProperties.SNS_RETRY, AwsProperties.SNS_RETRY_DEFAULT); + + this.retryIntervalMs = PropertyUtil.propertyAsInt( + properties, AwsProperties.SNS_RETRY_INTERVAL_MS, AwsProperties.SNS_RETRY_INTERVAL_MS_DEFAULT); + } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java index 71ead546c7e4..09304f625fdd 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java @@ -19,8 +19,13 @@ package org.apache.iceberg.aws.sqs; +import java.util.Map; +import org.apache.iceberg.aws.AwsClientFactories; +import org.apache.iceberg.aws.AwsClientFactory; +import org.apache.iceberg.aws.AwsProperties; import org.apache.iceberg.events.Listener; import org.apache.iceberg.util.EventParser; +import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,10 +37,13 @@ public class SQSListener implements Listener { private static final Logger LOG = LoggerFactory.getLogger(SQSListener.class); - private final String queueUrl; - private final SqsClient sqs; - private final int retry; - private final long retryIntervalMs; + private String queueUrl; + private SqsClient sqs; + private int retry; + private long retryIntervalMs; + + public SQSListener() { + } public SQSListener(String queueUrl, SqsClient sqs, int retry, long retryIntervalMs) { this.sqs = sqs; @@ -63,5 +71,27 @@ public void notify(Object event) { LOG.error("Failed to add to queue", e); } } + + @Override + public void initialize(String listenerName, Map properties) { + AwsClientFactory factory = AwsClientFactories.from(properties); + this.sqs = factory.sqs(); + + if (listenerName == null) { + throw new NullPointerException("Listener Name cannot be null"); + } + + if (properties.get(AwsProperties.SQS_QUEUE_URL) == null) { + throw new NullPointerException("SNS queue url cannot be null"); + } + + this.queueUrl = properties.get(AwsProperties.SQS_QUEUE_URL); + + this.retry = PropertyUtil.propertyAsInt( + properties, AwsProperties.SQS_RETRY, AwsProperties.SQS_RETRY_DEFAULT); + + this.retryIntervalMs = PropertyUtil.propertyAsInt( + properties, AwsProperties.SQS_RETRY_INTERVAL_MS, AwsProperties.SQS_RETRY_INTERVAL_MS_DEFAULT); + } } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 9cea03d2f1ae..1ee1263f7415 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -20,19 +20,26 @@ package org.apache.iceberg; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.events.Listener; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; 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.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public abstract class BaseMetastoreCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(BaseMetastoreCatalog.class); + private static final Pattern LISTENER_REGEX = Pattern.compile("^listeners[.](?[^[.]]+)[.](?.+)$"); + private static final String LISTENER_PROPERTY_REGEX_GROUP_NAME = "name"; + private static final String LISTENER_PROPERTY_REGEX_GROUP_CONFIG = "config"; @Override public Table loadTable(TableIdentifier identifier) { @@ -68,6 +75,37 @@ public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { return new BaseMetastoreCatalogTableBuilder(identifier, schema); } + @Override + public void initialize(String name, Map properties) { + Map> listenerProperties = createListenerProperties(properties); + + for (String listenerName : listenerProperties.keySet()) { + if (listenerProperties.get(listenerName).get("impl") != null) { + Listener listener = CatalogUtil.loadListener( + listenerProperties.get(listenerName).get("impl"), + listenerName, + listenerProperties.get(listenerName)); + } + } + } + + public static Map> createListenerProperties(Map properties) { + Map> listenerProperties = Maps.newHashMap(); + + for (String key : properties.keySet()) { + Matcher match = LISTENER_REGEX.matcher(key); + if (match.matches()) { + if (!listenerProperties.containsKey(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME))) { + listenerProperties.put(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME), Maps.newHashMap()); + } + listenerProperties.get(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME)) + .put(match.group(LISTENER_PROPERTY_REGEX_GROUP_CONFIG), properties.get(key)); + } + } + + return listenerProperties; + } + private Table loadMetadataTable(TableIdentifier identifier) { String tableName = identifier.name(); MetadataTableType type = MetadataTableType.from(tableName); diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index d5daedee812b..d16ae1721bcf 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -81,4 +81,5 @@ private CatalogProperties() { public static final String APP_ID = "app-id"; public static final String USER = "user"; + public static final String LISTENERS = "listeners"; } diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 07b69c17fa82..43513268f95f 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -20,6 +20,7 @@ package org.apache.iceberg; import java.io.IOException; +import java.util.Arrays; import java.util.Locale; import java.util.Map; import java.util.Set; @@ -27,6 +28,11 @@ import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.events.IncrementalScanEvent; +import org.apache.iceberg.events.Listener; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; @@ -275,6 +281,46 @@ public static FileIO loadFileIO( return fileIO; } + public static Listener loadListener(String listenerClass, String listenerName, Map properties) { + DynConstructors.Ctor ctor; + try { + ctor = DynConstructors.builder(Listener.class).impl(listenerClass).buildChecked(); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException(String.format( + "Cannot initialize Listener, missing no-arg constructor: %s", listenerClass), e); + } + + Listener listener; + try { + listener = ctor.newInstance(); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format("Cannot initialize Listener, %s does not implement Listener.", listenerClass), e); + } + + listener.initialize(listenerName, properties); + String registerEvents = properties.get("event-types"); + + if (registerEvents == null) { + Listeners.register(listener, CreateSnapshotEvent.class); + Listeners.register(listener, ScanEvent.class); + Listeners.register(listener, IncrementalScanEvent.class); + } else { + Set events = Sets.newHashSet(Arrays.asList(registerEvents.split(", "))); + if (events.contains("scan")) { + Listeners.register(listener, ScanEvent.class); + } + if (events.contains("create-snapshot")) { + Listeners.register(listener, CreateSnapshotEvent.class); + } + if (events.contains("incremental-scan")) { + Listeners.register(listener, IncrementalScanEvent.class); + } + } + + return listener; + } + /** * Dynamically detects whether an object is a Hadoop Configurable and calls setConf. * @param maybeConfigurable an object that may be Configurable diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java index 78224e27bb0a..0adc65155653 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java @@ -98,6 +98,7 @@ public HadoopCatalog() { @Override public void initialize(String name, Map properties) { + super.initialize(name, properties); String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); Preconditions.checkArgument(inputWarehouseLocation != null && !inputWarehouseLocation.equals(""), "Cannot instantiate hadoop catalog. No location provided for warehouse (Set warehouse config)"); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 549c0cf7e34a..037bad3597d3 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -78,6 +78,7 @@ public JdbcCatalog() { @Override public void initialize(String name, Map properties) { + super.initialize(name, properties); String uri = properties.get(CatalogProperties.URI); Preconditions.checkNotNull(uri, "JDBC connection URI is required"); diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 2295b771dd28..479e51c0165b 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -21,11 +21,15 @@ import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.events.Listener; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; @@ -170,6 +174,65 @@ public void buildCustomCatalog_withTypeSet() { "both type and catalog-impl are set", () -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)); } + @Test + public void testSingleRegEx() { + Pattern pattern = Pattern.compile("^listeners[.](?.+)[.]impl$"); + Matcher matchTrue = pattern.matcher("listeners.prod.impl"); + Matcher matchFalse = pattern.matcher("listeners.prod.iampl"); + Assert.assertTrue(matchTrue.matches()); + Assert.assertFalse(matchFalse.matches()); + Assert.assertEquals("prod", matchTrue.group("name")); + } + + @Test + public void testLoadListener() { + Map properties = Maps.newHashMap(); + String listenerName = "ListenerName"; + properties.put("impl", TestListener.class.getName()); + properties.put("test.client", "Client-Info"); + properties.put("test.info", "Information"); + + Listener listener = CatalogUtil.loadListener(TestListener.class.getName(), listenerName, properties); + Assertions.assertThat(listener).isInstanceOf(TestListener.class); + Assert.assertEquals("Client-Info", ((TestListener) listener).client); + Assert.assertEquals("Information", ((TestListener) listener).info); + } + + @Test + public void testListenerProperties() { + Map properties = Maps.newHashMap(); + properties.put("listeners.listenerOne.impl", TestListener.class.getName()); + properties.put("listeners.listenerOne.test.info", "Information"); + properties.put("listeners.listenerOne.test.client", "Client-Info"); + properties.put("listeners.listenerTwo.impl", TestListener.class.getName()); + properties.put("listeners.listenerTwo.test.info", "Information"); + properties.put("listeners.listenerTwo.test.client", "Client-Info"); + Map> listenerProperties = BaseMetastoreCatalog.createListenerProperties(properties); + Assert.assertEquals(listenerProperties.get("listenerOne").get("test.client"), "Client-Info"); + Assert.assertEquals(listenerProperties.get("listenerTwo").get("test.info"), "Information"); + } + + @Test + public void loadBadListenerClass() { + Map properties = Maps.newHashMap(); + properties.put("key", "val"); + String name = "custom"; + String impl = "ListenerDoesNotExist"; + AssertHelpers.assertThrows("Listener must exist", + IllegalArgumentException.class, + "Cannot initialize Listener", + () -> CatalogUtil.loadListener(impl, name, properties)); + } + + @Test + public void loadBadListenerConstructor() { + String name = "custom"; + AssertHelpers.assertThrows("cannot find constructor", + IllegalArgumentException.class, + "missing no-arg constructor", + () -> CatalogUtil.loadListener(TestListenerBadConstructor.class.getName(), name, Maps.newHashMap())); + } + public static class TestCatalog extends BaseMetastoreCatalog { private String catalogName; @@ -180,6 +243,7 @@ public TestCatalog() { @Override public void initialize(String name, Map properties) { + super.initialize(name, properties); this.catalogName = name; this.flinkOptions = properties; } @@ -402,4 +466,43 @@ public static class TestFileIONotImpl { public TestFileIONotImpl() { } } + + public static class TestListener implements Listener { + private String client; + private String info; + private String name; + public static final AtomicInteger NOTIFY_TIMES = new AtomicInteger(); + + public TestListener() { + } + + @Override + public void notify(Object event) { + NOTIFY_TIMES.incrementAndGet(); + } + + @Override + public void initialize(String listenerName, Map properties) { + this.name = listenerName; + this.info = properties.get("test.info"); + this.client = properties.get("test.client"); + } + } + + public static class TestListenerBadConstructor implements Listener { + private String arg; + + public TestListenerBadConstructor(String arg) { + this.arg = arg; + } + + @Override + public void notify(Object event) { + } + + @Override + public void initialize(String listenerName, Map properties) { + this.arg = listenerName; + } + } } 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 0d1ba8e34bb1..f71b3908bc4b 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 @@ -78,6 +78,7 @@ public HiveCatalog() { @Override public void initialize(String inputName, Map properties) { + super.initialize(name, properties); this.name = inputName; if (conf == null) { LOG.warn("No Hadoop Configuration was set, using the default environment Configuration"); diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java index b558c276daaa..8d379423f6c8 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java @@ -93,6 +93,7 @@ public NessieCatalog() { @Override public void initialize(String inputName, Map options) { + super.initialize(name, options); this.catalogOptions = ImmutableMap.copyOf(options); String fileIOImpl = options.get(CatalogProperties.FILE_IO_IMPL); this.fileIO = fileIOImpl == null ? new HadoopFileIO(config) : CatalogUtil.loadFileIO(fileIOImpl, options, config); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java new file mode 100644 index 000000000000..c1394dbe92ea --- /dev/null +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.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.spark; + +import java.util.Map; +import org.apache.iceberg.TestCatalogUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + + +@RunWith(Parameterized.class) +public class TestSparkCatalogEndToEnd extends SparkTestBaseWithCatalog { + public static Integer notifyTimes = 0; + + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] {{ + TestSparkCatalogConfig.TEST.catalogName(), + TestSparkCatalogConfig.TEST.implementation(), + TestSparkCatalogConfig.TEST.properties() + } + }; + } + + public TestSparkCatalogEndToEnd(String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Before + public void createTable() { + TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } + + @After + public void removeTable() { + TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testEndToEnd() throws NoSuchTableException { + TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); + + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + Assert.assertEquals(1, TestCatalogUtil.TestListener.NOTIFY_TIMES.get()); + + sql("SELECT * FROM %s", tableName); + Assert.assertEquals(2, TestCatalogUtil.TestListener.NOTIFY_TIMES.get()); + TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); + } + + public enum TestSparkCatalogConfig { + TEST("hive", SparkCatalog.class.getName(), ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "listeners.listenerOne.impl", TestCatalogUtil.TestListener.class.getName(), + "listeners.listenerOne.test.info", "Information", + "listeners.listenerOne.test.client", "Client-Info" + )); + + private final String catalogName; + private final String implementation; + private final Map properties; + + TestSparkCatalogConfig(String catalogName, String implementation, Map properties) { + this.catalogName = catalogName; + this.implementation = implementation; + this.properties = properties; + } + + public String catalogName() { + return catalogName; + } + + public String implementation() { + return implementation; + } + + public Map properties() { + return properties; + } + } +} From f1cbe930735b850f8871af75ff78ae263fb9de03 Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Thu, 24 Feb 2022 23:25:05 -0800 Subject: [PATCH 07/19] fix rebase --- .../iceberg/aws/AwsClientFactories.java | 4 +- .../apache/iceberg/BaseMetastoreCatalog.java | 2 +- .../spark/TestSparkCatalogEndToEnd.java | 107 ------------------ 3 files changed, 3 insertions(+), 110 deletions(-) delete mode 100644 spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 400ef0a2bd14..85b83e2cbedf 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -112,12 +112,12 @@ public DynamoDbClient dynamo() { @Override public SnsClient sns() { - return SnsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build(); + return SnsClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); } @Override public SqsClient sqs() { - return SqsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build(); + return SqsClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 1ee1263f7415..7ee32c2d8094 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -37,7 +37,7 @@ public abstract class BaseMetastoreCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(BaseMetastoreCatalog.class); - private static final Pattern LISTENER_REGEX = Pattern.compile("^listeners[.](?[^[.]]+)[.](?.+)$"); + private static final Pattern LISTENER_REGEX = Pattern.compile("^listeners[.](?[^\\.]+)[.](?.+)$"); private static final String LISTENER_PROPERTY_REGEX_GROUP_NAME = "name"; private static final String LISTENER_PROPERTY_REGEX_GROUP_CONFIG = "config"; diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java deleted file mode 100644 index c1394dbe92ea..000000000000 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogEndToEnd.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg.spark; - -import java.util.Map; -import org.apache.iceberg.TestCatalogUtil; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - - -@RunWith(Parameterized.class) -public class TestSparkCatalogEndToEnd extends SparkTestBaseWithCatalog { - public static Integer notifyTimes = 0; - - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") - public static Object[][] parameters() { - return new Object[][] {{ - TestSparkCatalogConfig.TEST.catalogName(), - TestSparkCatalogConfig.TEST.implementation(), - TestSparkCatalogConfig.TEST.properties() - } - }; - } - - public TestSparkCatalogEndToEnd(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before - public void createTable() { - TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); - sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - } - - @After - public void removeTable() { - TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); - sql("DROP TABLE IF EXISTS %s", tableName); - } - - @Test - public void testEndToEnd() throws NoSuchTableException { - TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); - - sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); - Assert.assertEquals(1, TestCatalogUtil.TestListener.NOTIFY_TIMES.get()); - - sql("SELECT * FROM %s", tableName); - Assert.assertEquals(2, TestCatalogUtil.TestListener.NOTIFY_TIMES.get()); - TestCatalogUtil.TestListener.NOTIFY_TIMES.set(0); - } - - public enum TestSparkCatalogConfig { - TEST("hive", SparkCatalog.class.getName(), ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "listeners.listenerOne.impl", TestCatalogUtil.TestListener.class.getName(), - "listeners.listenerOne.test.info", "Information", - "listeners.listenerOne.test.client", "Client-Info" - )); - - private final String catalogName; - private final String implementation; - private final Map properties; - - TestSparkCatalogConfig(String catalogName, String implementation, Map properties) { - this.catalogName = catalogName; - this.implementation = implementation; - this.properties = properties; - } - - public String catalogName() { - return catalogName; - } - - public String implementation() { - return implementation; - } - - public Map properties() { - return properties; - } - } -} From ed16ef5869bc8f52d7386ebfc84fcf05fad4ae36 Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Fri, 25 Feb 2022 01:15:32 -0800 Subject: [PATCH 08/19] refactor listener and catalog initialization logics --- .../apache/iceberg/aws/sns/SNSListener.java | 4 +- .../apache/iceberg/aws/sqs/SQSListener.java | 2 +- .../apache/iceberg/BaseMetastoreCatalog.java | 31 +------ .../org/apache/iceberg/CatalogProperties.java | 60 ++++++++++++- .../java/org/apache/iceberg/CatalogUtil.java | 86 +++++++++++++------ .../org/apache/iceberg/TestCatalogUtil.java | 39 +-------- 6 files changed, 127 insertions(+), 95 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java index 2a1c9ad10b49..31e1113288e5 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java @@ -42,7 +42,7 @@ public class SNSListener implements Listener { private int retry; private long retryIntervalMs; - public SNSListener() { + public SNSListener(Class clazz) { } public SNSListener(String topicArn, SnsClient sns, int retry, long retryIntervalMs) { @@ -53,7 +53,7 @@ public SNSListener(String topicArn, SnsClient sns, int retry, long retryInterval } @Override - public void notify(Object event) { + public void notify(T event) { try { String msg = EventParser.toJson(event); PublishRequest request = PublishRequest.builder() diff --git a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java index 09304f625fdd..9eb74086df8c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java +++ b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java @@ -42,7 +42,7 @@ public class SQSListener implements Listener { private int retry; private long retryIntervalMs; - public SQSListener() { + public SQSListener(Class clazz) { } public SQSListener(String queueUrl, SqsClient sqs, int retry, long retryIntervalMs) { diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 7ee32c2d8094..060cb2d559c8 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -37,9 +37,6 @@ public abstract class BaseMetastoreCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(BaseMetastoreCatalog.class); - private static final Pattern LISTENER_REGEX = Pattern.compile("^listeners[.](?[^\\.]+)[.](?.+)$"); - private static final String LISTENER_PROPERTY_REGEX_GROUP_NAME = "name"; - private static final String LISTENER_PROPERTY_REGEX_GROUP_CONFIG = "config"; @Override public Table loadTable(TableIdentifier identifier) { @@ -77,33 +74,7 @@ public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { @Override public void initialize(String name, Map properties) { - Map> listenerProperties = createListenerProperties(properties); - - for (String listenerName : listenerProperties.keySet()) { - if (listenerProperties.get(listenerName).get("impl") != null) { - Listener listener = CatalogUtil.loadListener( - listenerProperties.get(listenerName).get("impl"), - listenerName, - listenerProperties.get(listenerName)); - } - } - } - - public static Map> createListenerProperties(Map properties) { - Map> listenerProperties = Maps.newHashMap(); - - for (String key : properties.keySet()) { - Matcher match = LISTENER_REGEX.matcher(key); - if (match.matches()) { - if (!listenerProperties.containsKey(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME))) { - listenerProperties.put(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME), Maps.newHashMap()); - } - listenerProperties.get(match.group(LISTENER_PROPERTY_REGEX_GROUP_NAME)) - .put(match.group(LISTENER_PROPERTY_REGEX_GROUP_CONFIG), properties.get(key)); - } - } - - return listenerProperties; + CatalogUtil.initializeListeners(properties); } private Table loadMetadataTable(TableIdentifier identifier) { diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index d16ae1721bcf..8a4c3457f282 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -19,7 +19,16 @@ package org.apache.iceberg; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.events.IncrementalScanEvent; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.Pair; public class CatalogProperties { @@ -81,5 +90,54 @@ private CatalogProperties() { public static final String APP_ID = "app-id"; public static final String USER = "user"; - public static final String LISTENERS = "listeners"; + /** + * Listeners are registered using catalog properties following the pattern of + * listeners.(listener-name).(listener-property)=(property-value) + *

+ * A listener name cannot contain dot (.) character + * The specified listener is registered when a catalog is initialized + *

+ * For example, there is the set of catalog properties registering an AWS SQS listener of name prod: + *

    + *
  • listener.prod.impl=org.apache.iceberg.aws.sns.SnsListener + *
  • listener.prod.event-types=scan,incremental-scan + *
  • listener.prod.sns.topic-arn=arn:aws:sns:us-east-2:123456789012:MyTopic + *
+ */ + public static String listenerCatalogProperty(String listenerName, String listenerProperty) { + return "listener." + listenerName + "." + listenerProperty; + } + + /** + * Parse the listener name and listener property from a catalog property string + * @param listenerCatalogProperty listener catalog property + * @return a pair of the listener name and listener property + */ + public static Optional> parseListenerCatalogProperty(String listenerCatalogProperty) { + Matcher matcher = Pattern.compile("^listener[.](?[^\\.]+)[.](?.+)$") + .matcher(listenerCatalogProperty); + if (matcher.matches()) { + return Optional.of(Pair.of(matcher.group("name"), matcher.group("property"))); + } + + return Optional.empty(); + } + + /** + * Listener property describing the implementation Java class name of the listener for dynamic loading + */ + public static final String LISTENER_PROPERTY_IMPL = "impl"; + + /** + * Listener property describing the event types that a listener subscribes to. + * The value is a comma delimited list of event types (Java class name), + * e.g. org.apache.iceberg.events.ScanEvent,org.apache.iceberg.events.IncrementalScanEvent. + * If not specified, the listener subscribes to events listed in {@link #LISTENER_EVENT_TYPES_DEFAULT} + */ + public static final String LISTENER_PROPERTY_EVENT_TYPES = "event-types"; + public static final Set> LISTENER_EVENT_TYPES_DEFAULT = ImmutableSet.of( + ScanEvent.class, + IncrementalScanEvent.class, + CreateSnapshotEvent.class + ); } diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 43513268f95f..d557179a6d21 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -21,9 +21,14 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; @@ -37,11 +42,15 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -281,43 +290,70 @@ public static FileIO loadFileIO( return fileIO; } - public static Listener loadListener(String listenerClass, String listenerName, Map properties) { - DynConstructors.Ctor ctor; + public static void initializeListeners(Map properties) { + Map> propertiesSummary = Maps.newHashMap(); + Map commonProperties = Maps.newHashMap(); + for (String key : properties.keySet()) { + Optional> listenerInfo = CatalogProperties.parseListenerCatalogProperty(key); + if (listenerInfo.isPresent()) { + String name = listenerInfo.get().first(); + String property = listenerInfo.get().second(); + propertiesSummary.computeIfAbsent(name, k -> Maps.newHashMap()); + propertiesSummary.get(name).put(property, properties.get(key)); + } else { + commonProperties.put(key, properties.get(key)); + } + } + + // inherit all common properties during listener initialization + propertiesSummary.forEach((k, v) -> v.putAll(commonProperties)); + + for (String listenerName : propertiesSummary.keySet()) { + Map listenerProperties = propertiesSummary.get(listenerName); + String listenerImpl = listenerProperties.get(CatalogProperties.LISTENER_PROPERTY_IMPL); + ValidationException.check(listenerImpl != null, + "Cannot initialize listener %s, missing %s property", + listenerName, CatalogProperties.LISTENER_PROPERTY_IMPL); + + String eventTypesString = listenerProperties.get(CatalogProperties.LISTENER_PROPERTY_EVENT_TYPES); + Set> eventTypes = eventTypesString != null ? Arrays.stream(eventTypesString.split(",")) + .map(s -> { + try { + return Class.forName(s); + } catch (ClassNotFoundException e) { + throw new ValidationException(e, "Cannot find listener event type class %s", s); + } + }) + .collect(Collectors.toSet()) : CatalogProperties.LISTENER_EVENT_TYPES_DEFAULT; + + eventTypes.forEach(t -> CatalogUtil.loadAndRegisterListener(listenerImpl, listenerName, t, listenerProperties)); + } + } + + @VisibleForTesting + static Listener loadAndRegisterListener( + String listenerClass, + String listenerName, + Class eventType, + Map properties) { + DynConstructors.Ctor> ctor; try { - ctor = DynConstructors.builder(Listener.class).impl(listenerClass).buildChecked(); + ctor = DynConstructors.builder(Listener.class).impl(listenerClass, eventType).buildChecked(); } catch (NoSuchMethodException e) { throw new IllegalArgumentException(String.format( "Cannot initialize Listener, missing no-arg constructor: %s", listenerClass), e); } - Listener listener; + Listener listener; try { listener = ctor.newInstance(); } catch (ClassCastException e) { - throw new IllegalArgumentException( - String.format("Cannot initialize Listener, %s does not implement Listener.", listenerClass), e); + throw new IllegalArgumentException(String.format( + "Cannot initialize Listener, %s does not implement org.apache.iceberg.events.Listener", listenerClass), e); } listener.initialize(listenerName, properties); - String registerEvents = properties.get("event-types"); - - if (registerEvents == null) { - Listeners.register(listener, CreateSnapshotEvent.class); - Listeners.register(listener, ScanEvent.class); - Listeners.register(listener, IncrementalScanEvent.class); - } else { - Set events = Sets.newHashSet(Arrays.asList(registerEvents.split(", "))); - if (events.contains("scan")) { - Listeners.register(listener, ScanEvent.class); - } - if (events.contains("create-snapshot")) { - Listeners.register(listener, CreateSnapshotEvent.class); - } - if (events.contains("incremental-scan")) { - Listeners.register(listener, IncrementalScanEvent.class); - } - } - + Listeners.register(listener, eventType); return listener; } diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 479e51c0165b..3d8d431b0dc7 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -30,6 +30,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.events.Listener; +import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; @@ -192,47 +193,13 @@ public void testLoadListener() { properties.put("test.client", "Client-Info"); properties.put("test.info", "Information"); - Listener listener = CatalogUtil.loadListener(TestListener.class.getName(), listenerName, properties); + Listener listener = CatalogUtil.loadAndRegisterListener( + TestListener.class.getName(), listenerName, ScanEvent.class, properties); Assertions.assertThat(listener).isInstanceOf(TestListener.class); Assert.assertEquals("Client-Info", ((TestListener) listener).client); Assert.assertEquals("Information", ((TestListener) listener).info); } - @Test - public void testListenerProperties() { - Map properties = Maps.newHashMap(); - properties.put("listeners.listenerOne.impl", TestListener.class.getName()); - properties.put("listeners.listenerOne.test.info", "Information"); - properties.put("listeners.listenerOne.test.client", "Client-Info"); - properties.put("listeners.listenerTwo.impl", TestListener.class.getName()); - properties.put("listeners.listenerTwo.test.info", "Information"); - properties.put("listeners.listenerTwo.test.client", "Client-Info"); - Map> listenerProperties = BaseMetastoreCatalog.createListenerProperties(properties); - Assert.assertEquals(listenerProperties.get("listenerOne").get("test.client"), "Client-Info"); - Assert.assertEquals(listenerProperties.get("listenerTwo").get("test.info"), "Information"); - } - - @Test - public void loadBadListenerClass() { - Map properties = Maps.newHashMap(); - properties.put("key", "val"); - String name = "custom"; - String impl = "ListenerDoesNotExist"; - AssertHelpers.assertThrows("Listener must exist", - IllegalArgumentException.class, - "Cannot initialize Listener", - () -> CatalogUtil.loadListener(impl, name, properties)); - } - - @Test - public void loadBadListenerConstructor() { - String name = "custom"; - AssertHelpers.assertThrows("cannot find constructor", - IllegalArgumentException.class, - "missing no-arg constructor", - () -> CatalogUtil.loadListener(TestListenerBadConstructor.class.getName(), name, Maps.newHashMap())); - } - public static class TestCatalog extends BaseMetastoreCatalog { private String catalogName; From da57f741217b27675f98dd173a92d6a9473a257f Mon Sep 17 00:00:00 2001 From: Kunal Bhargava Date: Mon, 7 Mar 2022 14:03:38 -0600 Subject: [PATCH 09/19] Added Event Types in Json (#10) * Added Event Types in Json * Changed Reflection Initialization * Changed Event Type Input in Json * Fixed Listener Initialization --- .../org/apache/iceberg/BaseMetastoreCatalog.java | 4 ---- .../main/java/org/apache/iceberg/CatalogUtil.java | 12 +++--------- .../java/org/apache/iceberg/util/EventParser.java | 8 ++++++++ .../java/org/apache/iceberg/TestCatalogUtil.java | 2 +- 4 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 060cb2d559c8..0805cc3ff6ea 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -20,18 +20,14 @@ package org.apache.iceberg; import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.events.Listener; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; 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.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index d557179a6d21..562fc92f58f9 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -21,23 +21,17 @@ import java.io.IOException; import java.util.Arrays; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; -import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.events.IncrementalScanEvent; import org.apache.iceberg.events.Listener; import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; @@ -45,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -331,6 +324,7 @@ public static void initializeListeners(Map properties) { } @VisibleForTesting + @SuppressWarnings("GetClassOnClass") static Listener loadAndRegisterListener( String listenerClass, String listenerName, @@ -338,7 +332,7 @@ static Listener loadAndRegisterListener( Map properties) { DynConstructors.Ctor> ctor; try { - ctor = DynConstructors.builder(Listener.class).impl(listenerClass, eventType).buildChecked(); + ctor = DynConstructors.builder(Listener.class).impl(listenerClass, eventType.getClass()).buildChecked(); } catch (NoSuchMethodException e) { throw new IllegalArgumentException(String.format( "Cannot initialize Listener, missing no-arg constructor: %s", listenerClass), e); @@ -346,7 +340,7 @@ static Listener loadAndRegisterListener( Listener listener; try { - listener = ctor.newInstance(); + listener = ctor.newInstance(eventType); } catch (ClassCastException e) { throw new IllegalArgumentException(String.format( "Cannot initialize Listener, %s does not implement org.apache.iceberg.events.Listener", listenerClass), e); diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java index ecf018a1a8e8..2005998d30d6 100644 --- a/core/src/main/java/org/apache/iceberg/util/EventParser.java +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -31,6 +31,8 @@ import org.apache.iceberg.expressions.ExpressionParser; public class EventParser { + private static final String EVENT_TYPE = "event-type"; + private static final String TABLE_NAME = "table-name"; private static final String SNAPSHOT_ID = "snapshot-id"; private static final String PROJECTION = "projection"; @@ -65,6 +67,8 @@ public static String toJson(Object event) { public static void toJson(ScanEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); + generator.writeFieldName(EVENT_TYPE); + generator.writeString(event.getClass().getName()); generator.writeFieldName(TABLE_NAME); generator.writeString(event.tableName()); generator.writeFieldName(SNAPSHOT_ID); @@ -78,6 +82,8 @@ public static void toJson(ScanEvent event, JsonGenerator generator) throws IOExc public static void toJson(CreateSnapshotEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); + generator.writeFieldName(EVENT_TYPE); + generator.writeString(event.getClass().getName()); generator.writeFieldName(TABLE_NAME); generator.writeString(event.tableName()); generator.writeFieldName(OPERATION); @@ -97,6 +103,8 @@ public static void toJson(CreateSnapshotEvent event, JsonGenerator generator) th public static void toJson(IncrementalScanEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); + generator.writeFieldName(EVENT_TYPE); + generator.writeString(event.getClass().getName()); generator.writeFieldName(TABLE_NAME); generator.writeString(event.tableName()); generator.writeFieldName(FROM_SNAPSHOT_ID); diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 3d8d431b0dc7..fbecdcd1833c 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -440,7 +440,7 @@ public static class TestListener implements Listener { private String name; public static final AtomicInteger NOTIFY_TIMES = new AtomicInteger(); - public TestListener() { + public TestListener(Class clazz) { } @Override From 7aec26f9ce5781a0c50394d0b3f3ad3553eb3292 Mon Sep 17 00:00:00 2001 From: Kunal Bhargava Date: Wed, 9 Mar 2022 23:21:43 -0600 Subject: [PATCH 10/19] EventParser and Expression fromJson Serialization (#11) * Added Event Types in Json * Changed Reflection Initialization * Changed Event Type Input in Json * Fixed Listener Initialization * Added Partial fromJson in ExpressionParser Still missing Literal serialization. * Added Partial fromJson in ExpressionParser Still missing Literal serialization. * Added Literal Serialization * Fixed DecimalLiteral Serialization * Changed to Iceberg Private Classes (Original) --- .../apache/iceberg/expressions/Literals.java | 35 +++- .../iceberg/expressions/ExpressionParser.java | 191 +++++++++++++++++- .../expressions/TestExpressionParser.java | 125 +++++++++--- 3 files changed, 312 insertions(+), 39 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 19722e912aca..c21378e10000 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -23,6 +23,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -88,6 +89,34 @@ static Literal from(T value) { "Cannot create expression literal from %s: %s", value.getClass().getName(), value)); } + static Type typeFromLiteralValue(Object value) { + if (value instanceof BooleanLiteral) { + return Types.BooleanType.get(); + } else if (value instanceof IntegerLiteral) { + return Types.IntegerType.get(); + } else if (value instanceof LongLiteral) { + return Types.LongType.get(); + } else if (value instanceof FloatLiteral) { + return Types.FloatType.get(); + } else if (value instanceof DoubleLiteral) { + return Types.DoubleType.get(); + } else if (value instanceof StringLiteral) { + return Types.StringType.get(); + } else if (value instanceof UUIDLiteral) { + return Types.UUIDType.get(); + } else if (value instanceof FixedLiteral) { + return Types.FixedType.ofLength( + StandardCharsets.UTF_8.decode(((FixedLiteral) value).toByteBuffer()).length()); + } else if (value instanceof BinaryLiteral) { + return Types.BinaryType.get(); + } else if (value instanceof DecimalLiteral) { + return Types.DecimalType.of( + ((DecimalLiteral) value).value().precision(), ((DecimalLiteral) value).value().scale()); + } else { + throw new IllegalArgumentException("Cannot find valid Literal Type for " + value + "."); + } + } + @SuppressWarnings("unchecked") static AboveMax aboveMax() { return AboveMax.INSTANCE; @@ -98,7 +127,7 @@ static BelowMin belowMin() { return BelowMin.INSTANCE; } - protected abstract static class BaseLiteral implements Literal { + private abstract static class BaseLiteral implements Literal { private final T value; private transient volatile ByteBuffer byteBuffer = null; @@ -171,7 +200,7 @@ public Comparator comparator() { static class AboveMax implements Literal { private static final AboveMax INSTANCE = new AboveMax(); - protected AboveMax() { + private AboveMax() { } @Override @@ -198,7 +227,7 @@ public String toString() { static class BelowMin implements Literal { private static final BelowMin INSTANCE = new BelowMin(); - protected BelowMin() { + private BelowMin() { } @Override diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 1abf6a754829..62a4cd72cb07 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -20,17 +20,23 @@ package org.apache.iceberg.expressions; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Set; +import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; public class ExpressionParser { - private static final String TYPE = "type"; private static final String VALUE = "value"; private static final String OPERATION = "operation"; @@ -48,17 +54,46 @@ public class ExpressionParser { private static final String BOUNDED_LITERAL_PREDICATE = "bounded-literal-predicate"; private static final String BOUNDED_SET_PREDICATE = "bounded-set-predicate"; private static final String BOUNDED_UNARY_PREDICATE = "bounded-unary-predicate"; + private static final Set PREDICATE_TYPES = Sets.newHashSet( + UNBOUNDED_PREDICATE, + BOUNDED_LITERAL_PREDICATE, + BOUNDED_SET_PREDICATE, + BOUNDED_UNARY_PREDICATE); private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; private static final String ABOVE_MAX = "above-max"; private static final String BELOW_MIN = "below-min"; - private static final Set oneInputs = ImmutableSet.of( + private static final String IS_NULL = "is"; + private static final String NOT_NULL = "not_null"; + private static final String IS_NAN = "is_nan"; + private static final String NOT_NAN = "not_nan"; + + private static final Set ONE_INPUTS = ImmutableSet.of( Expression.Operation.IS_NULL, Expression.Operation.NOT_NULL, Expression.Operation.IS_NAN, Expression.Operation.NOT_NAN); + private static final Set ONE_INPUTS_STRINGS = Sets.newHashSet( + Expression.Operation.IS_NULL.name().toLowerCase(), + Expression.Operation.NOT_NULL.name().toLowerCase(), + Expression.Operation.IS_NAN.name().toLowerCase(), + Expression.Operation.NOT_NAN.name().toLowerCase()); + + private static final String BOOLEAN = "boolean"; + private static final String INTEGER = "int"; + private static final String LONG = "long"; + private static final String FLOAT = "float"; + private static final String DOUBLE = "double"; + private static final String DATE = "date"; + private static final String TIME = "time"; + private static final String TIMESTAMP = "timestamp"; + private static final String STRING = "string"; + private static final String UUID = "uuid"; + private static final String FIXED = "fixed"; + private static final String BINARY = "binary"; + private static final String DECIMAL = "decimal"; private ExpressionParser() { } @@ -157,7 +192,7 @@ public static void toJson(UnboundPredicate predicate, JsonGenerator generator) t generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); generator.writeFieldName(TERM); toJson(predicate.term(), generator); - if (!(oneInputs.contains(predicate.op()))) { + if (!(ONE_INPUTS.contains(predicate.op()))) { generator.writeFieldName(LITERALS); toJson(predicate.literals(), generator); } @@ -216,10 +251,158 @@ public static void toJson(Literal literal, JsonGenerator generator) throws IOExc } else if (literal instanceof Literals.BelowMin) { generator.writeStringField(TYPE, BELOW_MIN); } else { - generator.writeStringField(TYPE, ((Literals.BaseLiteral) literal).typeId().toString().toLowerCase()); + generator.writeStringField(TYPE, Literals.typeFromLiteralValue(literal).toString()); generator.writeStringField(VALUE, StandardCharsets.UTF_8.decode(literal.toByteBuffer()).toString()); } generator.writeEndObject(); } + + public static Expression fromJson(String json) { + try { + ObjectMapper mapper = new ObjectMapper(); + return fromJson(mapper.readTree(json)); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + public static Expression fromJson(JsonNode json) { + String expressionType = JsonUtil.getString("type", json); + + if (AND.equals(expressionType)) { + return new And(fromJson(json.get(LEFT_OPERAND)), fromJson(json.get(RIGHT_OPERAND))); + } else if (OR.equals(expressionType)) { + return new Or(fromJson(json.get(LEFT_OPERAND)), fromJson(json.get(RIGHT_OPERAND))); + } else if (NOT.equals(expressionType)) { + return new Not(fromJson(json.get(OPERAND))); + } else if (TRUE.equals(expressionType)) { + return True.INSTANCE; + } else if (FALSE.equals(expressionType)) { + return False.INSTANCE; + } else if (PREDICATE_TYPES.contains(expressionType)) { + return fromJsonToPredicate(json, expressionType); + } else { + throw new IllegalArgumentException("Invalid Operation Type"); + } + } + + public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { + if (UNBOUNDED_PREDICATE.equals(predicateType)) { + return fromJsonUnboundPredicate(json); + } else if (BOUNDED_LITERAL_PREDICATE.equals(predicateType)) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundLiteralPredicate is not currently supported."); + } else if (BOUNDED_SET_PREDICATE.equals(predicateType)) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundSetPredicate is not currently supported."); + } else if (BOUNDED_UNARY_PREDICATE.equals(predicateType)) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundUnaryPredicate is not currently supported."); + } else { + throw new IllegalArgumentException("Invalid Predicate Type"); + } + } + + @SuppressWarnings("CyclomaticComplexity") + public static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { + String operation = json.get(OPERATION).textValue(); + + if (ONE_INPUTS_STRINGS.contains(operation)) { + switch (operation) { + case IS_NULL: + return new UnboundPredicate( + Expression.Operation.IS_NULL, fromJsonToTerm(json.get(TERM))); + case NOT_NULL: + return new UnboundPredicate( + Expression.Operation.NOT_NULL, fromJsonToTerm(json.get(TERM))); + case IS_NAN: + return new UnboundPredicate( + Expression.Operation.IS_NAN, fromJsonToTerm(json.get(TERM))); + case NOT_NAN: + return new UnboundPredicate( + Expression.Operation.NOT_NAN, fromJsonToTerm(json.get(TERM))); + default: + throw new IllegalArgumentException("Cannot find valid Operation Type for " + operation + "."); + } + + } else if (operation.equals(Expression.Operation.LT.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.LT, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.LT_EQ.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.LT_EQ, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.GT.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.GT, fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.GT_EQ.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.GT_EQ, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.EQ.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.EQ, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.NOT_EQ.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.NOT_EQ, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.IN.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.IN, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.NOT_IN.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.NOT_IN, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.STARTS_WITH.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.STARTS_WITH, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else if (operation.equals(Expression.Operation.NOT_STARTS_WITH.name().toLowerCase())) { + return new UnboundPredicate( + Expression.Operation.NOT_STARTS_WITH, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiterals(json.get(LITERALS))); + } else { + throw new IllegalArgumentException("Cannot find valid Operation Type for " + operation + "."); + } + } + + public static UnboundTerm fromJsonToTerm(JsonNode json) { + String referenceType = json.get(TYPE).textValue(); + + if (referenceType.equals(NAMED_REFERENCE)) { + return new NamedReference(json.get(VALUE).textValue()); + } else if (referenceType.equals(BOUND_REFERENCE)) { + throw new UnsupportedOperationException( + "Serialization of Predicate type BoundReference is not currently supported."); + } else { + throw new IllegalArgumentException("Invalid Term Reference Type"); + } + } + + public static List fromJsonToLiterals(JsonNode json) { + List literals = Lists.newArrayList(); + String literalType; + for (int i = 0; i < json.size(); i++) { + literalType = json.get(i).get(TYPE).textValue(); + literals.add(Conversions.fromByteBuffer( + Types.fromPrimitiveString(literalType), + StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue()))); + } + + return literals; + } } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index bc57825a7114..56a2aae2018a 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -19,6 +19,10 @@ package org.apache.iceberg.expressions; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; import org.junit.Test; @@ -33,7 +37,7 @@ public void testPredicate() { " \"value\" : \"Column-Name\"\n" + " },\n" + " \"literals\" : [ {\n" + - " \"type\" : \"integer\",\n" + + " \"type\" : \"int\",\n" + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + " } ]\n" + "}"; @@ -46,6 +50,7 @@ public void testPredicate() { String actual = ExpressionParser.toJson(inPredicate, true); Assert.assertEquals(expected, actual); } + @Test public void testAnd() { String expected = "{\n" + @@ -58,7 +63,7 @@ public void testAnd() { " \"value\" : \"Column1-Name\"\n" + " },\n" + " \"literals\" : [ {\n" + - " \"type\" : \"integer\",\n" + + " \"type\" : \"int\",\n" + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + " } ]\n" + " },\n" + @@ -87,6 +92,9 @@ public void testAnd() { Literal.of("Check")); And andExpression = new And(gtEqPredicate, inPredicate); + + Expression test = ExpressionParser.fromJson(expected); + String actual = ExpressionParser.toJson(andExpression, true); Assert.assertEquals(expected, actual); } @@ -103,7 +111,7 @@ public void testOr() { " \"value\" : \"Column1-Name\"\n" + " },\n" + " \"literals\" : [ {\n" + - " \"type\" : \"integer\",\n" + + " \"type\" : \"int\",\n" + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + " } ]\n" + " },\n" + @@ -143,7 +151,7 @@ public void testNot() { " \"value\" : \"Column1-Name\"\n" + " },\n" + " \"literals\" : [ {\n" + - " \"type\" : \"integer\",\n" + + " \"type\" : \"int\",\n" + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + " } ]\n" + " }\n" + @@ -173,7 +181,7 @@ public void testNestedExpression() { " \"value\" : \"Column1-Name\"\n" + " },\n" + " \"literals\" : [ {\n" + - " \"type\" : \"integer\",\n" + + " \"type\" : \"int\",\n" + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + " } ]\n" + " },\n" + @@ -221,45 +229,98 @@ public void testNestedExpression() { } @Test - public void aboveMaxAndBelowMinTest() { + public void testParserBothWays() { String expected = "{\n" + - " \"type\" : \"and\",\n" + + " \"type\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + - " \"operation\" : \"lt\",\n" + - " \"term\" : {\n" + - " \"type\" : \"named-reference\",\n" + - " \"value\" : \"Column1-Name\"\n" + + " \"type\" : \"and\",\n" + + " \"left-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"in\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column1-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"int\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + " },\n" + - " \"literals\" : [ {\n" + - " \"type\" : \"above-max\"\n" + - " } ]\n" + + " \"right-operand\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column2-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"string\",\n" + + " \"value\" : \"Test\"\n" + + " } ]\n" + + " }\n" + " },\n" + " \"right-operand\" : {\n" + " \"type\" : \"unbounded-predicate\",\n" + - " \"operation\" : \"gt_eq\",\n" + + " \"operation\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + - " \"value\" : \"Column2-Name\"\n" + - " },\n" + - " \"literals\" : [ {\n" + - " \"type\" : \"below-min\"\n" + - " } ]\n" + + " \"value\" : \"Column3-Name\"\n" + + " }\n" + " }\n" + "}"; - UnboundPredicate aboveMaxPredicate = new UnboundPredicate( - Expression.Operation.LT, - new NamedReference("Column1-Name"), - new Literals.AboveMax()); + Expression actualExpression = ExpressionParser.fromJson(expected); + String actualJsonExpression = ExpressionParser.toJson(actualExpression, true); - UnboundPredicate belowMinPredicate = new UnboundPredicate( - Expression.Operation.GT_EQ, - new NamedReference("Column2-Name"), - new Literals.BelowMin<>()); + Assert.assertEquals(expected, actualJsonExpression); + } - And andExpression = new And(aboveMaxPredicate, belowMinPredicate); - String actual = ExpressionParser.toJson(andExpression, true); - Assert.assertEquals(expected, actual); + @Test + public void testFixedLiteral() { + String expected = ""; + String testString = "2\\u0000\\u0000\\u0000"; + ByteBuffer testByteBuffer = StandardCharsets.UTF_8.encode(testString); + + byte[] testByteArray = new byte[testByteBuffer.remaining()]; + testByteBuffer.get(testByteArray); + + Literal testLiteral = Literals.from(testByteArray); + System.out.println(StandardCharsets.UTF_8.decode(testLiteral.toByteBuffer()).toString()); + + UnboundPredicate expectedExpression = new UnboundPredicate( + Expression.Operation.EQ, + new NamedReference("Column-Name"), + Lists.newArrayList(testByteArray)); + System.out.println(ExpressionParser.toJson(expectedExpression, true)); + } + + @Test + public void testDecimalLiteral() { + String expected = "{\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"in\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"decimal(3, 2)\",\n" + + " \"value\" : \"\\u0001:\"\n" + + " } ]\n" + + "}"; + + UnboundPredicate expectedExpression = new UnboundPredicate( + Expression.Operation.IN, + new NamedReference("Column-Name"), + Lists.newArrayList(new BigDecimal("3.14"))); + + String actualJsonExpression = ExpressionParser.toJson(ExpressionParser.fromJson(expected), true); + Expression actualExpression = ExpressionParser.fromJson(expected); + + Assert.assertEquals(expected, ExpressionParser.toJson(expectedExpression, true)); + Assert.assertEquals(expected, actualJsonExpression); + Assert.assertEquals( + ((UnboundPredicate) actualExpression).literals().get(0).toString(), + expectedExpression.literals().get(0).toString()); } } From b969fb30a498fd301431a50c00c0bafcf76d696e Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Wed, 9 Mar 2022 21:44:17 -0800 Subject: [PATCH 11/19] add some fixes to parser --- .../apache/iceberg/expressions/Literals.java | 28 --- .../iceberg/expressions/ExpressionParser.java | 219 ++++++------------ 2 files changed, 67 insertions(+), 180 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index c21378e10000..49ff3a23f29b 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -89,34 +89,6 @@ static Literal from(T value) { "Cannot create expression literal from %s: %s", value.getClass().getName(), value)); } - static Type typeFromLiteralValue(Object value) { - if (value instanceof BooleanLiteral) { - return Types.BooleanType.get(); - } else if (value instanceof IntegerLiteral) { - return Types.IntegerType.get(); - } else if (value instanceof LongLiteral) { - return Types.LongType.get(); - } else if (value instanceof FloatLiteral) { - return Types.FloatType.get(); - } else if (value instanceof DoubleLiteral) { - return Types.DoubleType.get(); - } else if (value instanceof StringLiteral) { - return Types.StringType.get(); - } else if (value instanceof UUIDLiteral) { - return Types.UUIDType.get(); - } else if (value instanceof FixedLiteral) { - return Types.FixedType.ofLength( - StandardCharsets.UTF_8.decode(((FixedLiteral) value).toByteBuffer()).length()); - } else if (value instanceof BinaryLiteral) { - return Types.BinaryType.get(); - } else if (value instanceof DecimalLiteral) { - return Types.DecimalType.of( - ((DecimalLiteral) value).value().precision(), ((DecimalLiteral) value).value().scale()); - } else { - throw new IllegalArgumentException("Cannot find valid Literal Type for " + value + "."); - } - } - @SuppressWarnings("unchecked") static AboveMax aboveMax() { return AboveMax.INSTANCE; diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 62a4cd72cb07..47aa756e96f0 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -25,14 +25,19 @@ import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.List; +import java.util.Locale; import java.util.Set; +import java.util.UUID; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; @@ -61,13 +66,6 @@ public class ExpressionParser { BOUNDED_UNARY_PREDICATE); private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; - private static final String ABOVE_MAX = "above-max"; - private static final String BELOW_MIN = "below-min"; - - private static final String IS_NULL = "is"; - private static final String NOT_NULL = "not_null"; - private static final String IS_NAN = "is_nan"; - private static final String NOT_NAN = "not_nan"; private static final Set ONE_INPUTS = ImmutableSet.of( Expression.Operation.IS_NULL, @@ -75,26 +73,6 @@ public class ExpressionParser { Expression.Operation.IS_NAN, Expression.Operation.NOT_NAN); - private static final Set ONE_INPUTS_STRINGS = Sets.newHashSet( - Expression.Operation.IS_NULL.name().toLowerCase(), - Expression.Operation.NOT_NULL.name().toLowerCase(), - Expression.Operation.IS_NAN.name().toLowerCase(), - Expression.Operation.NOT_NAN.name().toLowerCase()); - - private static final String BOOLEAN = "boolean"; - private static final String INTEGER = "int"; - private static final String LONG = "long"; - private static final String FLOAT = "float"; - private static final String DOUBLE = "double"; - private static final String DATE = "date"; - private static final String TIME = "time"; - private static final String TIMESTAMP = "timestamp"; - private static final String STRING = "string"; - private static final String UUID = "uuid"; - private static final String FIXED = "fixed"; - private static final String BINARY = "binary"; - private static final String DECIMAL = "decimal"; - private ExpressionParser() { } @@ -126,7 +104,7 @@ public static void toJson(Expression expression, JsonGenerator generator) throws } else if (expression instanceof False) { toJson((False) expression, generator); } else if (expression instanceof Predicate) { - toJson((Predicate) expression, generator); + toJson((Predicate) expression, generator); } else { throw new IllegalArgumentException("Invalid Operation Type"); } @@ -172,21 +150,15 @@ public static void toJson(False expression, JsonGenerator generator) throws IOEx generator.writeEndObject(); } - public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { + public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { if (predicate instanceof UnboundPredicate) { - toJson((UnboundPredicate) predicate, generator); - } else if (predicate instanceof BoundLiteralPredicate) { - toJson((BoundLiteralPredicate) predicate, generator); - } else if (predicate instanceof BoundSetPredicate) { - toJson((BoundSetPredicate) predicate, generator); - } else if (predicate instanceof BoundUnaryPredicate) { - toJson((BoundUnaryPredicate) predicate, generator); + toJson((UnboundPredicate) predicate, generator); } else { - throw new IllegalArgumentException("Cannot find valid Predicate Type for " + predicate + "."); + throw new IllegalArgumentException("Cannot convert predicate " + predicate); } } - public static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { + public static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeStringField(TYPE, UNBOUNDED_PREDICATE); generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); @@ -194,49 +166,32 @@ public static void toJson(UnboundPredicate predicate, JsonGenerator generator) t toJson(predicate.term(), generator); if (!(ONE_INPUTS.contains(predicate.op()))) { generator.writeFieldName(LITERALS); - toJson(predicate.literals(), generator); + generator.writeStartArray(); + for (Literal literal : predicate.literals()) { + toJson(literal, generator); + } + generator.writeEndArray(); } generator.writeEndObject(); } - public static void toJson(BoundLiteralPredicate predicate, JsonGenerator generator) { - throw new UnsupportedOperationException( - "Serialization of Predicate type BoundLiteralPredicate is not currently supported."); - } - - public static void toJson(BoundSetPredicate predicate, JsonGenerator generator) { - throw new UnsupportedOperationException( - "Serialization of Predicate type BoundSetPredicate is not currently supported."); - } - - public static void toJson(BoundUnaryPredicate predicate, JsonGenerator generator) { - throw new UnsupportedOperationException( - "Serialization of Predicate type BoundUnaryPredicate is not currently supported."); - } - public static void toJson(Term term, JsonGenerator generator) throws IOException { if (term instanceof NamedReference) { - toJson((NamedReference) term, generator); - } else if (term instanceof BoundReference) { - toJson((BoundReference) term, generator); // Need to Implement + toJson((NamedReference) term, generator); } else { - throw new IllegalArgumentException("Cannot find valid Term Type for " + term + "."); + throw new IllegalArgumentException("Cannot convert term " + term); } } - public static void toJson(NamedReference term, JsonGenerator generator) throws IOException { + public static void toJson(NamedReference term, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeStringField(TYPE, NAMED_REFERENCE); generator.writeStringField(VALUE, term.name()); generator.writeEndObject(); } - public static void toJson(BoundReference term, JsonGenerator generator) { - throw new UnsupportedOperationException("Serialization of Term type BoundReference is not currently supported"); - } - - public static void toJson(List literals, JsonGenerator generator) throws IOException { + public static void toJson(List> literals, JsonGenerator generator) throws IOException { generator.writeStartArray(); for (int i = 0; i < literals.size(); i++) { toJson(literals.get(i), generator); @@ -244,17 +199,38 @@ public static void toJson(List literals, JsonGenerator generator) throw generator.writeEndArray(); } - public static void toJson(Literal literal, JsonGenerator generator) throws IOException { + public static void toJson(Literal literal, JsonGenerator generator) throws IOException { generator.writeStartObject(); - if (literal instanceof Literals.AboveMax) { - generator.writeStringField(TYPE, ABOVE_MAX); - } else if (literal instanceof Literals.BelowMin) { - generator.writeStringField(TYPE, BELOW_MIN); + + Object value = literal.value(); + Type type; + if (value instanceof Boolean) { + type = Types.BooleanType.get(); + } else if (value instanceof Integer) { + type = Types.IntegerType.get(); + } else if (value instanceof Long) { + type = Types.LongType.get(); + } else if (value instanceof Float) { + type = Types.FloatType.get(); + } else if (value instanceof Double) { + type = Types.DoubleType.get(); + } else if (value instanceof CharSequence) { + type = Types.StringType.get(); + } else if (value instanceof UUID) { + type = Types.UUIDType.get(); + } else if (value instanceof byte[]) { + type = Types.FixedType.ofLength(((byte[]) value).length); + } else if (value instanceof ByteBuffer) { + type = Types.BinaryType.get(); + } else if (value instanceof BigDecimal) { + BigDecimal decimal = (BigDecimal) value; + type = Types.DecimalType.of(decimal.precision(), decimal.scale()); } else { - generator.writeStringField(TYPE, Literals.typeFromLiteralValue(literal).toString()); - generator.writeStringField(VALUE, StandardCharsets.UTF_8.decode(literal.toByteBuffer()).toString()); + throw new IllegalArgumentException("Cannot find literal type for value class " + value.getClass().getName()); } + generator.writeStringField(TYPE, type.toString()); + generator.writeStringField(VALUE, StandardCharsets.UTF_8.decode(literal.toByteBuffer()).toString()); generator.writeEndObject(); } @@ -268,7 +244,7 @@ public static Expression fromJson(String json) { } public static Expression fromJson(JsonNode json) { - String expressionType = JsonUtil.getString("type", json); + String expressionType = JsonUtil.getString(TYPE, json); if (AND.equals(expressionType)) { return new And(fromJson(json.get(LEFT_OPERAND)), fromJson(json.get(RIGHT_OPERAND))); @@ -287,7 +263,7 @@ public static Expression fromJson(JsonNode json) { } } - public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { + public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { if (UNBOUNDED_PREDICATE.equals(predicateType)) { return fromJsonUnboundPredicate(json); } else if (BOUNDED_LITERAL_PREDICATE.equals(predicateType)) { @@ -305,86 +281,25 @@ public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) } @SuppressWarnings("CyclomaticComplexity") - public static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { - String operation = json.get(OPERATION).textValue(); - - if (ONE_INPUTS_STRINGS.contains(operation)) { - switch (operation) { - case IS_NULL: - return new UnboundPredicate( - Expression.Operation.IS_NULL, fromJsonToTerm(json.get(TERM))); - case NOT_NULL: - return new UnboundPredicate( - Expression.Operation.NOT_NULL, fromJsonToTerm(json.get(TERM))); - case IS_NAN: - return new UnboundPredicate( - Expression.Operation.IS_NAN, fromJsonToTerm(json.get(TERM))); - case NOT_NAN: - return new UnboundPredicate( - Expression.Operation.NOT_NAN, fromJsonToTerm(json.get(TERM))); - default: - throw new IllegalArgumentException("Cannot find valid Operation Type for " + operation + "."); - } + public static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { + Expression.Operation operation = Expression.Operation.valueOf( + JsonUtil.getString(OPERATION, json).toUpperCase(Locale.ENGLISH)); - } else if (operation.equals(Expression.Operation.LT.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.LT, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.LT_EQ.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.LT_EQ, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.GT.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.GT, fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.GT_EQ.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.GT_EQ, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.EQ.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.EQ, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.NOT_EQ.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.NOT_EQ, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.IN.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.IN, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.NOT_IN.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.NOT_IN, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.STARTS_WITH.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.STARTS_WITH, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); - } else if (operation.equals(Expression.Operation.NOT_STARTS_WITH.name().toLowerCase())) { - return new UnboundPredicate( - Expression.Operation.NOT_STARTS_WITH, - fromJsonToTerm(json.get(TERM)), - fromJsonToLiterals(json.get(LITERALS))); + if (ONE_INPUTS.contains(operation)) { + return new UnboundPredicate<>(operation, fromJsonToTerm(json.get(TERM))); } else { - throw new IllegalArgumentException("Cannot find valid Operation Type for " + operation + "."); + return new UnboundPredicate( + operation, + fromJsonToTerm(json.get(TERM)), + fromJsonToLiteralValues(json.get(LITERALS))); } } - public static UnboundTerm fromJsonToTerm(JsonNode json) { + public static UnboundTerm fromJsonToTerm(JsonNode json) { String referenceType = json.get(TYPE).textValue(); if (referenceType.equals(NAMED_REFERENCE)) { - return new NamedReference(json.get(VALUE).textValue()); + return new NamedReference<>(json.get(VALUE).textValue()); } else if (referenceType.equals(BOUND_REFERENCE)) { throw new UnsupportedOperationException( "Serialization of Predicate type BoundReference is not currently supported."); @@ -393,14 +308,14 @@ public static UnboundTerm fromJsonToTerm(JsonNode json) { } } - public static List fromJsonToLiterals(JsonNode json) { - List literals = Lists.newArrayList(); - String literalType; + public static List fromJsonToLiteralValues(JsonNode json) { + List literals = Lists.newArrayList(); for (int i = 0; i < json.size(); i++) { - literalType = json.get(i).get(TYPE).textValue(); - literals.add(Conversions.fromByteBuffer( - Types.fromPrimitiveString(literalType), - StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue()))); + String literalType = json.get(i).get(TYPE).textValue(); + Object value = Conversions.fromByteBuffer( + Types.fromPrimitiveString(literalType), + StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue())); + literals.add(value); } return literals; From 55c1505558e6589fea435bf4593ff1edefde1623 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Thu, 10 Mar 2022 11:45:51 -0600 Subject: [PATCH 12/19] Updated FixedLiteral and Testing --- .../apache/iceberg/expressions/Literals.java | 3 +-- .../iceberg/expressions/ExpressionParser.java | 23 +++++++++++++----- .../expressions/TestExpressionParser.java | 24 +++++++++++++++---- 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 49ff3a23f29b..9a1981217cf5 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -23,7 +23,6 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -99,7 +98,7 @@ static BelowMin belowMin() { return BelowMin.INSTANCE; } - private abstract static class BaseLiteral implements Literal { + protected abstract static class BaseLiteral implements Literal { private final T value; private transient volatile ByteBuffer byteBuffer = null; diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 47aa756e96f0..c0c2e1332a40 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -67,6 +67,8 @@ public class ExpressionParser { private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; + private static final String FIXED = "fixed"; + private static final Set ONE_INPUTS = ImmutableSet.of( Expression.Operation.IS_NULL, Expression.Operation.NOT_NULL, @@ -104,7 +106,7 @@ public static void toJson(Expression expression, JsonGenerator generator) throws } else if (expression instanceof False) { toJson((False) expression, generator); } else if (expression instanceof Predicate) { - toJson((Predicate) expression, generator); + toJson((Predicate) expression, generator); } else { throw new IllegalArgumentException("Invalid Operation Type"); } @@ -150,7 +152,7 @@ public static void toJson(False expression, JsonGenerator generator) throws IOEx generator.writeEndObject(); } - public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { + public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { if (predicate instanceof UnboundPredicate) { toJson((UnboundPredicate) predicate, generator); } else { @@ -221,7 +223,11 @@ public static void toJson(Literal literal, JsonGenerator generator) throws IO } else if (value instanceof byte[]) { type = Types.FixedType.ofLength(((byte[]) value).length); } else if (value instanceof ByteBuffer) { - type = Types.BinaryType.get(); + if (((Literals.BaseLiteral) literal).typeId() == Type.TypeID.FIXED) { + type = Types.FixedType.ofLength(((ByteBuffer) value).remaining()); + } else { + type = Types.BinaryType.get(); + } } else if (value instanceof BigDecimal) { BigDecimal decimal = (BigDecimal) value; type = Types.DecimalType.of(decimal.precision(), decimal.scale()); @@ -263,7 +269,7 @@ public static Expression fromJson(JsonNode json) { } } - public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { + public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { if (UNBOUNDED_PREDICATE.equals(predicateType)) { return fromJsonUnboundPredicate(json); } else if (BOUNDED_LITERAL_PREDICATE.equals(predicateType)) { @@ -313,8 +319,13 @@ public static List fromJsonToLiteralValues(JsonNode json) { for (int i = 0; i < json.size(); i++) { String literalType = json.get(i).get(TYPE).textValue(); Object value = Conversions.fromByteBuffer( - Types.fromPrimitiveString(literalType), - StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue())); + Types.fromPrimitiveString(literalType), + StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue())); + if (literalType.startsWith(FIXED)) { + byte[] valueByteArray = new byte[((ByteBuffer) value).remaining()]; + ((ByteBuffer) value).get(valueByteArray); + value = valueByteArray; + } literals.add(value); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index 56a2aae2018a..e12bc77fccb3 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -277,21 +277,37 @@ public void testParserBothWays() { @Test public void testFixedLiteral() { - String expected = ""; - String testString = "2\\u0000\\u0000\\u0000"; + String expected = "{\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"Column-Name\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"fixed[10]\",\n" + + " \"value\" : \"testString\"\n" + + " } ]\n" + + "}"; + String testString = "testString"; ByteBuffer testByteBuffer = StandardCharsets.UTF_8.encode(testString); byte[] testByteArray = new byte[testByteBuffer.remaining()]; testByteBuffer.get(testByteArray); Literal testLiteral = Literals.from(testByteArray); - System.out.println(StandardCharsets.UTF_8.decode(testLiteral.toByteBuffer()).toString()); UnboundPredicate expectedExpression = new UnboundPredicate( Expression.Operation.EQ, new NamedReference("Column-Name"), Lists.newArrayList(testByteArray)); - System.out.println(ExpressionParser.toJson(expectedExpression, true)); + + String actualJsonExpression = ExpressionParser.toJson(expectedExpression, true); + Expression actualExpression = ExpressionParser.fromJson(actualJsonExpression); + String newActualJsonExpression = ExpressionParser.toJson(actualExpression, true); + + Assert.assertEquals(expected, actualJsonExpression); + Assert.assertEquals(actualJsonExpression, newActualJsonExpression); } @Test From e61ce99535de4395c8c49dc589f3f3e9e41422cb Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Thu, 10 Mar 2022 12:39:27 -0600 Subject: [PATCH 13/19] Added fromJson Event Serialization --- .../org/apache/iceberg/util/EventParser.java | 60 ++++++++++++ .../iceberg/events/TestEventParser.java | 95 +++++++++++++++++++ 2 files changed, 155 insertions(+) create mode 100644 core/src/test/java/org/apache/iceberg/events/TestEventParser.java diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java index 2005998d30d6..4e79457d9ed7 100644 --- a/core/src/main/java/org/apache/iceberg/util/EventParser.java +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -20,14 +20,19 @@ package org.apache.iceberg.util; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; import java.util.Map; +import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.IncrementalScanEvent; import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.ExpressionParser; public class EventParser { @@ -47,9 +52,16 @@ private EventParser() { } public static String toJson(Object event) { + return toJson(event, false); + } + + public static String toJson(Object event, boolean pretty) { try { StringWriter writer = new StringWriter(); JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + if (pretty) { + generator.useDefaultPrettyPrinter(); + } if (event instanceof ScanEvent) { toJson((ScanEvent) event, generator); } else if (event instanceof CreateSnapshotEvent) { @@ -117,4 +129,52 @@ public static void toJson(IncrementalScanEvent event, JsonGenerator generator) t SchemaParser.toJson(event.projection(), generator); generator.writeEndObject(); } + + public static Object fromJson(String json) { + try { + ObjectMapper mapper = new ObjectMapper(); + return fromJson(mapper.readTree(json)); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + public static Object fromJson(JsonNode json) { + String eventType = JsonUtil.getString(EVENT_TYPE, json); + if (eventType.equals(ScanEvent.class.getName())) { + return fromJsonToScanEvent(json); + } else if (eventType.equals(CreateSnapshotEvent.class.getName())) { + return fromJsonToCreateSnapshotEvent(json); + } else if (eventType.equals(IncrementalScanEvent.class.getName())) { + return fromJsonToIncrementalScanEvent(json); + } else { + throw new IllegalArgumentException("Invalid Event Type"); + } + } + + public static ScanEvent fromJsonToScanEvent(JsonNode json) { + String tableName = JsonUtil.getString(TABLE_NAME, json); + Long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, json); + Expression filter = ExpressionParser.fromJson(json.get(EXPRESSION)); + Schema schema = SchemaParser.fromJson(json.get(PROJECTION)); + return new ScanEvent(tableName, snapshotId, filter, schema); + } + + public static CreateSnapshotEvent fromJsonToCreateSnapshotEvent(JsonNode json) { + String tableName = JsonUtil.getString(TABLE_NAME, json); + String operation = JsonUtil.getString(OPERATION, json); + Long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, json); + Long sequenceNumber = JsonUtil.getLong(SEQUENCE_NUMBER, json); + Map summary = JsonUtil.getStringMap(SUMMARY, json); + return new CreateSnapshotEvent(tableName, operation, snapshotId, sequenceNumber, summary); + } + + public static IncrementalScanEvent fromJsonToIncrementalScanEvent(JsonNode json) { + String tableName = JsonUtil.getString(TABLE_NAME, json); + Long fromSnapshotId = JsonUtil.getLong(FROM_SNAPSHOT_ID, json); + Long toSnapshotId = JsonUtil.getLong(TO_SNAPSHOT_ID, json); + Expression filter = ExpressionParser.fromJson(json.get(EXPRESSION)); + Schema schema = SchemaParser.fromJson(json.get(PROJECTION)); + return new IncrementalScanEvent(tableName, fromSnapshotId, toSnapshotId, filter, schema); + } } diff --git a/core/src/test/java/org/apache/iceberg/events/TestEventParser.java b/core/src/test/java/org/apache/iceberg/events/TestEventParser.java new file mode 100644 index 000000000000..c1f7182661a4 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/events/TestEventParser.java @@ -0,0 +1,95 @@ +/* + * 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.events; + +import org.apache.iceberg.util.EventParser; +import org.junit.Assert; +import org.junit.Test; + +public class TestEventParser { + @Test + public void testCreateSnapshotEventSerialization() { + String createSnapshotEventMessageBefore = "{\n" + + " \"event-type\" : \"org.apache.iceberg.events.CreateSnapshotEvent\",\n" + + " \"table-name\" : \"glue.default1.demo\",\n" + + " \"operation\" : \"append\",\n" + + " \"snapshot-id\" : 3898248035543915183,\n" + + " \"sequence-number\" : 0,\n" + + " \"summary\" : {\n" + + " \"spark.app.id\" : \"local-1645224366005\",\n" + + " \"added-data-files\" : \"3\",\n" + + " \"added-records\" : \"3\",\n" + + " \"added-files-size\" : \"1929\",\n" + + " \"changed-partition-count\" : \"1\",\n" + + " \"total-records\" : \"3\",\n" + + " \"total-files-size\" : \"1929\",\n" + + " \"total-data-files\" : \"3\",\n" + + " \"total-delete-files\" : \"0\",\n" + + " \"total-position-deletes\" : \"0\",\n" + + " \"total-equality-deletes\" : \"0\"\n" + + " }\n" + + "}"; + + Object createSnapShotEvent = EventParser.fromJson(createSnapshotEventMessageBefore); + String createSnapshotEventMessageAfter = EventParser.toJson(createSnapShotEvent, true); + Assert.assertEquals(createSnapshotEventMessageBefore, createSnapshotEventMessageAfter); + } + + @Test + public void testScanEventSerialization() { + String scanEventMessageBefore = "{\n" + + " \"event-type\" : \"org.apache.iceberg.events.ScanEvent\",\n" + + " \"table-name\" : \"glue.default1.check\",\n" + + " \"snapshot-id\" : 4466314262163129668,\n" + + " \"expression\" : {\n" + + " \"type\" : \"unbounded-predicate\",\n" + + " \"operation\" : \"eq\",\n" + + " \"term\" : {\n" + + " \"type\" : \"named-reference\",\n" + + " \"value\" : \"id\"\n" + + " },\n" + + " \"literals\" : [ {\n" + + " \"type\" : \"int\",\n" + + " \"value\" : \"2\\u0000\\u0000\\u0000\"\n" + + " } ]\n" + + " },\n" + + " \"projection\" : {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"id\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"int\"\n" + + " }, {\n" + + " \"id\" : 2,\n" + + " \"name\" : \"data\",\n" + + " \"required\" : false,\n" + + " \"type\" : \"string\"\n" + + " } ]\n" + + " }\n" + + "}"; + + Object scanEvent = EventParser.fromJson(scanEventMessageBefore); + + String scanEventMessageAfter = EventParser.toJson(scanEvent, true); + Assert.assertEquals(scanEventMessageBefore, scanEventMessageAfter); + } +} From 0aa2f8ccaf9ba6227fede80ea9bc80a055ad193e Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Thu, 10 Mar 2022 13:30:06 -0600 Subject: [PATCH 14/19] Fixed BaseLiteral --- .../java/org/apache/iceberg/expressions/Literals.java | 2 +- .../apache/iceberg/expressions/ExpressionParser.java | 10 +++++----- .../iceberg/expressions/TestExpressionParser.java | 2 -- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 9a1981217cf5..483c2a1a7a79 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -98,7 +98,7 @@ static BelowMin belowMin() { return BelowMin.INSTANCE; } - protected abstract static class BaseLiteral implements Literal { + private abstract static class BaseLiteral implements Literal { private final T value; private transient volatile ByteBuffer byteBuffer = null; diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index c0c2e1332a40..5d507901aff1 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -67,8 +67,6 @@ public class ExpressionParser { private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; - private static final String FIXED = "fixed"; - private static final Set ONE_INPUTS = ImmutableSet.of( Expression.Operation.IS_NULL, Expression.Operation.NOT_NULL, @@ -223,7 +221,7 @@ public static void toJson(Literal literal, JsonGenerator generator) throws IO } else if (value instanceof byte[]) { type = Types.FixedType.ofLength(((byte[]) value).length); } else if (value instanceof ByteBuffer) { - if (((Literals.BaseLiteral) literal).typeId() == Type.TypeID.FIXED) { + if (literal instanceof Literals.FixedLiteral) { type = Types.FixedType.ofLength(((ByteBuffer) value).remaining()); } else { type = Types.BinaryType.get(); @@ -318,14 +316,16 @@ public static List fromJsonToLiteralValues(JsonNode json) { List literals = Lists.newArrayList(); for (int i = 0; i < json.size(); i++) { String literalType = json.get(i).get(TYPE).textValue(); + Type primitiveType = Types.fromPrimitiveString(literalType); Object value = Conversions.fromByteBuffer( - Types.fromPrimitiveString(literalType), + primitiveType, StandardCharsets.UTF_8.encode(json.get(i).get(VALUE).textValue())); - if (literalType.startsWith(FIXED)) { + if (primitiveType.typeId() == Type.TypeID.FIXED) { byte[] valueByteArray = new byte[((ByteBuffer) value).remaining()]; ((ByteBuffer) value).get(valueByteArray); value = valueByteArray; } + literals.add(value); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index e12bc77fccb3..0d5bdf05d006 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -295,8 +295,6 @@ public void testFixedLiteral() { byte[] testByteArray = new byte[testByteBuffer.remaining()]; testByteBuffer.get(testByteArray); - Literal testLiteral = Literals.from(testByteArray); - UnboundPredicate expectedExpression = new UnboundPredicate( Expression.Operation.EQ, new NamedReference("Column-Name"), From b545adee74d25446e5c438fb58114f039e6df492 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Thu, 10 Mar 2022 13:58:39 -0600 Subject: [PATCH 15/19] Fixed Space --- .../java/org/apache/iceberg/aws/glue/GlueTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index fc37de5ad097..c47673ab7fa2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -73,7 +73,7 @@ public class GlueTestBase { TableProperties.WRITE_METADATA_LOCATION, "s3://" + testBucketName + "/writeMetaDataLoc", TableProperties.WRITE_FOLDER_STORAGE_LOCATION, "s3://" + testBucketName + "/writeFolderStorageLoc"); -@BeforeClass + @BeforeClass public static void beforeClass() { String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; S3FileIO fileIO = new S3FileIO(clientFactory::s3); From 39dee63b770fb7ccc813c6cd1c998f2e442db7a6 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Thu, 10 Mar 2022 15:42:14 -0600 Subject: [PATCH 16/19] Removed Listener Implementations --- .../org/apache/iceberg/events/Listener.java | 5 - .../glue/TestGlueCatalogSqsNotification.java | 276 ------------------ .../aws/AssumeRoleAwsClientFactory.java | 12 - .../iceberg/aws/AwsClientFactories.java | 12 - .../apache/iceberg/aws/AwsClientFactory.java | 6 - .../org/apache/iceberg/aws/AwsProperties.java | 12 - .../iceberg/aws/dynamodb/DynamoDbCatalog.java | 1 - .../apache/iceberg/aws/glue/GlueCatalog.java | 1 - .../apache/iceberg/aws/sns/SNSListener.java | 97 ------ .../apache/iceberg/aws/sqs/SQSListener.java | 97 ------ .../iceberg/aws/TestAwsClientFactories.java | 12 - build.gradle | 2 - .../apache/iceberg/BaseMetastoreCatalog.java | 5 - .../org/apache/iceberg/CatalogProperties.java | 59 ---- .../java/org/apache/iceberg/CatalogUtil.java | 76 ----- .../apache/iceberg/hadoop/HadoopCatalog.java | 1 - .../org/apache/iceberg/jdbc/JdbcCatalog.java | 1 - .../org/apache/iceberg/TestCatalogUtil.java | 70 ----- .../org/apache/iceberg/hive/HiveCatalog.java | 1 - .../apache/iceberg/nessie/NessieCatalog.java | 1 - 20 files changed, 747 deletions(-) delete mode 100644 aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java delete mode 100644 aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java delete mode 100644 aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java diff --git a/api/src/main/java/org/apache/iceberg/events/Listener.java b/api/src/main/java/org/apache/iceberg/events/Listener.java index 0011cd414e12..843a1de91402 100644 --- a/api/src/main/java/org/apache/iceberg/events/Listener.java +++ b/api/src/main/java/org/apache/iceberg/events/Listener.java @@ -19,14 +19,9 @@ package org.apache.iceberg.events; -import java.util.Map; - /** * A listener interface that can receive notifications. */ public interface Listener { void notify(E event); - - default void initialize(String listenerName, Map properties) { - } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java deleted file mode 100644 index 1cbe3f9ae9bf..000000000000 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogSqsNotification.java +++ /dev/null @@ -1,276 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg.aws.glue; - -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.aws.sqs.SQSListener; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.events.IncrementalScanEvent; -import org.apache.iceberg.events.Listener; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.JsonUtil; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.CreateQueueRequest; -import software.amazon.awssdk.services.sqs.model.DeleteQueueRequest; -import software.amazon.awssdk.services.sqs.model.GetQueueUrlRequest; -import software.amazon.awssdk.services.sqs.model.Message; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; - -public class TestGlueCatalogSqsNotification extends GlueTestBase { - - private static final SqsClient sqs = clientFactory.sqs(); - private static final DataFile testDataFile = DataFiles.builder(partitionSpec) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); - private static final int RETRY = 3; - private static final long RETRY_INTERVAL_MS = 1000; - - private String queueUrl; - - @Before - public void before() { - String queueName = getRandomName(); - sqs.createQueue(CreateQueueRequest.builder() - .queueName(queueName) - .build()); - this.queueUrl = sqs.getQueueUrl(GetQueueUrlRequest.builder() - .queueName(queueName) - .build()) - .queueUrl(); - } - - @After - public void after() { - DeleteQueueRequest deleteRequest = DeleteQueueRequest.builder().queueUrl(queueUrl).build(); - sqs.deleteQueue(deleteRequest); - } - - @Test - public void testNotifyOnCreateSnapshotEvent() throws IOException { - Listeners.register(listener(), CreateSnapshotEvent.class); - - String namespace = createNamespace(); - String tableName = getRandomName(); - createTable(namespace, tableName); - Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - - table.newAppend().appendFile(testDataFile).commit(); - - List messages = receiveAllMessages(queueUrl); - Assert.assertEquals(1, messages.size()); - - ObjectMapper objectMapper = new ObjectMapper(); - JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); - - String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"operation\":\"append\",\"snapshot-id\":" + table.currentSnapshot().snapshotId() + "," + - "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\"," + - "\"added-records\":\"1\",\"added-files-size\":\"10\"," + - "\"changed-partition-count\":\"1\",\"total-records\":\"1\"," + - "\"total-files-size\":\"10\",\"total-data-files\":\"1\"," + - "\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\"," + - "\"total-equality-deletes\":\"0\"}}"; - - Assert.assertEquals(expectedMessage, bodyNode.toString()); - } - - @Test - public void testNotifyOnScanEvent() throws IOException { - Listeners.register(listener(), ScanEvent.class); - - String namespace = createNamespace(); - String tableName = getRandomName(); - createTable(namespace, tableName); - Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - - table.newAppend().appendFile(testDataFile).commit(); - table.refresh(); - - Expression andExpression = Expressions.and(Expressions.equal("c1", "First"), Expressions.equal("c1", "Second")); - table.newScan().filter(andExpression).planFiles(); - - List messages = receiveAllMessages(queueUrl); - Assert.assertEquals(1, messages.size()); - - ObjectMapper objectMapper = new ObjectMapper(); - JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); - - String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"snapshot-id\":" + table.currentSnapshot().snapshotId() + "," + - "\"expression\":{\"type\":\"and\"," + - "\"left-operand\":{\"type\":\"unbounded-predicate\"," + - "\"operation\":\"eq\",\"term\":{\"type\":\"named-reference\",\"value\":\"c1\"}," + - "\"literals\":[{\"type\":\"string\",\"value\":\"First\"}]}," + - "\"right-operand\":{\"type\":\"unbounded-predicate\"," + - "\"operation\":\"eq\",\"term\":{\"type\":\"named-reference\",\"value\":\"c1\"}," + - "\"literals\":[{\"type\":\"string\",\"value\":\"Second\"}]}}," + - "\"projection\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; - - Assert.assertEquals(expectedMessage, bodyNode.toString()); - } - - @Test - public void testNotifyOnIncrementalScan() throws IOException { - Listeners.register(listener(), IncrementalScanEvent.class); - - String namespace = createNamespace(); - String tableName = getRandomName(); - createTable(namespace, tableName); - Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - - table.newAppend().appendFile(testDataFile).commit(); - table.newAppend().appendFile(testDataFile).commit(); - table.refresh(); - - Iterable snapshots = table.snapshots(); - table.newScan().appendsBetween( - Iterables.get(snapshots, 0).snapshotId(), - Iterables.get(snapshots, 1).snapshotId()) - .planFiles(); - - List messages = receiveAllMessages(queueUrl); - Assert.assertEquals(1, messages.size()); - - ObjectMapper objectMapper = new ObjectMapper(); - JsonNode bodyNode = objectMapper.readTree(messages.get(0).body()); - - String expectedMessage = "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"from-snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + - "\"to-snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + - "\"expression\":{\"type\":\"true\"}," + - "\"projection\":{\"type\":\"struct\"," + - "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}"; - - Assert.assertEquals(expectedMessage, bodyNode.toString()); - } - - @Test - public void testNotifyOnAllEvents() { - Listeners.register(listener(), CreateSnapshotEvent.class); - Listeners.register(listener(), ScanEvent.class); - Listeners.register(listener(), IncrementalScanEvent.class); - - String namespace = createNamespace(); - String tableName = getRandomName(); - createTable(namespace, tableName); - Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - - table.newAppend().appendFile(testDataFile).commit(); - table.newScan().planFiles(); - - table.newAppend().appendFile(testDataFile).commit(); - table.refresh(); - - Iterable snapshots = table.snapshots(); - table.newScan().appendsBetween( - Iterables.get(snapshots, 0).snapshotId(), - Iterables.get(snapshots, 1).snapshotId()) - .planFiles(); - - List messages = receiveAllMessages(queueUrl); - Assert.assertEquals(4, messages.size()); - - Set actualBodyNodesMessages = messages.stream() - .map(m -> { - try { - return JsonUtil.mapper().readTree(m.body()).toString(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }) - .collect(Collectors.toSet()); - - Set expectedBodyNodesMessages = Sets.newHashSet( - "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + - "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + - "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}", - "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + - "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + - "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"2\"," + - "\"total-files-size\":\"20\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\"," + - "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}", - "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"from-snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + - "\"to-snapshot-id\":" + Iterables.get(snapshots, 1).snapshotId() + "," + - "\"expression\":{\"type\":\"true\"},\"projection\":{\"type\":\"struct\"," + - "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c1\"," + - "\"required\":true,\"type\":\"string\",\"doc\":\"c1\"}]}}", - "{\"table-name\":\"" + "glue." + namespace + "." + tableName + "\"," + - "\"operation\":\"append\",\"snapshot-id\":" + Iterables.get(snapshots, 0).snapshotId() + "," + - "\"sequence-number\":0,\"summary\":{\"added-data-files\":\"1\",\"added-records\":\"1\"," + - "\"added-files-size\":\"10\",\"changed-partition-count\":\"1\",\"total-records\":\"1\"," + - "\"total-files-size\":\"10\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\"," + - "\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}}" - ); - - Assert.assertEquals(expectedBodyNodesMessages, actualBodyNodesMessages); - } - - public static List receiveAllMessages(String sqsUrl) { - ReceiveMessageRequest receiveMessageRequest = ReceiveMessageRequest.builder() - .queueUrl(sqsUrl) - .visibilityTimeout(100) - .waitTimeSeconds(2) - .maxNumberOfMessages(10) - .build(); - - List messages = Lists.newArrayList(); - messages.addAll(sqs.receiveMessage(receiveMessageRequest).messages()); - int prevCounter = -1; - while (prevCounter != messages.size()) { - prevCounter = messages.size(); - messages.addAll(sqs.receiveMessage(receiveMessageRequest).messages()); - } - - return messages; - } - - private Listener listener() { - return new SQSListener<>(queueUrl, sqs, RETRY, RETRY_INTERVAL_MS); - } -} diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java index 1114d077e74f..1d3a53dce627 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -31,8 +31,6 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; @@ -68,16 +66,6 @@ public DynamoDbClient dynamo() { return DynamoDbClient.builder().applyMutation(this::configure).build(); } - @Override - public SnsClient sns() { - return SnsClient.builder().applyMutation(this::configure).build(); - } - - @Override - public SqsClient sqs() { - return SqsClient.builder().applyMutation(this::configure).build(); - } - @Override public void initialize(Map properties) { roleArn = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_ARN); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 85b83e2cbedf..fdbc258c2413 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -35,8 +35,6 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sqs.SqsClient; public class AwsClientFactories { @@ -110,16 +108,6 @@ public DynamoDbClient dynamo() { return DynamoDbClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); } - @Override - public SnsClient sns() { - return SnsClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); - } - - @Override - public SqsClient sqs() { - return SqsClient.builder().httpClientBuilder(UrlConnectionHttpClient.builder()).build(); - } - @Override public void initialize(Map properties) { this.s3Endpoint = properties.get(AwsProperties.S3FILEIO_ENDPOINT); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java index 3b24c851b729..1ed5e6f5a9c0 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactory.java @@ -25,8 +25,6 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sqs.SqsClient; /** * Interface to customize AWS clients used by Iceberg. @@ -58,10 +56,6 @@ public interface AwsClientFactory extends Serializable { */ DynamoDbClient dynamo(); - SnsClient sns(); - - SqsClient sqs(); - /** * Initialize AWS client factory from catalog properties. * @param properties catalog properties diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index 76cd173dc2b4..fef845a0a005 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -221,18 +221,6 @@ public class AwsProperties implements Serializable { @Deprecated public static final boolean CLIENT_ENABLE_ETAG_CHECK_DEFAULT = false; - public static final String SQS_QUEUE_URL = "sqs.queue-url"; - public static final String SQS_RETRY = "sqs.retry"; - public static final String SQS_RETRY_INTERVAL_MS = "sqs.retryIntervalMs"; - public static final Integer SQS_RETRY_DEFAULT = 3; - public static final Integer SQS_RETRY_INTERVAL_MS_DEFAULT = 1000; - - public static final String SNS_TOPIC_ARN = "sns.topic-arn"; - public static final String SNS_RETRY = "sns.retry"; - public static final String SNS_RETRY_INTERVAL_MS = "sns.retryIntervalMs"; - public static final Integer SNS_RETRY_DEFAULT = 3; - public static final Integer SNS_RETRY_INTERVAL_MS_DEFAULT = 1000; - private String s3FileIoSseType; private String s3FileIoSseKey; private String s3FileIoSseMd5; diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index 01e6a75009a7..b9443e4a8164 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -117,7 +117,6 @@ public DynamoDbCatalog() { @Override public void initialize(String name, Map properties) { - super.initialize(name, properties); initialize( name, properties.get(CatalogProperties.WAREHOUSE_LOCATION), diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index e9cc0f346014..4dafcb9b4325 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -104,7 +104,6 @@ public GlueCatalog() { @Override public void initialize(String name, Map properties) { - super.initialize(name, properties); initialize( name, properties.get(CatalogProperties.WAREHOUSE_LOCATION), diff --git a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java deleted file mode 100644 index 31e1113288e5..000000000000 --- a/aws/src/main/java/org/apache/iceberg/aws/sns/SNSListener.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg.aws.sns; - -import java.util.Map; -import org.apache.iceberg.aws.AwsClientFactories; -import org.apache.iceberg.aws.AwsClientFactory; -import org.apache.iceberg.aws.AwsProperties; -import org.apache.iceberg.events.Listener; -import org.apache.iceberg.util.EventParser; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sns.model.NotFoundException; -import software.amazon.awssdk.services.sns.model.PublishRequest; -import software.amazon.awssdk.services.sns.model.SnsException; - -public class SNSListener implements Listener { - private static final Logger LOG = LoggerFactory.getLogger(SNSListener.class); - - private String topicArn; - private SnsClient sns; - private int retry; - private long retryIntervalMs; - - public SNSListener(Class clazz) { - } - - public SNSListener(String topicArn, SnsClient sns, int retry, long retryIntervalMs) { - this.sns = sns; - this.topicArn = topicArn; - this.retry = retry; - this.retryIntervalMs = retryIntervalMs; - } - - @Override - public void notify(T event) { - try { - String msg = EventParser.toJson(event); - PublishRequest request = PublishRequest.builder() - .message(msg) - .topicArn(topicArn) - .build(); - Tasks.foreach(request) - .exponentialBackoff(retryIntervalMs, retryIntervalMs, retryIntervalMs, 1 /* scale factor */) - .retry(retry) - .onlyRetryOn(NotFoundException.class) - .run(sns::publish); - } catch (SnsException e) { - LOG.error("Failed to send notification event to SNS topic", e); - } catch (RuntimeException e) { - LOG.error("Failed to notify subscriber", e); - } - } - - @Override - public void initialize(String listenerName, Map properties) { - AwsClientFactory factory = AwsClientFactories.from(properties); - this.sns = factory.sns(); - - if (listenerName == null) { - throw new NullPointerException("Listener Name cannot be null"); - } - - if (properties.get(AwsProperties.SNS_TOPIC_ARN) == null) { - throw new NullPointerException("SNS queue url cannot be null"); - } - - this.topicArn = properties.get(AwsProperties.SNS_TOPIC_ARN); - - this.retry = PropertyUtil.propertyAsInt( - properties, AwsProperties.SNS_RETRY, AwsProperties.SNS_RETRY_DEFAULT); - - this.retryIntervalMs = PropertyUtil.propertyAsInt( - properties, AwsProperties.SNS_RETRY_INTERVAL_MS, AwsProperties.SNS_RETRY_INTERVAL_MS_DEFAULT); - } -} - diff --git a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java b/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java deleted file mode 100644 index 9eb74086df8c..000000000000 --- a/aws/src/main/java/org/apache/iceberg/aws/sqs/SQSListener.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg.aws.sqs; - -import java.util.Map; -import org.apache.iceberg.aws.AwsClientFactories; -import org.apache.iceberg.aws.AwsClientFactory; -import org.apache.iceberg.aws.AwsProperties; -import org.apache.iceberg.events.Listener; -import org.apache.iceberg.util.EventParser; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.QueueDoesNotExistException; -import software.amazon.awssdk.services.sqs.model.SendMessageRequest; -import software.amazon.awssdk.services.sqs.model.SqsException; - -public class SQSListener implements Listener { - private static final Logger LOG = LoggerFactory.getLogger(SQSListener.class); - - private String queueUrl; - private SqsClient sqs; - private int retry; - private long retryIntervalMs; - - public SQSListener(Class clazz) { - } - - public SQSListener(String queueUrl, SqsClient sqs, int retry, long retryIntervalMs) { - this.sqs = sqs; - this.queueUrl = queueUrl; - this.retry = retry; - this.retryIntervalMs = retryIntervalMs; - } - - @Override - public void notify(Object event) { - try { - String msg = EventParser.toJson(event); - SendMessageRequest request = SendMessageRequest.builder() - .queueUrl(queueUrl) - .messageBody(msg) - .build(); - Tasks.foreach(request) - .exponentialBackoff(retryIntervalMs, retryIntervalMs, retryIntervalMs, 1 /* scale factor */) - .retry(retry) - .onlyRetryOn(QueueDoesNotExistException.class) - .run(sqs::sendMessage); - } catch (SqsException e) { - LOG.error("Failed to send notification event to SQS", e); - } catch (RuntimeException e) { - LOG.error("Failed to add to queue", e); - } - } - - @Override - public void initialize(String listenerName, Map properties) { - AwsClientFactory factory = AwsClientFactories.from(properties); - this.sqs = factory.sqs(); - - if (listenerName == null) { - throw new NullPointerException("Listener Name cannot be null"); - } - - if (properties.get(AwsProperties.SQS_QUEUE_URL) == null) { - throw new NullPointerException("SNS queue url cannot be null"); - } - - this.queueUrl = properties.get(AwsProperties.SQS_QUEUE_URL); - - this.retry = PropertyUtil.propertyAsInt( - properties, AwsProperties.SQS_RETRY, AwsProperties.SQS_RETRY_DEFAULT); - - this.retryIntervalMs = PropertyUtil.propertyAsInt( - properties, AwsProperties.SQS_RETRY_INTERVAL_MS, AwsProperties.SQS_RETRY_INTERVAL_MS_DEFAULT); - } -} - diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index ad0ece3e5631..86a10e491ae3 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -33,8 +33,6 @@ import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.kms.KmsClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.sns.SnsClient; -import software.amazon.awssdk.services.sqs.SqsClient; public class TestAwsClientFactories { @@ -110,16 +108,6 @@ public DynamoDbClient dynamo() { return null; } - @Override - public SnsClient sns() { - return null; - } - - @Override - public SqsClient sqs() { - return null; - } - @Override public void initialize(Map properties) { diff --git a/build.gradle b/build.gradle index edce810177bc..77b15e8eb08c 100644 --- a/build.gradle +++ b/build.gradle @@ -317,8 +317,6 @@ project(':iceberg-aws') { compileOnly 'software.amazon.awssdk:glue' compileOnly 'software.amazon.awssdk:sts' compileOnly 'software.amazon.awssdk:dynamodb' - compileOnly 'software.amazon.awssdk:sns' - compileOnly 'software.amazon.awssdk:sqs' compileOnly("org.apache.hadoop:hadoop-common") { exclude group: 'org.apache.avro', module: 'avro' diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index 0805cc3ff6ea..9cea03d2f1ae 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -68,11 +68,6 @@ public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { return new BaseMetastoreCatalogTableBuilder(identifier, schema); } - @Override - public void initialize(String name, Map properties) { - CatalogUtil.initializeListeners(properties); - } - private Table loadMetadataTable(TableIdentifier identifier) { String tableName = identifier.name(); MetadataTableType type = MetadataTableType.from(tableName); diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index 8a4c3457f282..d5daedee812b 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -19,16 +19,7 @@ package org.apache.iceberg; -import java.util.Optional; -import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.events.IncrementalScanEvent; -import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.util.Pair; public class CatalogProperties { @@ -90,54 +81,4 @@ private CatalogProperties() { public static final String APP_ID = "app-id"; public static final String USER = "user"; - /** - * Listeners are registered using catalog properties following the pattern of - * listeners.(listener-name).(listener-property)=(property-value) - *

- * A listener name cannot contain dot (.) character - * The specified listener is registered when a catalog is initialized - *

- * For example, there is the set of catalog properties registering an AWS SQS listener of name prod: - *

    - *
  • listener.prod.impl=org.apache.iceberg.aws.sns.SnsListener - *
  • listener.prod.event-types=scan,incremental-scan - *
  • listener.prod.sns.topic-arn=arn:aws:sns:us-east-2:123456789012:MyTopic - *
- */ - public static String listenerCatalogProperty(String listenerName, String listenerProperty) { - return "listener." + listenerName + "." + listenerProperty; - } - - /** - * Parse the listener name and listener property from a catalog property string - * @param listenerCatalogProperty listener catalog property - * @return a pair of the listener name and listener property - */ - public static Optional> parseListenerCatalogProperty(String listenerCatalogProperty) { - Matcher matcher = Pattern.compile("^listener[.](?[^\\.]+)[.](?.+)$") - .matcher(listenerCatalogProperty); - if (matcher.matches()) { - return Optional.of(Pair.of(matcher.group("name"), matcher.group("property"))); - } - - return Optional.empty(); - } - - /** - * Listener property describing the implementation Java class name of the listener for dynamic loading - */ - public static final String LISTENER_PROPERTY_IMPL = "impl"; - - /** - * Listener property describing the event types that a listener subscribes to. - * The value is a comma delimited list of event types (Java class name), - * e.g. org.apache.iceberg.events.ScanEvent,org.apache.iceberg.events.IncrementalScanEvent. - * If not specified, the listener subscribes to events listed in {@link #LISTENER_EVENT_TYPES_DEFAULT} - */ - public static final String LISTENER_PROPERTY_EVENT_TYPES = "event-types"; - public static final Set> LISTENER_EVENT_TYPES_DEFAULT = ImmutableSet.of( - ScanEvent.class, - IncrementalScanEvent.class, - CreateSnapshotEvent.class - ); } diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 562fc92f58f9..07b69c17fa82 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -20,30 +20,22 @@ package org.apache.iceberg; import java.io.IOException; -import java.util.Arrays; import java.util.Locale; import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; -import org.apache.iceberg.events.Listener; -import org.apache.iceberg.events.Listeners; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -283,74 +275,6 @@ public static FileIO loadFileIO( return fileIO; } - public static void initializeListeners(Map properties) { - Map> propertiesSummary = Maps.newHashMap(); - Map commonProperties = Maps.newHashMap(); - for (String key : properties.keySet()) { - Optional> listenerInfo = CatalogProperties.parseListenerCatalogProperty(key); - if (listenerInfo.isPresent()) { - String name = listenerInfo.get().first(); - String property = listenerInfo.get().second(); - propertiesSummary.computeIfAbsent(name, k -> Maps.newHashMap()); - propertiesSummary.get(name).put(property, properties.get(key)); - } else { - commonProperties.put(key, properties.get(key)); - } - } - - // inherit all common properties during listener initialization - propertiesSummary.forEach((k, v) -> v.putAll(commonProperties)); - - for (String listenerName : propertiesSummary.keySet()) { - Map listenerProperties = propertiesSummary.get(listenerName); - String listenerImpl = listenerProperties.get(CatalogProperties.LISTENER_PROPERTY_IMPL); - ValidationException.check(listenerImpl != null, - "Cannot initialize listener %s, missing %s property", - listenerName, CatalogProperties.LISTENER_PROPERTY_IMPL); - - String eventTypesString = listenerProperties.get(CatalogProperties.LISTENER_PROPERTY_EVENT_TYPES); - Set> eventTypes = eventTypesString != null ? Arrays.stream(eventTypesString.split(",")) - .map(s -> { - try { - return Class.forName(s); - } catch (ClassNotFoundException e) { - throw new ValidationException(e, "Cannot find listener event type class %s", s); - } - }) - .collect(Collectors.toSet()) : CatalogProperties.LISTENER_EVENT_TYPES_DEFAULT; - - eventTypes.forEach(t -> CatalogUtil.loadAndRegisterListener(listenerImpl, listenerName, t, listenerProperties)); - } - } - - @VisibleForTesting - @SuppressWarnings("GetClassOnClass") - static Listener loadAndRegisterListener( - String listenerClass, - String listenerName, - Class eventType, - Map properties) { - DynConstructors.Ctor> ctor; - try { - ctor = DynConstructors.builder(Listener.class).impl(listenerClass, eventType.getClass()).buildChecked(); - } catch (NoSuchMethodException e) { - throw new IllegalArgumentException(String.format( - "Cannot initialize Listener, missing no-arg constructor: %s", listenerClass), e); - } - - Listener listener; - try { - listener = ctor.newInstance(eventType); - } catch (ClassCastException e) { - throw new IllegalArgumentException(String.format( - "Cannot initialize Listener, %s does not implement org.apache.iceberg.events.Listener", listenerClass), e); - } - - listener.initialize(listenerName, properties); - Listeners.register(listener, eventType); - return listener; - } - /** * Dynamically detects whether an object is a Hadoop Configurable and calls setConf. * @param maybeConfigurable an object that may be Configurable diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java index 0adc65155653..78224e27bb0a 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java @@ -98,7 +98,6 @@ public HadoopCatalog() { @Override public void initialize(String name, Map properties) { - super.initialize(name, properties); String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); Preconditions.checkArgument(inputWarehouseLocation != null && !inputWarehouseLocation.equals(""), "Cannot instantiate hadoop catalog. No location provided for warehouse (Set warehouse config)"); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 037bad3597d3..549c0cf7e34a 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -78,7 +78,6 @@ public JdbcCatalog() { @Override public void initialize(String name, Map properties) { - super.initialize(name, properties); String uri = properties.get(CatalogProperties.URI); Preconditions.checkNotNull(uri, "JDBC connection URI is required"); diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index fbecdcd1833c..2295b771dd28 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -21,16 +21,11 @@ import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.events.Listener; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; @@ -175,31 +170,6 @@ public void buildCustomCatalog_withTypeSet() { "both type and catalog-impl are set", () -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)); } - @Test - public void testSingleRegEx() { - Pattern pattern = Pattern.compile("^listeners[.](?.+)[.]impl$"); - Matcher matchTrue = pattern.matcher("listeners.prod.impl"); - Matcher matchFalse = pattern.matcher("listeners.prod.iampl"); - Assert.assertTrue(matchTrue.matches()); - Assert.assertFalse(matchFalse.matches()); - Assert.assertEquals("prod", matchTrue.group("name")); - } - - @Test - public void testLoadListener() { - Map properties = Maps.newHashMap(); - String listenerName = "ListenerName"; - properties.put("impl", TestListener.class.getName()); - properties.put("test.client", "Client-Info"); - properties.put("test.info", "Information"); - - Listener listener = CatalogUtil.loadAndRegisterListener( - TestListener.class.getName(), listenerName, ScanEvent.class, properties); - Assertions.assertThat(listener).isInstanceOf(TestListener.class); - Assert.assertEquals("Client-Info", ((TestListener) listener).client); - Assert.assertEquals("Information", ((TestListener) listener).info); - } - public static class TestCatalog extends BaseMetastoreCatalog { private String catalogName; @@ -210,7 +180,6 @@ public TestCatalog() { @Override public void initialize(String name, Map properties) { - super.initialize(name, properties); this.catalogName = name; this.flinkOptions = properties; } @@ -433,43 +402,4 @@ public static class TestFileIONotImpl { public TestFileIONotImpl() { } } - - public static class TestListener implements Listener { - private String client; - private String info; - private String name; - public static final AtomicInteger NOTIFY_TIMES = new AtomicInteger(); - - public TestListener(Class clazz) { - } - - @Override - public void notify(Object event) { - NOTIFY_TIMES.incrementAndGet(); - } - - @Override - public void initialize(String listenerName, Map properties) { - this.name = listenerName; - this.info = properties.get("test.info"); - this.client = properties.get("test.client"); - } - } - - public static class TestListenerBadConstructor implements Listener { - private String arg; - - public TestListenerBadConstructor(String arg) { - this.arg = arg; - } - - @Override - public void notify(Object event) { - } - - @Override - public void initialize(String listenerName, Map properties) { - this.arg = listenerName; - } - } } 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 f71b3908bc4b..0d1ba8e34bb1 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 @@ -78,7 +78,6 @@ public HiveCatalog() { @Override public void initialize(String inputName, Map properties) { - super.initialize(name, properties); this.name = inputName; if (conf == null) { LOG.warn("No Hadoop Configuration was set, using the default environment Configuration"); diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java index 8d379423f6c8..b558c276daaa 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java @@ -93,7 +93,6 @@ public NessieCatalog() { @Override public void initialize(String inputName, Map options) { - super.initialize(name, options); this.catalogOptions = ImmutableMap.copyOf(options); String fileIOImpl = options.get(CatalogProperties.FILE_IO_IMPL); this.fileIO = fileIOImpl == null ? new HadoopFileIO(config) : CatalogUtil.loadFileIO(fileIOImpl, options, config); From d990df1b767cdc0699a99a4e6cdca7c7ea763ebc Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Fri, 8 Apr 2022 12:29:12 -0700 Subject: [PATCH 17/19] Updated Parser Made Parser function private that aren't called outside the classes and changed Operation type to "operation" instead of "type" --- .../iceberg/expressions/ExpressionParser.java | 107 +++++++----------- .../org/apache/iceberg/util/EventParser.java | 18 +-- .../expressions/TestExpressionParser.java | 44 +++---- 3 files changed, 76 insertions(+), 93 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 5d507901aff1..ffc6641602ae 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -35,7 +35,6 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -50,20 +49,10 @@ public class ExpressionParser { private static final String LEFT_OPERAND = "left-operand"; private static final String RIGHT_OPERAND = "right-operand"; private static final String OPERAND = "operand"; - private static final String AND = "and"; - private static final String OR = "or"; - private static final String NOT = "not"; - private static final String TRUE = "true"; - private static final String FALSE = "false"; - private static final String UNBOUNDED_PREDICATE = "unbounded-predicate"; - private static final String BOUNDED_LITERAL_PREDICATE = "bounded-literal-predicate"; - private static final String BOUNDED_SET_PREDICATE = "bounded-set-predicate"; - private static final String BOUNDED_UNARY_PREDICATE = "bounded-unary-predicate"; - private static final Set PREDICATE_TYPES = Sets.newHashSet( - UNBOUNDED_PREDICATE, - BOUNDED_LITERAL_PREDICATE, - BOUNDED_SET_PREDICATE, - BOUNDED_UNARY_PREDICATE); + private static final String UNBOUND_PREDICATE = "unbound-predicate"; + private static final String BOUND_LITERAL_PREDICATE = "bound-literal-predicate"; + private static final String BOUND_SET_PREDICATE = "bound-set-predicate"; + private static final String BOUND_UNARY_PREDICATE = "bound-unary-predicate"; private static final String NAMED_REFERENCE = "named-reference"; private static final String BOUND_REFERENCE = "bound-reference"; @@ -110,9 +99,9 @@ public static void toJson(Expression expression, JsonGenerator generator) throws } } - public static void toJson(And expression, JsonGenerator generator) throws IOException { + private static void toJson(And expression, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, AND); + generator.writeStringField(OPERATION, Expression.Operation.AND.name().toLowerCase()); generator.writeFieldName(LEFT_OPERAND); toJson(expression.left(), generator); generator.writeFieldName(RIGHT_OPERAND); @@ -120,9 +109,9 @@ public static void toJson(And expression, JsonGenerator generator) throws IOExce generator.writeEndObject(); } - public static void toJson(Or expression, JsonGenerator generator) throws IOException { + private static void toJson(Or expression, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, OR); + generator.writeStringField(OPERATION, Expression.Operation.OR.name().toLowerCase()); generator.writeFieldName(LEFT_OPERAND); toJson(expression.left(), generator); generator.writeFieldName(RIGHT_OPERAND); @@ -130,27 +119,27 @@ public static void toJson(Or expression, JsonGenerator generator) throws IOExcep generator.writeEndObject(); } - public static void toJson(Not expression, JsonGenerator generator) throws IOException { + private static void toJson(Not expression, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, NOT); + generator.writeStringField(OPERATION, Expression.Operation.NOT.name().toLowerCase()); generator.writeFieldName(OPERAND); toJson(expression.child(), generator); generator.writeEndObject(); } - public static void toJson(True expression, JsonGenerator generator) throws IOException { + private static void toJson(True expression, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, TRUE); + generator.writeStringField(OPERATION, Expression.Operation.TRUE.name().toLowerCase()); generator.writeEndObject(); } - public static void toJson(False expression, JsonGenerator generator) throws IOException { + private static void toJson(False expression, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, FALSE); + generator.writeStringField(OPERATION, Expression.Operation.FALSE.name().toLowerCase()); generator.writeEndObject(); } - public static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { + private static void toJson(Predicate predicate, JsonGenerator generator) throws IOException { if (predicate instanceof UnboundPredicate) { toJson((UnboundPredicate) predicate, generator); } else { @@ -158,13 +147,13 @@ public static void toJson(Predicate predicate, JsonGenerator generator) th } } - public static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { + private static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, UNBOUNDED_PREDICATE); + generator.writeStringField(TYPE, UNBOUND_PREDICATE); generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); generator.writeFieldName(TERM); toJson(predicate.term(), generator); - if (!(ONE_INPUTS.contains(predicate.op()))) { + if (!ONE_INPUTS.contains(predicate.op())) { generator.writeFieldName(LITERALS); generator.writeStartArray(); for (Literal literal : predicate.literals()) { @@ -176,7 +165,7 @@ public static void toJson(UnboundPredicate predicate, JsonGenerator generator generator.writeEndObject(); } - public static void toJson(Term term, JsonGenerator generator) throws IOException { + private static void toJson(Term term, JsonGenerator generator) throws IOException { if (term instanceof NamedReference) { toJson((NamedReference) term, generator); } else { @@ -184,22 +173,14 @@ public static void toJson(Term term, JsonGenerator generator) throws IOException } } - public static void toJson(NamedReference term, JsonGenerator generator) throws IOException { + private static void toJson(NamedReference term, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeStringField(TYPE, NAMED_REFERENCE); generator.writeStringField(VALUE, term.name()); generator.writeEndObject(); } - public static void toJson(List> literals, JsonGenerator generator) throws IOException { - generator.writeStartArray(); - for (int i = 0; i < literals.size(); i++) { - toJson(literals.get(i), generator); - } - generator.writeEndArray(); - } - - public static void toJson(Literal literal, JsonGenerator generator) throws IOException { + private static void toJson(Literal literal, JsonGenerator generator) throws IOException { generator.writeStartObject(); Object value = literal.value(); @@ -248,44 +229,44 @@ public static Expression fromJson(String json) { } public static Expression fromJson(JsonNode json) { - String expressionType = JsonUtil.getString(TYPE, json); + String expressionType; + if (json.hasNonNull(TYPE)) { + expressionType = JsonUtil.getString(TYPE, json); + } else if (json.hasNonNull(OPERATION)) { + expressionType = JsonUtil.getString(OPERATION, json); + } else { + return null; + } - if (AND.equals(expressionType)) { + if (Expression.Operation.AND.name().toLowerCase().equals(expressionType)) { return new And(fromJson(json.get(LEFT_OPERAND)), fromJson(json.get(RIGHT_OPERAND))); - } else if (OR.equals(expressionType)) { + } else if (Expression.Operation.OR.name().toLowerCase().equals(expressionType)) { return new Or(fromJson(json.get(LEFT_OPERAND)), fromJson(json.get(RIGHT_OPERAND))); - } else if (NOT.equals(expressionType)) { + } else if (Expression.Operation.NOT.name().toLowerCase().equals(expressionType)) { return new Not(fromJson(json.get(OPERAND))); - } else if (TRUE.equals(expressionType)) { + } else if (Expression.Operation.TRUE.name().toLowerCase().equals(expressionType)) { return True.INSTANCE; - } else if (FALSE.equals(expressionType)) { + } else if (Expression.Operation.FALSE.name().toLowerCase().equals(expressionType)) { return False.INSTANCE; - } else if (PREDICATE_TYPES.contains(expressionType)) { - return fromJsonToPredicate(json, expressionType); } else { - throw new IllegalArgumentException("Invalid Operation Type"); + return fromJsonToPredicate(json, expressionType); } } - public static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { - if (UNBOUNDED_PREDICATE.equals(predicateType)) { + private static Predicate fromJsonToPredicate(JsonNode json, String predicateType) { + if (UNBOUND_PREDICATE.equals(predicateType)) { return fromJsonUnboundPredicate(json); - } else if (BOUNDED_LITERAL_PREDICATE.equals(predicateType)) { - throw new UnsupportedOperationException( - "Serialization of Predicate type BoundLiteralPredicate is not currently supported."); - } else if (BOUNDED_SET_PREDICATE.equals(predicateType)) { - throw new UnsupportedOperationException( - "Serialization of Predicate type BoundSetPredicate is not currently supported."); - } else if (BOUNDED_UNARY_PREDICATE.equals(predicateType)) { + } else if (BOUND_LITERAL_PREDICATE.equals(predicateType) || + BOUND_SET_PREDICATE.equals(predicateType) || + BOUND_UNARY_PREDICATE.equals(predicateType)) { throw new UnsupportedOperationException( - "Serialization of Predicate type BoundUnaryPredicate is not currently supported."); + "Serialization of Bound Predicates is not currently supported."); } else { throw new IllegalArgumentException("Invalid Predicate Type"); } } - @SuppressWarnings("CyclomaticComplexity") - public static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { + private static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { Expression.Operation operation = Expression.Operation.valueOf( JsonUtil.getString(OPERATION, json).toUpperCase(Locale.ENGLISH)); @@ -299,7 +280,7 @@ public static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { } } - public static UnboundTerm fromJsonToTerm(JsonNode json) { + private static UnboundTerm fromJsonToTerm(JsonNode json) { String referenceType = json.get(TYPE).textValue(); if (referenceType.equals(NAMED_REFERENCE)) { @@ -312,7 +293,7 @@ public static UnboundTerm fromJsonToTerm(JsonNode json) { } } - public static List fromJsonToLiteralValues(JsonNode json) { + private static List fromJsonToLiteralValues(JsonNode json) { List literals = Lists.newArrayList(); for (int i = 0; i < json.size(); i++) { String literalType = json.get(i).get(TYPE).textValue(); diff --git a/core/src/main/java/org/apache/iceberg/util/EventParser.java b/core/src/main/java/org/apache/iceberg/util/EventParser.java index 4e79457d9ed7..3a75406cc2b0 100644 --- a/core/src/main/java/org/apache/iceberg/util/EventParser.java +++ b/core/src/main/java/org/apache/iceberg/util/EventParser.java @@ -37,7 +37,6 @@ public class EventParser { private static final String EVENT_TYPE = "event-type"; - private static final String TABLE_NAME = "table-name"; private static final String SNAPSHOT_ID = "snapshot-id"; private static final String PROJECTION = "projection"; @@ -62,12 +61,15 @@ public static String toJson(Object event, boolean pretty) { if (pretty) { generator.useDefaultPrettyPrinter(); } + if (event instanceof ScanEvent) { toJson((ScanEvent) event, generator); } else if (event instanceof CreateSnapshotEvent) { toJson((CreateSnapshotEvent) event, generator); } else if (event instanceof IncrementalScanEvent) { toJson((IncrementalScanEvent) event, generator); + } else { + return "Unable to Serialize Event " + event; } generator.flush(); @@ -77,7 +79,7 @@ public static String toJson(Object event, boolean pretty) { } } - public static void toJson(ScanEvent event, JsonGenerator generator) throws IOException { + private static void toJson(ScanEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeFieldName(EVENT_TYPE); generator.writeString(event.getClass().getName()); @@ -92,7 +94,7 @@ public static void toJson(ScanEvent event, JsonGenerator generator) throws IOExc generator.writeEndObject(); } - public static void toJson(CreateSnapshotEvent event, JsonGenerator generator) throws IOException { + private static void toJson(CreateSnapshotEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeFieldName(EVENT_TYPE); generator.writeString(event.getClass().getName()); @@ -113,7 +115,7 @@ public static void toJson(CreateSnapshotEvent event, JsonGenerator generator) th generator.writeEndObject(); } - public static void toJson(IncrementalScanEvent event, JsonGenerator generator) throws IOException { + private static void toJson(IncrementalScanEvent event, JsonGenerator generator) throws IOException { generator.writeStartObject(); generator.writeFieldName(EVENT_TYPE); generator.writeString(event.getClass().getName()); @@ -139,7 +141,7 @@ public static Object fromJson(String json) { } } - public static Object fromJson(JsonNode json) { + private static Object fromJson(JsonNode json) { String eventType = JsonUtil.getString(EVENT_TYPE, json); if (eventType.equals(ScanEvent.class.getName())) { return fromJsonToScanEvent(json); @@ -152,7 +154,7 @@ public static Object fromJson(JsonNode json) { } } - public static ScanEvent fromJsonToScanEvent(JsonNode json) { + private static ScanEvent fromJsonToScanEvent(JsonNode json) { String tableName = JsonUtil.getString(TABLE_NAME, json); Long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, json); Expression filter = ExpressionParser.fromJson(json.get(EXPRESSION)); @@ -160,7 +162,7 @@ public static ScanEvent fromJsonToScanEvent(JsonNode json) { return new ScanEvent(tableName, snapshotId, filter, schema); } - public static CreateSnapshotEvent fromJsonToCreateSnapshotEvent(JsonNode json) { + private static CreateSnapshotEvent fromJsonToCreateSnapshotEvent(JsonNode json) { String tableName = JsonUtil.getString(TABLE_NAME, json); String operation = JsonUtil.getString(OPERATION, json); Long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, json); @@ -169,7 +171,7 @@ public static CreateSnapshotEvent fromJsonToCreateSnapshotEvent(JsonNode json) { return new CreateSnapshotEvent(tableName, operation, snapshotId, sequenceNumber, summary); } - public static IncrementalScanEvent fromJsonToIncrementalScanEvent(JsonNode json) { + private static IncrementalScanEvent fromJsonToIncrementalScanEvent(JsonNode json) { String tableName = JsonUtil.getString(TABLE_NAME, json); Long fromSnapshotId = JsonUtil.getLong(FROM_SNAPSHOT_ID, json); Long toSnapshotId = JsonUtil.getLong(TO_SNAPSHOT_ID, json); diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index 0d5bdf05d006..ca7b845d2ff1 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -30,7 +30,7 @@ public class TestExpressionParser { @Test public void testPredicate() { String expected = "{\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -54,9 +54,9 @@ public void testPredicate() { @Test public void testAnd() { String expected = "{\n" + - " \"type\" : \"and\",\n" + + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"gt_eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -68,7 +68,7 @@ public void testAnd() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -102,9 +102,9 @@ public void testAnd() { @Test public void testOr() { String expected = "{\n" + - " \"type\" : \"or\",\n" + + " \"operation\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -116,7 +116,7 @@ public void testOr() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"not_null\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -142,9 +142,9 @@ public void testOr() { @Test public void testNot() { String expected = "{\n" + - " \"type\" : \"not\",\n" + + " \"operation\" : \"not\",\n" + " \"operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -170,11 +170,11 @@ public void testNot() { @Test public void testNestedExpression() { String expected = "{\n" + - " \"type\" : \"or\",\n" + + " \"operation\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"and\",\n" + + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -186,7 +186,7 @@ public void testNestedExpression() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -199,7 +199,7 @@ public void testNestedExpression() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -229,13 +229,13 @@ public void testNestedExpression() { } @Test - public void testParserBothWays() { + public void testParserRoundTrip() { String expected = "{\n" + - " \"type\" : \"or\",\n" + + " \"operation\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"and\",\n" + + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -247,7 +247,7 @@ public void testParserBothWays() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -260,7 +260,7 @@ public void testParserBothWays() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -278,7 +278,7 @@ public void testParserBothWays() { @Test public void testFixedLiteral() { String expected = "{\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + @@ -311,7 +311,7 @@ public void testFixedLiteral() { @Test public void testDecimalLiteral() { String expected = "{\n" + - " \"type\" : \"unbounded-predicate\",\n" + + " \"type\" : \"unbound-predicate\",\n" + " \"operation\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + From 2bde5a8e0fac29e31702642581d335726dedad69 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Mon, 11 Apr 2022 00:17:46 -0700 Subject: [PATCH 18/19] Updated Bound Predicates Json Serialization --- .../iceberg/expressions/ExpressionParser.java | 19 +++--- .../expressions/TestExpressionParser.java | 58 +++++++++---------- 2 files changed, 37 insertions(+), 40 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index ffc6641602ae..13ee02a60ae0 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.List; -import java.util.Locale; import java.util.Set; import java.util.UUID; import org.apache.iceberg.exceptions.RuntimeIOException; @@ -149,8 +148,8 @@ private static void toJson(Predicate predicate, JsonGenerator generator) t private static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, UNBOUND_PREDICATE); - generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); + generator.writeStringField(OPERATION, UNBOUND_PREDICATE); + generator.writeStringField(TYPE, predicate.op().name().toLowerCase()); generator.writeFieldName(TERM); toJson(predicate.term(), generator); if (!ONE_INPUTS.contains(predicate.op())) { @@ -230,9 +229,7 @@ public static Expression fromJson(String json) { public static Expression fromJson(JsonNode json) { String expressionType; - if (json.hasNonNull(TYPE)) { - expressionType = JsonUtil.getString(TYPE, json); - } else if (json.hasNonNull(OPERATION)) { + if (json.hasNonNull(OPERATION)) { expressionType = JsonUtil.getString(OPERATION, json); } else { return null; @@ -267,14 +264,14 @@ public static Expression fromJson(JsonNode json) { } private static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { - Expression.Operation operation = Expression.Operation.valueOf( - JsonUtil.getString(OPERATION, json).toUpperCase(Locale.ENGLISH)); + Expression.Operation operationType = Expression.Operation.valueOf( + JsonUtil.getString(TYPE, json).toUpperCase()); - if (ONE_INPUTS.contains(operation)) { - return new UnboundPredicate<>(operation, fromJsonToTerm(json.get(TERM))); + if (ONE_INPUTS.contains(operationType)) { + return new UnboundPredicate<>(operationType, fromJsonToTerm(json.get(TERM))); } else { return new UnboundPredicate( - operation, + operationType, fromJsonToTerm(json.get(TERM)), fromJsonToLiteralValues(json.get(LITERALS))); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index ca7b845d2ff1..2a548793368c 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -30,8 +30,8 @@ public class TestExpressionParser { @Test public void testPredicate() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" + @@ -56,8 +56,8 @@ public void testAnd() { String expected = "{\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"gt_eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"gt_eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -68,8 +68,8 @@ public void testAnd() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -104,8 +104,8 @@ public void testOr() { String expected = "{\n" + " \"operation\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"lt\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -116,8 +116,8 @@ public void testOr() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"not_null\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"not_null\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -144,8 +144,8 @@ public void testNot() { String expected = "{\n" + " \"operation\" : \"not\",\n" + " \"operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"lt\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -174,8 +174,8 @@ public void testNestedExpression() { " \"left-operand\" : {\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -186,8 +186,8 @@ public void testNestedExpression() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -199,8 +199,8 @@ public void testNestedExpression() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"is_nan\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column3-Name\"\n" + @@ -235,8 +235,8 @@ public void testParserRoundTrip() { " \"left-operand\" : {\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -247,8 +247,8 @@ public void testParserRoundTrip() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -260,8 +260,8 @@ public void testParserRoundTrip() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"is_nan\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column3-Name\"\n" + @@ -271,15 +271,15 @@ public void testParserRoundTrip() { Expression actualExpression = ExpressionParser.fromJson(expected); String actualJsonExpression = ExpressionParser.toJson(actualExpression, true); - + System.out.println(expected); Assert.assertEquals(expected, actualJsonExpression); } @Test public void testFixedLiteral() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" + @@ -311,8 +311,8 @@ public void testFixedLiteral() { @Test public void testDecimalLiteral() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" + From 636c9b89d854356a9bcd52b44b834d79e8f34443 Mon Sep 17 00:00:00 2001 From: kunal0829 Date: Mon, 11 Apr 2022 00:17:46 -0700 Subject: [PATCH 19/19] Updated Unbound Predicates Json Serialization --- .../iceberg/expressions/ExpressionParser.java | 19 +++--- .../expressions/TestExpressionParser.java | 58 +++++++++---------- 2 files changed, 37 insertions(+), 40 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index ffc6641602ae..13ee02a60ae0 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.List; -import java.util.Locale; import java.util.Set; import java.util.UUID; import org.apache.iceberg.exceptions.RuntimeIOException; @@ -149,8 +148,8 @@ private static void toJson(Predicate predicate, JsonGenerator generator) t private static void toJson(UnboundPredicate predicate, JsonGenerator generator) throws IOException { generator.writeStartObject(); - generator.writeStringField(TYPE, UNBOUND_PREDICATE); - generator.writeStringField(OPERATION, predicate.op().name().toLowerCase()); + generator.writeStringField(OPERATION, UNBOUND_PREDICATE); + generator.writeStringField(TYPE, predicate.op().name().toLowerCase()); generator.writeFieldName(TERM); toJson(predicate.term(), generator); if (!ONE_INPUTS.contains(predicate.op())) { @@ -230,9 +229,7 @@ public static Expression fromJson(String json) { public static Expression fromJson(JsonNode json) { String expressionType; - if (json.hasNonNull(TYPE)) { - expressionType = JsonUtil.getString(TYPE, json); - } else if (json.hasNonNull(OPERATION)) { + if (json.hasNonNull(OPERATION)) { expressionType = JsonUtil.getString(OPERATION, json); } else { return null; @@ -267,14 +264,14 @@ public static Expression fromJson(JsonNode json) { } private static UnboundPredicate fromJsonUnboundPredicate(JsonNode json) { - Expression.Operation operation = Expression.Operation.valueOf( - JsonUtil.getString(OPERATION, json).toUpperCase(Locale.ENGLISH)); + Expression.Operation operationType = Expression.Operation.valueOf( + JsonUtil.getString(TYPE, json).toUpperCase()); - if (ONE_INPUTS.contains(operation)) { - return new UnboundPredicate<>(operation, fromJsonToTerm(json.get(TERM))); + if (ONE_INPUTS.contains(operationType)) { + return new UnboundPredicate<>(operationType, fromJsonToTerm(json.get(TERM))); } else { return new UnboundPredicate( - operation, + operationType, fromJsonToTerm(json.get(TERM)), fromJsonToLiteralValues(json.get(LITERALS))); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index ca7b845d2ff1..2a548793368c 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -30,8 +30,8 @@ public class TestExpressionParser { @Test public void testPredicate() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" + @@ -56,8 +56,8 @@ public void testAnd() { String expected = "{\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"gt_eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"gt_eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -68,8 +68,8 @@ public void testAnd() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -104,8 +104,8 @@ public void testOr() { String expected = "{\n" + " \"operation\" : \"or\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"lt\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -116,8 +116,8 @@ public void testOr() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"not_null\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"not_null\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -144,8 +144,8 @@ public void testNot() { String expected = "{\n" + " \"operation\" : \"not\",\n" + " \"operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"lt\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"lt\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -174,8 +174,8 @@ public void testNestedExpression() { " \"left-operand\" : {\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -186,8 +186,8 @@ public void testNestedExpression() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -199,8 +199,8 @@ public void testNestedExpression() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"is_nan\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column3-Name\"\n" + @@ -235,8 +235,8 @@ public void testParserRoundTrip() { " \"left-operand\" : {\n" + " \"operation\" : \"and\",\n" + " \"left-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column1-Name\"\n" + @@ -247,8 +247,8 @@ public void testParserRoundTrip() { " } ]\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column2-Name\"\n" + @@ -260,8 +260,8 @@ public void testParserRoundTrip() { " }\n" + " },\n" + " \"right-operand\" : {\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"is_nan\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"is_nan\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column3-Name\"\n" + @@ -271,15 +271,15 @@ public void testParserRoundTrip() { Expression actualExpression = ExpressionParser.fromJson(expected); String actualJsonExpression = ExpressionParser.toJson(actualExpression, true); - + System.out.println(expected); Assert.assertEquals(expected, actualJsonExpression); } @Test public void testFixedLiteral() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"eq\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"eq\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" + @@ -311,8 +311,8 @@ public void testFixedLiteral() { @Test public void testDecimalLiteral() { String expected = "{\n" + - " \"type\" : \"unbound-predicate\",\n" + - " \"operation\" : \"in\",\n" + + " \"operation\" : \"unbound-predicate\",\n" + + " \"type\" : \"in\",\n" + " \"term\" : {\n" + " \"type\" : \"named-reference\",\n" + " \"value\" : \"Column-Name\"\n" +