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 2022/07/28 21:56:04 UTC

[GitHub] [iceberg] szehon-ho opened a new pull request, #5376: Core: Add readable metrics columns to files metadata tables

szehon-ho opened a new pull request, #5376:
URL: https://github.com/apache/iceberg/pull/5376

   This adds following columns to all files tables:
   
   - column_sizes_metrics
   - value_counts_metrics
   - null_value_counts_metrics
   - nan_value_counts_metrics
   - lower_bounds_metrics
   - upper_bounds_metrics
   
   This is to keep backward compatibility as the existing metrics columns can not be changed.
   
   The first four return Map<String, Long>.  Key is the human-readable column name (dot separated for nested columns).
   The last two return Map<String, String>.  Key is like above, Value is human-readable upper/lower bound.
   
   Example: upper_bounds_metrics = Map ("mystruct.timestamp" => "1970-01-01T00:00:00.000002")
   
   This makes Iceberg metadata tables is a bit closer to Trino, where the last two columns are <Long, String> (column id to human readable bound).  It goes beyond and even resolves the column to make it readable.
   
   Implementation detail:  Not that we add new columns to files table, it becomes not a 1 to 1 mapping with the "DataFile" java object, so we have to add code to handle column mapping in projection case.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r934960703


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore
+        return Optional.empty();

Review Comment:
   This happens in some cases, I found it in some case of importing external files to Iceberg table, ie TestIcebergSourceHadoopTables.testFilesTableWithSnapshotIdInheritance, where the I think columns are out of order of the original schema and the metrics are corrupt (underflow exception in this case).  
   
   Not sure if we should error out the files tables, in that case, I was leaning towards just returning null.  User has original column to see why the error happened.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025811376


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   I think this can just be for field in schema.columns?
   Can't we get the name then from field?
   
   I don't think "id" is actually used in this loop



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038490304


##########
core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java:
##########
@@ -527,6 +527,63 @@ public void testDeleteFilesTableSelection() throws IOException {
     Assert.assertEquals(expected, scan.schema().asStruct());
   }
 
+  @Test
+  public void testFilesTableReadableMetricsSchema() {
+
+    Table filesTable = new FilesTable(table.ops(), table);
+    Types.StructType actual = filesTable.newScan().schema().select("readable_metrics").asStruct();
+

Review Comment:
   Just to make this a little easier in the future you may just want to do something like
   
   firstAssigned = (schema.highestId - 15)
   Then do 
   1001 = firstAssigned +1; ....
   
   not sure this really helps that much though



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038597531


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +69,293 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricColDefinition> READABLE_METRIC_COLS =
+      ImmutableList.of(
+          new ReadableMetricColDefinition(
+              "column_size",
+              "Total size on disk",
+              DataFile.COLUMN_SIZES,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.columnSizes() == null ? null : file.columnSizes().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "value_count",
+              "Total count, including null and NaN",
+              DataFile.VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.valueCounts() == null ? null : file.valueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "null_value_count",
+              "Null value count",
+              DataFile.NULL_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nullValueCounts() == null
+                      ? null
+                      : file.nullValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "nan_value_count",
+              "NaN value count",
+              DataFile.NAN_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nanValueCounts() == null
+                      ? null
+                      : file.nanValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "lower_bound",
+              "Lower bound",
+              DataFile.LOWER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.lowerBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.lowerBounds().get(field.fieldId()))),
+          new ReadableMetricColDefinition(
+              "upper_bound",
+              "Upper bound",
+              DataFile.UPPER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.upperBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.upperBounds().get(field.fieldId()))));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  /**
+   * Fixed definition of a readable metric column, ie a mapping of a raw metric to a readable metric
+   */
+  public static class ReadableMetricColDefinition {
+    private final String name;
+    private final String doc;
+    private final Types.NestedField originalCol;
+    private final TypeFunction typeFunction;
+    private final MetricFunction metricFunction;
+
+    public interface TypeFunction {
+      Type type(Types.NestedField originalCol);
+    }
+
+    public interface MetricFunction {
+      Object metric(ContentFile<?> file, Types.NestedField originalCol);
+    }
+
+    /**
+     * @param name column name
+     * @param doc column doc
+     * @param originalCol original (raw) metric column field on metadata table
+     * @param typeFunction function that returns the readable metric column type from original field
+     *     type
+     * @param metricFunction function that returns readable metric from data file
+     */
+    ReadableMetricColDefinition(
+        String name,
+        String doc,
+        Types.NestedField originalCol,
+        TypeFunction typeFunction,
+        MetricFunction metricFunction) {
+      this.name = name;
+      this.doc = doc;
+      this.originalCol = originalCol;
+      this.typeFunction = typeFunction;
+      this.metricFunction = metricFunction;
+    }
+
+    Types.NestedField originalCol() {
+      return originalCol;
+    }
+
+    Type colType(Types.NestedField field) {
+      return typeFunction.type(field);
+    }
+
+    String name() {
+      return name;
+    }
+
+    String doc() {
+      return doc;
+    }
+
+    Object value(ContentFile<?> dataFile, Types.NestedField dataField) {
+      return metricFunction.metric(dataFile, dataField);
+    }
+  }
+
+  /** A struct of readable metric values for a primitive column */
+  public static class ReadableColMetricsStruct implements StructLike {
+
+    private final String columnName;
+    private final Map<Integer, Integer> projectionMap;
+    private final Object[] metrics;
+
+    public ReadableColMetricsStruct(
+        String columnName, Types.NestedField projection, Object... metrics) {
+      this.columnName = columnName;
+      this.projectionMap = readableMetricsProjection(projection);
+      this.metrics = metrics;
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      return metrics[projectedPos];
+    }
+
+    /** Returns map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_METRIC_COLS.size(); fieldIndex++) {
+        ReadableMetricColDefinition readableMetric = READABLE_METRIC_COLS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * A struct, consisting of all {@link ReadableColMetricsStruct} for all primitive columns of the
+   * table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metadataTableSchema) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    AtomicInteger nextId =

Review Comment:
   Ah AtomicInteger is because there is a lambda function in there (the map) and compiler complains :
   
   Variable used in lambda expression should be final or effectively final



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014322081


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +142,72 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    private static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);

Review Comment:
   Added a comment..



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014323668


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Sure I guess the counts can be string, though its weird. 
   
   To clarify, are you suggesting:
   `Types.StructType.of(readableStruct(LOWER_BOUNDS), readable_struct(UPPER_BOUNDS), ...?)`
   
   or something else?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949704253


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -82,11 +107,20 @@ private static CloseableIterable<FileScanTask> planFiles(
     Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter;
     ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter);
 
+    Map<Integer, String> fieldById = TypeUtil.indexNameById(table.schema().asStruct());

Review Comment:
   Made a public Schema API idToName()



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015603424


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   I did a lot of work prototyping keeping the original struct (which I think you also bringing up).  That is a ton of work and complexity, in writing schema utils to clone the schema for each struct.  In the end I felt all that complexity is not really worth it, a map is much simpler.
   
   And it is more in line  with what trino has (all the bounds as string).  



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1302866207

   updated and rebased, @RussellSpitzer if you have time to take a look 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014038296


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Is it crazy for us to have a different struct for every value type so we can return structs with the actual bound types? Probably ... just thinking about this a little. We could define them programmatically so it wouldn't be so bad? I guess strings are probably fine since most folks will be reading these values with human eyes and not using them in another system. 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014211767


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =

Review Comment:
   Rather than this it may make sense to just use our "sql" method.
   
   Could change this to 
   ```code
   assertEquals("Failed nested select",
     expected,
     sql("SELECT ..."),
     )
     
   ```
   
   We use this pattern in a bunch of places like
   
   ```
       assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName));
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949458152


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,123 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param namesById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexNameById(Types.StructType)}
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, Map<Integer, String> namesById, ContentFile<?> contentFile) {

Review Comment:
   If we go with `schema.findColumnName`, we may drop `namesById` here. If you shorten contentFile to file, it would even fit on one line.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +235,51 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {
+      List<Function<ContentFile<?>, Object>> accessors =
+          Lists.newArrayList(
+              file -> file.content().id(),
+              ContentFile::path,
+              file -> file.format().toString(),
+              ContentFile::specId,
+              ContentFile::partition,
+              ContentFile::recordCount,
+              ContentFile::fileSizeInBytes,
+              ContentFile::columnSizes,
+              ContentFile::valueCounts,
+              ContentFile::nullValueCounts,
+              ContentFile::nanValueCounts,
+              ContentFile::lowerBounds,
+              ContentFile::upperBounds,
+              ContentFile::keyMetadata,
+              ContentFile::splitOffsets,
+              ContentFile::equalityFieldIds,
+              ContentFile::sortOrderId,
+              file -> MetricsUtil.readableMetricsMap(dataTableSchema, dataTableFields, file));
+      return partitioned
+          ? accessors
+          : Stream.concat(
+                  accessors.subList(0, 4).stream(), accessors.subList(5, accessors.size()).stream())
+              .collect(Collectors.toList());
+    }
+
+    private List<Object> projectedFields(

Review Comment:
   Are we calling this per row now?



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,123 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param namesById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexNameById(Types.StructType)}
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, Map<Integer, String> namesById, ContentFile<?> contentFile) {
+    Map<String, StructLike> metricsStruct = Maps.newHashMapWithExpectedSize(namesById.size());
+
+    Map<Integer, Long> columnSizes = contentFile.columnSizes();
+    Map<Integer, Long> valueCounts = contentFile.valueCounts();
+    Map<Integer, Long> nullValueCounts = contentFile.nullValueCounts();
+    Map<Integer, Long> nanValueCounts = contentFile.nanValueCounts();
+    Map<Integer, ByteBuffer> lowerBounds = contentFile.lowerBounds();
+    Map<Integer, ByteBuffer> upperBounds = contentFile.upperBounds();
+
+    for (int id : namesById.keySet()) {
+      Types.NestedField field = schema.findField(id);
+      if (field.type().isPrimitiveType()) {
+        // Iceberg stores metrics only for primitive types
+        String colName = namesById.get(id);
+        ReadableMetricsStruct struct =
+            new ReadableMetricsStruct(
+                columnSizes == null ? null : columnSizes.get(id),
+                valueCounts == null ? null : valueCounts.get(id),
+                nullValueCounts == null ? null : nullValueCounts.get(id),
+                nanValueCounts == null ? null : nanValueCounts.get(id),
+                lowerBounds == null ? null : convertToReadable(field, lowerBounds.get(id)),
+                upperBounds == null ? null : convertToReadable(field, upperBounds.get(id)));
+        metricsStruct.put(colName, struct);
+      }
+    }
+    return metricsStruct;

Review Comment:
   nit: What about an empty line before the return statement to separate the for loop above?



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,123 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param namesById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexNameById(Types.StructType)}
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, Map<Integer, String> namesById, ContentFile<?> contentFile) {
+    Map<String, StructLike> metricsStruct = Maps.newHashMapWithExpectedSize(namesById.size());

Review Comment:
   nit: `metricsStruct` -> `metricsMap`?



##########
spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java:
##########
@@ -1743,11 +1749,99 @@ private GenericData.Record manifestRecord(
         .build();
   }
 
-  private void asMetadataRecord(GenericData.Record file) {
+  static final Types.StructType EXPECTED_METRICS_VALUE_TYPE =

Review Comment:
   If we move the field to `DataFile`, we can reuse it in tests as well.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -18,25 +18,50 @@
  */
 package org.apache.iceberg;
 
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.ManifestEvaluator;
 import org.apache.iceberg.expressions.ResidualEvaluator;
 import org.apache.iceberg.io.CloseableIterable;
 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.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 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.TypeUtil;
+import org.apache.iceberg.types.Types;
 import org.apache.iceberg.types.Types.StructType;
 
 /** Base class logic for files metadata tables */
 abstract class BaseFilesTable extends BaseMetadataTable {
 
+  static final Types.StructType READABLE_METRICS_VALUE =

Review Comment:
   The current solution seems safe enough but your idea to put it in `DataFile` can be a bit better, I guess. Then we can just reuse the next ID that we were supposed to assign and this field will be accessible in all other places too. We don't have to return the new field in `DataFile$getType`.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   Oh, I see, let me think.



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,123 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param namesById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexNameById(Types.StructType)}
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, Map<Integer, String> namesById, ContentFile<?> contentFile) {
+    Map<String, StructLike> metricsStruct = Maps.newHashMapWithExpectedSize(namesById.size());
+
+    Map<Integer, Long> columnSizes = contentFile.columnSizes();
+    Map<Integer, Long> valueCounts = contentFile.valueCounts();
+    Map<Integer, Long> nullValueCounts = contentFile.nullValueCounts();
+    Map<Integer, Long> nanValueCounts = contentFile.nanValueCounts();
+    Map<Integer, ByteBuffer> lowerBounds = contentFile.lowerBounds();
+    Map<Integer, ByteBuffer> upperBounds = contentFile.upperBounds();
+
+    for (int id : namesById.keySet()) {
+      Types.NestedField field = schema.findField(id);
+      if (field.type().isPrimitiveType()) {
+        // Iceberg stores metrics only for primitive types
+        String colName = namesById.get(id);
+        ReadableMetricsStruct struct =
+            new ReadableMetricsStruct(
+                columnSizes == null ? null : columnSizes.get(id),
+                valueCounts == null ? null : valueCounts.get(id),
+                nullValueCounts == null ? null : nullValueCounts.get(id),
+                nanValueCounts == null ? null : nanValueCounts.get(id),
+                lowerBounds == null ? null : convertToReadable(field, lowerBounds.get(id)),
+                upperBounds == null ? null : convertToReadable(field, upperBounds.get(id)));
+        metricsStruct.put(colName, struct);
+      }
+    }
+    return metricsStruct;
+  }
+
+  public static String convertToReadable(Types.NestedField field, ByteBuffer value) {
+    if (field == null || value == null) {
+      return null;
+    }
+    try {
+      return Transforms.identity(field.type())
+          .toHumanString(Conversions.fromByteBuffer(field.type(), value));
+    } catch (Exception e) {
+      LOG.warn("Error converting metric to readable form", e);
+      return null;
+    }
+  }
+
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final String lowerBound;
+    private final String upperBound;
+
+    public ReadableMetricsStruct(
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        String lowerBound,
+        String upperBound) {
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+    }
+
+    @Override
+    public int size() {
+      return 6;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value;
+      switch (pos) {

Review Comment:
   nit: Would having `private Object get(int pos)` help separate getting the value from casting?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +177,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema filesTableSchema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> dataTableFields;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
-        ResidualEvaluator residuals) {
+        ResidualEvaluator residuals,
+        Map<Integer, String> dataTableFields) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.dataTableSchema = table.schema();
+      this.dataTableFields = dataTableFields;
+      this.isPartitioned = filesTableSchema.findField(DataFile.PARTITION_ID) != null;
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      return CloseableIterable.transform(
+          manifestEntries(),
+          fileEntry ->
+              StaticDataTask.Row.of(
+                  projectedFields(fileEntry, accessors(isPartitioned)).toArray()));
     }
 
     private CloseableIterable<? extends ContentFile<?>> manifestEntries() {

Review Comment:
   Not directly related to this PR but it should probably be called `files` as we are returning `ContentFile` and not the manifest entry objects.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -82,11 +107,20 @@ private static CloseableIterable<FileScanTask> planFiles(
     Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter;
     ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter);
 
+    Map<Integer, String> fieldById = TypeUtil.indexNameById(table.schema().asStruct());

Review Comment:
   Actually, my original comment was only partially true. I thought we were quoting, which would generate new strings but we just need to index columns. If so, we can probably use `schema$findColumnName` directly.
   
   I believe `Schema` uses `TypeUtil` under the hood and will cache the map for us already.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r954090486


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());

