This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new b7b781d  Remove block_size_in_bytes from DataFile (#184)
b7b781d is described below

commit b7b781d6dd8c1f7a2ace15cf190c57d84579316f
Author: Samarth Jain <[email protected]>
AuthorDate: Wed May 15 16:21:30 2019 -0700

    Remove block_size_in_bytes from DataFile (#184)
    
    This field is not used and won't be used for better split planning based on 
row group or stripe offsets.
---
 api/src/main/java/org/apache/iceberg/DataFile.java |  5 ---
 .../test/java/org/apache/iceberg/TestHelpers.java  |  5 ---
 .../main/java/org/apache/iceberg/DataFiles.java    | 37 +++++++---------------
 .../java/org/apache/iceberg/GenericDataFile.java   | 28 ++++++----------
 .../org/apache/iceberg/spark/SparkTableUtil.scala  |  1 -
 5 files changed, 21 insertions(+), 55 deletions(-)

diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java 
b/api/src/main/java/org/apache/iceberg/DataFile.java
index 1a86e12..b8c02be 100644
--- a/api/src/main/java/org/apache/iceberg/DataFile.java
+++ b/api/src/main/java/org/apache/iceberg/DataFile.java
@@ -89,11 +89,6 @@ public interface DataFile {
   long fileSizeInBytes();
 
   /**
-   * @return the data file block size in bytes (for split planning)
-   */
-  long blockSizeInBytes();
-
-  /**
    * @return file ordinal if written in a global ordering, or null
    */
   Integer fileOrdinal();
diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java 
b/api/src/test/java/org/apache/iceberg/TestHelpers.java
index ae6a69f..ab92634 100644
--- a/api/src/test/java/org/apache/iceberg/TestHelpers.java
+++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java
@@ -333,11 +333,6 @@ public class TestHelpers {
     }
 
     @Override
-    public long blockSizeInBytes() {
-      return 0;
-    }
-
-    @Override
     public Integer fileOrdinal() {
       return null;
     }
diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java 
b/core/src/main/java/org/apache/iceberg/DataFiles.java
index 045891f..9a3111a 100644
--- a/core/src/main/java/org/apache/iceberg/DataFiles.java
+++ b/core/src/main/java/org/apache/iceberg/DataFiles.java
@@ -36,8 +36,6 @@ public class DataFiles {
 
   private DataFiles() {}
 
-  private static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;
-
   private static PartitionData newPartitionData(PartitionSpec spec) {
     return new PartitionData(spec.partitionType());
   }
@@ -98,21 +96,21 @@ public class DataFiles {
   public static DataFile fromStat(FileStatus stat, long rowCount) {
     String location = stat.getPath().toString();
     FileFormat format = FileFormat.fromFileName(location);
-    return new GenericDataFile(location, format, rowCount, stat.getLen(), 
stat.getBlockSize());
+    return new GenericDataFile(location, format, rowCount, stat.getLen());
   }
 
   public static DataFile fromStat(FileStatus stat, PartitionData partition, 
long rowCount) {
     String location = stat.getPath().toString();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, rowCount, stat.getLen(), 
stat.getBlockSize());
+        location, format, partition, rowCount, stat.getLen());
   }
 
   public static DataFile fromStat(FileStatus stat, PartitionData partition, 
Metrics metrics) {
     String location = stat.getPath().toString();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, stat.getLen(), stat.getBlockSize(), 
metrics);
+        location, format, partition, stat.getLen(), metrics);
   }
 
   public static DataFile fromStat(FileStatus stat, PartitionData partition, 
Metrics metrics,
@@ -120,7 +118,7 @@ public class DataFiles {
     String location = stat.getPath().toString();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, stat.getLen(), stat.getBlockSize(), 
metrics, keyMetadata.buffer());
+        location, format, partition, stat.getLen(), metrics, 
keyMetadata.buffer());
   }
 
   public static DataFile fromInputFile(InputFile file, PartitionData 
partition, long rowCount) {
@@ -131,7 +129,7 @@ public class DataFiles {
     String location = file.location();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, rowCount, file.getLength(), 
DEFAULT_BLOCK_SIZE);
+        location, format, partition, rowCount, file.getLength());
   }
 
   public static DataFile fromInputFile(InputFile file, PartitionData 
partition, Metrics metrics) {
@@ -142,7 +140,7 @@ public class DataFiles {
     String location = file.location();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, file.getLength(), DEFAULT_BLOCK_SIZE, 
metrics);
+        location, format, partition, file.getLength(), metrics);
   }
 
   public static DataFile fromInputFile(InputFile file, long rowCount) {
@@ -152,7 +150,7 @@ public class DataFiles {
 
     String location = file.location();
     FileFormat format = FileFormat.fromFileName(location);
-    return new GenericDataFile(location, format, rowCount, file.getLength(), 
DEFAULT_BLOCK_SIZE);
+    return new GenericDataFile(location, format, rowCount, file.getLength());
   }
 
   public static DataFile fromEncryptedOutputFile(EncryptedOutputFile 
encryptedFile, PartitionData partition,
@@ -166,7 +164,7 @@ public class DataFiles {
     String location = file.location();
     FileFormat format = FileFormat.fromFileName(location);
     return new GenericDataFile(
-        location, format, partition, file.getLength(), DEFAULT_BLOCK_SIZE, 
metrics, keyMetadata.buffer());
+        location, format, partition, file.getLength(), metrics, 
keyMetadata.buffer());
   }
 
   public static DataFile fromParquetInputFile(InputFile file,
@@ -179,14 +177,14 @@ public class DataFiles {
     String location = file.location();
     FileFormat format = FileFormat.PARQUET;
     return new GenericDataFile(
-        location, format, partition, file.getLength(), DEFAULT_BLOCK_SIZE, 
metrics);
+        location, format, partition, file.getLength(), metrics);
   }
 
   public static DataFile fromParquetStat(FileStatus stat, PartitionData 
partition, Metrics metrics) {
     String location = stat.getPath().toString();
     FileFormat format = FileFormat.PARQUET;
     return new GenericDataFile(
-        location, format, partition, stat.getLen(), stat.getBlockSize(), 
metrics);
+        location, format, partition, stat.getLen(), metrics);
   }
 
   public static Builder builder(PartitionSpec spec) {
@@ -205,7 +203,6 @@ public class DataFiles {
     private FileFormat format = null;
     private long recordCount = -1L;
     private long fileSizeInBytes = -1L;
-    private long blockSizeInBytes = -1L;
 
     // optional fields
     private Map<Integer, Long> columnSizes = null;
@@ -235,7 +232,6 @@ public class DataFiles {
       this.format = null;
       this.recordCount = -1L;
       this.fileSizeInBytes = -1L;
-      this.blockSizeInBytes = -1L;
       this.columnSizes = null;
       this.valueCounts = null;
       this.nullValueCounts = null;
@@ -251,7 +247,6 @@ public class DataFiles {
       this.format = toCopy.format();
       this.recordCount = toCopy.recordCount();
       this.fileSizeInBytes = toCopy.fileSizeInBytes();
-      this.blockSizeInBytes = toCopy.blockSizeInBytes();
       this.columnSizes = toCopy.columnSizes();
       this.valueCounts = toCopy.valueCounts();
       this.nullValueCounts = toCopy.nullValueCounts();
@@ -265,7 +260,6 @@ public class DataFiles {
     public Builder withStatus(FileStatus stat) {
       this.filePath = stat.getPath().toString();
       this.fileSizeInBytes = stat.getLen();
-      this.blockSizeInBytes = stat.getBlockSize();
       return this;
     }
 
@@ -315,11 +309,6 @@ public class DataFiles {
       return this;
     }
 
-    public Builder withBlockSizeInBytes(long newBlockSizeInBytes) {
-      this.blockSizeInBytes = newBlockSizeInBytes;
-      return this;
-    }
-
     public Builder withPartitionPath(String newPartitionPath) {
       Preconditions.checkArgument(isPartitioned || newPartitionPath.isEmpty(),
           "Cannot add partition data for an unpartitioned table");
@@ -360,13 +349,9 @@ public class DataFiles {
       Preconditions.checkArgument(fileSizeInBytes >= 0, "File size is 
required");
       Preconditions.checkArgument(recordCount >= 0, "Record count is 
required");
 
-      if (blockSizeInBytes < 0) {
-        this.blockSizeInBytes = DEFAULT_BLOCK_SIZE; // assume 64MB blocks
-      }
-
       return new GenericDataFile(
           filePath, format, isPartitioned ? partitionData.copy() : null,
-          fileSizeInBytes, blockSizeInBytes, new Metrics(
+          fileSizeInBytes, new Metrics(
               recordCount, columnSizes, valueCounts, nullValueCounts, 
lowerBounds, upperBounds), keyMetadata);
     }
   }
diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java 
b/core/src/main/java/org/apache/iceberg/GenericDataFile.java
index 5e9904a..e38217d 100644
--- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java
+++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java
@@ -54,7 +54,6 @@ class GenericDataFile
   private PartitionData partitionData = null;
   private Long recordCount = null;
   private long fileSizeInBytes = -1L;
-  private long blockSizeInBytes = -1L;
 
   // optional fields
   private Integer fileOrdinal = null; // boxed for nullability
@@ -69,6 +68,8 @@ class GenericDataFile
   // cached schema
   private transient org.apache.avro.Schema avroSchema = null;
 
+  private static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;
+
   /**
    * Used by Avro reflection to instantiate this class when reading manifest 
files.
    */
@@ -107,14 +108,13 @@ class GenericDataFile
   }
 
   GenericDataFile(String filePath, FileFormat format, long recordCount,
-                  long fileSizeInBytes, long blockSizeInBytes) {
+                  long fileSizeInBytes) {
     this.filePath = filePath;
     this.format = format;
     this.partitionData = EMPTY_PARTITION_DATA;
     this.partitionType = EMPTY_PARTITION_DATA.getPartitionType();
     this.recordCount = recordCount;
     this.fileSizeInBytes = fileSizeInBytes;
-    this.blockSizeInBytes = blockSizeInBytes;
     this.fileOrdinal = null;
     this.sortColumns = null;
     this.columnSizes = null;
@@ -126,14 +126,13 @@ class GenericDataFile
   }
 
   GenericDataFile(String filePath, FileFormat format, PartitionData partition,
-                  long recordCount, long fileSizeInBytes, long 
blockSizeInBytes) {
+                  long recordCount, long fileSizeInBytes) {
     this.filePath = filePath;
     this.format = format;
     this.partitionData = partition;
     this.partitionType = partition.getPartitionType();
     this.recordCount = recordCount;
     this.fileSizeInBytes = fileSizeInBytes;
-    this.blockSizeInBytes = blockSizeInBytes;
     this.fileOrdinal = null;
     this.sortColumns = null;
     this.columnSizes = null;
@@ -145,7 +144,7 @@ class GenericDataFile
   }
 
   GenericDataFile(String filePath, FileFormat format, PartitionData partition,
-                  long fileSizeInBytes, long blockSizeInBytes, Metrics 
metrics) {
+                  long fileSizeInBytes, Metrics metrics) {
     this.filePath = filePath;
     this.format = format;
 
@@ -161,7 +160,6 @@ class GenericDataFile
     // this will throw NPE if metrics.recordCount is null
     this.recordCount = metrics.recordCount();
     this.fileSizeInBytes = fileSizeInBytes;
-    this.blockSizeInBytes = blockSizeInBytes;
     this.fileOrdinal = null;
     this.sortColumns = null;
     this.columnSizes = metrics.columnSizes();
@@ -173,9 +171,9 @@ class GenericDataFile
   }
 
   GenericDataFile(String filePath, FileFormat format, PartitionData partition,
-                  long fileSizeInBytes, long blockSizeInBytes, Metrics metrics,
+                  long fileSizeInBytes, Metrics metrics,
                   ByteBuffer keyMetadata) {
-    this(filePath, format, partition, fileSizeInBytes, blockSizeInBytes, 
metrics);
+    this(filePath, format, partition, fileSizeInBytes, metrics);
     this.keyMetadata = keyMetadata;
   }
 
@@ -191,7 +189,6 @@ class GenericDataFile
     this.partitionType = toCopy.partitionType;
     this.recordCount = toCopy.recordCount;
     this.fileSizeInBytes = toCopy.fileSizeInBytes;
-    this.blockSizeInBytes = toCopy.blockSizeInBytes;
     this.fileOrdinal = toCopy.fileOrdinal;
     this.sortColumns = copy(toCopy.sortColumns);
     // TODO: support lazy conversion to/from map
@@ -236,11 +233,6 @@ class GenericDataFile
   }
 
   @Override
-  public long blockSizeInBytes() {
-    return blockSizeInBytes;
-  }
-
-  @Override
   public Integer fileOrdinal() {
     return fileOrdinal;
   }
@@ -314,7 +306,6 @@ class GenericDataFile
         this.fileSizeInBytes = (Long) v;
         return;
       case 5:
-        this.blockSizeInBytes = (Long) v;
         return;
       case 6:
         this.fileOrdinal = (Integer) v;
@@ -369,7 +360,9 @@ class GenericDataFile
       case 4:
         return fileSizeInBytes;
       case 5:
-        return blockSizeInBytes;
+        // block_size_in_bytes is not used. However, it is a required avro 
field in DataFile. So
+        // to maintain compatibility, we need to return something.
+        return DEFAULT_BLOCK_SIZE;
       case 6:
         return fileOrdinal;
       case 7:
@@ -416,7 +409,6 @@ class GenericDataFile
         .add("partition", partitionData)
         .add("record_count", recordCount)
         .add("file_size_in_bytes", fileSizeInBytes)
-        .add("block_size_in_bytes", blockSizeInBytes)
         .add("column_sizes", columnSizes)
         .add("value_counts", valueCounts)
         .add("null_value_counts", nullValueCounts)
diff --git a/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala 
b/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
index 90bc16a..ca835d6 100644
--- a/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
+++ b/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
@@ -114,7 +114,6 @@ object SparkTableUtil {
           .withFormat(format)
           .withPartitionPath(partitionKey)
           .withFileSizeInBytes(fileSize)
-          .withBlockSizeInBytes(rowGroupSize)
           .withMetrics(new Metrics(rowCount,
             arrayToMap(columnSizes),
             arrayToMap(valueCounts),

Reply via email to