Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 15 additions & 0 deletions api/src/main/java/org/apache/iceberg/DataFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,21 @@ static StructType getType(StructType partitionType) {
);
}

/**
* @return the status of the file, whether EXISTING, ADDED, or DELETED
*/
FileStatus status();

/**
* @return id of the snapshot in which the file was added to the table
*/
Long snapshotId();

/**
* @return the sequence number of the snapshot in which the file was added to the table
*/
Long sequenceNumber();

/**
* @return fully qualified path to the file, suitable for constructing a Hadoop Path
*/
Expand Down
42 changes: 42 additions & 0 deletions api/src/main/java/org/apache/iceberg/FileStatus.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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;

/**
* The status of a data or delete file in a manifest:
* 0 - existing (added in a different manifest)
* 1 - added to the table in the manifest
* 2 - deleted from the table in the manifest
*/
public enum FileStatus {
EXISTING(0),
ADDED(1),
DELETED(2);

private final int id;

FileStatus(int id) {
this.id = id;
}

public int id() {
return id;
}
}
15 changes: 15 additions & 0 deletions api/src/test/java/org/apache/iceberg/TestHelpers.java
Original file line number Diff line number Diff line change
Expand Up @@ -303,6 +303,21 @@ public TestDataFile(String path, StructLike partition, long recordCount,
this.upperBounds = upperBounds;
}

@Override
public FileStatus status() {
return FileStatus.ADDED;
}

@Override
public Long snapshotId() {
return 0L;
}

@Override
public Long sequenceNumber() {
return 0L;
}