Review Comment:
   Added and actually found two bugs:
   
   - When readable_metrics is selected, was projecting out the metrics needed to calculate it
   - Need to add projection logic on struct itself (ie, the value of readable_metrics map), otherwise MetricsStruct becomes out of order.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1213260204

   Let me check in a bit.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026578079


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Yea I think that would be nice, changing these tests are definitely the most painful part of this (and related changes).
   
   But the GenericRecord here is an Avro class that doesnt have any select methods.  It has a get() but it returns a field and not a projected record.  We'd have to maybe try to make a struct based on get() all 15 non-derived DataFile fields, not sure if that's cleaner?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1203087530

   All Spark tests are updated/fixed now.
   
   Fyi @RussellSpitzer @aokolnychyi @rdblue if you guys have time to leave some feedback.  There are new tests added but its a bit big to show 'Files Changed': [TestMetadataTableMetricsColumns.java](https://github.com/apache/iceberg/pull/5376/files#diff-b2056d565254122d3c10959f8cb9b69e23d2e4d0cdf91418ef32d44c83c3d30f)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944973038


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +174,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema filesTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> quotedNameById;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.projectedSchema = projectedSchema;
+      this.dataTableSchema = table.schema();
+      this.quotedNameById = TypeUtil.indexQuotedNameById(table.schema().asStruct(), name -> name);

Review Comment:
   Done, now I use TypeUtil.indexNameById().  I still pre-compute it outside to avoid cost per row, instead of using schema.findColumnName, I'm not sure is what you meant?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho closed pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho closed pull request #5376: Core: Add readable metrics columns to files metadata tables
URL: https://github.com/apache/iceberg/pull/5376


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues:
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the table will never request that field.
   
   To me a list of explicit accessors for this table is better as it decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to encapsulate this somehow in a StructLike class ?  



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues that break the projection.
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the table will never request that field.
   
   To me a list of explicit accessors for this table is better as it decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to encapsulate this somehow in a StructLike class ?  



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015582931


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Actually I was thinking about this more over the weekend, what if we had 
   
   lowerBound <Struct <Table's Schema>>
   upperBound <Struct<Table's Schema>>
   
   That way all of the types would be their original column types, I think the only thing we need to do is make all the fields optional since we don't have metrics for certain types. Or you could just select not on those columns.
   
   This would kind of invert the current layout and i'm not sure how well that fits but the layout would instead look like
   
   metricName.columnToRead



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014235434


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("file_path", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult = rowsToJava(mixedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(dataFile.path().toString(), 2L)),
+        mixedSelectResult);
+
+    Dataset<Row> mixedSelect2 =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("sort_order_id", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult2 = rowsToJava(mixedSelect2.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row(0, 2L)), mixedSelectResult2);
+
+    Dataset<Row> reversedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("readable_metrics.longCol.value_count", "file_path");
+    List<Object[]> reversedSelectResult = rowsToJava(reversedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(2L, dataFile.path().toString())),
+        reversedSelectResult);
+  }
+
+  @Test
+  public void testNullNanValues() throws Exception {
+    String tableName = "testNullNanValues";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false, 0, 0, Float.NaN, Double.NaN, null, "0", null, null, null, null, null),
+            createPrimitiveRecord(
+                false,
+                0,
+                1,
+                Float.NaN,
+                1.0,
+                new BigDecimal("1.00"),
+                "1",
+                null,
+                null,
+                null,
+                null,
+                null));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    checkMetricValues(
+        row,
+        "null_value_count",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", 0L),
+            Maps.immutableEntry("stringCol", 0L),
+            Maps.immutableEntry("intCol", 0L),
+            Maps.immutableEntry("longCol", 0L),
+            Maps.immutableEntry("floatCol", 0L),
+            Maps.immutableEntry("doubleCol", 0L),
+            Maps.immutableEntry("decimalCol", 1L),
+            Maps.immutableEntry("binaryCol", 2L),
+            Maps.immutableEntry("fixedCol", 2L),
+            Maps.immutableEntry("dateCol", 2L),
+            Maps.immutableEntry("timeCol", 2L),
+            Maps.immutableEntry("timestampCol", 2L)));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("booleanCol", null);
+    expectedNanValues.put("stringCol", null);
+    expectedNanValues.put("intCol", null);
+    expectedNanValues.put("longCol", null);
+    expectedNanValues.put("floatCol", 2L);
+    expectedNanValues.put("doubleCol", 1L);
+    expectedNanValues.put("decimalCol", null);
+    expectedNanValues.put("binaryCol", null);
+    expectedNanValues.put("fixedCol", null);
+    expectedNanValues.put("dateCol", null);
+    expectedNanValues.put("timeCol", null);
+    expectedNanValues.put("timestampCol", null);
+
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+  }
+
+  @Test
+  public void testNestedValues() throws Exception {
+    String tableName = "testNestedValues";
+    Table table = createTable(tableName, NESTED_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createNestedRecord(0L, 0.0),
+            createNestedRecord(1L, Double.NaN),
+            createNestedRecord(null, null));
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    String[] nestedColumns =
+        new String[] {
+          "nestedStructCol.leafStructCol.leafDoubleCol", "nestedStructCol.leafStructCol.leafLongCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        nestedColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Value count should be 3", l.longValue(), 3L));
+    checkMetric(
+        row,
+        "null_value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Null value count should be 1", l.longValue(), 1L));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafDoubleCol", 1L);
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafLongCol", null);
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "0",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "1",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+  }
+
+  private <T> void checkCollectionValues(Row row, String columnName, Set<T> expectedValues) {
+    Set<Long> actualValues =
+        Sets.newHashSet(JavaConverters.asJavaCollection(row.getAs(columnName)));
+    Assert.assertEquals("Collection values should match", expectedValues, actualValues);
+  }
+
+  private void checkMetric(Row row, String metricName, String[] columns, Consumer<Long> check) {

Review Comment:
   So this is for checking if a certain metric is present for all columns and then run an arbitrary long function on that value?
   
   I think it may help to break this into a function that reports missing metrics and a function which checks if a boolean expression is true for all such values. Currently we end up passing in check functions that have to implement their own assertions which seems a little confusing to me. 
   
   Another option maybe something like
   
   ```
     private <T> void checkMetricValueForAllColumns(Row row, String metricName, String[] columns, Predicate<T> check) {
       scala.collection.Map<String, Row> metrics = row.getAs("readable_metrics");
       for (String column : columns) {
         Option<Row> rowOption = metrics.get(column);
         Assert.assertTrue("Missing metric for column: " + column, rowOption.isDefined());
         T metric = rowOption.get().getAs(metricName);
         Assert.assertTrue("Check on metric ....", check.test(metric));
       }
     }
   ```
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015604784


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("file_path", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult = rowsToJava(mixedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(dataFile.path().toString(), 2L)),
+        mixedSelectResult);
+
+    Dataset<Row> mixedSelect2 =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("sort_order_id", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult2 = rowsToJava(mixedSelect2.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row(0, 2L)), mixedSelectResult2);
+
+    Dataset<Row> reversedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("readable_metrics.longCol.value_count", "file_path");
+    List<Object[]> reversedSelectResult = rowsToJava(reversedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(2L, dataFile.path().toString())),
+        reversedSelectResult);
+  }
+
+  @Test
+  public void testNullNanValues() throws Exception {
+    String tableName = "testNullNanValues";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =

Review Comment:
   Yea, done, removed the extra nullNan test and do all the checks in the primitive test



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r956441487


##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -148,6 +148,16 @@ public static Schema join(Schema left, Schema right) {
     return new Schema(joinedColumns);
   }
 
+  public static Schema joinCommon(Schema left, Schema right) {

Review Comment:
   What about simply adapting the existing `join` method? Are there any scenarios where we want to skip the validation and simply add all columns (old logic)?



##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -148,6 +148,16 @@ public static Schema join(Schema left, Schema right) {
     return new Schema(joinedColumns);
   }
 
