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/12/16 22:34:57 UTC

[GitHub] [iceberg] yyanyy opened a new pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

yyanyy opened a new pull request #1946:
URL: https://github.com/apache/iceberg/pull/1946


   This change is a smaller PR broken down from #1935. There is no change in behavior from this PR. It covers the following:
    - add comparator for byte array
    - updated field metrics bound type to allow object to byte buffer translation to occur later
    - add `metrics()` method to `ValueWriter` for Avro, currently default to empty stream
    - create `MetricsAwareDatumWriter` that exposes writer metrics, and replace `DatumWriter` with it in various classes
    - add metrics config to avro writer and builder
    - create a `AvroMetrics` class that resembles current behavior for producing metrics for avro writer


----------------------------------------------------------------
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] yyanyy commented on a change in pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/FieldMetrics.java
##########
@@ -30,15 +28,15 @@
   private final long valueCount;
   private final long nullValueCount;
   private final long nanValueCount;
-  private final ByteBuffer lowerBound;
-  private final ByteBuffer upperBound;
+  private final Object lowerBound;
+  private final Object upperBound;

Review comment:
       I think if we convert this to `ByteBuffer` now we may still need to check type when doing truncation (based on metrics mode), and I think string with non-unicode characters will not vend the same result if truncated by `BinaryUtil.truncateBinary`, so we will either convert the byte buffer back to char sequence and use `UnicodeUtil.truncateString` or create a new `BinaryUtil.truncateString`. Whereas if we do conversion later when evaluating metrics, I think the [code needed for the conversion itself](https://github.com/apache/iceberg/pull/1935/files#diff-32411391d7c2962fe8c090ca56f1e9f63cb9bd473f0e55a9a1581863e9542deaR109) isn't that bad since we know the type of the field, and that's the reason for me to do this change. 
   
   But one thing that may worth noting is that for the current approach, in order ensure the `Conversions.toByteBuffer` could work, for certain writers I have to make sure the min/max from the value writers return the type that `Conversions.toByteBuffer` knows how to translate, if the data type in `write` is not of that type (that is, usage of [this method](https://github.com/apache/iceberg/pull/1935/files#diff-6ccdc1daa757d215170dc57a4b3d4f31a62c8b2993ae2289511e97b2df52ef02R641-R652)). I think we still need to maintain a similar function for translation in each value writer if we return bytebuffer for bounds in field metrics, but it will directly translate input data type to byte buffer instead of doing two hops, and that might be easier to understand. 
   
   




----------------------------------------------------------------
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] jackye1995 commented on a change in pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/types/Comparators.java
##########
@@ -272,6 +276,30 @@ public int compare(ByteBuffer buf1, ByteBuffer buf2) {
     }
   }
 
+  private static class UnsignedByteArrayComparator implements Comparator<byte[]> {
+    private static final UnsignedByteArrayComparator INSTANCE = new UnsignedByteArrayComparator();
+
+    private UnsignedByteArrayComparator() {
+    }
+
+    @Override
+    public int compare(byte[] array1, byte[] array2) {
+      int len = Math.min(array1.length, array2.length);
+
+      // find the first difference and return
+      for (int i = 0; i < len; i += 1) {
+        // Conversion to int is what Byte.toUnsignedInt would do
+        int cmp = Integer.compare(((int) array1[i]) & 0xff, ((int) array2[i]) & 0xff);
+        if (cmp != 0) {
+          return cmp;
+        }
+      }
+
+      // if there are no differences, then the shorter seq is first

Review comment:
       nit: "the shorter seq is first" is a bit confusing to me, maybe "is smaller" is a better word.




----------------------------------------------------------------
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] yyanyy commented on a change in pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/FieldMetrics.java
##########
@@ -30,15 +28,15 @@
   private final long valueCount;
   private final long nullValueCount;
   private final long nanValueCount;
-  private final ByteBuffer lowerBound;
-  private final ByteBuffer upperBound;
+  private final Object lowerBound;
+  private final Object upperBound;

Review comment:
       I was mostly following the pattern of ORC and Parquet to evaluate metrics mode when collecting metrics (which has to be since the file formats collects stats themselves), but I think there's nothing prevent us from ingesting metrics mode during value writers creation, it will just make the visitor pattern a little bit more complicated. I'll give it a try, and thanks for bringing up this idea! 
   
   I guess for now I'll revert the change to `FieldMetrics` in this PR and include it in the next one that updates value writers if we need to change it. Hopefully that doesn't add too much to the next PR! 




----------------------------------------------------------------
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] yyanyy commented on a change in pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
##########
@@ -27,22 +27,31 @@
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.io.DatumWriter;
 import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.OutputFile;
 import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 class AvroFileAppender<D> implements FileAppender<D> {
   private PositionOutputStream stream = null;
   private DataFileWriter<D> writer = null;
+  private MetricsAwareDatumWriter<?> metricsAwareDatumWriter = null;
+  private org.apache.iceberg.Schema icebergSchema;
+  private MetricsConfig metricsConfig;
   private long numRecords = 0L;
+  private boolean isClosed = false;
 
-  AvroFileAppender(Schema schema, OutputFile file,
-                   Function<Schema, DatumWriter<?>> createWriterFunc,
+  AvroFileAppender(org.apache.iceberg.Schema icebergSchema, Schema schema, OutputFile file,

Review comment:
       I think here I'm adding iceberg schema to the constructor, in addition to an avro `Schema`, so we need to fully qualify one of them. The reason to add the iceberg schema is to avoid converting Avro schema back to Iceberg one when using it to evaluating metrics modes, but I guess we can directly input Iceberg schema in constructor and let constructor to do the conversion to Avro schema. Do you have a recommendation? 




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
##########
@@ -57,7 +66,10 @@ public void add(D datum) {
 
   @Override
   public Metrics metrics() {
-    return new Metrics(numRecords, null, null, null);
+    Preconditions.checkState(isClosed,
+        "Cannot return metrics while appending to an open file.");

Review comment:
       Nit: Does this need a newline? I think it would all fit.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/FieldMetrics.java
##########
@@ -30,15 +28,15 @@
   private final long valueCount;
   private final long nullValueCount;
   private final long nanValueCount;
-  private final ByteBuffer lowerBound;
-  private final ByteBuffer upperBound;
+  private final Object lowerBound;
+  private final Object upperBound;

Review comment:
       Why change this to `Object` rather than `ByteBuffer`? Seems like conversion to ByteBuffer would be cleaner if this was done in each writer because the writer already has its type because it is going to call the right method on the encoder.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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


   Thanks, @yyanyy! This looks good now so I merged it. That should unblock the next steps.


----------------------------------------------------------------
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] yyanyy commented on a change in pull request #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/Avro.java
##########
@@ -123,7 +127,7 @@ public WriteBuilder named(String newName) {
       return this;
     }
 
-    public WriteBuilder createWriterFunc(Function<Schema, DatumWriter<?>> writerFunction) {
+    public WriteBuilder createWriterFunc(Function<Schema, MetricsAwareDatumWriter<?>> writerFunction) {

Review comment:
       Sounds good, I didn't think of the case where people have their own implementation of these interfaces so I totally missed this. Will update and keep in mind!




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/types/Comparators.java
##########
@@ -157,6 +157,10 @@ public int compare(List<T> o1, List<T> o2) {
     return UnsignedByteBufComparator.INSTANCE;
   }
 
+  public static Comparator<byte[]> unsignedByteArray() {

Review comment:
       Other method names are plural. Could we use `unsignedByteArrays()`?




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/Avro.java
##########
@@ -123,7 +127,7 @@ public WriteBuilder named(String newName) {
       return this;
     }
 
-    public WriteBuilder createWriterFunc(Function<Schema, DatumWriter<?>> writerFunction) {
+    public WriteBuilder createWriterFunc(Function<Schema, MetricsAwareDatumWriter<?>> writerFunction) {

Review comment:
       This is going to break existing uses of `createWriterFunc` in projects that build on Iceberg. I think this should keep the old parameter and just check whether the implementation is `MetricsAwareDatumWriter` in the appender to return metrics.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/Avro.java
##########
@@ -363,6 +378,11 @@ public void write(PositionDelete<D> delete, Encoder out) throws IOException {
       POS_WRITER.write(delete.pos(), out);
       rowWriter.write(delete.row(), out);
     }
+
+    @Override
+    public Stream<FieldMetrics> metrics() {
+      return Stream.concat(PATH_WRITER.metrics(), POS_WRITER.metrics());

Review comment:
       This can be fixed in a follow-up.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
##########
@@ -27,22 +27,31 @@
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.io.DatumWriter;
 import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.OutputFile;
 import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 class AvroFileAppender<D> implements FileAppender<D> {
   private PositionOutputStream stream = null;
   private DataFileWriter<D> writer = null;
+  private MetricsAwareDatumWriter<?> metricsAwareDatumWriter = null;
+  private org.apache.iceberg.Schema icebergSchema;
+  private MetricsConfig metricsConfig;
   private long numRecords = 0L;
+  private boolean isClosed = false;
 
-  AvroFileAppender(Schema schema, OutputFile file,
-                   Function<Schema, DatumWriter<?>> createWriterFunc,
+  AvroFileAppender(org.apache.iceberg.Schema icebergSchema, Schema schema, OutputFile file,

Review comment:
       Does `Schema` need to be fully qualified?




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: api/src/main/java/org/apache/iceberg/FieldMetrics.java
##########
@@ -30,15 +28,15 @@
   private final long valueCount;
   private final long nullValueCount;
   private final long nanValueCount;
-  private final ByteBuffer lowerBound;
-  private final ByteBuffer upperBound;
+  private final Object lowerBound;
+  private final Object upperBound;

Review comment:
       I guess I was thinking that truncation would happen when `FieldMetrics` is constructed, in the leaf writers. If that's not the case, then I think it makes sense to do the conversion later.
   
   If the conversion happens later, then I think this class should be parameterized. I never like to have classes that track just `Object`. We should at least guarantee that both lower and upper bounds are the same type, for example.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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


   Thanks @yyanyy, I'll take a look at this one soon!


----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
##########
@@ -77,15 +89,16 @@ public void close() throws IOException {
     if (writer != null) {
       writer.close();
       this.writer = null;
+      isClosed = true;
     }
   }
 
   @SuppressWarnings("unchecked")
   private static <D> DataFileWriter<D> newAvroWriter(
-      Schema schema, PositionOutputStream stream, Function<Schema, DatumWriter<?>> createWriterFunc,
+      Schema schema, PositionOutputStream stream, DatumWriter<?> metricsAwareDatumWriter,
       CodecFactory codec, Map<String, String> metadata) throws IOException {
     DataFileWriter<D> writer = new DataFileWriter<>(
-        (DatumWriter<D>) createWriterFunc.apply(schema));
+        (DatumWriter<D>) metricsAwareDatumWriter);

Review comment:
       Minor: I don't think this needs to be a `MetricsAwareDatumWriter`, right? It isn't in the type signature, so we should name it just `datumWriter`.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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


   @yyanyy, can you rebase this to fix the conflict?


----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
##########
@@ -27,22 +27,31 @@
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.io.DatumWriter;
 import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.OutputFile;
 import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 class AvroFileAppender<D> implements FileAppender<D> {
   private PositionOutputStream stream = null;
   private DataFileWriter<D> writer = null;
+  private MetricsAwareDatumWriter<?> metricsAwareDatumWriter = null;
+  private org.apache.iceberg.Schema icebergSchema;
+  private MetricsConfig metricsConfig;
   private long numRecords = 0L;
+  private boolean isClosed = false;
 
-  AvroFileAppender(Schema schema, OutputFile file,
-                   Function<Schema, DatumWriter<?>> createWriterFunc,
+  AvroFileAppender(org.apache.iceberg.Schema icebergSchema, Schema schema, OutputFile file,

Review comment:
       I just didn't see that Avro's Schema is already imported. This should be fine.




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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



##########
File path: core/src/main/java/org/apache/iceberg/avro/Avro.java
##########
@@ -363,6 +378,11 @@ public void write(PositionDelete<D> delete, Encoder out) throws IOException {
       POS_WRITER.write(delete.pos(), out);
       rowWriter.write(delete.row(), out);
     }
+
+    @Override
+    public Stream<FieldMetrics> metrics() {
+      return Stream.concat(PATH_WRITER.metrics(), POS_WRITER.metrics());

Review comment:
       This should also include metrics from the `rowWriter`, right?




----------------------------------------------------------------
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 #1946: Avro metrics support: create MetricsAwareDatumWriter and some refactors for Avro

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


   


----------------------------------------------------------------
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