You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/01 09:53:59 UTC

[GitHub] [iceberg] openinx opened a new pull request #1153: Flink: Add Avro value reader, writer implementations

openinx opened a new pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449234842



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {
+  private static final int NUM_RECORDS = 20_000;

Review comment:
       How many records have you tested with?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449649097



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java
##########
@@ -0,0 +1,570 @@
+/*
+ * 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.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestRowProjection {

Review comment:
       Looks good. We may want to also apply these cases to ORC and Parquet also.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449235870



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {

Review comment:
       The implementation looks fine since it relies on the parent class, but this would help catch regressions in the future.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449235477



##########
File path: core/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
##########
@@ -50,13 +50,17 @@ public void setSchema(Schema schema) {
     this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new WriteBuilder());
   }
 
+  protected void setWriter(ValueWriter<T> writer) {
+    this.writer = writer;

Review comment:
       Should `setWriter` be used in `setSchema` above for this class as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449349952



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {

Review comment:
       Thanks for reminding,  will address this in the next patch.

##########
File path: core/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
##########
@@ -50,13 +50,17 @@ public void setSchema(Schema schema) {
     this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new WriteBuilder());
   }
 
+  protected void setWriter(ValueWriter<T> writer) {
+    this.writer = writer;

Review comment:
       Fine,  let address it in the next patch. Thanks.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {
+  private static final int NUM_RECORDS = 20_000;

Review comment:
       I ran 1_000_000 rows under my host, it all passed.  Seems works fine.

##########
File path: core/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
##########
@@ -50,13 +50,17 @@ public void setSchema(Schema schema) {
     this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new WriteBuilder());
   }
 
+  protected void setWriter(ValueWriter<T> writer) {
+    this.writer = writer;
+  }
+
   @Override
   public void write(T datum, Encoder out) throws IOException {
     writer.write(datum, out);
   }
 
-  private static class WriteBuilder extends AvroSchemaVisitor<ValueWriter<?>> {
-    private WriteBuilder() {
+  protected static class WriteBuilder extends AvroSchemaVisitor<ValueWriter<?>> {
+    protected WriteBuilder() {

Review comment:
       The current `FlinkAvroWriter#WriterBuilder` is extending the `DataWriter#WriteBuilder `, so I marked this constructor as `protected` before, seems I could also abstract the generic codes in a `BaseDataWriter` so that we don't need to expose the `WriterBuilder` to its subclass then. (similar with the https://github.com/apache/iceberg/commit/41cc1334bfe50c7dccb58baeb664b236732f0e2b).  let me address this in the next patch.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449475298



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {

Review comment:
       @rdblue I've addressed this by porting the unit test in `TestGenericReadProjection` to flink avro reader writer. Thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449234566



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
+
+public class TestFlinkAvroReaderWriter {

Review comment:
       To make sure the projection is done correctly in the record reader, can you add a projection test? Here's the one we use for Iceberg generics and Avro: https://github.com/apache/iceberg/blob/master/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449236225



##########
File path: core/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
##########
@@ -50,13 +50,17 @@ public void setSchema(Schema schema) {
     this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new WriteBuilder());
   }
 
+  protected void setWriter(ValueWriter<T> writer) {
+    this.writer = writer;
+  }
+
   @Override
   public void write(T datum, Encoder out) throws IOException {
     writer.write(datum, out);
   }
 
-  private static class WriteBuilder extends AvroSchemaVisitor<ValueWriter<?>> {
-    private WriteBuilder() {
+  protected static class WriteBuilder extends AvroSchemaVisitor<ValueWriter<?>> {
+    protected WriteBuilder() {

Review comment:
       Can we remove the constructor since it is empty?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#discussion_r449398538



##########
File path: core/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
##########
@@ -50,13 +50,17 @@ public void setSchema(Schema schema) {
     this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new WriteBuilder());
   }
 
+  protected void setWriter(ValueWriter<T> writer) {
+    this.writer = writer;

Review comment:
       The `setWriter` can be removed now in the new patch because we don't need to expose it in new way. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#issuecomment-653194140


   Looks good overall. Mainly, I think we should have this run through the projection test to ensure we don't have future regressions.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1153: Flink: Add Avro value reader, writer implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1153:
URL: https://github.com/apache/iceberg/pull/1153#issuecomment-653611751


   Looks good, I'll merge this. Thanks @openinx!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org