Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
Show all changes
40 commits
Select commit Hold shift + click to select a range
41041f3
Spark3 structured streaming micro_batch read support
SreeramGarlapati Jun 2, 2021
51c9e87
integrate with spark3 checkpointing.
SreeramGarlapati Jun 4, 2021
1b7dbb0
remove guava dependency
SreeramGarlapati Jun 7, 2021
5a1020f
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati Jun 7, 2021
6f57b55
fix NPE in SnapshotUtil
SreeramGarlapati Jun 8, 2021
6fe5657
rename OffsetLog to InitialOffsetStore
SreeramGarlapati Jun 8, 2021
15efe95
optimization: use snapshot summary to get number of added files inste…
SreeramGarlapati Jun 8, 2021
9c9b4de
handle the case when read stream on iceberg table source restarts fro…
SreeramGarlapati Jun 8, 2021
b4acade
refactor test code
SreeramGarlapati Jun 8, 2021
cab843b
mark initialOffset final
SreeramGarlapati Jun 8, 2021
5e59082
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati Jun 10, 2021
fa85859
refresh table metadata before computing offsets.
SreeramGarlapati Jun 10, 2021
96aaa22
remove dependency on HDFSMetadataLog for checkpointing
SreeramGarlapati Jun 10, 2021
f286cee
reduce SparkScan.ReaderFactory visibility from public to package-private
SreeramGarlapati Jun 11, 2021
2ac1269
fix latestOffset value when latestOffset is the startingOffset and st…
SreeramGarlapati Jun 11, 2021
7ed9781
rename getFilesScanTasks to calculateFilesScanTasks
SreeramGarlapati Jun 11, 2021
6ee958c
remove unused instance variable - spark
SreeramGarlapati Jun 11, 2021
c101a9e
refactor SparkMicroBatchStream constructor
SreeramGarlapati Jun 11, 2021
ce16a76
remove unused log variable.
SreeramGarlapati Jun 11, 2021
baa03d6
remove unused imports.
SreeramGarlapati Jun 11, 2021
78e5bdb
replace the optimization to use snapshot.summary with data file enume…
SreeramGarlapati Jun 12, 2021
00fe477
checkstyle: remove unused imports
SreeramGarlapati Jun 12, 2021
cb4f200
confirm to codebase error message fmt's
SreeramGarlapati Jun 16, 2021
33bf3c5
refactor initialOffsetStore
SreeramGarlapati Jun 16, 2021
353f274
remove dependency on Hadoop's path - use SLASH instead
SreeramGarlapati Jun 16, 2021
f22a587
change the default behavior scanning the first snapshot of spark3 str…
SreeramGarlapati Jun 21, 2021
75e8430
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati Jun 21, 2021
3c60ef8
rename variable in SnapshotUtil.snapshotAfter from pointer to current
SreeramGarlapati Jun 21, 2021
d671d5e
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati Jun 21, 2021
344ed1f
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati Jun 21, 2021
f33b26e
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati Jun 21, 2021
9b2fd4e
refactor SnapshotUtil.snapshotAfter
SreeramGarlapati Jun 22, 2021
cc7c887
Add streaming read tests for Catalog based tables
daksha121 Jun 24, 2021
da2528b
checkstyle
SreeramGarlapati Jun 24, 2021
fadd15c
replace usage of Streams with Iterables
SreeramGarlapati Jun 24, 2021
8bb4048
add comment to explain how streaming stop and resume is simulated
SreeramGarlapati Jun 24, 2021
12c675a
revert the usage of 'delete from' statement to the original appraoach…
SreeramGarlapati Jun 24, 2021
507f7d8
remove redundant creation of sparkSession object
SreeramGarlapati Jun 25, 2021
2baebb1
Remove unnecessary rules from tests
rdblue Jun 25, 2021
7467c0c
Remove unused imports.
rdblue Jun 25, 2021
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
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hasher;
import com.google.common.hash.Hashing;
import com.google.common.io.CharStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.io.Files;
import com.google.common.primitives.Bytes;
Expand Down Expand Up @@ -92,6 +93,7 @@ public class GuavaClasses {
ThreadFactoryBuilder.class.getName();
Iterables.class.getName();
CountingOutputStream.class.getName();
CharStreams.class.getName();
}

}
Expand Down
14 changes: 14 additions & 0 deletions core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,20 @@ public static List<Long> currentAncestors(Table table) {
return ancestorIds(table.currentSnapshot(), table::snapshot);
}

/**
* Find the oldest Snapshot of a table.
* @param table the table to find the oldest snapshot on.
* @return null if the table is empty, else the oldest Snapshot.
*/
public static Snapshot oldestSnapshot(Table table) {
Snapshot current = table.currentSnapshot();
while (current.parentId() != null) {
current = table.snapshot(current.parentId());
}

return current;
}

/**
* Returns list of snapshot ids in the range - (fromSnapshotId, toSnapshotId]
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.apache.spark.sql.connector.read.Scan;
import org.apache.spark.sql.connector.read.Statistics;
import org.apache.spark.sql.connector.read.SupportsReportStatistics;
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
import org.apache.spark.sql.vectorized.ColumnarBatch;
Expand All @@ -62,6 +63,7 @@ abstract class SparkBatchScan implements Scan, Batch, SupportsReportStatistics {
private static final Logger LOG = LoggerFactory.getLogger(SparkBatchScan.class);

private final JavaSparkContext sparkContext;
private final SparkSession spark;
private final Table table;
private final boolean caseSensitive;
private final boolean localityPreferred;
Expand All @@ -76,6 +78,7 @@ abstract class SparkBatchScan implements Scan, Batch, SupportsReportStatistics {
SparkBatchScan(SparkSession spark, Table table, boolean caseSensitive, Schema expectedSchema,
List<Expression> filters, CaseInsensitiveStringMap options) {
this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
this.spark = spark;
this.table = table;
this.caseSensitive = caseSensitive;
this.expectedSchema = expectedSchema;
Expand Down Expand Up @@ -108,6 +111,12 @@ public Batch toBatch() {
return this;
}

@Override
public MicroBatchStream toMicroBatchStream(String checkpointLocation) {
return new SparkMicroBatchStream(
spark, sparkContext, table, caseSensitive, expectedSchema, options, checkpointLocation);
}

@Override
public StructType readSchema() {
if (readSchema == null) {
Expand Down Expand Up @@ -213,10 +222,10 @@ public String description() {
return String.format("%s [filters=%s]", table, filters);
}

private static class ReaderFactory implements PartitionReaderFactory {
public static class ReaderFactory implements PartitionReaderFactory {
private final int batchSize;

private ReaderFactory(int batchSize) {
ReaderFactory(int batchSize) {
this.batchSize = batchSize;
}

Expand Down Expand Up @@ -256,7 +265,7 @@ private static class BatchReader extends BatchDataReader implements PartitionRea
}
}

private static class ReadTask implements InputPartition, Serializable {
public static class ReadTask implements InputPartition, Serializable {
private final CombinedScanTask task;
private final Broadcast<Table> tableBroadcast;
private final String expectedSchemaString;
Expand Down
Loading