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/02/16 17:10:46 UTC

[GitHub] [iceberg] szehon-ho opened a new pull request #4142: [Core] Add utility to print an Iceberg table metadata

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


   Was debugging why delete files are not being removed, and wrote this small utility to dump the metadata in the console that proved quite helpful for that. 
   
   I was thinking it can help to find in various other problems invoking maintenance procedures, for example why are the files/metadata files not being compacted, etc.
   
   For small tables it works well, maybe for big tables it will be a bigger dump to read through, though it writes in a streaming fashion so should not OOM.
   
   Example:
   
   ```
       ReachableFileUtil.printCurrentSnapshot(table, System.out);
   ```
   
   ```
   \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/metadata/f3e35c2f-5dd9-4a13-a262-b050b74c248f-m1.avro, length=6560, partition_spec_id=0, added_snapshot_id=709093599914280039, added_data_files_count=1, added_rows_count=3, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], specId=0, key_metadata=null, sequence_number=3, min_sequence_number=3}
       +---GenericManifestEntry{status=ADDED, snapshot_id=709093599914280039, sequence_number=3, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/data/00000-7-79c9c8a4-9bb0-4b4d-8e34-2fcee70fb357-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=3, file_size_in_bytes=654, column_sizes={1=51, 2=57}, value_counts={1=3, 2=3}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
   \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/metadata/f3e35c2f-5dd9-4a13-a262-b050b74c248f-m0.avro, length=6602, partition_spec_id=0, added_snapshot_id=709093599914280039, added_data_files_count=0, added_rows_count=0, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=2, deleted_rows_count=4, partitions=[], specId=0, key_metadata=null, sequence_number=3, min_sequence_number=3}
       +---GenericManifestEntry{status=DELETED, snapshot_id=709093599914280039, sequence_number=3, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/data/00000-0-903b06f3-f54b-471d-891b-44ed7022f671-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=2, file_size_in_bytes=644, column_sizes={1=49, 2=51}, value_counts={1=2, 2=2}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
       +---GenericManifestEntry{status=DELETED, snapshot_id=709093599914280039, sequence_number=3, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/data/00001-1-2a15729e-3940-4231-b5ad-47962462acca-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=2, file_size_in_bytes=644, column_sizes={1=49, 2=51}, value_counts={1=2, 2=2}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
   \---GenericManifestFile{content=DELETES, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/metadata/f4c479d4-fd21-49fd-9903-636f1fed09ac-m0.avro, length=6603, partition_spec_id=0, added_snapshot_id=7445530922696444799, added_data_files_count=1, added_rows_count=1, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], specId=0, key_metadata=null, sequence_number=2, min_sequence_number=2}
        +---GenericManifestEntry{status=ADDED, snapshot_id=7445530922696444799, sequence_number=2, file=GenericDeleteFile{content=position_deletes, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive3369326090081521705/table/data/00003-3-d496c17a-f752-4bc8-b555-9ee4a4b006a5-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=1, file_size_in_bytes=1670, column_sizes={2147483546=170, 2147483545=46}, value_counts={2147483546=1, 2147483545=1}, null_value_counts={2147483546=0, 2147483545=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@2686ed34, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@2686ed34, key_metadata=null, split_offsets=null, equality_ids=null, sort_order_id=null}}
   
   
   ```
   
   This also adds a few missing fields to the ManifestFile's toString


-- 
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 change in pull request #4142: [Core] Add utility to print an Iceberg table metadata

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4142:
URL: https://github.com/apache/iceberg/pull/4142#discussion_r811503419



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -114,4 +119,47 @@ private static TableMetadata findFirstExistentPreviousMetadata(List<MetadataLogE
     }
     return manifestListLocations;
   }
+
+  /**
+   * Emits a human-readable metadata graph of the current snapshot of the given table
+   * @param table Iceberg table
+   * @param out output print stream to emit to
+   */
+  public static void printCurrentSnapshot(Table table, PrintStream out) {

Review comment:
       Makes sense, will do.




-- 
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] flyrain edited a comment on pull request #4142: [Core] Add utility to print an Iceberg table metadata

Posted by GitBox <gi...@apache.org>.
flyrain edited a comment on pull request #4142:
URL: https://github.com/apache/iceberg/pull/4142#issuecomment-1045454895


   This is going to be pretty useful for unit test and debugging. Thanks @szehon-ho.