+  public static Schema joinCommon(Schema left, Schema right) {
+    List<Types.NestedField> joinedColumns = Lists.newArrayList(left.columns());
+    for (Types.NestedField column : right.columns()) {
+      if (!joinedColumns.contains(column)) {

Review Comment:
   We are calling `contains` on a list, which is suboptimal. We also don't do any validation when IDs are conflicting but the rest of the field metadata (like type) is different.
   
   What about something like this?
   
   ```
   List<Types.NestedField> joinedColumns = Lists.newArrayList(left.columns());
   
   for (Types.NestedField rightColumn : right.columns()) {
     Types.NestedField leftColumn = left.findField(rightColumn.fieldId());
   
     Preconditions.checkArgument(
        leftColumn == null || leftColumn.equals(rightColumn),
        "...");
   
     if (leftColumn == null) {
       joinedColumns.add(rightColumn);
     }
   }
   
   return new Schema(joinedColumns);
   ```



##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =

Review Comment:
   Are there better names we can use?
   
   `READABLE_METRICS_VALUE_TYPE` - the struct type or the type of the value in the map
   `READABLE_METRICS_VALUE ` - the type of the map with metrics
   
   Will `READABLE_METRICS_VALUE_TYPE` and `READABLE_METRICS_TYPE` make more sense?
   
   
   
   



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =
+            TypeUtil.joinCommon(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);
+        return CloseableIterable.transform(files(minProjection), this::withReadableMetrics);
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike withReadableMetrics(ContentFile<?> file) {
+      int structSize = projection.columns().size();
+      Map<String, StructLike> metrics =
+          MetricsUtil.readableMetricsMap(
+              dataTableSchema, file, readableMetricsProjection(projection));
+      return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
+    }
+
+    // Handles projections for readable metrics struct
+    private Map<Integer, Integer> readableMetricsProjection(Schema projectedSchema) {

Review Comment:
   Optional: It probably belongs more to `MetricsUtil` than here. We could pass `StructType metricsType` and compute the map in the utility so that if there is another place that needs it, we won't have to move this method around. The utility can also make sure the map of indices is serializable.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =
+            TypeUtil.joinCommon(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);
+        return CloseableIterable.transform(files(minProjection), this::withReadableMetrics);
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike withReadableMetrics(ContentFile<?> file) {
+      int structSize = projection.columns().size();
+      Map<String, StructLike> metrics =
+          MetricsUtil.readableMetricsMap(
+              dataTableSchema, file, readableMetricsProjection(projection));
+      return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
+    }
+
+    // Handles projections for readable metrics struct
+    private Map<Integer, Integer> readableMetricsProjection(Schema projectedSchema) {
+      Map<Integer, Integer> projectionMap = Maps.newHashMap();
+      Type type = projectedSchema.findType(DataFile.READABLE_METRICS_VALUE.valueId());
+      if (type != null) {

Review Comment:
   I don't think `type` can be null. We don't have to add more validation as it is checked above. If the type is null, the projection map will be empty, which will break `ReadableMetricsStruct` later. Seems like we can simply remove the if statement.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =

Review Comment:
   Any particular reasons not to make it private?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +239,41 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final Map<String, StructLike> readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize, StructLike fileAsStruct, Map<String, StructLike> readableMetrics) {
+      this.structSize = structSize;
+      this.fileAsStruct = fileAsStruct;
+      this.readableMetrics = readableMetrics;
+    }
+
+    @Override
+    public int size() {
+      return structSize;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      if (pos < (structSize - 1)) {
+        return fileAsStruct.get(pos, javaClass);
+      } else if (pos == (structSize - 1)) {
+        return javaClass.cast(readableMetrics);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Illegal position access for ContentFileStructWithMetrics: %d, max allowed is %d",
+                pos, (structSize - 1)));
+      }
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("FileEntryRow is read only");

Review Comment:
   Still references `FileEntryRow`?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think this logic should be part of `BaseFilesTableScan` and `BaseAllFilesTableScan`.
   Otherwise, our scans won't report the correct schema in `Scan$schema()`.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =
+            TypeUtil.joinCommon(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);
+        return CloseableIterable.transform(files(minProjection), this::withReadableMetrics);
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike withReadableMetrics(ContentFile<?> file) {
+      int structSize = projection.columns().size();
+      Map<String, StructLike> metrics =
+          MetricsUtil.readableMetricsMap(
+              dataTableSchema, file, readableMetricsProjection(projection));
+      return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
+    }
+
+    // Handles projections for readable metrics struct
+    private Map<Integer, Integer> readableMetricsProjection(Schema projectedSchema) {
+      Map<Integer, Integer> projectionMap = Maps.newHashMap();
+      Type type = projectedSchema.findType(DataFile.READABLE_METRICS_VALUE.valueId());
+      if (type != null) {
+        Set<Types.NestedField> projectedFields = Sets.newHashSet(type.asStructType().fields());
+        int projectedIndex = 0;
+        for (int fieldIndex = 0;
+            fieldIndex < DataFile.READABLE_METRICS_VALUE_TYPE.fields().size();

Review Comment:
   nit: Will a helper variable avoid the need to split this into multiple lines?
   
   ```
   List<Types.NestedField> allMetricFields = READABLE_METRICS_STRUCT_TYPE.fields();
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho merged pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho merged PR #5376:
URL: https://github.com/apache/iceberg/pull/5376


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038597531


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +69,293 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricColDefinition> READABLE_METRIC_COLS =
+      ImmutableList.of(
+          new ReadableMetricColDefinition(
+              "column_size",
+              "Total size on disk",
+              DataFile.COLUMN_SIZES,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.columnSizes() == null ? null : file.columnSizes().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "value_count",
+              "Total count, including null and NaN",
+              DataFile.VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.valueCounts() == null ? null : file.valueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "null_value_count",
+              "Null value count",
+              DataFile.NULL_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nullValueCounts() == null
+                      ? null
+                      : file.nullValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "nan_value_count",
+              "NaN value count",
+              DataFile.NAN_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nanValueCounts() == null
+                      ? null
+                      : file.nanValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "lower_bound",
+              "Lower bound",
+              DataFile.LOWER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.lowerBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.lowerBounds().get(field.fieldId()))),
+          new ReadableMetricColDefinition(
+              "upper_bound",
+              "Upper bound",
+              DataFile.UPPER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.upperBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.upperBounds().get(field.fieldId()))));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  /**
+   * Fixed definition of a readable metric column, ie a mapping of a raw metric to a readable metric
+   */
+  public static class ReadableMetricColDefinition {
+    private final String name;
+    private final String doc;
+    private final Types.NestedField originalCol;
+    private final TypeFunction typeFunction;
+    private final MetricFunction metricFunction;
+
+    public interface TypeFunction {
+      Type type(Types.NestedField originalCol);
+    }
+
+    public interface MetricFunction {
+      Object metric(ContentFile<?> file, Types.NestedField originalCol);
+    }
+
+    /**
+     * @param name column name
+     * @param doc column doc
+     * @param originalCol original (raw) metric column field on metadata table
+     * @param typeFunction function that returns the readable metric column type from original field
+     *     type
+     * @param metricFunction function that returns readable metric from data file
+     */
+    ReadableMetricColDefinition(
+        String name,
+        String doc,
+        Types.NestedField originalCol,
+        TypeFunction typeFunction,
+        MetricFunction metricFunction) {
+      this.name = name;
+      this.doc = doc;
+      this.originalCol = originalCol;
+      this.typeFunction = typeFunction;
+      this.metricFunction = metricFunction;
+    }
+
+    Types.NestedField originalCol() {
+      return originalCol;
+    }
+
+    Type colType(Types.NestedField field) {
+      return typeFunction.type(field);
+    }
+
+    String name() {
+      return name;
+    }
+
+    String doc() {
+      return doc;
+    }
+
+    Object value(ContentFile<?> dataFile, Types.NestedField dataField) {
+      return metricFunction.metric(dataFile, dataField);
+    }
+  }
+
+  /** A struct of readable metric values for a primitive column */
+  public static class ReadableColMetricsStruct implements StructLike {
+
+    private final String columnName;
+    private final Map<Integer, Integer> projectionMap;
+    private final Object[] metrics;
+
+    public ReadableColMetricsStruct(
+        String columnName, Types.NestedField projection, Object... metrics) {
+      this.columnName = columnName;
+      this.projectionMap = readableMetricsProjection(projection);
+      this.metrics = metrics;
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      return metrics[projectedPos];
+    }
+
+    /** Returns map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_METRIC_COLS.size(); fieldIndex++) {
+        ReadableMetricColDefinition readableMetric = READABLE_METRIC_COLS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * A struct, consisting of all {@link ReadableColMetricsStruct} for all primitive columns of the
+   * table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metadataTableSchema) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    AtomicInteger nextId =

Review Comment:
   Ah because there is a lambda function in there and compiler complains :
   
   Variable used in lambda expression should be final or effectively final



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r950416982


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {

Review Comment:
   Optional and not strictly necessarily but thought its cleaner as now passing a projection with only fields directly known from ManifestFiles.read(), as anyway its calculated in the caller.  let me know if it makes sense or not



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r954089250


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +201,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final Schema projectedFilesSchema;
+    private final StructLike fileAsStruct;
+    private final Map<String, StructLike> readableMetrics;
+
+    ContentFileStructWithMetrics(
+        Schema projectedFileSchema,
+        StructLike fileAsStruct,
+        Map<String, StructLike> readableMetrics) {
+      this.projectedFilesSchema = projectedFileSchema;
+      this.fileAsStruct = fileAsStruct;
+      this.readableMetrics = readableMetrics;
+    }
+
+    @Override
+    public int size() {
+      return projectedFilesSchema.columns().size() + 1;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      if (pos < projectedFilesSchema.columns().size()) {
+        return fileAsStruct.get(pos, javaClass);
+      } else if (pos == projectedFilesSchema.columns().size()) {
+        return javaClass.cast(readableMetrics);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Illegal position access for FileRow: %d, max allowed is %d",
+                pos, fileAsStruct.size()));
+      }
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("FileEntryRow is read only");

Review Comment:
   Sorry, earlier version of name of this class



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957690407


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think putting it there will break the scan right, as its not the projection the user requested.  
   
   Note, this is actually a bit subtle here.  Because we are doing the join, (original projection + minimum metrics), the file's schema becomes 
   {any_projected_field_on_file} : {readable_metrics because its also projected} : {un-projected but required metrics fields}
   
   So the ContentFileWithMetrics works because it will discard any of the "un-projected but required metrics fields", given they are outside the range it will read.  For the remaining fields it uses the existing logic (delegate to file for the first n-1, and then get from MetricsStruct for nth field).
   
   I mean, we could add a select method to GenericDataFile to modify its internal 'fromProjectionPos' map to conform back to the original projection (dropping the "un-projected but required metrics fields").  But it would mainly be for clarity, and not strictly needed.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957690407


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think putting it there will break the scan right, as its not the projection the user requested.  
   
   Note, this is actually a bit subtle here.  Because we are doing the join, (original projection + minimum metrics), the file's schema becomes 
   {any_projected_field_on_file} : {readable_metrics because its also projected} : {un-projected but required metrics fields}
   
   So the ContentFileWithMetrics works because it will discard any of the "un-projected but required metrics fields", given they are outside the range it will read.  For the remaining fields it uses the existing logic (delegate to file for the first n-1, and then get from MetricsStruct for nth field).
   
   I mean, we could add a select method to GenericDataFile to modify its internal 'fromProjectionPos' map to conform back to the original projection (dropping the "un-projected but required metrics fields").  But it would mainly be for safety, and not strictly needed.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957730353


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =
+            TypeUtil.joinCommon(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);
+        return CloseableIterable.transform(files(minProjection), this::withReadableMetrics);
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike withReadableMetrics(ContentFile<?> file) {
+      int structSize = projection.columns().size();
+      Map<String, StructLike> metrics =
+          MetricsUtil.readableMetricsMap(
+              dataTableSchema, file, readableMetricsProjection(projection));
+      return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
+    }
+
+    // Handles projections for readable metrics struct
+    private Map<Integer, Integer> readableMetricsProjection(Schema projectedSchema) {
+      Map<Integer, Integer> projectionMap = Maps.newHashMap();
+      Type type = projectedSchema.findType(DataFile.READABLE_METRICS_VALUE.valueId());
+      if (type != null) {

Review Comment:
   You are right, removed.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1035319939


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   In that case we could still probably just iterate over columns, and call 
   ```java
   schema.findColumnName(col.id)
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1017037292


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +142,75 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    private static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+
+        // If readable_metrics is selected,
+        // original metrics columns need to be selected for derivation
+        Schema minProjection = TypeUtil.join(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);

Review Comment:
   As we discussed offline I think this is a bit mislabeled, this is actually the maximal projection required correct? Ie if I only select VALUE_COUNTS for example, I do not actually need to get the rest of the metrics. I think this is fine for now since that's just an optimization but the name of the variable probably should be changed?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] chenjunjiedada commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1011640444


##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -142,9 +142,21 @@ public static Schema selectNot(Schema schema, Set<Integer> fieldIds) {
   }
 
   public static Schema join(Schema left, Schema right) {
-    List<Types.NestedField> joinedColumns = Lists.newArrayList();
-    joinedColumns.addAll(left.columns());

Review Comment:
   nit: This changes the original behavior, why not add a new function?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014199642


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";

Review Comment:
   Test Primitive, test Select and Test non-null all use the same underlying table and records, it may make sense to just create the table at suite start rather than making 3 different tables, this would also shorten the tests so they only include actual assertions



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues that break the projection.
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the table will never request that field.
   
   My initial idea for list of explicit accessors was its cleanly decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to keep that pattern and encapsulate the logic in a StructLike class like this ?  Or let me know which approach is better for you



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949703996


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   Made the suggested change.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949553559


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   One way to make the custom struct approach to work is to pass the projected schema, which will tell us the number of projected columns. Based on that, we can either delegate to the content file struct or return metrics.
   
   ```
   public CloseableIterable<StructLike> rows() {
     if (...) {
       return CloseableIterable.transform(files(), this::wrapWithMetrics);
     } else {
       return CloseableIterable.transform(files(), file -> (StructLike) file);
     }
   }
   
   private StructLike wrapWithMetrics(ContentFile<?> file) {
     Map<String, StructLike> metrics = MetricsUtil.readableMetricsMap(dataTableSchema, file);
     return new ContentFileStructWithMetrics(projectedSchema, (StructLike) file, metrics);
   }
   ```
   
   I don't mind using the accessor-based approach too but we can't initialize the accessor list, pick columns, etc for every row. We also can't assign them to a field as the task will be serialized. We will have to build them lazily.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026571367


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   I do use it to get the colName, because its the qualified column name, which I need for readable_metrics.  
   
   I think field.name() is not qualified.  Let me know if I miss something though



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */

Review Comment:
   Fixed



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026570061


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {

Review Comment:
   The problem is the partition field ids are also dynamic?  Didnt want to conflict with those, hence doing a check if it id is already in the schema before assigning.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1024931871


##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to {@link #findColumnName(int)}.
+   *
+   * @return a map of field id to qualified field names
+   */
+  public Map<Integer, String> idToName() {

Review Comment:
   Yea I had a memory of discussing this with @aokolnychyi but cant remember now for sure :)
   
   From looking now, I guess I need the list of ids as well, and so thought might as well return the map (its just as expensive to compute list of ids as the whole map).  I updated the comment, but let me know if you prefer something else.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho closed pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho closed pull request #5376: Core: Add readable metrics columns to files metadata tables
URL: https://github.com/apache/iceberg/pull/5376


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957730131


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +239,41 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final Map<String, StructLike> readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize, StructLike fileAsStruct, Map<String, StructLike> readableMetrics) {
+      this.structSize = structSize;
+      this.fileAsStruct = fileAsStruct;
+      this.readableMetrics = readableMetrics;
+    }
+
+    @Override
+    public int size() {
+      return structSize;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      if (pos < (structSize - 1)) {
+        return fileAsStruct.get(pos, javaClass);
+      } else if (pos == (structSize - 1)) {
+        return javaClass.cast(readableMetrics);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Illegal position access for ContentFileStructWithMetrics: %d, max allowed is %d",
+                pos, (structSize - 1)));
+      }
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("FileEntryRow is read only");

Review Comment:
   Sorry thought I updated, should be updated now



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =
+            TypeUtil.joinCommon(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);
+        return CloseableIterable.transform(files(minProjection), this::withReadableMetrics);
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike withReadableMetrics(ContentFile<?> file) {
+      int structSize = projection.columns().size();
+      Map<String, StructLike> metrics =
+          MetricsUtil.readableMetricsMap(
+              dataTableSchema, file, readableMetricsProjection(projection));
+      return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
+    }
+
+    // Handles projections for readable metrics struct
+    private Map<Integer, Integer> readableMetricsProjection(Schema projectedSchema) {

Review Comment:
   Good idea, moved.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r954092992


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());

Review Comment:
   The test is located in "TestMetadataTableReadableMetrics#testSelect", which was collapsed by github as its too big



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r942755436


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore

Review Comment:
   Yea I tried a put a comment, but unfortunately got disassociated with this line after a rebase.
   
   > This happens in some cases, I found it in some case of importing external files to Iceberg table, ie TestIcebergSourceHadoopTables.testFilesTableWithSnapshotIdInheritance, where the I think columns are out of order of the original schema and the metrics are corrupt (underflow exception in this case).
   > 
   > Not sure if we should error out the files tables, in that case, I was leaning towards just returning null. User has original column to see why the error happened.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949553559


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   One way to make the custom struct approach to work is to pass the projected schema, which will tell us the number of projected columns. Based on that, we can either delegate to the content file struct or return metrics.
   
   ```
   public CloseableIterable<StructLike> rows() {
     if (...) {
       return CloseableIterable.transform(files(), this::wrapWithMetrics);
     } else {
       return CloseableIterable.transform(files(), file -> (StructLike) file);
     }
   }
   ```
   
   I don't mind using the accessor-based approach too but we can't initialize the accessor list, pick columns, etc for every row. We also can't assign them to a field as the task will be serialized. We will have to build them lazily.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026578079


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Yea I think that would be nice, updating these tests are definitely the most painful part of this change.
   
   But the GenericRecord here is an Avro class that doesnt have any select methods.  It has a get() but it returns a field and not a projected record.  We'd have to maybe make a struct based on get() all 15 non-derived DataFile fields, not sure if that's cleaner?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025812897


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {

Review Comment:
   What's the purpose of this class? I'm not sure I understand the "usedIds.contains" portion? I think you also would probably be safe just using schema.highestFieldID?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025805720


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,
+        StructLike fileAsStruct,
+        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+      this.structSize = structSize;

Review Comment:
   Since we only use structSize -1, why not instead store (and change variable name) to
   ```
   lastStructIndex = fileAsStruct.size() - 1;
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1335702965

   @RussellSpitzer  should be good now for another look when you get a chance, 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014239621


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("file_path", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult = rowsToJava(mixedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(dataFile.path().toString(), 2L)),
+        mixedSelectResult);
+
+    Dataset<Row> mixedSelect2 =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("sort_order_id", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult2 = rowsToJava(mixedSelect2.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row(0, 2L)), mixedSelectResult2);
+
+    Dataset<Row> reversedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("readable_metrics.longCol.value_count", "file_path");
+    List<Object[]> reversedSelectResult = rowsToJava(reversedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(2L, dataFile.path().toString())),
+        reversedSelectResult);
+  }
+
+  @Test
+  public void testNullNanValues() throws Exception {
+    String tableName = "testNullNanValues";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =

Review Comment:
   This record could just be added to the base table all the tests use as well 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015598536


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Initially was done with <metric_type>.<column>, and I changed it from @aokolnychyi 's  comment:  https://github.com/apache/iceberg/pull/5376#issuecomment-1210148871
   
   Not sure if that is what you are referring?
   
   I guess its all subjective, but Anton's way is slightly easier if you want to select columns you want to see metrics for.  This way is easier if you want to select by metric type.
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r946368363


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore

Review Comment:
   @aokolnychyi filed an issue:  https://github.com/apache/iceberg/issues/5543



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957690407


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think putting it there will break the scan right, as its not the projection the user requested.  
   
   Note, this is actually a bit subtle here.  Because we are doing the join, (original projection + minimum metrics), the file becomes 
   {any_projected_field_on_file} : {readable_metrics because its also projected} : {un-projected but required metrics fields}
   
   So the ContentFileWithMetrics works because it will discard any of the "un-projected but required metrics fields", given they are outside the range it will read.  For the remaining fields it uses the existing logic (delegate to file for the first n-1, and then get from MetricsStruct for nth field).
   
   I mean, we could add a select method to GenericDataFile to modify its internal 'fromProjectionPos' map to conform back to the original projection (without the "un-projected but required metrics fields") for safety, but it's strictly needed.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944700255


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +174,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema filesTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> quotedNameById;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.projectedSchema = projectedSchema;
+      this.dataTableSchema = table.schema();
+      this.quotedNameById = TypeUtil.indexQuotedNameById(table.schema().asStruct(), name -> name);
+      this.isPartitioned = Partitioning.partitionType(table).fields().size() > 0;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      return CloseableIterable.transform(
+          manifestEntries(),
+          fileEntry ->
+              StaticDataTask.Row.of(
+                  projectedFields(fileEntry, accessors(isPartitioned)).toArray()));
     }
 
     private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+      Schema finalProjectedSchema =
+          TypeUtil.selectNot(filesTableSchema, TypeUtil.getProjectedIds(READABLE_METRICS.type()));

Review Comment:
   I think this broke file projection as we should not be using the full metadata schema. I believe `schema` previously referred to a projection.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -18,25 +18,50 @@
  */
 package org.apache.iceberg;
 
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.ManifestEvaluator;
 import org.apache.iceberg.expressions.ResidualEvaluator;
 import org.apache.iceberg.io.CloseableIterable;
 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.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 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.TypeUtil;
+import org.apache.iceberg.types.Types;
 import org.apache.iceberg.types.Types.StructType;
 
 /** Base class logic for files metadata tables */
 abstract class BaseFilesTable extends BaseMetadataTable {
 
+  static final Types.StructType READABLE_METRICS_VALUE =
+      Types.StructType.of(
+          optional(303, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(304, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(305, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(306, "nan_value_count", Types.LongType.get(), "Nan value count"),
+          optional(307, "lower_bound", Types.StringType.get(), "Lower bound in string form"),
+          optional(308, "upper_bound", Types.StringType.get(), "Upper bound in string form"));
+
+  static final Types.NestedField READABLE_METRICS =
+      required(
+          300,

Review Comment:
   We start from 300 to avoid future conflicts with the file schema, right?



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,127 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param quotedNameById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexQuotedNameById}
+   * @param columnSizes column size metrics
+   * @param valueCounts value count metrics
+   * @param nullValueCounts null value metrics
+   * @param nanValueCounts nan value metrics
+   * @param lowerBounds lower bound metrics
+   * @param upperBounds upper bound metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema,
+      Map<Integer, String> quotedNameById,
+      Map<Integer, Long> columnSizes,

Review Comment:
   What about passing `ContentFile<?>` directly to reduce the number of arguments?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +174,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema filesTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> quotedNameById;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.projectedSchema = projectedSchema;
+      this.dataTableSchema = table.schema();
+      this.quotedNameById = TypeUtil.indexQuotedNameById(table.schema().asStruct(), name -> name);

Review Comment:
   Do we actually need it since we are not quoting? Can't we use `schema.findColumnName(id)`?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   These two methods are hard to grasp. I think our goal is to wrap the struct we get back from reading manifests and attach metrics to it if they were projected. Let me think more on this one.



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -18,14 +18,22 @@
  */
 package org.apache.iceberg;
 
+import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class MetricsUtil {
 
+  private static final Logger logger = LoggerFactory.getLogger(MetricsUtil.class);

Review Comment:
   nit: `logger` -> `LOG` as it is a constant.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +174,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema filesTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> quotedNameById;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.projectedSchema = projectedSchema;
+      this.dataTableSchema = table.schema();
+      this.quotedNameById = TypeUtil.indexQuotedNameById(table.schema().asStruct(), name -> name);

Review Comment:
   I am not sure it is a good idea to compute this for every manifest task as there may be a lot of columns. I don't think it will degrade the performance too much but will definitely generate some extra objects. I'd probably compute it once and reuse to avoid surprises.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -18,25 +18,50 @@
  */
 package org.apache.iceberg;
 
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.ManifestEvaluator;
 import org.apache.iceberg.expressions.ResidualEvaluator;
 import org.apache.iceberg.io.CloseableIterable;
 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.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 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.TypeUtil;
+import org.apache.iceberg.types.Types;
 import org.apache.iceberg.types.Types.StructType;
 
 /** Base class logic for files metadata tables */
 abstract class BaseFilesTable extends BaseMetadataTable {
 
+  static final Types.StructType READABLE_METRICS_VALUE =
+      Types.StructType.of(
+          optional(303, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(304, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(305, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(306, "nan_value_count", Types.LongType.get(), "Nan value count"),

Review Comment:
   nit: `Nan` -> `NaN`



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +174,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema filesTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> quotedNameById;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.projectedSchema = projectedSchema;
+      this.dataTableSchema = table.schema();
+      this.quotedNameById = TypeUtil.indexQuotedNameById(table.schema().asStruct(), name -> name);
+      this.isPartitioned = Partitioning.partitionType(table).fields().size() > 0;

Review Comment:
   Can we check if the schema contains `DataFile.PARTITION_ID` instead of creating a common partition type? I think we constructed this common partition type earlier and included/excluded `DataFile.PARTITION_ID` based on that.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014235434


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("file_path", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult = rowsToJava(mixedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(dataFile.path().toString(), 2L)),
+        mixedSelectResult);
+
+    Dataset<Row> mixedSelect2 =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("sort_order_id", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult2 = rowsToJava(mixedSelect2.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row(0, 2L)), mixedSelectResult2);
+
+    Dataset<Row> reversedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("readable_metrics.longCol.value_count", "file_path");
+    List<Object[]> reversedSelectResult = rowsToJava(reversedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(2L, dataFile.path().toString())),
+        reversedSelectResult);
+  }
+
+  @Test
+  public void testNullNanValues() throws Exception {
+    String tableName = "testNullNanValues";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false, 0, 0, Float.NaN, Double.NaN, null, "0", null, null, null, null, null),
+            createPrimitiveRecord(
+                false,
+                0,
+                1,
+                Float.NaN,
+                1.0,
+                new BigDecimal("1.00"),
+                "1",
+                null,
+                null,
+                null,
+                null,
+                null));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    checkMetricValues(
+        row,
+        "null_value_count",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", 0L),
+            Maps.immutableEntry("stringCol", 0L),
+            Maps.immutableEntry("intCol", 0L),
+            Maps.immutableEntry("longCol", 0L),
+            Maps.immutableEntry("floatCol", 0L),
+            Maps.immutableEntry("doubleCol", 0L),
+            Maps.immutableEntry("decimalCol", 1L),
+            Maps.immutableEntry("binaryCol", 2L),
+            Maps.immutableEntry("fixedCol", 2L),
+            Maps.immutableEntry("dateCol", 2L),
+            Maps.immutableEntry("timeCol", 2L),
+            Maps.immutableEntry("timestampCol", 2L)));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("booleanCol", null);
+    expectedNanValues.put("stringCol", null);
+    expectedNanValues.put("intCol", null);
+    expectedNanValues.put("longCol", null);
+    expectedNanValues.put("floatCol", 2L);
+    expectedNanValues.put("doubleCol", 1L);
+    expectedNanValues.put("decimalCol", null);
+    expectedNanValues.put("binaryCol", null);
+    expectedNanValues.put("fixedCol", null);
+    expectedNanValues.put("dateCol", null);
+    expectedNanValues.put("timeCol", null);
+    expectedNanValues.put("timestampCol", null);
+
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+  }
+
+  @Test
+  public void testNestedValues() throws Exception {
+    String tableName = "testNestedValues";
+    Table table = createTable(tableName, NESTED_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createNestedRecord(0L, 0.0),
+            createNestedRecord(1L, Double.NaN),
+            createNestedRecord(null, null));
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    String[] nestedColumns =
+        new String[] {
+          "nestedStructCol.leafStructCol.leafDoubleCol", "nestedStructCol.leafStructCol.leafLongCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        nestedColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Value count should be 3", l.longValue(), 3L));
+    checkMetric(
+        row,
+        "null_value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Null value count should be 1", l.longValue(), 1L));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafDoubleCol", 1L);
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafLongCol", null);
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "0",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "1",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+  }
+
+  private <T> void checkCollectionValues(Row row, String columnName, Set<T> expectedValues) {
+    Set<Long> actualValues =
+        Sets.newHashSet(JavaConverters.asJavaCollection(row.getAs(columnName)));
+    Assert.assertEquals("Collection values should match", expectedValues, actualValues);
+  }
+
+  private void checkMetric(Row row, String metricName, String[] columns, Consumer<Long> check) {

Review Comment:
   So this is for checking if a certain metric is present for all columns and then run an arbitrary long function on that value?
   
   I think it may help to break this into a function that reports missing metrics and a function which checks if a boolean expression is true for all such values. Currently we end up passing in check functions that have to implement their own assertions which seems a little confusing to me. 
   
   Another option maybe something like
   
   ```
     private <T> void checkMetric(Row row, String metricName, String[] columns, Predicate<T> check) {
       scala.collection.Map<String, Row> metrics = row.getAs("readable_metrics");
       for (String column : columns) {
         Option<Row> rowOption = metrics.get(column);
         Assert.assertTrue("Missing metric for column: " + column, rowOption.isDefined());
         T metric = rowOption.get().getAs(metricName);
         Assert.assertTrue("Check on metric ....", check.test(metric));
       }
     }
   ```
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015583682


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   For the other metrics in this struct, did we not have all the counts correctly before? I thought those all came out as longs anyway



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015605206


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("file_path", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult = rowsToJava(mixedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(dataFile.path().toString(), 2L)),
+        mixedSelectResult);
+
+    Dataset<Row> mixedSelect2 =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("sort_order_id", "readable_metrics.longCol.value_count");
+
+    List<Object[]> mixedSelectResult2 = rowsToJava(mixedSelect2.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row(0, 2L)), mixedSelectResult2);
+
+    Dataset<Row> reversedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select("readable_metrics.longCol.value_count", "file_path");
+    List<Object[]> reversedSelectResult = rowsToJava(reversedSelect.collectAsList());
+    assertEquals(
+        "Rows do not match",
+        ImmutableList.of(row(2L, dataFile.path().toString())),
+        reversedSelectResult);
+  }
+
+  @Test
+  public void testNullNanValues() throws Exception {
+    String tableName = "testNullNanValues";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false, 0, 0, Float.NaN, Double.NaN, null, "0", null, null, null, null, null),
+            createPrimitiveRecord(
+                false,
+                0,
+                1,
+                Float.NaN,
+                1.0,
+                new BigDecimal("1.00"),
+                "1",
+                null,
+                null,
+                null,
+                null,
+                null));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    checkMetricValues(
+        row,
+        "null_value_count",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", 0L),
+            Maps.immutableEntry("stringCol", 0L),
+            Maps.immutableEntry("intCol", 0L),
+            Maps.immutableEntry("longCol", 0L),
+            Maps.immutableEntry("floatCol", 0L),
+            Maps.immutableEntry("doubleCol", 0L),
+            Maps.immutableEntry("decimalCol", 1L),
+            Maps.immutableEntry("binaryCol", 2L),
+            Maps.immutableEntry("fixedCol", 2L),
+            Maps.immutableEntry("dateCol", 2L),
+            Maps.immutableEntry("timeCol", 2L),
+            Maps.immutableEntry("timestampCol", 2L)));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("booleanCol", null);
+    expectedNanValues.put("stringCol", null);
+    expectedNanValues.put("intCol", null);
+    expectedNanValues.put("longCol", null);
+    expectedNanValues.put("floatCol", 2L);
+    expectedNanValues.put("doubleCol", 1L);
+    expectedNanValues.put("decimalCol", null);
+    expectedNanValues.put("binaryCol", null);
+    expectedNanValues.put("fixedCol", null);
+    expectedNanValues.put("dateCol", null);
+    expectedNanValues.put("timeCol", null);
+    expectedNanValues.put("timestampCol", null);
+
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+  }
+
+  @Test
+  public void testNestedValues() throws Exception {
+    String tableName = "testNestedValues";
+    Table table = createTable(tableName, NESTED_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createNestedRecord(0L, 0.0),
+            createNestedRecord(1L, Double.NaN),
+            createNestedRecord(null, null));
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+
+    String[] nestedColumns =
+        new String[] {
+          "nestedStructCol.leafStructCol.leafDoubleCol", "nestedStructCol.leafStructCol.leafLongCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        nestedColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Value count should be 3", l.longValue(), 3L));
+    checkMetric(
+        row,
+        "null_value_count",
+        nestedColumns,
+        l -> Assert.assertEquals("Null value count should be 1", l.longValue(), 1L));
+
+    Map<String, Long> expectedNanValues = Maps.newHashMap();
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafDoubleCol", 1L);
+    expectedNanValues.put("nestedStructCol.leafStructCol.leafLongCol", null);
+    checkMetricValues(row, "nan_value_count", expectedNanValues);
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "0",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.of(
+            "nestedStructCol.leafStructCol.leafLongCol", "1",
+            "nestedStructCol.leafStructCol.leafDoubleCol", "0.0"));
+  }
+
+  private <T> void checkCollectionValues(Row row, String columnName, Set<T> expectedValues) {
+    Set<Long> actualValues =
+        Sets.newHashSet(JavaConverters.asJavaCollection(row.getAs(columnName)));
+    Assert.assertEquals("Collection values should match", expectedValues, actualValues);
+  }
+
+  private void checkMetric(Row row, String metricName, String[] columns, Consumer<Long> check) {

Review Comment:
   Got rid of these custom methods, used assertEquals() and sql() for everything



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026573190


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,

Review Comment:
   I tried to change the code and didnt work.  This is actually the expectedSize ('expected' to use the terminology in other places, aka the size of user 'projection').  The fileAsStruct.size() is actually always all the possible DataFile values, if they are not projected they are just null.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1223303757

   Let me take a look today.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038472949


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +69,293 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricColDefinition> READABLE_METRIC_COLS =
+      ImmutableList.of(
+          new ReadableMetricColDefinition(
+              "column_size",
+              "Total size on disk",
+              DataFile.COLUMN_SIZES,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.columnSizes() == null ? null : file.columnSizes().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "value_count",
+              "Total count, including null and NaN",
+              DataFile.VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.valueCounts() == null ? null : file.valueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "null_value_count",
+              "Null value count",
+              DataFile.NULL_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nullValueCounts() == null
+                      ? null
+                      : file.nullValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "nan_value_count",
+              "NaN value count",
+              DataFile.NAN_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nanValueCounts() == null
+                      ? null
+                      : file.nanValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "lower_bound",
+              "Lower bound",
+              DataFile.LOWER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.lowerBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.lowerBounds().get(field.fieldId()))),
+          new ReadableMetricColDefinition(
+              "upper_bound",
+              "Upper bound",
+              DataFile.UPPER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.upperBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.upperBounds().get(field.fieldId()))));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  /**
+   * Fixed definition of a readable metric column, ie a mapping of a raw metric to a readable metric
+   */
+  public static class ReadableMetricColDefinition {
+    private final String name;
+    private final String doc;
+    private final Types.NestedField originalCol;
+    private final TypeFunction typeFunction;
+    private final MetricFunction metricFunction;
+
+    public interface TypeFunction {
+      Type type(Types.NestedField originalCol);
+    }
+
+    public interface MetricFunction {
+      Object metric(ContentFile<?> file, Types.NestedField originalCol);
+    }
+
+    /**
+     * @param name column name
+     * @param doc column doc
+     * @param originalCol original (raw) metric column field on metadata table
+     * @param typeFunction function that returns the readable metric column type from original field
+     *     type
+     * @param metricFunction function that returns readable metric from data file
+     */
+    ReadableMetricColDefinition(
+        String name,
+        String doc,
+        Types.NestedField originalCol,
+        TypeFunction typeFunction,
+        MetricFunction metricFunction) {
+      this.name = name;
+      this.doc = doc;
+      this.originalCol = originalCol;
+      this.typeFunction = typeFunction;
+      this.metricFunction = metricFunction;
+    }
+
+    Types.NestedField originalCol() {
+      return originalCol;
+    }
+
+    Type colType(Types.NestedField field) {
+      return typeFunction.type(field);
+    }
+
+    String name() {
+      return name;
+    }
+
+    String doc() {
+      return doc;
+    }
+
+    Object value(ContentFile<?> dataFile, Types.NestedField dataField) {
+      return metricFunction.metric(dataFile, dataField);
+    }
+  }
+
+  /** A struct of readable metric values for a primitive column */
+  public static class ReadableColMetricsStruct implements StructLike {
+
+    private final String columnName;
+    private final Map<Integer, Integer> projectionMap;
+    private final Object[] metrics;
+
+    public ReadableColMetricsStruct(
+        String columnName, Types.NestedField projection, Object... metrics) {
+      this.columnName = columnName;
+      this.projectionMap = readableMetricsProjection(projection);
+      this.metrics = metrics;
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      return metrics[projectedPos];
+    }
+
+    /** Returns map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_METRIC_COLS.size(); fieldIndex++) {
+        ReadableMetricColDefinition readableMetric = READABLE_METRIC_COLS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * A struct, consisting of all {@link ReadableColMetricsStruct} for all primitive columns of the
+   * table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metadataTableSchema) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    AtomicInteger nextId =

Review Comment:
   I don't believe this has to be atomic, and metadataTableSchema should already have a method `highestFieldId()` which according to the doc includes nested fields



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1037486854


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   Hm as I can see, schema.columns() will return only the top level columns, whereas Schema.idToName returns ids of all columns even nested ones (which I need)



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1335310296

   Actually hold on a second, looking at a small refactor to make it more generic to add a readable_metric definition in 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r950416982


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {

Review Comment:
   Not strictly necessarily but thought its cleaner as now passing a projection with only fields directly from the file



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1024565829


##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to {@link #findColumnName(int)}.
+   *
+   * @return a map of field id to qualified field names
+   */
+  public Map<Integer, String> idToName() {

Review Comment:
   I'm generally a bit nervous about expanding the api here, is this better than just re-using findColumnName(id)? 
   
   That said I don't think lazyIdToName() is that dangerous, but the java doc for this method contains a lot of very specific implementation details which should probably be removed. 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1024931871


##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to {@link #findColumnName(int)}.
+   *
+   * @return a map of field id to qualified field names
+   */
+  public Map<Integer, String> idToName() {

Review Comment:
   Yea I had a memory of discussing this with @aokolnychyi but cant remember now for sure :)
   
   From looking now, I guess I need the list of ids as well, and so thought might as well return the map (as laziness wise its just as expensive to compute list of ids as the whole map).  I updated the comment, but let me know if you prefer something else.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1037486854


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   Hm as I can see, schema.columns() will return only the top level columns, whereas Schema.idToName returns ids of all columns even nested ones. 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1035316094


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {

Review Comment:
   Yeah but shouldn't we already have the schema for the table without our new metrics columns? So should we just be able to start at highestFieldId from that pre union schema?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038463689


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java:
##########
@@ -486,6 +489,11 @@ public void testFilesTableWithSnapshotIdInheritance() throws Exception {
     Dataset<Row> inputDF = spark.createDataFrame(records, SimpleRecord.class);
     inputDF.select("data", "id").write().mode("overwrite").insertInto("parquet_table");
 
+    NameMapping mapping = MappingUtil.create(table.schema());

Review Comment:
   Test used to write the wrong metrics to imported table, without these lines.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038474080


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +69,293 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricColDefinition> READABLE_METRIC_COLS =
+      ImmutableList.of(
+          new ReadableMetricColDefinition(
+              "column_size",
+              "Total size on disk",
+              DataFile.COLUMN_SIZES,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.columnSizes() == null ? null : file.columnSizes().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "value_count",
+              "Total count, including null and NaN",
+              DataFile.VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.valueCounts() == null ? null : file.valueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "null_value_count",
+              "Null value count",
+              DataFile.NULL_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nullValueCounts() == null
+                      ? null
+                      : file.nullValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "nan_value_count",
+              "NaN value count",
+              DataFile.NAN_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nanValueCounts() == null
+                      ? null
+                      : file.nanValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "lower_bound",
+              "Lower bound",
+              DataFile.LOWER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.lowerBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.lowerBounds().get(field.fieldId()))),
+          new ReadableMetricColDefinition(
+              "upper_bound",
+              "Upper bound",
+              DataFile.UPPER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.upperBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.upperBounds().get(field.fieldId()))));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  /**
+   * Fixed definition of a readable metric column, ie a mapping of a raw metric to a readable metric
+   */
+  public static class ReadableMetricColDefinition {
+    private final String name;
+    private final String doc;
+    private final Types.NestedField originalCol;
+    private final TypeFunction typeFunction;
+    private final MetricFunction metricFunction;
+
+    public interface TypeFunction {
+      Type type(Types.NestedField originalCol);
+    }
+
+    public interface MetricFunction {
+      Object metric(ContentFile<?> file, Types.NestedField originalCol);
+    }
+
+    /**
+     * @param name column name
+     * @param doc column doc
+     * @param originalCol original (raw) metric column field on metadata table
+     * @param typeFunction function that returns the readable metric column type from original field
+     *     type
+     * @param metricFunction function that returns readable metric from data file
+     */
+    ReadableMetricColDefinition(
+        String name,
+        String doc,
+        Types.NestedField originalCol,
+        TypeFunction typeFunction,
+        MetricFunction metricFunction) {
+      this.name = name;
+      this.doc = doc;
+      this.originalCol = originalCol;
+      this.typeFunction = typeFunction;
+      this.metricFunction = metricFunction;
+    }
+
+    Types.NestedField originalCol() {
+      return originalCol;
+    }
+
+    Type colType(Types.NestedField field) {
+      return typeFunction.type(field);
+    }
+
+    String name() {
+      return name;
+    }
+
+    String doc() {
+      return doc;
+    }
+
+    Object value(ContentFile<?> dataFile, Types.NestedField dataField) {
+      return metricFunction.metric(dataFile, dataField);
+    }
+  }
+
+  /** A struct of readable metric values for a primitive column */
+  public static class ReadableColMetricsStruct implements StructLike {
+
+    private final String columnName;
+    private final Map<Integer, Integer> projectionMap;
+    private final Object[] metrics;
+
+    public ReadableColMetricsStruct(
+        String columnName, Types.NestedField projection, Object... metrics) {
+      this.columnName = columnName;
+      this.projectionMap = readableMetricsProjection(projection);
+      this.metrics = metrics;
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      return metrics[projectedPos];
+    }
+
+    /** Returns map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_METRIC_COLS.size(); fieldIndex++) {
+        ReadableMetricColDefinition readableMetric = READABLE_METRIC_COLS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * A struct, consisting of all {@link ReadableColMetricsStruct} for all primitive columns of the
+   * table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metadataTableSchema) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    AtomicInteger nextId =

Review Comment:
   If you prefer incrementAndGet() to ++nextId though I think using the Atomic just for readability is probably 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] chenjunjiedada commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1300269495

   Really nice PR, thanks @szehon-ho and @aokolnychyi for the effort!  When can we merge this? I think it is ready and has been two months since the last review, which will lead to more conflicts if leave it.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1210148871

   It seems like a great idea to add readable metrics. It is hard to make sense of them otherwise.
   
   @szehon-ho, what do you think about adding a single map column, let's say called `readable_metrics`, that will hold a mapping from a column name into a struct that would represent metrics? The type will be `Map<String, StructType>` and we will have individual struct fields for each type of metric.
   
   We can then easily access them via SQL.
   ```
   SELECT readable_metrics['col1'].lower_bound FROM db.t.files
   ```
   
   I am okay with individual columns too but it seems a bit cleaner to just have one.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1017644858


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Discussed offline, will look into how hard it is to make into a dynamic struct with the right types, assuming we keep this current hierarchy (readable_metric, col, metric_type)



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1022594631


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +142,75 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    private static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+
+        // If readable_metrics is selected,
+        // original metrics columns need to be selected for derivation
+        Schema minProjection = TypeUtil.join(fileProjection, MIN_PROJECTION_FOR_READABLE_METRICS);

Review Comment:
   Hm not sure what is a good name.  To me max projection is not right (that would just be all the columns of files table).  So I just put projection for now, and we can optimize it later?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1315100949

   @RussellSpitzer i think it should be as we discussed now, the readable_metrics is a dynamic type of column metrics for all primitive columns by qualified name.  Each column metric is a struct , of which upper/lower bounds is the original column type.
   
   Added code to generate the dynamic schema and handle projection (previously it was a map , so no projection needed).


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026573190


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,

Review Comment:
   I tried to change the code and didnt work.  This is actually the expectedSize (to use the terminology in other places, aka the user-requested size).  The fileAsStruct.size() is actually always all the possible DataFile values, if they are not projected they are just null..



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026574052


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,
+        StructLike fileAsStruct,
+        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+      this.structSize = structSize;

Review Comment:
   I do use the variable in size(), so kept the field declaration as is.  I added a  local variable in get() that is fileAsStruct.size() - 1 as suggested, hope that works.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r943043336


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =

Review Comment:
   It's actually for an existing test (TestIcebergSourceTablesBase), where it needs it to construct the 'expected' record from files table.  In the end, I moved it to BaseFilesTable and then just duplicated it in all the test classes.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r950416982


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {

Review Comment:
   Optional and not strictly necessarily but thought its cleaner as now passing a projection with only fields directly from the file, as anyway its calculated in the caller.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {

Review Comment:
   Optional and not strictly necessarily but thought its cleaner as now passing a projection with only fields directly from the file, as anyway its calculated in the caller, let me know if it makes sense or not



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957690407


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think putting it there will break the scan right, as its not the projection the user requested.  
   
   Note, this is actually a bit subtle here.  Because we are doing the join, (original projection + minimum metrics), the file's schema becomes 
   {any_projected_field_on_file} : {readable_metrics because its also projected} : {un-projected but required metrics fields}
   
   So the ContentFileWithMetrics works because it will discard any of the "un-projected but required metrics fields", given they are outside the range it will read.  For the remaining fields it uses the existing logic (delegate to file for the first n-1, and then get from MetricsStruct for nth field).
   
   I mean, we could add a select method to GenericDataFile to modify its internal 'fromProjectionPos' map to conform back to the original projection (without the "un-projected but required metrics fields") for safety, but it's strictly needed.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1233534601

   Added additional test, looks it is working even when readable_metric column is selected before other columns (spark somehow calls the rows in their original order)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r941997542


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =

Review Comment:
   Can we make it package-private in the table and have tests in the corresponding module?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944973387


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -18,25 +18,50 @@
  */
 package org.apache.iceberg;
 
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.ManifestEvaluator;
 import org.apache.iceberg.expressions.ResidualEvaluator;
 import org.apache.iceberg.io.CloseableIterable;
 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.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 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.TypeUtil;
+import org.apache.iceberg.types.Types;
 import org.apache.iceberg.types.Types.StructType;
 
 /** Base class logic for files metadata tables */
 abstract class BaseFilesTable extends BaseMetadataTable {
 
+  static final Types.StructType READABLE_METRICS_VALUE =
+      Types.StructType.of(
+          optional(303, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(304, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(305, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(306, "nan_value_count", Types.LongType.get(), "Nan value count"),
+          optional(307, "lower_bound", Types.StringType.get(), "Lower bound in string form"),
+          optional(308, "upper_bound", Types.StringType.get(), "Upper bound in string form"));
+
+  static final Types.NestedField READABLE_METRICS =
+      required(
+          300,

Review Comment:
   Yea that's right, DataFile currently is from 100-200.  Not sure if its the right way, or we should put these fields on DataFile.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015605561


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> df = spark.read().format("iceberg").load("default." + tableName + ".files");
+
+    List<Row> rows = df.collectAsList();
+    Assert.assertEquals("Expected only one data file", 1, rows.size());
+    Row row = rows.get(0);
+    String[] primitiveColumns =
+        new String[] {
+          "booleanCol",
+          "intCol",
+          "longCol",
+          "floatCol",
+          "doubleCol",
+          "decimalCol",
+          "stringCol",
+          "dateCol",
+          "timeCol",
+          "timestampCol",
+          "fixedCol",
+          "binaryCol"
+        };
+    checkMetric(
+        row,
+        "column_size",
+        primitiveColumns,
+        l -> Assert.assertTrue("Column size should be greater than 0", l > 0));
+    checkMetric(
+        row,
+        "value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Value count should be 2", l.longValue(), 2L));
+    checkMetric(
+        row,
+        "null_value_count",
+        primitiveColumns,
+        l -> Assert.assertEquals("Null value count should be 0", l.longValue(), 0L));
+    checkMetric(
+        row,
+        "nan_value_count",
+        new String[] {"floatCol", "doubleCol"},
+        l -> Assert.assertEquals("Nan value count should be 0", l.longValue(), 0L));
+
+    checkMetricValues(
+        row,
+        "lower_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "false"),
+            Maps.immutableEntry("stringCol", "1"),
+            Maps.immutableEntry("intCol", "1"),
+            Maps.immutableEntry("longCol", "1"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "1.0"),
+            Maps.immutableEntry("decimalCol", "1.00"),
+            Maps.immutableEntry("binaryCol", "1111"),
+            Maps.immutableEntry("fixedCol", "1111"),
+            Maps.immutableEntry("dateCol", "1970-01-02"),
+            Maps.immutableEntry("timeCol", "00:00:00.000001"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000001")));
+
+    checkMetricValues(
+        row,
+        "upper_bound",
+        ImmutableMap.ofEntries(
+            Maps.immutableEntry("booleanCol", "true"),
+            Maps.immutableEntry("stringCol", "2"),
+            Maps.immutableEntry("intCol", "2"),
+            Maps.immutableEntry("longCol", "2"),
+            Maps.immutableEntry("floatCol", "0.0"),
+            Maps.immutableEntry("doubleCol", "2.0"),
+            Maps.immutableEntry("decimalCol", "2.00"),
+            Maps.immutableEntry("binaryCol", "2222"),
+            Maps.immutableEntry("fixedCol", "2222"),
+            Maps.immutableEntry("dateCol", "1970-01-03"),
+            Maps.immutableEntry("timeCol", "00:00:00.000002"),
+            Maps.immutableEntry("timestampCol", "1970-01-01T00:00:00.000002")));
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    String tableName = "testselect";
+    Table table = createTable(tableName, PRIMITIVE_SCHEMA, PartitionSpec.unpartitioned());
+
+    List<Record> records =
+        Lists.newArrayList(
+            createPrimitiveRecord(
+                false,
+                1,
+                1L,
+                0,
+                1.0D,
+                new BigDecimal("1.00"),
+                "1",
+                DateTimeUtil.dateFromDays(1),
+                DateTimeUtil.timeFromMicros(1),
+                DateTimeUtil.timestampFromMicros(1L),
+                Base64.getDecoder().decode("1111"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("1111"))),
+            createPrimitiveRecord(
+                true,
+                2,
+                2L,
+                0,
+                2.0D,
+                new BigDecimal("2.00"),
+                "2",
+                DateTimeUtil.dateFromDays(2),
+                DateTimeUtil.timeFromMicros(2),
+                DateTimeUtil.timestampFromMicros(2L),
+                Base64.getDecoder().decode("2222"),
+                ByteBuffer.wrap(Base64.getDecoder().decode("2222"))));
+
+    DataFile dataFile =
+        FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records);
+    table.newAppend().appendFile(dataFile).commit();
+
+    Dataset<Row> nestedSelect =
+        spark
+            .read()
+            .format("iceberg")
+            .load("default." + tableName + ".files")
+            .select(
+                "readable_metrics.intCol.lower_bound", "readable_metrics.booleanCol.upper_bound");
+
+    List<Object[]> metricSelectResult = rowsToJava(nestedSelect.collectAsList());
+    assertEquals("Rows do not match", ImmutableList.of(row("1", "true")), metricSelectResult);
+
+    Dataset<Row> mixedSelect =

Review Comment:
   Done, refactored all the tests to use this.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.source;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+public class TestMetadataTableReadableMetrics extends SparkTestBaseWithCatalog {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private static final Types.StructType LEAF_STRUCT_TYPE =
+      Types.StructType.of(
+          optional(1, "leafLongCol", Types.LongType.get()),
+          optional(2, "leafDoubleCol", Types.DoubleType.get()));
+
+  private static final Types.StructType NESTED_STRUCT_TYPE =
+      Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE));
+
+  private static final Schema NESTED_SCHEMA =
+      new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE));
+
+  private static final Schema PRIMITIVE_SCHEMA =
+      new Schema(
+          required(1, "booleanCol", Types.BooleanType.get()),
+          required(2, "intCol", Types.IntegerType.get()),
+          required(3, "longCol", Types.LongType.get()),
+          required(4, "floatCol", Types.FloatType.get()),
+          required(5, "doubleCol", Types.DoubleType.get()),
+          optional(6, "decimalCol", Types.DecimalType.of(10, 2)),
+          optional(7, "stringCol", Types.StringType.get()),
+          optional(8, "dateCol", Types.DateType.get()),
+          optional(9, "timeCol", Types.TimeType.get()),
+          optional(10, "timestampCol", Types.TimestampType.withoutZone()),
+          optional(11, "fixedCol", Types.FixedType.ofLength(3)),
+          optional(12, "binaryCol", Types.BinaryType.get()));
+
+  public TestMetadataTableReadableMetrics() {
+    // to support metadata table sql queries
+    super(SparkCatalogConfig.HIVE);
+  }
+
+  protected Table createTable(String name, Schema schema, PartitionSpec spec) {
+    return catalog.createTable(
+        TableIdentifier.of("default", name), schema, spec, ImmutableMap.of());
+  }
+
+  protected void dropTable(String name) {
+    catalog.dropTable(TableIdentifier.of("default", name));
+  }
+
+  protected GenericRecord createPrimitiveRecord(
+      boolean booleanCol,
+      int intCol,
+      long longCol,
+      float floatCol,
+      double doubleCol,
+      BigDecimal decimalCol,
+      String stringCol,
+      LocalDate dateCol,
+      LocalTime timeCol,
+      LocalDateTime timestampCol,
+      byte[] fixedCol,
+      ByteBuffer binaryCol) {
+    GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA);
+    record.set(0, booleanCol);
+    record.set(1, intCol);
+    record.set(2, longCol);
+    record.set(3, floatCol);
+    record.set(4, doubleCol);
+    record.set(5, decimalCol);
+    record.set(6, stringCol);
+    record.set(7, dateCol);
+    record.set(8, timeCol);
+    record.set(9, timestampCol);
+    record.set(10, fixedCol);
+    record.set(11, binaryCol);
+    return record;
+  }
+
+  private GenericRecord createNestedRecord(Long longCol, Double doubleCol) {
+    GenericRecord record = GenericRecord.create(NESTED_SCHEMA);
+    GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE);
+    GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE);
+    leaf.set(0, longCol);
+    leaf.set(1, doubleCol);
+    nested.set(0, leaf);
+    record.set(0, nested);
+    return record;
+  }
+
+  @Test
+  public void testPrimitiveColumns() throws Exception {
+    String tableName = "primitiveColumns";

Review Comment:
   Done



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014048088


##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -142,9 +142,21 @@ public static Schema selectNot(Schema schema, Set<Integer> fieldIds) {
   }
 
   public static Schema join(Schema left, Schema right) {
-    List<Types.NestedField> joinedColumns = Lists.newArrayList();
-    joinedColumns.addAll(left.columns());

Review Comment:
   Technically this is changing a public API which previously would have allowed these combos. This is ... maybe ok since it's a utility method but we may end up breaking users of the function at runtime. That said I think Anton is right and any schema with multiple columns with the same ID would always be wrong.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1022604659


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore

Review Comment:
   Following up on this, this is a non-issue as the spark procedures set the flag: schema.name-mapping.default , just this test does not.  Fixed the test.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1315643542

   Transitive error downloading, restarting


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1035320608


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -102,7 +102,8 @@ public interface DataFile extends ContentFile<DataFile> {
   int PARTITION_ID = 102;
   String PARTITION_NAME = "partition";
   String PARTITION_DOC = "Partition data tuple, schema based on the partition spec";
-  // NEXT ID TO ASSIGN: 142
+
+  int NEXT_ID_TO_ASSIGN = 142;

Review Comment:
   I think I mention this below but I think we don't need this as a constant, since we can just pick the highest value from the base generated for the files table?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1038602262


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +69,293 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricColDefinition> READABLE_METRIC_COLS =
+      ImmutableList.of(
+          new ReadableMetricColDefinition(
+              "column_size",
+              "Total size on disk",
+              DataFile.COLUMN_SIZES,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.columnSizes() == null ? null : file.columnSizes().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "value_count",
+              "Total count, including null and NaN",
+              DataFile.VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.valueCounts() == null ? null : file.valueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "null_value_count",
+              "Null value count",
+              DataFile.NULL_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nullValueCounts() == null
+                      ? null
+                      : file.nullValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "nan_value_count",
+              "NaN value count",
+              DataFile.NAN_VALUE_COUNTS,
+              field -> Types.LongType.get(),
+              (file, field) ->
+                  file.nanValueCounts() == null
+                      ? null
+                      : file.nanValueCounts().get(field.fieldId())),
+          new ReadableMetricColDefinition(
+              "lower_bound",
+              "Lower bound",
+              DataFile.LOWER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.lowerBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.lowerBounds().get(field.fieldId()))),
+          new ReadableMetricColDefinition(
+              "upper_bound",
+              "Upper bound",
+              DataFile.UPPER_BOUNDS,
+              Types.NestedField::type,
+              (file, field) ->
+                  file.upperBounds() == null
+                      ? null
+                      : Conversions.fromByteBuffer(
+                          field.type(), file.upperBounds().get(field.fieldId()))));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  /**
+   * Fixed definition of a readable metric column, ie a mapping of a raw metric to a readable metric
+   */
+  public static class ReadableMetricColDefinition {
+    private final String name;
+    private final String doc;
+    private final Types.NestedField originalCol;
+    private final TypeFunction typeFunction;
+    private final MetricFunction metricFunction;
+
+    public interface TypeFunction {
+      Type type(Types.NestedField originalCol);
+    }
+
+    public interface MetricFunction {
+      Object metric(ContentFile<?> file, Types.NestedField originalCol);
+    }
+
+    /**
+     * @param name column name
+     * @param doc column doc
+     * @param originalCol original (raw) metric column field on metadata table
+     * @param typeFunction function that returns the readable metric column type from original field
+     *     type
+     * @param metricFunction function that returns readable metric from data file
+     */
+    ReadableMetricColDefinition(
+        String name,
+        String doc,
+        Types.NestedField originalCol,
+        TypeFunction typeFunction,
+        MetricFunction metricFunction) {
+      this.name = name;
+      this.doc = doc;
+      this.originalCol = originalCol;
+      this.typeFunction = typeFunction;
+      this.metricFunction = metricFunction;
+    }
+
+    Types.NestedField originalCol() {
+      return originalCol;
+    }
+
+    Type colType(Types.NestedField field) {
+      return typeFunction.type(field);
+    }
+
+    String name() {
+      return name;
+    }
+
+    String doc() {
+      return doc;
+    }
+
+    Object value(ContentFile<?> dataFile, Types.NestedField dataField) {
+      return metricFunction.metric(dataFile, dataField);
+    }
+  }
+
+  /** A struct of readable metric values for a primitive column */
+  public static class ReadableColMetricsStruct implements StructLike {
+
+    private final String columnName;
+    private final Map<Integer, Integer> projectionMap;
+    private final Object[] metrics;
+
+    public ReadableColMetricsStruct(
+        String columnName, Types.NestedField projection, Object... metrics) {
+      this.columnName = columnName;
+      this.projectionMap = readableMetricsProjection(projection);
+      this.metrics = metrics;
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      return metrics[projectedPos];
+    }
+
+    /** Returns map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_METRIC_COLS.size(); fieldIndex++) {
+        ReadableMetricColDefinition readableMetric = READABLE_METRIC_COLS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * A struct, consisting of all {@link ReadableColMetricsStruct} for all primitive columns of the
+   * table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metadataTableSchema) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    AtomicInteger nextId =

Review Comment:
   Re: highestFieldId(), good to know!  Done.



##########
core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java:
##########
@@ -527,6 +527,63 @@ public void testDeleteFilesTableSelection() throws IOException {
     Assert.assertEquals(expected, scan.schema().asStruct());
   }
 
+  @Test
+  public void testFilesTableReadableMetricsSchema() {
+
+    Table filesTable = new FilesTable(table.ops(), table);
+    Types.StructType actual = filesTable.newScan().schema().select("readable_metrics").asStruct();
+

Review Comment:
   Done



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1335928345

   Yep , test should be here:   https://github.com/apache/iceberg/blob/6681dba9bc7dc0d793aa8de739d2b9962260b0ff/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025804938


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,

Review Comment:
   Do we need to pass this? StructLike has a size() operator



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r942000239


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore

Review Comment:
   Do you have examples when this throws an exception?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949704407


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,123 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param namesById pre-computed map of all column ids in schema to readable name, see {@link
+   *     org.apache.iceberg.types.TypeUtil#indexNameById(Types.StructType)}
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, Map<Integer, String> namesById, ContentFile<?> contentFile) {
+    Map<String, StructLike> metricsStruct = Maps.newHashMapWithExpectedSize(namesById.size());
+
+    Map<Integer, Long> columnSizes = contentFile.columnSizes();
+    Map<Integer, Long> valueCounts = contentFile.valueCounts();
+    Map<Integer, Long> nullValueCounts = contentFile.nullValueCounts();
+    Map<Integer, Long> nanValueCounts = contentFile.nanValueCounts();
+    Map<Integer, ByteBuffer> lowerBounds = contentFile.lowerBounds();
+    Map<Integer, ByteBuffer> upperBounds = contentFile.upperBounds();
+
+    for (int id : namesById.keySet()) {
+      Types.NestedField field = schema.findField(id);
+      if (field.type().isPrimitiveType()) {
+        // Iceberg stores metrics only for primitive types
+        String colName = namesById.get(id);
+        ReadableMetricsStruct struct =
+            new ReadableMetricsStruct(
+                columnSizes == null ? null : columnSizes.get(id),
+                valueCounts == null ? null : valueCounts.get(id),
+                nullValueCounts == null ? null : nullValueCounts.get(id),
+                nanValueCounts == null ? null : nanValueCounts.get(id),
+                lowerBounds == null ? null : convertToReadable(field, lowerBounds.get(id)),
+                upperBounds == null ? null : convertToReadable(field, upperBounds.get(id)));
+        metricsStruct.put(colName, struct);
+      }
+    }
+    return metricsStruct;
+  }
+
+  public static String convertToReadable(Types.NestedField field, ByteBuffer value) {
+    if (field == null || value == null) {
+      return null;
+    }
+    try {
+      return Transforms.identity(field.type())
+          .toHumanString(Conversions.fromByteBuffer(field.type(), value));
+    } catch (Exception e) {
+      LOG.warn("Error converting metric to readable form", e);
+      return null;
+    }
+  }
+
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final String lowerBound;
+    private final String upperBound;
+
+    public ReadableMetricsStruct(
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        String lowerBound,
+        String upperBound) {
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+    }
+
+    @Override
+    public int size() {
+      return 6;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value;
+      switch (pos) {

Review Comment:
   Done



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r954092992


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());

Review Comment:
   The test is located in "TestMetadataTableReadableMetrics", which was collapsed by github as its too big



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1035314891


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {
+      private int next;
+
+      NextFieldId() {
+        this.next = baseId;
+      }
+
+      int next() {
+        do {
+          next++;
+        } while (usedIds.contains(next));
+        return next;
+      }
+    }
+    NextFieldId next = new NextFieldId();
+
+    Map<Integer, String> idToName = dataTableSchema.idToName();
+    for (int id : idToName.keySet()) {

Review Comment:
   if we just start at Schema.highestFieldId shouldn't that 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1037681151


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */
+    private Map<Integer, Integer> readableMetricsProjection(Types.NestedField projection) {
+      Map<Integer, Integer> result = Maps.newHashMap();
+
+      Set<String> projectedFields =
+          Sets.newHashSet(
+              projection.type().asStructType().fields().stream()
+                  .map(Types.NestedField::name)
+                  .collect(Collectors.toSet()));
+
+      int projectedIndex = 0;
+      for (int fieldIndex = 0; fieldIndex < READABLE_COL_METRICS.size(); fieldIndex++) {
+        ReadableMetricCol readableMetric = READABLE_COL_METRICS.get(fieldIndex);
+
+        if (projectedFields.contains(readableMetric.name())) {
+          result.put(projectedIndex, fieldIndex);
+          projectedIndex++;
+        }
+      }
+      return result;
+    }
+
+    String columnName() {
+      return columnName;
+    }
+  }
+
+  /**
+   * Represents a struct, consisting of all {@link ReadableColMetricsStruct} for all primitive
+   * columns of the table
+   */
+  public static class ReadableMetricsStruct implements StructLike {
+
+    private final List<StructLike> columnMetrics;
+
+    public ReadableMetricsStruct(List<StructLike> columnMetrics) {
+      this.columnMetrics = columnMetrics;
+    }
+
+    @Override
+    public int size() {
+      return columnMetrics.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return javaClass.cast(columnMetrics.get(pos));
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+  }
+
+  /**
+   * Calculates a dynamic schema for readable_metrics to add to metadata tables. The type will be
+   * the struct {@link ReadableColMetricsStruct}, composed of {@link ReadableMetricsStruct} for all
+   * primitive columns in the data table
+   *
+   * @param dataTableSchema schema of data table
+   * @param metadataTableSchema schema of existing metadata table (to ensure id uniqueness)
+   * @param baseId first id to assign. This algorithm assigns field ids by incrementing this value
+   *     and avoiding conflict with existing metadata table schema
+   * @return schema of readable_metrics struct
+   */
+  public static Schema readableMetricsSchema(
+      Schema dataTableSchema, Schema metadataTableSchema, int baseId) {
+    List<Types.NestedField> fields = Lists.newArrayList();
+    Set<Integer> usedIds = metadataTableSchema.idToName().keySet();
+
+    class NextFieldId {

Review Comment:
   Done



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1337877894

   Thanks @RussellSpitzer @aokolnychyi @chenjunjiedada for detailed reviews


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026595133


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Actually I missed it, the actual comes from Spark, let me take another look.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026578079


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Yea I think that would be nice, changing these tests are definitely the most painful part of this change.
   
   But the GenericRecord here is an Avro class that doesnt have any select methods.  It has a get() but it returns a field and not a projected record.  We'd have to maybe make a struct based on get() all 15 non-derived DataFile fields, not sure if that's cleaner?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r952132633


##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to #{@link #findColumnName(int)}

Review Comment:
   nit: Redundant `#` before `{@link ...}`?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,

Review Comment:
   nit: You can probably call it just `projection`, give that you call the derived var as `fileProjection`.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +201,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final Schema projectedFilesSchema;
+    private final StructLike fileAsStruct;
+    private final Map<String, StructLike> readableMetrics;
+
+    ContentFileStructWithMetrics(
+        Schema projectedFileSchema,
+        StructLike fileAsStruct,
+        Map<String, StructLike> readableMetrics) {
+      this.projectedFilesSchema = projectedFileSchema;
+      this.fileAsStruct = fileAsStruct;
+      this.readableMetrics = readableMetrics;
+    }
+
+    @Override
+    public int size() {
+      return projectedFilesSchema.columns().size() + 1;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      if (pos < projectedFilesSchema.columns().size()) {
+        return fileAsStruct.get(pos, javaClass);
+      } else if (pos == projectedFilesSchema.columns().size()) {
+        return javaClass.cast(readableMetrics);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Illegal position access for FileRow: %d, max allowed is %d",
+                pos, fileAsStruct.size()));
+      }
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("FileEntryRow is read only");

Review Comment:
   nit: What's `FileEntryRow`?



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +64,116 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  /**
+   * Return a readable metrics map
+   *
+   * @param schema schema of original data table
+   * @param contentFile content file with metrics
+   * @return map of readable column name to column metric, of which the bounds are made readable
+   */
+  public static Map<String, StructLike> readableMetricsMap(
+      Schema schema, ContentFile<?> contentFile) {

Review Comment:
   nit: If you call it `file` instead of `contentFile`, the arg definition would fit on one line.



##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to #{@link #findColumnName(int)}

Review Comment:
   nit: Missing `.` at the end of the sentence?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);

Review Comment:
   nit: Strictly speaking, you can directly use `projection` when metrics are not projection.
   
   ```
   if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
     return CloseableIterable.transform(files(projection), file -> (StructLike) file);
   } else {
     Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
     return CloseableIterable.transform(files(fileProjection), ...);
   }
   ```



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {
       switch (manifest.content()) {
         case DATA:
-          return ManifestFiles.read(manifest, io, specsById).project(schema);
+          return ManifestFiles.read(manifest, io, specsById).project(fileProjection);
         case DELETES:
-          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(schema);
+          return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection);
         default:
           throw new IllegalArgumentException(
               "Unsupported manifest content type:" + manifest.content());
       }
     }
 
+    private StructLike wrapWithMetrics(ContentFile<?> file, Schema fileProjection) {

Review Comment:
   If you decide to compute `fileProjection` only if metrics are projected, you could remove the second arg and the place that calls this method will fit on one line.
   
   ```
   private StructLike wrapWithMetrics(ContentFile<?> file) {
     int structSize = projection.columns().size();
     Map<String, StructLike> metrics = ...
     return new ContentFileStructWithMetrics(structSize, (StructLike) file, metrics);
   }
   ```



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());

Review Comment:
   What about defining a constant for this in `ManifestReadTask` and having more specific name?
   Something like `READABLE_METRICS_FIELD_IDS`?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +201,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final Schema projectedFilesSchema;

Review Comment:
   I'd probably just define a variable called `size` instead of referencing the complete schema.
   
   Overall, `StructLike` is serializable so we have to be careful with what we put into fields. Serializing an int will be much more efficient and would work across different serialization mechanisms.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +201,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final Schema projectedFilesSchema;
+    private final StructLike fileAsStruct;
+    private final Map<String, StructLike> readableMetrics;
+
+    ContentFileStructWithMetrics(
+        Schema projectedFileSchema,
+        StructLike fileAsStruct,
+        Map<String, StructLike> readableMetrics) {
+      this.projectedFilesSchema = projectedFileSchema;
+      this.fileAsStruct = fileAsStruct;
+      this.readableMetrics = readableMetrics;
+    }
+
+    @Override
+    public int size() {
+      return projectedFilesSchema.columns().size() + 1;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      if (pos < projectedFilesSchema.columns().size()) {
+        return fileAsStruct.get(pos, javaClass);
+      } else if (pos == projectedFilesSchema.columns().size()) {
+        return javaClass.cast(readableMetrics);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Illegal position access for FileRow: %d, max allowed is %d",

Review Comment:
   nit: What's `FileRow`?



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());

Review Comment:
   Could you please add tests where we project only some subfields in the struct value?
   I don't expect schema pruning to happen on the Iceberg side but it should work correctly.
   
   ```
   SELECT readable_metrics['col1'].lower_bound, readable_metrics['col1'].upper_bound ...
   ```



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,39 +144,53 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Set<Integer> projectedIds = TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+      Schema fileProjection = TypeUtil.selectNot(projectedSchema, projectedIds);
+      if (projectedSchema.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(fileProjection), file -> (StructLike) file);
+      } else {
+        return CloseableIterable.transform(
+            files(fileProjection), file -> wrapWithMetrics(file, fileProjection));
+      }
     }
 
-    private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+    private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) {

Review Comment:
   Looks good to me.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1228789100

   Let me take a look now.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r934961737


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =

Review Comment:
   Moved it here so I could access it for the TestHadoopSourceTableBase to construct the expected record.  Not sure if there is a better location.  These probably belong in BaseFilesTable which is not a public class, one option is to make it public.



##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =

Review Comment:
   Moved it here so I could access it for tests to construct the expected record.  Not sure if there is a better location.  These probably belong in BaseFilesTable which is not a public class, one option is to make it public.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r934960703


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =
+      new Schema(
+          COLUMN_SIZES_METRICS,
+          VALUE_COUNT_METRICS,
+          NULL_VALUE_COUNTS_METRICS,
+          NAN_VALUE_COUNTS_METRICS,
+          LOWER_BOUNDS_METRICS,
+          UPPER_BOUNDS_METRICS);
+
+  public static class Metric {
+    private final String quotedName;
+    private final Types.NestedField field;
+    private final ByteBuffer value;
+
+    Metric(String quotedName, Types.NestedField field, ByteBuffer value) {
+      this.quotedName = quotedName;
+      this.field = field;
+      this.value = value;
+    }
+
+    String quotedName() {
+      return quotedName;
+    }
+
+    boolean valid() {
+      return quotedName != null && field != null && value != null;
+    }
+
+    Optional<String> convertToReadable() {
+      try {
+        return Optional.of(
+            Transforms.identity(field.type())
+                .toHumanString(Conversions.fromByteBuffer(field.type(), value)));
+      } catch (Exception e) { // Ignore
+        return Optional.empty();

Review Comment:
   This happens in some cases, I found it in some case of importing external files to Iceberg table, ie TestIcebergSourceHadoopTables.testFilesTableWithSnapshotIdInheritance, where the I think columns are out of order of the original schema and the metrics become corrupt (underflow exception in this case).  
   
   Not sure if we should error out the files tables, in that case, I was leaning towards just returning null.  User has original column to see why the error happened.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r934961737


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +63,125 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  // Utilities for Displaying Metrics
+
+  static final Types.NestedField COLUMN_SIZES_METRICS =
+      optional(
+          300,
+          "column_sizes_metrics",
+          Types.MapType.ofRequired(301, 302, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total size on disk");
+  static final Types.NestedField VALUE_COUNT_METRICS =
+      optional(
+          303,
+          "value_counts_metrics",
+          Types.MapType.ofRequired(304, 305, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to total count, including null and NaN");
+  static final Types.NestedField NULL_VALUE_COUNTS_METRICS =
+      optional(
+          306,
+          "null_value_counts_metrics",
+          Types.MapType.ofRequired(307, 308, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to null value count");
+  static final Types.NestedField NAN_VALUE_COUNTS_METRICS =
+      optional(
+          309,
+          "nan_value_counts_metrics",
+          Types.MapType.ofRequired(310, 311, Types.StringType.get(), Types.LongType.get()),
+          "Map of column name to number of NaN values in the column");
+  static final Types.NestedField LOWER_BOUNDS_METRICS =
+      optional(
+          312,
+          "lower_bounds_metrics",
+          Types.MapType.ofRequired(313, 314, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to lower bound in string format");
+  static final Types.NestedField UPPER_BOUNDS_METRICS =
+      optional(
+          315,
+          "upper_bounds_metrics",
+          Types.MapType.ofRequired(316, 317, Types.StringType.get(), Types.StringType.get()),
+          "Map of column name to upper bound in string format");
+  public static final Schema METRICS_DISPLAY_SCHEMA =

Review Comment:
   Moved it here so I could access it for the TestHadoopSourceTableBase to construct the expected record.  Not sure if there is a better location.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015603424


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   I spent some time prototyping keeping the original struct (which I think you also bringing up).  That is a ton of work and complexity, in writing schema utils to clone the schema for each struct.  In the end I felt all that complexity is not really worth it, a map is much simpler.
   
   And it is more in line  with what trino has (everything there is converted to human readable string)



##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   I spent some time prototyping keeping the original struct (which I think you also bringing up).  That is a ton of work and complexity, in writing schema utils to clone the schema for each struct.  In the end I felt all that complexity is not really worth it, a map is much simpler.
   
   And it is more in line  with what trino has (everything there is converted to readable string type)



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1013485645


##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -142,9 +142,21 @@ public static Schema selectNot(Schema schema, Set<Integer> fieldIds) {
   }
 
   public static Schema join(Schema left, Schema right) {
-    List<Types.NestedField> joinedColumns = Lists.newArrayList();
-    joinedColumns.addAll(left.columns());

Review Comment:
   @chenjunjiedada Yea that was my original version, and changed after comment of @aokolnychyi  https://github.com/apache/iceberg/pull/5376#discussion_r956441487



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014312886


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +142,72 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    private static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);

Review Comment:
   Yes, took me awhile too to remember ..
   
   This is the case where user selects readable_metrics but not the original metrics columns from files_table.  So I have no way to calculate readable_metrics.  So hence I have to add it to the selection in order to calculate readable_metrics.
   
   Ill add a comment or let me know if theres a better 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014323668


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Sure I guess the counts can be string, though long might be more convenient.
   
   To clarify, are you suggesting:
   `Types.StructType.of(readableStruct(LOWER_BOUNDS), readable_struct(UPPER_BOUNDS), ...?)`
   
   or something else?  



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] atifiu commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by "atifiu (via GitHub)" <gi...@apache.org>.
atifiu commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1722457597

   > Closes #4362
   > 
   > This adds following columns to all files tables:
   > 
   > * readable_metrics, which is struct of:
   > * column_sizes
   > * value_counts
   > * null_value_counts
   > * nan_value_counts
   > * lower_bounds
   > * upper_bounds
   > 
   > These are then a map of column_name to value.
   
   @szehon-ho Actual column names are without 's' in the end


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] atifiu commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by "atifiu (via GitHub)" <gi...@apache.org>.
atifiu commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1722453441

   @szehon-ho @RussellSpitzer Is there any document about these readable metrics ? All these metrics are exposed using files metadata only ?


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r949550004


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +177,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema filesTableSchema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> dataTableFields;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
-        ResidualEvaluator residuals) {
+        ResidualEvaluator residuals,
+        Map<Integer, String> dataTableFields) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.dataTableSchema = table.schema();
+      this.dataTableFields = dataTableFields;
+      this.isPartitioned = filesTableSchema.findField(DataFile.PARTITION_ID) != null;
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      return CloseableIterable.transform(
+          manifestEntries(),
+          fileEntry ->
+              StaticDataTask.Row.of(
+                  projectedFields(fileEntry, accessors(isPartitioned)).toArray()));
     }
 
     private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+      Schema fileProjection =
+          TypeUtil.selectNot(projectedSchema, TypeUtil.getProjectedIds(READABLE_METRICS.type()));

Review Comment:
   If you define a var to hold the result of `TypeUtil.getProjectedIds()`, this would fit on one line.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -143,33 +177,49 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask {
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema filesTableSchema;
+    private final Schema dataTableSchema;
+    private final Schema projectedSchema;
+    private final Map<Integer, String> dataTableFields;
+    private final boolean isPartitioned;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
         Schema schema,
+        Schema projectedSchema,
         String schemaString,
         String specString,
-        ResidualEvaluator residuals) {
+        ResidualEvaluator residuals,
+        Map<Integer, String> dataTableFields) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.filesTableSchema = schema;
+      this.dataTableSchema = table.schema();
+      this.dataTableFields = dataTableFields;
+      this.isPartitioned = filesTableSchema.findField(DataFile.PARTITION_ID) != null;
+      this.projectedSchema = projectedSchema;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      return CloseableIterable.transform(
+          manifestEntries(),
+          fileEntry ->
+              StaticDataTask.Row.of(
+                  projectedFields(fileEntry, accessors(isPartitioned)).toArray()));
     }
 
     private CloseableIterable<? extends ContentFile<?>> manifestEntries() {
+      Schema fileProjection =
+          TypeUtil.selectNot(projectedSchema, TypeUtil.getProjectedIds(READABLE_METRICS.type()));

Review Comment:
   nit: If you define a var to hold the result of `TypeUtil.getProjectedIds()`, this would fit on one line.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1014095953


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +142,72 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    private static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);

Review Comment:
   I'm a little lost on this, we are removing the readable metrics then adding back the min_proj? (if they weren't requested?)



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015598536


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   Initially was done with $metric_type.$column and I changed it from @aokolnychyi 's  comment:  https://github.com/apache/iceberg/pull/5376#issuecomment-1210148871
   
   Not sure if that is what you are referring?
   
   I guess its all subjective, but Anton's way is slightly easier if you want to select columns you want to see metrics for.  This way is easier if you want to select by metric type.
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1015603424


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -99,10 +99,24 @@ public interface DataFile extends ContentFile<DataFile> {
       optional(140, "sort_order_id", IntegerType.get(), "Sort order ID");
   Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID");
 
+  Types.StructType READABLE_METRICS_VALUE_TYPE =
+      Types.StructType.of(
+          optional(142, "column_size", Types.LongType.get(), "Total size on disk"),
+          optional(143, "value_count", Types.LongType.get(), "Total count, including null and NaN"),
+          optional(144, "null_value_count", Types.LongType.get(), "Null value count"),
+          optional(145, "nan_value_count", Types.LongType.get(), "NaN value count"),
+          optional(146, "lower_bound", Types.StringType.get(), "Lower bound in string form"),

Review Comment:
   I spent some time prototyping keeping the original struct (which I think you also bringing up).  That is a ton of work and complexity, in writing schema utils to clone the schema for each struct.  In the end I felt all that complexity is not really worth it, a map is much simpler.
   
   And it is more in line  with what trino has (all the bounds as string).  



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1308426921

   Update. chatted offline with @RussellSpitzer will spend a few days if its possible to make the type dynamic struct instead of static map, to get the right types for lower, upper bounds.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r957690407


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +143,92 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    static final Set<Integer> READABLE_METRICS_FIELD_IDS =
+        TypeUtil.getProjectedIds(DataFile.READABLE_METRICS.type());
+    static final Schema MIN_PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      if (projection.findColumnName(DataFile.READABLE_METRICS.fieldId()) == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Schema fileProjection = TypeUtil.selectNot(projection, READABLE_METRICS_FIELD_IDS);
+        Schema minProjection =

Review Comment:
   I think putting it there will break the scan right, as its not the projection the user requested.  
   
   Note, this is actually a bit subtle here.  Because we are doing the join, adding the minimum metrics, the file becomes 
   {any_projected_field_on_file} : {readable_metrics} : {un-projected but required metrics fields}
   
   So the ContentFileWithMetrics works because it will discard any of the "un-projected but required metrics fields", given they are outside the range it will read.
   
   I mean, we could add a select method to GenericDataFile to modify its internal 'fromProjectionPos' map to conform back to the original projection (without the "un-projected but required metrics fields", but it's strictly needed.
   
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues that prevent projection from working with this approach:
   
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the files table will never request that field.
   
   My initial idea for list of explicit accessors was its cleanly decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to keep that pattern and encapsulate the logic in a StructLike class like this ?  Or let me know which approach is better for you



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] aokolnychyi commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944907127


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   What if you created a StructLike that would wrap the StructLike you get from reading manifests and the metrics map?
   Something like this.
   
   ```
   private static class SomeReasonableNameStructLike implements StructLike {
     private final StructLike fileAsStruct;
     private final Map<String, StructLike> readableMetrics;
   
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
       if (pos < fileAsStruct.size()) {
         return fileAsStruct.get(pos, javaClass);
       } else if (pos == fileAsStruct.size()) {
         return javaClass.cast(readableMetrics);
       } else {
         throw new ...
       }
     }
   ```



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   What if you created a StructLike that would wrap the StructLike you get from reading manifests and the metrics map? Something like this.
   
   ```
   private static class SomeReasonableNameStructLike implements StructLike {
     private final StructLike fileAsStruct;
     private final Map<String, StructLike> readableMetrics;
   
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
       if (pos < fileAsStruct.size()) {
         return fileAsStruct.get(pos, javaClass);
       } else if (pos == fileAsStruct.size()) {
         return javaClass.cast(readableMetrics);
       } else {
         throw new ...
       }
     }
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues that break the projection.
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the files table will never request that field.
   
   My initial idea for list of explicit accessors was its cleanly decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to keep that pattern and encapsulate the logic in a StructLike class like this ?  Or let me know which approach is better for you



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r944972504


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -185,5 +232,60 @@ public Iterable<FileScanTask> split(long splitSize) {
     ManifestFile manifest() {
       return manifest;
     }
+
+    private List<Function<ContentFile<?>, Object>> accessors(boolean partitioned) {

Review Comment:
   I gave a try but there are two issues that break the projection.
   1.  BaseFile::size() returns a fixed size (DataFile.getType()), regardless of the projection schema.  I guess this can be fixed, but hopefully doesnt break anything.
   2. A more serious issue, BaseFile has a field called 'fileOrdinal', but it is not on the FilesTable schemas (again DataFile.getType()).  It seems its working today because its the last field, and projection on the table will never request that field.
   
   My initial idea for list of explicit accessors was its cleanly decouples the table from the messiness of the underlying BaseFile/DataFile classes.  Maybe we can try to keep that pattern and encapsulate this somehow in a StructLike class ?  Or let me know which approach is better for you



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025803239


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +144,76 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Schema PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Types.NestedField readableMetricsField = projection.findField(MetricsUtil.READABLE_METRICS);
+
+      if (readableMetricsField == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
+        Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);

Review Comment:
   I think the comment below might modified a little since I think this still a little confusing,
   
   Maybe "Remove virtual columns from the file projection and then make sure the underlying metrics used to create those columns are part of the file projection." ?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025808348


##########
core/src/main/java/org/apache/iceberg/MetricsUtil.java:
##########
@@ -56,4 +72,270 @@ public static MetricsModes.MetricsMode metricsMode(
     String columnName = inputSchema.findColumnName(fieldId);
     return metricsConfig.columnMode(columnName);
   }
+
+  public static final List<ReadableMetricCol> READABLE_COL_METRICS =
+      ImmutableList.of(
+          new ReadableMetricCol("column_size", f -> Types.LongType.get(), "Total size on disk"),
+          new ReadableMetricCol(
+              "value_count", f -> Types.LongType.get(), "Total count, including null and NaN"),
+          new ReadableMetricCol("null_value_count", f -> Types.LongType.get(), "Null value count"),
+          new ReadableMetricCol("nan_value_count", f -> Types.LongType.get(), "NaN value count"),
+          new ReadableMetricCol("lower_bound", Types.NestedField::type, "Lower bound"),
+          new ReadableMetricCol("upper_bound", Types.NestedField::type, "Upper bound"));
+
+  public static final String READABLE_METRICS = "readable_metrics";
+
+  public static class ReadableMetricCol {
+    private final String name;
+    private final Function<Types.NestedField, Type> typeFunction;
+    private final String doc;
+
+    ReadableMetricCol(String name, Function<Types.NestedField, Type> typeFunction, String doc) {
+      this.name = name;
+      this.typeFunction = typeFunction;
+      this.doc = doc;
+    }
+
+    String name() {
+      return name;
+    }
+
+    Type type(Types.NestedField field) {
+      return typeFunction.apply(field);
+    }
+
+    String doc() {
+      return doc;
+    }
+  }
+
+  /**
+   * Represents a struct of metrics for a primitive column
+   *
+   * @param <T> primitive column type
+   */
+  public static class ReadableColMetricsStruct<T> implements StructLike {
+
+    private final String columnName;
+    private final Long columnSize;
+    private final Long valueCount;
+    private final Long nullValueCount;
+    private final Long nanValueCount;
+    private final T lowerBound;
+    private final T upperBound;
+    private final Map<Integer, Integer> projectionMap;
+
+    public ReadableColMetricsStruct(
+        String columnName,
+        Long columnSize,
+        Long valueCount,
+        Long nullValueCount,
+        Long nanValueCount,
+        T lowerBound,
+        T upperBound,
+        Types.NestedField projection) {
+      this.columnName = columnName;
+      this.columnSize = columnSize;
+      this.valueCount = valueCount;
+      this.nullValueCount = nullValueCount;
+      this.nanValueCount = nanValueCount;
+      this.lowerBound = lowerBound;
+      this.upperBound = upperBound;
+      this.projectionMap = readableMetricsProjection(projection);
+    }
+
+    @Override
+    public int size() {
+      return projectionMap.size();
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      Object value = get(pos);
+      return value == null ? null : javaClass.cast(value);
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("ReadableMetricsStruct is read only");
+    }
+
+    private Object get(int pos) {
+      int projectedPos = projectionMap.get(pos);
+      switch (projectedPos) {
+        case 0:
+          return columnSize;
+        case 1:
+          return valueCount;
+        case 2:
+          return nullValueCount;
+        case 3:
+          return nanValueCount;
+        case 4:
+          return lowerBound;
+        case 5:
+          return upperBound;
+        default:
+          throw new IllegalArgumentException(
+              String.format("Invalid projected pos %d", projectedPos));
+      }
+    }
+
+    /** @return map of projected position to actual position of this struct's fields */

Review Comment:
   I didn't think we allowed 1 line java docs like this? There should be an alert that we don't do @return ...., for single line java docs it should just be "return ...." so that the summary is filled in the java doc.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025816613


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Would it be easier to just change our tests to not select readable metrics? Or is that more complicated, just looks like we have to copy a bunch of code here that is exactly the same as the code we are testing. Always gives me a bit of a worry.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] RussellSpitzer commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1025472556


##########
api/src/main/java/org/apache/iceberg/Schema.java:
##########
@@ -233,6 +233,16 @@ public Map<String, Integer> getAliases() {
     return aliasToId;
   }
 
+  /**
+   * Returns a map for this schema between field id and qualified field names. Initializes the map,
+   * if it has not been initialized by calls to {@link #findColumnName(int)}.
+   *
+   * @return a map of field id to qualified field names
+   */
+  public Map<Integer, String> idToName() {

Review Comment:
   If Anton voted to expand the API then I'm onboard.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1026574052


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -186,4 +224,43 @@ ManifestFile manifest() {
       return manifest;
     }
   }
+
+  static class ContentFileStructWithMetrics implements StructLike {
+    private final int structSize;
+    private final StructLike fileAsStruct;
+    private final MetricsUtil.ReadableMetricsStruct readableMetrics;
+
+    ContentFileStructWithMetrics(
+        int structSize,
+        StructLike fileAsStruct,
+        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+      this.structSize = structSize;

Review Comment:
   I do use the variable in size().  I changed to use a local variable in get() that is fileAsStruct.size() - 1, hope that works.



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -140,42 +144,76 @@ protected CloseableIterable<FileScanTask> doPlanFiles() {
   }
 
   static class ManifestReadTask extends BaseFileScanTask implements DataTask {
+
+    private static final Schema PROJECTION_FOR_READABLE_METRICS =
+        new Schema(
+            DataFile.COLUMN_SIZES,
+            DataFile.VALUE_COUNTS,
+            DataFile.NULL_VALUE_COUNTS,
+            DataFile.NAN_VALUE_COUNTS,
+            DataFile.LOWER_BOUNDS,
+            DataFile.UPPER_BOUNDS);
+
     private final FileIO io;
     private final Map<Integer, PartitionSpec> specsById;
     private final ManifestFile manifest;
-    private final Schema schema;
+    private final Schema dataTableSchema;
+    private final Schema projection;
 
     ManifestReadTask(
         Table table,
         ManifestFile manifest,
-        Schema schema,
+        Schema projection,
         String schemaString,
         String specString,
         ResidualEvaluator residuals) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = table.io();
       this.specsById = Maps.newHashMap(table.specs());
       this.manifest = manifest;
-      this.schema = schema;
+      this.dataTableSchema = table.schema();
+      this.projection = projection;
     }
 
     @Override
     public CloseableIterable<StructLike> rows() {
-      return CloseableIterable.transform(manifestEntries(), file -> (StructLike) file);
+      Types.NestedField readableMetricsField = projection.findField(MetricsUtil.READABLE_METRICS);
+
+      if (readableMetricsField == null) {
+        return CloseableIterable.transform(files(projection), file -> (StructLike) file);
+      } else {
+        Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
+        Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);

Review Comment:
   Changed.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1037554727


##########
api/src/main/java/org/apache/iceberg/DataFile.java:
##########
@@ -102,7 +102,8 @@ public interface DataFile extends ContentFile<DataFile> {
   int PARTITION_ID = 102;
   String PARTITION_NAME = "partition";
   String PARTITION_DOC = "Partition data tuple, schema based on the partition spec";
-  // NEXT ID TO ASSIGN: 142
+
+  int NEXT_ID_TO_ASSIGN = 142;

Review Comment:
   Changed to find the highest existing id from files table schema, and then assign ids from there



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1335928980

   Would love to see what is a good way to simplify it without breaking the checks.  Currently compares every single field.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#discussion_r1037554935


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java:
##########
@@ -817,4 +824,93 @@ public static Set<String> reachableManifestPaths(Table table) {
         .map(ManifestFile::path)
         .collect(Collectors.toSet());
   }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file) {
+    return asMetadataRecordWithMetrics(dataTable, file, FileContent.DATA);
+  }
+
+  public static GenericData.Record asMetadataRecordWithMetrics(
+      Table dataTable, GenericData.Record file, FileContent content) {
+
+    Table filesTable =
+        MetadataTableUtils.createMetadataTableInstance(dataTable, MetadataTableType.FILES);
+
+    GenericData.Record record =
+        new GenericData.Record(AvroSchemaUtil.convert(filesTable.schema(), "dummy"));
+    boolean isPartitioned = Partitioning.partitionType(dataTable).fields().size() != 0;
+    int filesFields = isPartitioned ? 17 : 16;
+    for (int i = 0; i < filesFields; i++) {
+      if (i == 0) {
+        record.put(0, content.id());
+      } else if (i == 3) {
+        record.put(3, 0); // spec id
+      } else {
+        record.put(i, file.get(i));
+      }
+    }
+    record.put(
+        isPartitioned ? 17 : 16,
+        expectedReadableMetrics(

Review Comment:
   Did the select to simplify the tests.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on pull request #5376: Core: Add readable metrics columns to files metadata tables

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on PR #5376:
URL: https://github.com/apache/iceberg/pull/5376#issuecomment-1335228779

   @RussellSpitzer addressed the comments, 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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