@Override
public CharSequence path() {
return path;
Expand Down
186 changes: 181 additions & 5 deletions core/src/main/java/org/apache/iceberg/GenericDataFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.specific.SpecificData;
import org.apache.iceberg.avro.AvroSchemaUtil;
Expand All @@ -49,6 +50,12 @@ public PartitionData copy() {
private int[] fromProjectionPos;
private Types.StructType partitionType;

// ManifestEntry fields
private ManifestEntry asEntry = null;
private FileStatus status = FileStatus.ADDED;
private Long snapshotId = null;
private Long sequenceNumber = null;

private String filePath = null;
private FileFormat format = null;
private PartitionData partitionData = null;
Expand All @@ -70,11 +77,11 @@ public PartitionData copy() {
private static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;

/**
* Used by Avro reflection to instantiate this class when reading manifest files.
* Used by AsManifestEntry to instantiate this class when reading manifest files.
*/
@SuppressWarnings("checkstyle:RedundantModifier") // Must be public
public GenericDataFile(org.apache.avro.Schema avroSchema) {
private GenericDataFile(org.apache.avro.Schema avroSchema, AsManifestEntry asEntry) {
this.avroSchema = avroSchema;
this.asEntry = asEntry;

Types.StructType schema = AvroSchemaUtil.convert(avroSchema).asNestedType().asStructType();

Expand Down Expand Up @@ -112,6 +119,7 @@ public GenericDataFile(org.apache.avro.Schema avroSchema) {
this.format = format;
this.partitionData = EMPTY_PARTITION_DATA;
this.partitionType = EMPTY_PARTITION_DATA.getPartitionType();
this.asEntry = new AsManifestEntry(this, partitionType);
this.recordCount = recordCount;
this.fileSizeInBytes = fileSizeInBytes;
}
Expand All @@ -122,6 +130,7 @@ public GenericDataFile(org.apache.avro.Schema avroSchema) {
this.format = format;
this.partitionData = partition;
this.partitionType = partition.getPartitionType();
this.asEntry = new AsManifestEntry(this, partitionType);
this.recordCount = recordCount;
this.fileSizeInBytes = fileSizeInBytes;
}
Expand All @@ -139,6 +148,7 @@ public GenericDataFile(org.apache.avro.Schema avroSchema) {
this.partitionData = partition;
this.partitionType = partition.getPartitionType();
}
this.asEntry = new AsManifestEntry(this, partitionType);

// this will throw NPE if metrics.recordCount is null
this.recordCount = metrics.recordCount();
Expand All @@ -165,10 +175,14 @@ public GenericDataFile(org.apache.avro.Schema avroSchema) {
* @param fullCopy whether to copy all fields or to drop column-level stats
*/
private GenericDataFile(GenericDataFile toCopy, boolean fullCopy) {
this.status = toCopy.status;
this.snapshotId = toCopy.snapshotId;
this.sequenceNumber = toCopy.sequenceNumber;
this.filePath = toCopy.filePath;
this.format = toCopy.format;
this.partitionData = toCopy.partitionData.copy();
this.partitionType = toCopy.partitionType;
this.asEntry = new AsManifestEntry(this, partitionType);
this.recordCount = toCopy.recordCount;
this.fileSizeInBytes = toCopy.fileSizeInBytes;
if (fullCopy) {
Expand Down Expand Up @@ -196,6 +210,25 @@ private GenericDataFile(GenericDataFile toCopy, boolean fullCopy) {
GenericDataFile() {
}

@Override
public FileStatus status() {
return status;
}

@Override
public Long snapshotId() {
return snapshotId;
}

@Override
public Long sequenceNumber() {
return sequenceNumber;
}

public ManifestEntry asEntry() {
return asEntry;
}

@Override
public CharSequence path() {
return filePath;
Expand Down Expand Up @@ -399,12 +432,12 @@ public String toString() {
}

@Override
public DataFile copyWithoutStats() {
public GenericDataFile copyWithoutStats() {
return new GenericDataFile(this, false /* drop stats */);
}

@Override
public DataFile copy() {
public GenericDataFile copy() {
return new GenericDataFile(this, true /* full copy */);
}

Expand All @@ -425,4 +458,147 @@ private static <E> List<E> copy(List<E> list) {
}
return null;
}

/**
* An adapter that makes a DataFile appear like a ManifestEntry for v1 metadata files.
*/
static class AsManifestEntry
implements ManifestEntry, IndexedRecord, SpecificData.SchemaConstructable, StructLike, Serializable {
private transient Schema avroSchema = null;
private GenericDataFile file = null;

/**
* Used by Avro reflection to instantiate this class when reading manifest files.
*
* @param avroSchema an Avro read schema
*/
protected AsManifestEntry(Schema avroSchema) {
this.avroSchema = avroSchema;
this.file = new GenericDataFile(avroSchema.getField("data_file").schema(), this);
}

/**
* Used by DataFile to create a ManifestEntry adapter.
*
* @param file a GenericDataFile that contains manifest entry data
*/
protected AsManifestEntry(GenericDataFile file, Types.StructType partitionType) {
this.avroSchema = AvroSchemaUtil.convert(ManifestEntry.getSchema(partitionType), "manifest_entry");
this.file = file;
}

/**
* Constructor for Java serialization.
*/
AsManifestEntry() {
}

@Override
public Schema getSchema() {
return avroSchema;
}

@Override
public Status status() {
return ManifestEntry.Status.values()[file.status.id()];
}

@Override
public Long snapshotId() {
return file.snapshotId;
}

@Override
public void setSnapshotId(long snapshotId) {
file.snapshotId = snapshotId;
}

@Override
public Long sequenceNumber() {
return file.sequenceNumber;
}

@Override
public void setSequenceNumber(long sequenceNumber) {
file.sequenceNumber = sequenceNumber;
}

@Override
public DataFile file() {
return file;
}

@Override
public ManifestEntry copy() {
return file.copy().asEntry;
}

@Override
public ManifestEntry copyWithoutStats() {
return file.copyWithoutStats().asEntry;
}

@Override
public void put(int pos, Object value) {
switch (pos) {
case 0:
file.status = FileStatus.values()[(Integer) value];
return;
case 1:
file.snapshotId = (Long) value;
return;
case 2:
file.sequenceNumber = (Long) value;
return;
case 3:
if (file != value) {
throw new IllegalArgumentException("Cannot replace data file");
}
return;
default:
// ignore the object, it must be from a newer version of the format
}
}

@Override
public <T> void set(int pos, T value) {
put(pos, value);
}

@Override
public Object get(int i) {
switch (i) {
case 0:
return file.status.id();
case 1:
return file.snapshotId;
case 2:
return file.sequenceNumber;
case 3:
return file;
default:
throw new UnsupportedOperationException("Unknown field ordinal: " + i);
}
}

@Override
public <T> T get(int pos, Class<T> javaClass) {
return javaClass.cast(get(pos));
}

@Override
public int size() {
return avroSchema.getFields().size();
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("status", file.status)
.add("snapshot_id", file.snapshotId)
.add("sequence_number", file.sequenceNumber)
.add("file", file)
.toString();
}
}
}
Loading