-- 
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 change in pull request #4142: [Core] Add utility to print an Iceberg table metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -404,6 +404,7 @@ public int hashCode() {
   @Override
   public String toString() {
     return MoreObjects.toStringHelper(this)

Review comment:
       I think we want this change even if we don't include the print method




-- 
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 change in pull request #4142: [Core] Add utility to print an Iceberg table metadata

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4142:
URL: https://github.com/apache/iceberg/pull/4142#discussion_r811503306



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -404,6 +404,7 @@ public int hashCode() {
   @Override
   public String toString() {
     return MoreObjects.toStringHelper(this)

Review comment:
       Done, split it out in #4185 




-- 
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] kbendick commented on pull request #4142: [Core] Add utility to print an Iceberg table metadata

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


   I think that even if this gets really big, it could be useful.
   
   I've talked to several people that have written their own scripts for importing the table metadata into things like [amundsen](https://github.com/lyft/amundsen) so that they can view very large table metadata.
   
   So there's definitely a demand for this capability (though I'd have to see how useful the output is).
   
   If we could get the entries into JSON eventually, using `jq` or `bat` might make it easier to consume.


-- 
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] flyrain commented on pull request #4142: [Core] Add utility to print an Iceberg table metadata

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


   This is going to be pretty useful for unit test and debugging.


-- 
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 #4142: [Core] Add utility to print an Iceberg table metadata

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


   Added new method as suggested @flyrain 
   
   Output for this method is now:
   ```
   BaseSnapshot{id=8051839554011841790, timestamp_ms=1645660550950, operation=append, summary={spark.app.id=local-1645660524540, added-data-files=1, added-records=1, added-files-size=643, changed-partition-count=1, total-records=5, total-files-size=1948, total-data-files=3, total-delete-files=0, total-position-deletes=0, total-equality-deletes=0}, manifest-list=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/snap-8051839554011841790-1-171220c0-d44c-47ff-9a28-92bbd2f32960.avro, schema-id=0}
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/171220c0-d44c-47ff-9a28-92bbd2f32960-m0.avro, length=6561, partition_spec_id=0, added_snapshot_id=8051839554011841790, added_data_files_count=1, added_rows_count=1, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=5, min_sequence_number=5}
        +---GenericManifestEntry{status=ADDED, snapshot_id=8051839554011841790, sequence_number=5, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-403-8ed90937-faae-4151-9a87-aaf82a4318ed-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=1, file_size_in_bytes=643, column_sizes={1=46, 2=48}, value_counts={1=1, 2=1}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/8bdf97d4-7ad4-428f-8ef3-65a9454f449a-m1.avro, length=6559, partition_spec_id=0, added_snapshot_id=2008302932940166540, added_data_files_count=1, added_rows_count=1, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=4, min_sequence_number=4}
        +---GenericManifestEntry{status=ADDED, snapshot_id=2008302932940166540, sequence_number=4, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-402-c135e7a5-98ee-4336-9d3e-264b33aa6606-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=1, file_size_in_bytes=643, column_sizes={1=46, 2=48}, value_counts={1=1, 2=1}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/45a03772-f196-49dd-8d9a-7395e4e7ea3f-m1.avro, length=6563, partition_spec_id=0, added_snapshot_id=8202500298836421079, added_data_files_count=1, added_rows_count=3, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=3, min_sequence_number=3}
        +---GenericManifestEntry{status=ADDED, snapshot_id=8202500298836421079, sequence_number=3, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-401-263e4856-3c17-4137-8146-ad6c9117a92e-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=3, file_size_in_bytes=662, column_sizes={1=57, 2=57}, value_counts={1=3, 2=3}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   ```
   
   versus for the original method (printTable):
   ```
   ------------------------------------------------------------
   Table spark_catalog.default.table
   ------------------------------------------------------------
   BaseSnapshot{id=8051839554011841790, timestamp_ms=1645660550950, operation=append, summary={spark.app.id=local-1645660524540, added-data-files=1, added-records=1, added-files-size=643, changed-partition-count=1, total-records=5, total-files-size=1948, total-data-files=3, total-delete-files=0, total-position-deletes=0, total-equality-deletes=0}, manifest-list=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/snap-8051839554011841790-1-171220c0-d44c-47ff-9a28-92bbd2f32960.avro, schema-id=0}
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/171220c0-d44c-47ff-9a28-92bbd2f32960-m0.avro, length=6561, partition_spec_id=0, added_snapshot_id=8051839554011841790, added_data_files_count=1, added_rows_count=1, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=5, min_sequence_number=5}
        +---GenericManifestEntry{status=ADDED, snapshot_id=8051839554011841790, sequence_number=5, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-403-8ed90937-faae-4151-9a87-aaf82a4318ed-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=1, file_size_in_bytes=643, column_sizes={1=46, 2=48}, value_counts={1=1, 2=1}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/8bdf97d4-7ad4-428f-8ef3-65a9454f449a-m1.avro, length=6559, partition_spec_id=0, added_snapshot_id=2008302932940166540, added_data_files_count=1, added_rows_count=1, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=4, min_sequence_number=4}
        +---GenericManifestEntry{status=ADDED, snapshot_id=2008302932940166540, sequence_number=4, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-402-c135e7a5-98ee-4336-9d3e-264b33aa6606-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=1, file_size_in_bytes=643, column_sizes={1=46, 2=48}, value_counts={1=1, 2=1}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f8d, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   
    \---GenericManifestFile{content=DATA, path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/metadata/45a03772-f196-49dd-8d9a-7395e4e7ea3f-m1.avro, length=6563, partition_spec_id=0, added_snapshot_id=8202500298836421079, added_data_files_count=1, added_rows_count=3, existing_data_files_count=0, existing_rows_count=0, deleted_data_files_count=0, deleted_rows_count=0, partitions=[], key_metadata=null, sequence_number=3, min_sequence_number=3}
        +---GenericManifestEntry{status=ADDED, snapshot_id=8202500298836421079, sequence_number=3, file=GenericDataFile{content=data, file_path=file:/var/folders/wy/5b87_qx57n974szn9_wrn6lw0000gn/T/hive15893599931471374639/table/data/00000-401-263e4856-3c17-4137-8146-ad6c9117a92e-00001.parquet, file_format=PARQUET, spec_id=0, partition=PartitionData{}, record_count=3, file_size_in_bytes=662, column_sizes={1=57, 2=57}, value_counts={1=3, 2=3}, null_value_counts={1=0, 2=0}, nan_value_counts={}, lower_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, upper_bounds=org.apache.iceberg.SerializableByteBufferMap@94446f85, key_metadata=null, split_offsets=[4], equality_ids=null, sort_order_id=0}}
   ```


-- 
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] flyrain commented on a change in pull request #4142: [Core] Add utility to print an Iceberg table metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -114,4 +119,54 @@ private static TableMetadata findFirstExistentPreviousMetadata(List<MetadataLogE
     }
     return manifestListLocations;
   }
+
+  /**
+   * Emits a human-readable metadata graph of the Iceberg table's current snapshot
+   * @param table Iceberg table
+   * @param out output stream to emit to
+   */
+  public static void printTable(Table table, PrintStream out) {
+    out.println("------------------------------------------------------------");
+    out.println("Table " + table.name());
+    out.println("------------------------------------------------------------");
+    printSnapshot(table, table.currentSnapshot(), out);
+  }
+
+  /**
+   * Emits a human-readable metadata graph of the given snapshot
+   * @param table Iceberg table
+   * @param snapshot Iceberg table snapshot
+   * @param out output print stream to emit to
+   */
+  public static void printSnapshot(Table table, Snapshot snapshot, PrintStream out) {
+    out.println(snapshot);
+    Iterator<ManifestFile> allManifests = snapshot.allManifests().iterator();
+    while (allManifests.hasNext()) {
+      ManifestFile mf = allManifests.next();
+      out.println(" \\---" + mf);
+
+      if (mf.content().equals(ManifestContent.DATA)) {

Review comment:
       Can we create a new read method which take `FileType` as a parameter? So that we don't have to distinguish datafile and deletefile.
   ```
   ManifestFiles.read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById, FileType fileType)
   ```




-- 
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] flyrain commented on a change in pull request #4142: [Core] Add utility to print an Iceberg table metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -114,4 +119,47 @@ private static TableMetadata findFirstExistentPreviousMetadata(List<MetadataLogE
     }
     return manifestListLocations;
   }
+
+  /**
+   * Emits a human-readable metadata graph of the current snapshot of the given table
+   * @param table Iceberg table
+   * @param out output print stream to emit to
+   */
+  public static void printCurrentSnapshot(Table table, PrintStream out) {

Review comment:
       I'd suggest to add one more method which accepts snapshot as an additional parameter, like this. People may also be interested in printing out a historical snapshot. Most of the code can be reused between these two methods.
   ```
   public static void printSnapshot(Table table, Snapshot snapshot, PrintStream out) {
   ```




-- 
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 #4142: [Core] Add utility to print an Iceberg table metadata

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


   Flink failure does not look related.
   
   @jackye1995 @rdblue @aokolnychyi  wondering any thoughts on whether people will find this useful?  
   
   @RussellSpitzer  and @flyrain did mention that it might be useful but probably will get unreadable for big tables


-- 
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 pull request #4142: [Core] Add utility to print an Iceberg table metadata

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


   I am a bit conflicted, I think from @kbendick 's notes there are really two distinct use cases,
   
   Getting a human readable display for debugging of issues, (only works with smaller table)
   Generating a machine readable display for other systems (for any size table)
   
   I'm not sure we can handle both in the same function 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