From 85795fe262bdf4530859130aa77430e5437ffbcf Mon Sep 17 00:00:00 2001 From: abhishekagarwal87 <1477457+abhishekagarwal87@users.noreply.github.com> Date: Tue, 19 Dec 2023 22:28:22 +0530 Subject: [PATCH 01/65] something --- processing/pom.xml | 19 +++ .../apache/druid/data/input/InputSource.java | 2 + .../data/input/impl/delta/DeltaInputRow.java | 66 +++++++++ .../input/impl/delta/DeltaInputSource.java | 133 ++++++++++++++++++ .../impl/delta/DeltaInputSourceReader.java | 73 ++++++++++ .../data/input/impl/delta/DeltaSplit.java | 40 ++++++ .../druid/data/input/impl/delta/RowSerde.java | 130 +++++++++++++++++ 7 files changed, 463 insertions(+) create mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java create mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java create mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java create mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java create mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java diff --git a/processing/pom.xml b/processing/pom.xml index 7f4a6d7fffd8..76b950e1cb1e 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -342,6 +342,25 @@ ${oshi.version} + + io.delta + delta-kernel-api + 3.0.0 + + + + + io.delta + delta-kernel-defaults + 3.0.0 + + + + org.apache.hadoop + hadoop-client-api + 3.3.6 + + diff --git a/processing/src/main/java/org/apache/druid/data/input/InputSource.java b/processing/src/main/java/org/apache/druid/data/input/InputSource.java index be815742be16..38ce91df2178 100644 --- a/processing/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/InputSource.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.impl.HttpInputSource; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.delta.DeltaInputSource; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.UOE; @@ -57,6 +58,7 @@ @Type(name = HttpInputSource.TYPE_KEY, value = HttpInputSource.class), @Type(name = InlineInputSource.TYPE_KEY, value = InlineInputSource.class), @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class) + @Type(name = DeltaInputSource.TYPE_KEY, value = DeltaInputSource.class) }) public interface InputSource { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java new file mode 100644 index 000000000000..651e144fa205 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java @@ -0,0 +1,66 @@ +package org.apache.druid.data.input.impl.delta; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.Row; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.List; + +public class DeltaInputRow implements InputRow +{ + private final io.delta.kernel.data.Row row; + private final InputRowSchema schema; + + public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema schema) + { + this.row = row; + this.schema = schema; + } + @Override + public List getDimensions() + { + return row.getSchema().fieldNames(); + } + + @Override + public long getTimestampFromEpoch() + { + return row.getLong(0); + } + + @Override + public DateTime getTimestamp() + { + String tsCol = schema.getTimestampSpec().getTimestampColumn(); + return new DateTime(row.getLong(tsCol)); + return null; + } + + @Override + public List getDimension(String dimension) + { + return null; + } + + @Nullable + @Override + public Object getRaw(String dimension) + { + return null; + } + + @Nullable + @Override + public Number getMetric(String metric) + { + return null; + } + + @Override + public int compareTo(Row o) + { + return 0; + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java new file mode 100644 index 000000000000..7b6dd29d1c7f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -0,0 +1,133 @@ +package org.apache.druid.data.input.impl.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.delta.kernel.Scan; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.TableNotFoundException; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.internal.util.Utils; +import io.delta.kernel.utils.CloseableIterator; +import io.delta.kernel.utils.*; + +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.data.input.impl.systemfield.SystemFieldDecoratorFactory; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.hadoop.conf.Configuration; + + +public class DeltaInputSource implements SplittableInputSource +{ + public static final String TYPE_KEY = "delta"; + private final String tablePath; + @Nullable + private final DeltaSplit deltaSplit; + + @JsonCreator + public DeltaInputSource( + @JsonProperty("tablePath") String tablePath, + @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit + ) + { + this.tablePath = tablePath; + this.deltaSplit = deltaSplit; + } + + @Override + public boolean needsFormat() + { + // Only support Parquet + return false; + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + File temporaryDirectory + ) + { + String myTablePath = tablePath; // fully qualified table path. Ex: file:/user/tables/myTable + Configuration hadoopConf = new Configuration(); + TableClient myTableClient = DefaultTableClient.create(hadoopConf); + try { + Row scanState = null; + List scanRowList = null; + if (deltaSplit != null) { + scanState = deserialize(myTableClient, deltaSplit.getStateRow()); + scanRowList = deltaSplit.getFileRows().stream().map(row -> deserialize(myTableClient, row)).collect(Collectors.toList()); + } else { + Table myTable = Table.forPath(myTableClient, myTablePath); + Snapshot mySnapshot = myTable.getLatestSnapshot(myTableClient); + Scan myScan = mySnapshot.getScanBuilder(myTableClient).build(); + scanState = myScan.getScanState(myTableClient); + CloseableIterator myScanFilesAsBatches = myScan.getScanFiles(myTableClient); + scanRowList = new ArrayList<>(); + while (myScanFilesAsBatches.hasNext()) { + FilteredColumnarBatch scanFileBatch = myScanFilesAsBatches.next(); + CloseableIterator myScanFilesAsRows = scanFileBatch.getRows(); + myScanFilesAsRows.forEachRemaining(scanRowList::add); + } + } + return new DeltaInputSourceReader(Scan.readData( + myTableClient, + scanState, + Utils.toCloseableIterator(scanRowList.iterator()), + Optional.empty() + )); + } + catch (TableNotFoundException e) { + throw InvalidInput.exception(e, "Table not found: %s", myTablePath); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + throws IOException + { + return null; + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException + { + return 0; + } + + @Override + public InputSource withSplit(InputSplit split) + { + return null; + } + + private Row deserialize(TableClient myTableClient, String row) + { + return RowSerde.deserializeRowFromJson(myTableClient, row); + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java new file mode 100644 index 000000000000..f50b6915b9fe --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -0,0 +1,73 @@ +package org.apache.druid.data.input.impl.delta; + +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; + +public class DeltaInputSourceReader implements InputSourceReader +{ + private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + + public DeltaInputSourceReader(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator) + { + + this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + } + + @Override + public CloseableIterator read(InputStats inputStats) throws IOException + { + return null; + } + + @Override + public CloseableIterator sample() throws IOException + { + return null; + } + + private static class DeltaInputSourceIterator implements CloseableIterator + { + private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + private io.delta.kernel.utils.CloseableIterator currentBatch = null; + + public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator) + { + this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + } + + @Override + public boolean hasNext() + { + if (currentBatch == null) + { + while (filteredColumnarBatchCloseableIterator.hasNext()) + { + currentBatch = filteredColumnarBatchCloseableIterator.next().getRows(); + if (currentBatch.hasNext()) { + return true; + } + } + } + return currentBatch != null && currentBatch.hasNext(); + } + + @Override + public InputRow next() + { + return currentBatch.next(); + } + + @Override + public void close() throws IOException + { + filteredColumnarBatchCloseableIterator.close(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java new file mode 100644 index 000000000000..8b1883a5fdb8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java @@ -0,0 +1,40 @@ +package org.apache.druid.data.input.impl.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class DeltaSplit +{ + private final String stateRow; + private final List fileRows; + + @JsonCreator + public DeltaSplit(@JsonProperty("state") String stateRow, @JsonProperty("file") List fileRows) + { + this.stateRow = stateRow; + this.fileRows = fileRows; + } + + @JsonProperty("state") + public String getStateRow() + { + return stateRow; + } + + @JsonProperty("file") + public List getFileRows() + { + return fileRows; + } + + @Override + public String toString() + { + return "DeltaSplit{" + + "stateRow=" + stateRow + + ", fileRow=" + fileRows + + "}"; + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java new file mode 100644 index 000000000000..ee63bb8f6d45 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java @@ -0,0 +1,130 @@ +package org.apache.druid.data.input.impl.delta; + +/* + * Copyright (2023) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.Map; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.Row; +import io.delta.kernel.types.*; + +import io.delta.kernel.internal.types.TableSchemaSerDe; +import io.delta.kernel.internal.util.VectorUtils; + +import io.delta.kernel.defaults.internal.data.DefaultJsonRow; + +/** + * Utility class to serialize and deserialize {@link Row} object. + */ +public class RowSerde { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private RowSerde() { + } + + /** + * Utility method to serialize a {@link Row} as a JSON string + */ + public static String serializeRowToJson(Row row) { + Map rowObject = convertRowToJsonObject(row); + try { + Map rowWithSchema = new HashMap<>(); + rowWithSchema.put("schema", TableSchemaSerDe.toJson(row.getSchema())); + rowWithSchema.put("row", rowObject); + return OBJECT_MAPPER.writeValueAsString(rowWithSchema); + } catch (JsonProcessingException e) { + throw new UncheckedIOException(e); + } + } + + /** + * Utility method to deserialize a {@link Row} object from the JSON form. + */ + public static Row deserializeRowFromJson(TableClient tableClient, String jsonRowWithSchema) { + try { + JsonNode jsonNode = OBJECT_MAPPER.readTree(jsonRowWithSchema); + JsonNode schemaNode = jsonNode.get("schema"); + StructType schema = + TableSchemaSerDe.fromJson(tableClient.getJsonHandler(), schemaNode.asText()); + return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); + } catch (JsonProcessingException ex) { + throw new UncheckedIOException(ex); + } + } + + private static Map convertRowToJsonObject(Row row) { + StructType rowType = row.getSchema(); + Map rowObject = new HashMap<>(); + for (int fieldId = 0; fieldId < rowType.length(); fieldId++) { + StructField field = rowType.at(fieldId); + DataType fieldType = field.getDataType(); + String name = field.getName(); + + if (row.isNullAt(fieldId)) { + rowObject.put(name, null); + continue; + } + + Object value; + if (fieldType instanceof BooleanType) { + value = row.getBoolean(fieldId); + } else if (fieldType instanceof ByteType) { + value = row.getByte(fieldId); + } else if (fieldType instanceof ShortType) { + value = row.getShort(fieldId); + } else if (fieldType instanceof IntegerType) { + value = row.getInt(fieldId); + } else if (fieldType instanceof LongType) { + value = row.getLong(fieldId); + } else if (fieldType instanceof FloatType) { + value = row.getFloat(fieldId); + } else if (fieldType instanceof DoubleType) { + value = row.getDouble(fieldId); + } else if (fieldType instanceof DateType) { + value = row.getInt(fieldId); + } else if (fieldType instanceof TimestampType) { + value = row.getLong(fieldId); + } else if (fieldType instanceof StringType) { + value = row.getString(fieldId); + } else if (fieldType instanceof ArrayType) { + value = VectorUtils.toJavaList(row.getArray(fieldId)); + } else if (fieldType instanceof MapType) { + value = VectorUtils.toJavaMap(row.getMap(fieldId)); + } else if (fieldType instanceof StructType) { + Row subRow = row.getStruct(fieldId); + value = convertRowToJsonObject(subRow); + } else { + throw new UnsupportedOperationException("NYI"); + } + + rowObject.put(name, value); + } + + return rowObject; + } + + private static Row parseRowFromJsonWithSchema(ObjectNode rowJsonNode, StructType rowType) { + return new DefaultJsonRow(rowJsonNode, rowType); + } +} \ No newline at end of file From a3baae9e149d8105e850c4b53f695b19c9c7aca1 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 12:07:35 -0500 Subject: [PATCH 02/65] test commit --- .../apache/druid/data/input/impl/delta/DeltaInputSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index 7b6dd29d1c7f..e9c2b161a434 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -38,7 +38,7 @@ import org.apache.druid.java.util.common.CloseableIterators; import org.apache.hadoop.conf.Configuration; - +// TODO public class DeltaInputSource implements SplittableInputSource { public static final String TYPE_KEY = "delta"; From 23d2616ef3344168bc3b539e66ddd722e7033a68 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 12:26:32 -0500 Subject: [PATCH 03/65] compilation fix --- .../src/main/java/org/apache/druid/data/input/InputSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/InputSource.java b/processing/src/main/java/org/apache/druid/data/input/InputSource.java index 38ce91df2178..912dbbfd54e3 100644 --- a/processing/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/InputSource.java @@ -57,7 +57,7 @@ @Type(name = LocalInputSource.TYPE_KEY, value = LocalInputSource.class), @Type(name = HttpInputSource.TYPE_KEY, value = HttpInputSource.class), @Type(name = InlineInputSource.TYPE_KEY, value = InlineInputSource.class), - @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class) + @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class), @Type(name = DeltaInputSource.TYPE_KEY, value = DeltaInputSource.class) }) public interface InputSource From 672894af33701efdf889bc2604d5223822dc995f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 12:29:21 -0500 Subject: [PATCH 04/65] more compilation fixes (fixme placeholders) --- .../org/apache/druid/data/input/impl/delta/DeltaInputRow.java | 2 +- .../druid/data/input/impl/delta/DeltaInputSourceReader.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java index 651e144fa205..0f969ade933b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java @@ -34,7 +34,7 @@ public long getTimestampFromEpoch() public DateTime getTimestamp() { String tsCol = schema.getTimestampSpec().getTimestampColumn(); - return new DateTime(row.getLong(tsCol)); + // return new DateTime(row.getLong(tsCol)); fixme return null; } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index f50b6915b9fe..560903f969a1 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -61,7 +61,8 @@ public boolean hasNext() @Override public InputRow next() { - return currentBatch.next(); + return null; +// return currentBatch.next(); fixme type mismatch } @Override From 7b67bd23849230d3f6712c5def0f736029f16fcf Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 14:00:59 -0500 Subject: [PATCH 05/65] Comment out druid-kereberos build since it conflicts with newly added transitive deps from delta-lake Will need to sort out the dependencies later. --- distribution/pom.xml | 5 +++-- pom.xml | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index ab8e082c9ab8..e15bb77471a5 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -234,8 +234,9 @@ -c org.apache.druid.extensions:postgresql-metadata-storage -c - org.apache.druid.extensions:druid-kerberos - -c + + + org.apache.druid.extensions:druid-s3-extensions -c org.apache.druid.extensions:druid-aws-rds-extensions diff --git a/pom.xml b/pom.xml index 81cb00bb0cf5..92e7b09f103b 100644 --- a/pom.xml +++ b/pom.xml @@ -175,7 +175,7 @@ extensions-core/azure-extensions extensions-core/datasketches extensions-core/druid-bloom-filter - extensions-core/druid-kerberos + extensions-core/druid-pac4j extensions-core/hdfs-storage extensions-core/histogram From e67f5cf6728d978e45e6239f86af2fa7ce267056 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 21:23:40 -0500 Subject: [PATCH 06/65] checkpoint --- .../input/impl/delta/DeltaInputSource.java | 110 ++++++++++++------ .../impl/delta/DeltaInputSourceReader.java | 109 ++++++++++++++++- .../data/input/impl/delta/DeltaSplit.java | 6 +- 3 files changed, 184 insertions(+), 41 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index e9c2b161a434..402c4f5f2079 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -2,6 +2,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; +import com.google.common.primitives.Ints; import io.delta.kernel.Scan; import io.delta.kernel.Snapshot; import io.delta.kernel.Table; @@ -11,38 +14,36 @@ import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; import io.delta.kernel.internal.util.Utils; +import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; -import io.delta.kernel.utils.*; - import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.InputEntityIteratingReader; import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.data.input.impl.systemfield.SystemFieldDecoratorFactory; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.utils.Streams; +import org.apache.hadoop.conf.Configuration; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.hadoop.conf.Configuration; -// TODO public class DeltaInputSource implements SplittableInputSource { public static final String TYPE_KEY = "delta"; + private final String tablePath; + @Nullable private final DeltaSplit deltaSplit; @@ -52,7 +53,7 @@ public DeltaInputSource( @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit ) { - this.tablePath = tablePath; + this.tablePath = Preconditions.checkNotNull(tablePath, "tablePath cannot be null"); this.deltaSplit = deltaSplit; } @@ -70,37 +71,45 @@ public InputSourceReader reader( File temporaryDirectory ) { - String myTablePath = tablePath; // fully qualified table path. Ex: file:/user/tables/myTable Configuration hadoopConf = new Configuration(); - TableClient myTableClient = DefaultTableClient.create(hadoopConf); + TableClient tableClient = DefaultTableClient.create(hadoopConf); try { - Row scanState = null; - List scanRowList = null; + final Row scanState; + final List scanRowList; + final StructType schema; + if (deltaSplit != null) { - scanState = deserialize(myTableClient, deltaSplit.getStateRow()); - scanRowList = deltaSplit.getFileRows().stream().map(row -> deserialize(myTableClient, row)).collect(Collectors.toList()); + scanState = deserialize(tableClient, deltaSplit.getStateRow()); + scanRowList = deltaSplit.getFile().stream().map(row -> deserialize(tableClient, row)).collect(Collectors.toList()); + schema = scanRowList.stream().findFirst().map(Row::getSchema).orElse(null); } else { - Table myTable = Table.forPath(myTableClient, myTablePath); - Snapshot mySnapshot = myTable.getLatestSnapshot(myTableClient); - Scan myScan = mySnapshot.getScanBuilder(myTableClient).build(); - scanState = myScan.getScanState(myTableClient); - CloseableIterator myScanFilesAsBatches = myScan.getScanFiles(myTableClient); + Table table = Table.forPath(tableClient, tablePath); + Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); + schema = latestSnapshot.getSchema(tableClient); + + Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + scanState = scan.getScanState(tableClient); + CloseableIterator scanFiles = scan.getScanFiles(tableClient); scanRowList = new ArrayList<>(); - while (myScanFilesAsBatches.hasNext()) { - FilteredColumnarBatch scanFileBatch = myScanFilesAsBatches.next(); - CloseableIterator myScanFilesAsRows = scanFileBatch.getRows(); - myScanFilesAsRows.forEachRemaining(scanRowList::add); + + while (scanFiles.hasNext()) { + FilteredColumnarBatch scanFileBatch = scanFiles.next(); + CloseableIterator scanFileRows = scanFileBatch.getRows(); + scanFileRows.forEachRemaining(scanRowList::add); } } - return new DeltaInputSourceReader(Scan.readData( - myTableClient, - scanState, - Utils.toCloseableIterator(scanRowList.iterator()), + return new DeltaInputSourceReader( + Scan.readData( + tableClient, + scanState, + Utils.toCloseableIterator(scanRowList.iterator()), Optional.empty() - )); + ), + schema + ); } catch (TableNotFoundException e) { - throw InvalidInput.exception(e, "Table not found: %s", myTablePath); + throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); } catch (IOException e) { throw new RuntimeException(e); @@ -111,19 +120,52 @@ public InputSourceReader reader( public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException { - return null; + TableClient tableClient = DefaultTableClient.create(new Configuration()); + final Snapshot latestSnapshot; + final Table table; + try { + table = Table.forPath(tableClient, tablePath); + latestSnapshot = table.getLatestSnapshot(tableClient); + } + catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + // scan files iterator for the current snapshot + CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient); + + Row scanState = scan.getScanState(tableClient); + String scanStateStr = RowSerde.serializeRowToJson(scanState); + + Iterator deltaSplitIterator = Iterators.transform( + scanFilesIterator, + scanFile -> { + CloseableIterator rows = scanFile.getRows(); + List fileRows = new ArrayList<>(); + while (rows.hasNext()) { + fileRows.add(RowSerde.serializeRowToJson(rows.next())); + } + return new DeltaSplit(scanStateStr, fileRows); + } + ); + + // TODO: account for the split spec as well -- getSplitHintSpecOrDefault(splitHintSpec).split() + return Streams.sequentialStreamFrom(deltaSplitIterator).map(InputSplit::new); } @Override public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException { - return 0; + return Ints.checkedCast(createSplits(inputFormat, splitHintSpec).count()); } @Override public InputSource withSplit(InputSplit split) { - return null; + return new DeltaInputSource( + tablePath, + split.get() + ); } private Row deserialize(TableClient myTableClient, String row) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 560903f969a1..557e9f8b2e75 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -2,6 +2,22 @@ import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampType; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; @@ -9,15 +25,30 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.NoSuchElementException; +import java.util.stream.IntStream; public class DeltaInputSourceReader implements InputSourceReader { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + private final StructType snapshotSchema; - public DeltaInputSourceReader(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator) + public DeltaInputSourceReader( + io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + StructType snapshotSchema + ) { - this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.snapshotSchema = snapshotSchema; + } + + @Override + public CloseableIterator read() + { + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, snapshotSchema); } @Override @@ -35,11 +66,16 @@ public CloseableIterator sample() throws IOException private static class DeltaInputSourceIterator implements CloseableIterator { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + private final StructType snapshotSchema; + private io.delta.kernel.utils.CloseableIterator currentBatch = null; - public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator) + public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + StructType snapshotSchema + ) { this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.snapshotSchema = snapshotSchema; } @Override @@ -61,8 +97,21 @@ public boolean hasNext() @Override public InputRow next() { - return null; -// return currentBatch.next(); fixme type mismatch + if (!hasNext()) { + throw new NoSuchElementException(); + } + + Row dataRow = currentBatch.next(); + int numCols = dataRow.getSchema().length(); + + Object[] rowValues = IntStream.range(0, numCols) + .mapToObj(colOrdinal -> getValue(dataRow, colOrdinal)) + .toArray(); + for (Object rowValue : rowValues) { + System.out.println(rowValue); + } + // TODO: construct schema? + return new DeltaInputRow(dataRow, null); } @Override @@ -71,4 +120,54 @@ public void close() throws IOException filteredColumnarBatchCloseableIterator.close(); } } + + /** + * Derive value using the type mappings from row's schema pertaining to an ordinal + */ + private static String getValue(Row dataRow, int columnOrdinal) { + DataType dataType = dataRow.getSchema().at(columnOrdinal).getDataType(); + if (dataRow.isNullAt(columnOrdinal)) { + return null; + } else if (dataType instanceof BooleanType) { + return Boolean.toString(dataRow.getBoolean(columnOrdinal)); + } else if (dataType instanceof ByteType) { + return Byte.toString(dataRow.getByte(columnOrdinal)); + } else if (dataType instanceof ShortType) { + return Short.toString(dataRow.getShort(columnOrdinal)); + } else if (dataType instanceof IntegerType) { + return Integer.toString(dataRow.getInt(columnOrdinal)); + } else if (dataType instanceof DateType) { + // DateType data is stored internally as the number of days since 1970-01-01 + int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); + return LocalDate.ofEpochDay(daysSinceEpochUTC).toString(); + } else if (dataType instanceof LongType) { + return Long.toString(dataRow.getLong(columnOrdinal)); + } else if (dataType instanceof TimestampType) { + // TimestampType data is stored internally as the number of microseconds since epoch + long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); + LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC); + return dateTime.toString(); + } else if (dataType instanceof FloatType) { + return Float.toString(dataRow.getFloat(columnOrdinal)); + } else if (dataType instanceof DoubleType) { + return Double.toString(dataRow.getDouble(columnOrdinal)); + } else if (dataType instanceof StringType) { + return dataRow.getString(columnOrdinal); + } else if (dataType instanceof BinaryType) { + return new String(dataRow.getBinary(columnOrdinal)); + } else if (dataType instanceof DecimalType) { + return dataRow.getDecimal(columnOrdinal).toString(); + } else if (dataType instanceof StructType) { + return "TODO: struct value"; + } else if (dataType instanceof ArrayType) { + return "TODO: list value"; + } else if (dataType instanceof MapType) { + return "TODO: map value"; + } else { + throw new UnsupportedOperationException("unsupported data type: " + dataType); + } + } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java index 8b1883a5fdb8..856f7dc8aeb6 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java @@ -2,6 +2,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.delta.kernel.data.Row; import java.util.List; @@ -24,17 +25,18 @@ public String getStateRow() } @JsonProperty("file") - public List getFileRows() + public List getFile() { return fileRows; } + @Override public String toString() { return "DeltaSplit{" + "stateRow=" + stateRow + - ", fileRow=" + fileRows + + ", file=" + fileRows + "}"; } } From da24f6238c0497a7123409ca80ccc72c215512ab Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 21:25:32 -0500 Subject: [PATCH 07/65] remove snapshot schema since we can get schema from the row --- .../data/input/impl/delta/DeltaInputSource.java | 6 +----- .../input/impl/delta/DeltaInputSourceReader.java | 12 +++--------- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index 402c4f5f2079..c5c2b9a19dd8 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -76,16 +76,13 @@ public InputSourceReader reader( try { final Row scanState; final List scanRowList; - final StructType schema; if (deltaSplit != null) { scanState = deserialize(tableClient, deltaSplit.getStateRow()); scanRowList = deltaSplit.getFile().stream().map(row -> deserialize(tableClient, row)).collect(Collectors.toList()); - schema = scanRowList.stream().findFirst().map(Row::getSchema).orElse(null); } else { Table table = Table.forPath(tableClient, tablePath); Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); - schema = latestSnapshot.getSchema(tableClient); Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); scanState = scan.getScanState(tableClient); @@ -104,8 +101,7 @@ public InputSourceReader reader( scanState, Utils.toCloseableIterator(scanRowList.iterator()), Optional.empty() - ), - schema + ) ); } catch (TableNotFoundException e) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 557e9f8b2e75..6f9e455ba05c 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -34,21 +34,18 @@ public class DeltaInputSourceReader implements InputSourceReader { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; - private final StructType snapshotSchema; public DeltaInputSourceReader( - io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, - StructType snapshotSchema + io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator ) { this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; - this.snapshotSchema = snapshotSchema; } @Override public CloseableIterator read() { - return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, snapshotSchema); + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator); } @Override @@ -66,16 +63,13 @@ public CloseableIterator sample() throws IOException private static class DeltaInputSourceIterator implements CloseableIterator { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; - private final StructType snapshotSchema; private io.delta.kernel.utils.CloseableIterator currentBatch = null; - public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, - StructType snapshotSchema + public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator ) { this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; - this.snapshotSchema = snapshotSchema; } @Override From 5bc39e1f0f8e3789d454be94c5e73bccec22323a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 22:27:54 -0500 Subject: [PATCH 08/65] iterator bug fix --- .../impl/delta/DeltaInputSourceReader.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 6f9e455ba05c..66c421da992e 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -73,19 +73,14 @@ public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator getValue(dataRow, colOrdinal)) .toArray(); + + // TODO: construct schema? remove this after debugging for (Object rowValue : rowValues) { System.out.println(rowValue); } - // TODO: construct schema? return new DeltaInputRow(dataRow, null); } From 27ba7b372a625570a86af2bd324a56724ed0decd Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 19 Dec 2023 23:15:20 -0500 Subject: [PATCH 09/65] json json json --- .../druid/data/input/impl/delta/DeltaInputSource.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index c5c2b9a19dd8..e3d0bb1a8dbf 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -14,7 +14,6 @@ import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; import io.delta.kernel.internal.util.Utils; -import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; @@ -24,6 +23,7 @@ import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; @@ -42,11 +42,15 @@ public class DeltaInputSource implements SplittableInputSource { public static final String TYPE_KEY = "delta"; + @JsonProperty private final String tablePath; + @JsonProperty @Nullable private final DeltaSplit deltaSplit; + private static final Logger log = new Logger(DeltaInputSource.class); + @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") String tablePath, @@ -71,6 +75,7 @@ public InputSourceReader reader( File temporaryDirectory ) { + log.info("Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); Configuration hadoopConf = new Configuration(); TableClient tableClient = DefaultTableClient.create(hadoopConf); try { From 1bf4f7ebe6a29e2e59ac5cb6c7531808bd3a1900 Mon Sep 17 00:00:00 2001 From: abhishekagarwal87 <1477457+abhishekagarwal87@users.noreply.github.com> Date: Wed, 20 Dec 2023 10:43:47 +0530 Subject: [PATCH 10/65] sampler flow --- .../data/input/impl/delta/DeltaInputRow.java | 6 +++++ .../impl/delta/DeltaInputSourceReader.java | 27 +++++++++++++++++-- .../druid/data/input/impl/delta/RowSerde.java | 2 +- 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java index 0f969ade933b..24b86dcf082b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java @@ -7,6 +7,7 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Map; public class DeltaInputRow implements InputRow { @@ -63,4 +64,9 @@ public int compareTo(Row o) { return 0; } + + public Map getRawRowAsMap() + { + return RowSerde.convertRowToJsonObject(row); + } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 66c421da992e..cf077bb86cca 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -1,5 +1,6 @@ package org.apache.druid.data.input.impl.delta; +import com.google.common.collect.Iterators; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; import io.delta.kernel.types.ArrayType; @@ -22,6 +23,7 @@ import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputStats; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; @@ -51,13 +53,34 @@ public CloseableIterator read() @Override public CloseableIterator read(InputStats inputStats) throws IOException { - return null; + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator); } @Override public CloseableIterator sample() throws IOException { - return null; + CloseableIterator inner = read(); + return new CloseableIterator() + { + @Override + public void close() throws IOException + { + inner.close(); + } + + @Override + public boolean hasNext() + { + return inner.hasNext(); + } + + @Override + public InputRowListPlusRawValues next() + { + DeltaInputRow deltaInputRow = (DeltaInputRow) inner.next(); + return InputRowListPlusRawValues.of(deltaInputRow, deltaInputRow.getRawRowAsMap()); + } + }; } private static class DeltaInputSourceIterator implements CloseableIterator diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java index ee63bb8f6d45..df769b31ea2b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java @@ -73,7 +73,7 @@ public static Row deserializeRowFromJson(TableClient tableClient, String jsonRow } } - private static Map convertRowToJsonObject(Row row) { + public static Map convertRowToJsonObject(Row row) { StructType rowType = row.getSchema(); Map rowObject = new HashMap<>(); for (int fieldId = 0; fieldId < rowType.length(); fieldId++) { From 934d92d036dd6509637e5c107416a7157b1b07a7 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 20 Dec 2023 00:14:16 -0500 Subject: [PATCH 11/65] empty impls for read(InputStats) and sample() --- .../input/impl/delta/DeltaInputSourceReader.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 66c421da992e..54769f422476 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -22,17 +22,22 @@ import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputStats; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.util.Collections; import java.util.NoSuchElementException; import java.util.stream.IntStream; public class DeltaInputSourceReader implements InputSourceReader { + private static final Logger log = new Logger(DeltaInputSourceReader.class); + private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; public DeltaInputSourceReader( @@ -51,13 +56,16 @@ public CloseableIterator read() @Override public CloseableIterator read(InputStats inputStats) throws IOException { - return null; + log.info("read(inputStats) -> re-routing to read()"); // todo: input stats need to be accounted for + return read(); } @Override public CloseableIterator sample() throws IOException { - return null; + log.info("sample() -> noop"); + return CloseableIterators.wrap(Collections.emptyIterator(), () -> { + }); } private static class DeltaInputSourceIterator implements CloseableIterator @@ -99,7 +107,7 @@ public InputRow next() // TODO: construct schema? remove this after debugging for (Object rowValue : rowValues) { - System.out.println(rowValue); + log.info("RowValue[%s]", rowValue); } return new DeltaInputRow(dataRow, null); } From 235e03a33f08d894dad0bdf57b862b2e7a314aa9 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 20 Dec 2023 11:21:15 +0530 Subject: [PATCH 12/65] conversion? --- .../data/input/impl/delta/DeltaInputRow.java | 116 ++++++++++++++++-- .../impl/delta/DeltaInputSourceReader.java | 2 +- 2 files changed, 104 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java index 0f969ade933b..0a7fa00c8afb 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java @@ -1,66 +1,156 @@ package org.apache.druid.data.input.impl.delta; +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampType; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.Rows; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.List; public class DeltaInputRow implements InputRow { private final io.delta.kernel.data.Row row; - private final InputRowSchema schema; + private final StructType schema; + private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); + private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TIMEZONE HANDLING????? - public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema schema) + public DeltaInputRow(io.delta.kernel.data.Row row) { this.row = row; - this.schema = schema; + this.schema = row.getSchema(); + List fieldNames = this.schema.fieldNames(); + for (int i = 0; i < fieldNames.size(); ++i) { + fieldNameToOrdinal.put(fieldNames.get(i), i); + } + fieldNameToOrdinal.defaultReturnValue(-1); } @Override public List getDimensions() { - return row.getSchema().fieldNames(); + return schema.fieldNames(); } @Override public long getTimestampFromEpoch() { - return row.getLong(0); + Object type = getRaw(ColumnHolder.TIME_COLUMN_NAME); + return type == null ? 0 : (long) type; } @Override public DateTime getTimestamp() { - String tsCol = schema.getTimestampSpec().getTimestampColumn(); - // return new DateTime(row.getLong(tsCol)); fixme - return null; + return DateTimes.utc(getTimestampFromEpoch()); } @Override public List getDimension(String dimension) { - return null; + return Rows.objectToStrings(getRaw(dimension)); } @Nullable @Override public Object getRaw(String dimension) { - return null; + StructField field = schema.get(dimension); + if (field == null) { + return null; + } else if (field.isMetadataColumn()) { + return null; + } + + int ordinal = fieldNameToOrdinal.getInt(dimension); + if (ordinal < 0) { + return null; + } + return getValue(field.getDataType(), row, ordinal); + } @Nullable @Override public Number getMetric(String metric) { - return null; + return Rows.objectToNumber(metric, getRaw(metric), true); } @Override public int compareTo(Row o) { - return 0; + return this.getTimestamp().compareTo(o.getTimestamp()); + } + + @Nullable + private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataRow, int columnOrdinal) { + if (dataRow.isNullAt(columnOrdinal)) { + return null; + } else if (dataType instanceof BooleanType) { + return dataRow.getBoolean(columnOrdinal); + } else if (dataType instanceof ByteType) { + return dataRow.getByte(columnOrdinal); + } else if (dataType instanceof ShortType) { + return dataRow.getShort(columnOrdinal); + } else if (dataType instanceof IntegerType) { + return dataRow.getInt(columnOrdinal); + } else if (dataType instanceof DateType) { + // DateType data is stored internally as the number of days since 1970-01-01 + int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); + return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); + } else if (dataType instanceof LongType) { + return dataRow.getLong(columnOrdinal); + } else if (dataType instanceof TimestampType) { + // TimestampType data is stored internally as the number of microseconds since epoch + long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); + LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC); + return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); + } else if (dataType instanceof FloatType) { + return dataRow.getFloat(columnOrdinal); + } else if (dataType instanceof DoubleType) { + return dataRow.getDouble(columnOrdinal); + } else if (dataType instanceof StringType) { + return dataRow.getString(columnOrdinal); + } else if (dataType instanceof BinaryType) { + return new String(dataRow.getBinary(columnOrdinal)); + } else if (dataType instanceof DecimalType) { + return dataRow.getDecimal(columnOrdinal).longValue(); + } else if (dataType instanceof StructType) { + return "TODO: struct value"; + } else if (dataType instanceof ArrayType) { + return "TODO: list value"; + } else if (dataType instanceof MapType) { + return "TODO: map value"; + } else { + throw new UnsupportedOperationException("unsupported data type: " + dataType); + } } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 66c421da992e..2303af90e063 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -101,7 +101,7 @@ public InputRow next() for (Object rowValue : rowValues) { System.out.println(rowValue); } - return new DeltaInputRow(dataRow, null); + return new DeltaInputRow(dataRow); } @Override From 43a4868013110522d936c6a1d83db2f7d475c41a Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 20 Dec 2023 12:47:35 +0530 Subject: [PATCH 13/65] conversion, without timestamp --- .../data/input/impl/delta/DeltaInputRow.java | 33 +++++++++++++++---- .../input/impl/delta/DeltaInputSource.java | 3 +- .../impl/delta/DeltaInputSourceReader.java | 19 +++++++---- 3 files changed, 41 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java index 6c8fce062f6f..4867b1ea52f7 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java @@ -20,8 +20,11 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.Rows; +import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.DateTime; @@ -31,6 +34,7 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZoneOffset; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -40,8 +44,9 @@ public class DeltaInputRow implements InputRow private final StructType schema; private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TIMEZONE HANDLING????? + private final InputRow delegateRow; - public DeltaInputRow(io.delta.kernel.data.Row row) + public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) { this.row = row; this.schema = row.getSchema(); @@ -50,35 +55,47 @@ public DeltaInputRow(io.delta.kernel.data.Row row) fieldNameToOrdinal.put(fieldNames.get(i), i); } fieldNameToOrdinal.defaultReturnValue(-1); + + Map theMap = new HashMap<>(); + for (String fieldName : fieldNames) { + theMap.put(fieldName, _getRaw(fieldName)); + } + delegateRow = MapInputRowParser.parse(inputRowSchema, theMap); + } @Override public List getDimensions() { - return schema.fieldNames(); + return delegateRow.getDimensions(); } @Override public long getTimestampFromEpoch() { - Object type = getRaw(ColumnHolder.TIME_COLUMN_NAME); - return type == null ? 0 : (long) type; + return delegateRow.getTimestampFromEpoch(); } @Override public DateTime getTimestamp() { - return DateTimes.utc(getTimestampFromEpoch()); + return delegateRow.getTimestamp(); } @Override public List getDimension(String dimension) { - return Rows.objectToStrings(getRaw(dimension)); + return delegateRow.getDimension(dimension); } @Nullable @Override public Object getRaw(String dimension) + { + return delegateRow.getRaw(dimension); + } + + @Nullable + public Object _getRaw(String dimension) { StructField field = schema.get(dimension); if (field == null) { @@ -87,6 +104,8 @@ public Object getRaw(String dimension) return null; } + + int ordinal = fieldNameToOrdinal.getInt(dimension); if (ordinal < 0) { return null; @@ -99,7 +118,7 @@ public Object getRaw(String dimension) @Override public Number getMetric(String metric) { - return Rows.objectToNumber(metric, getRaw(metric), true); + return delegateRow.getMetric(metric); } @Override diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index e3d0bb1a8dbf..8b6e256c1ed7 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -106,7 +106,8 @@ public InputSourceReader reader( scanState, Utils.toCloseableIterator(scanRowList.iterator()), Optional.empty() - ) + ), + inputRowSchema ); } catch (TableNotFoundException e) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 865c7b4a54d7..d785eeb7fb99 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -21,6 +21,7 @@ import io.delta.kernel.types.TimestampType; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.CloseableIterators; @@ -36,24 +37,27 @@ public class DeltaInputSourceReader implements InputSourceReader { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + private final InputRowSchema inputRowSchema; public DeltaInputSourceReader( - io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator - ) + io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + InputRowSchema inputRowSchema + ) { this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.inputRowSchema = inputRowSchema; } @Override public CloseableIterator read() { - return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator); + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); } @Override public CloseableIterator read(InputStats inputStats) throws IOException { - return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator); + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); } @Override @@ -88,11 +92,14 @@ private static class DeltaInputSourceIterator implements CloseableIterator filteredColumnarBatchCloseableIterator; private io.delta.kernel.utils.CloseableIterator currentBatch = null; + private final InputRowSchema inputRowSchema; - public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator + public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + InputRowSchema inputRowSchema ) { this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.inputRowSchema = inputRowSchema; } @Override @@ -124,7 +131,7 @@ public InputRow next() for (Object rowValue : rowValues) { System.out.println(rowValue); } - return new DeltaInputRow(dataRow); + return new DeltaInputRow(dataRow, inputRowSchema); } @Override From 935582c60188017fbda1c5153486dd67a0e2cec1 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 20 Dec 2023 04:12:29 -0500 Subject: [PATCH 14/65] Web console changes to show Delta Lake --- web-console/assets/deltalake.png | Bin 0 -> 4338 bytes .../ingestion-spec/ingestion-spec.tsx | 20 +++++++++++++++++- .../load-data-view.spec.tsx.snap | 14 ++++++++++++ .../views/load-data-view/load-data-view.tsx | 12 +++++++++++ 4 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 web-console/assets/deltalake.png diff --git a/web-console/assets/deltalake.png b/web-console/assets/deltalake.png new file mode 100644 index 0000000000000000000000000000000000000000..f7865be7e64a81074577ba5a0c39153417013cd0 GIT binary patch literal 4338 zcmVD0ISpA-`_ViHvj(rG&D3qK|&-XBrq^ACnqN>D=U+elQ1qXt*xz!ii*O* z!n3opyu7?CDJlS|)5yrkKR-W5M@Q7u)c5!I@bK_sVq)gz=6!vA@caB(Sy}4&{MPdK zrlzKNczDI^^=)l!PsHG>aynYb&8X8YePm$Z~GrZlc z{-I!R>|RLXlEv>NizTd5A#VxK~#90?Oh8)s!9+RcLfru z>_J*)sAWE(l8)9n_y7N`yDuKXEMk_R@!fmwRe82stLE**#M_n1yeS31R=Xd%MTb_LU7n-C`qS7 z|L{u^GfCM+6qDybEK7H%Q`$kV9Is%{hJ*6jLgLxcftMskVC;e{&yUU|Q5{By-S|g> zbZC6j61pZ#c?lE8_>fl-CF(?LMMcmk&>@HHT4f@c%22co0L<0VL5J*fNtrDcl1l5& z2Ro(xlQ4=IabFVK9c-0-DXBNnuxxFP9>ZxXwhl97mz5vl?JgRtxhF;iB0kVNNdE?h zCL|t>g(igf5J}>kNhO%}4hC}NBGLjSxR9=8{S}Twa0i(m^)<4dE4G zjFW5NRH2`evK&6Rko^Pf@sTzvQ%?w1W4r^)(#D|r$iMHB1D3MF++md@Bx{5yB_?rT zl^@tDA{>PxoYKoSP(bGBpq|wk$O@}zBo?5t`S5V0n@sRxh09A|`@y+g8 zR+2c3RYb7+fmYWyO~`EZ0d@|DP?CUKlu6BLd zRm8D`cgB@pFk~koomKRZokby#>{%0Q(7z#+9{Uskvho8frDOs81ZLs8Bv!IYdiJMg zC8av0*daTMy#Vr=mxDAK_a)B27LG^uPfET6h)Smtg>Tif_s&}bl1eXX$gXWzBg~v% z$sRu~?keI0L$!phhbI)WE4;C}n8~R$;6z(41x;j4{RIu#Wov{)=kDlpXtSNN=&UcP zvKKUDhg%zbnJwhamTZt_pgxv8s5LEMDFzO+1CV&m!rm-f7rs5hU?5b;l7zI8&lj4N z)$wR-Z#)Ao02vD6)j{RVVvoq6A|GM&25;IZBLTt6{{pM-s%|y${wQ)LwyPN z3gAM$KE%%8*Z3ubxgUamlQr`Bl==9N5kVVXM>~+5u0X4ZDNoxVgfTHsX54GZ zCM!_knVoXl3G6noQkZ8E!KH2L?HFU^3LwRbxvpPyqp4C}cNZ~J1=84Fk@fv3h_-rpH9@&*?JuO=w!mkNZ3Q>}vK{}&A z;5p;D&3^Uef@m|?gN4ToK6MopuClXcxty76;%OUfx!z>hk8aoqd+4~-;#0c#B#nQl z2#tRZbydh)B%M^O3l-|joKFb#wD;b8`awY4!{y)DJSF~V0=ni*?YI5KXkyDZ)50w{ znHRvn4VEH?{E<<&l+5M1KdYnp|BKU}52v3q=fnG;aAu#&)z@Nd$y*jskwSJK_Fz2o zTx3}|ghe8n(QGQ6>7;;5JGem4>IfS4YUU|~GbAgkwTe1(W=Jw1Odn;+e>V4J7|gsN zD1-DwW&q2bZB<2)KH@Pd262k%2*C?}A#u1Q z#+q8(Np|NXc|TU}tBBmAzk8{=PU?$M&otLd9!W+;X#5+D(rp}4y5**YE>Xa>Yg~ZF zzeVPEIYs4GEH_(b4fg$G=HIkDpHZ6Ayg6%GJ)(2^Ox-AxhnJQlrg=K{<(4nFi3!gu z>>K2lyt6sKke&Y30C4`4ox1C+Dw~)OWe4Zy-~Q14k6oD4DbV5Rot(TaQ+Zw?yX{!4 za<&6;lxg``_JXjSRuOHHJFRIv6mbhfAoIE;l4WKvl^(aT8~od$ao&mPJKp*v>G=bw zE8c86k6nVoUe07EKuR7DncGmV>{~wOHDt`HqLRm?*AHX`OV^u?SE6D`&>svxM{Zh_ zki&`+(|25%B=cnHx1zXW35DN)j8@NbSH<=IIx~X5e?6Dh>C+Ck{_UOkEcqrz%$86- zoL~v1KNt+B|Sq0wZSnkNz;YkKj`93jhcXVdiLm^6woBIn7 zSY+HlyC}Ix!S2ro)9HAz_?%7$ z^RV>z3vry*Ek&dw_IEJXg8!`lRH8(Q5+zEMkU^_O(7lUpx~nC@X4Wp>IoPoS>=y1YJ^+hmvso%{7ucXY8>Vz<#M9aecJ(-rkU7Le-lb-HrUSfM;UQ`aN``_dULH)nDe&PPQfnDx2UWd(BD-7|2}!O`WyE7U8fNoOPMj*H*3M46)6)6(p3Ii{Rb<2;gbYSh z<(G|+_}x9_s#XnVV!IG0>>jjlBYy60GB3c(Zy?@16ZU94NQj@(3#h7ItyOj5H}hjUSZ;|bemaK!~N@T zZ0kimycM_G6Ysp~g1Bl%d{juhCPtFK^@1~CY$Hpf$2;*n5@L_dOG<|Oy-u0XgotTG&%8?2EXdXGdDfmcM^iQ;QbT{jgr%u zk(GGmKPLX!4Me=X74Zvp3#;a7<-uS|t3yIaiFdfW(R4dK3{ht4waSgbBn4+QDe)^#muw-^IlH{?B%;KPwR7%3oZ6?@KSG>cNZpQjO&ItC z8$T+$akWYUGU8$`(@%(jKZ<*BnIvAVUM7gE+_h{+yxZdYcf!cc&<*2^z%YB;5NA;| z3Aw0Cd*q>~38aWJ;x}TBexg?tBzjtsxL%FqFTQi*0#j6hdkqrjS%uyXrZeODCICN4 zHsT^LQ}z3&r>7ndJ@vO_jwU1iAUL$i;m^MMZlc6>J;9t$XL|c05Lu@1L;%CZWus9h zQa0j`bumcvGh#DBMdoGvWd2p+8)|+CK(uEBna)#$xPH$vAexy4%(o9<_!+gEHzT~Y zT%=^cm5F#X+zf{tkfTX%?@Kb{qsP}m;zqOE7P)xaw`W0ImI)?_S6j`~)6<8Z0VFBy z(-1FVl2WZcMK*Nf#4r5Hjw)Ou|B}qn#8uuRC%dZnjyw)NqXpU7ZA-}OGZ$l=vGE}p z4ga2j9xcNyc$Yo)Hf6&)M_{dh)5wioe5k=iHAS4vAah6R;XF`*_?4# zrzUc=o<;r%u^B#1Zp{6#Yy zj&pvHnwZg&bsqh#a_cMnQ5=5tESx+YgwfK|V}r|RK1*tDQsQwZV(2d-q&`h8Jpsho zRT*(Uml2O1&NP>GW(0{GEm^nHdwyOP3H3U{gy`unDOpv_L`~l|p5ZF1lz2lA%tF_V zhWQ-0pQfosLCd@iQC`;d^-U%p<7^jt{Npv!YgL_HF6s)K z*ZfM)^d7R3Gbeuc#2M%fVyG8pQZxzi+tx{d_S^Gg#0Ne`HVLr8e&;^1p+(qe53Lir zfed1;DfU<^3iF?;F+m=yiIY9w@ZVa|IK0+l=WLO|e8E47L38&YE~Z~qZrcgbs Druid connects to raw data through{' '} @@ -892,6 +897,18 @@ export function getIoConfigFormFields(ingestionComboType: IngestionComboType): F inputSourceFilter, ]; + case 'index_parallel:delta': + return [ + inputSourceType, + { + name: 'inputSource.tablePath', + label: 'Delta table path', + type: 'string', + placeholder: '/path/to/deltaTable', + required: true, + }, + ]; + case 'index_parallel:hdfs': return [ inputSourceType, @@ -1082,6 +1099,7 @@ export function getIoConfigTuningFormFields( case 'index_parallel:s3': case 'index_parallel:azure': case 'index_parallel:google': + case 'index_parallel:delta': case 'index_parallel:hdfs': return [ { diff --git a/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap b/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap index f5956786afd9..cb96157e4745 100644 --- a/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap +++ b/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap @@ -202,6 +202,20 @@ exports[`LoadDataView matches snapshot batch 1`] = ` Google Cloud Storage

+ + Ingestion tile for index_parallel:delta +

+ Delta Lake +

+
); + case 'index_parallel:delta': + return ( + <> +

Load data from Delta Lake.

+

+ Data must be stored in the Delta Lake format. +

+ + ); + case 'index_parallel:druid': return ( <> @@ -1009,6 +1020,7 @@ export class LoadDataView extends React.PureComponent Date: Wed, 20 Dec 2023 05:02:29 -0500 Subject: [PATCH 15/65] Asset bug fix and tile load --- web-console/assets/{deltalake.png => delta.png} | Bin .../ingestion-spec/ingestion-spec.tsx | 2 +- .../__snapshots__/load-data-view.spec.tsx.snap | 14 +++++++------- 3 files changed, 8 insertions(+), 8 deletions(-) rename web-console/assets/{deltalake.png => delta.png} (100%) diff --git a/web-console/assets/deltalake.png b/web-console/assets/delta.png similarity index 100% rename from web-console/assets/deltalake.png rename to web-console/assets/delta.png diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index 3a9c3c8bed16..91955b3b9605 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -137,7 +137,7 @@ export function getIngestionComboType( switch (inputSource.type) { case 'local': case 'http': - case 'deltaLake': + case 'delta': case 'druid': case 'inline': case 's3': diff --git a/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap b/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap index cb96157e4745..b0a81a1800ac 100644 --- a/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap +++ b/web-console/src/views/load-data-view/__snapshots__/load-data-view.spec.tsx.snap @@ -208,13 +208,13 @@ exports[`LoadDataView matches snapshot batch 1`] = ` interactive={true} onClick={[Function]} > - Ingestion tile for index_parallel:delta -

- Delta Lake -

+ Ingestion tile for index_parallel:delta +

+ Delta Lake +

Date: Wed, 20 Dec 2023 05:34:25 -0500 Subject: [PATCH 16/65] Add missing pieces to input source info, etc. --- .../input-source/input-source.tsx | 25 ++++++++++++++++++- .../input-source-step/input-source-info.tsx | 3 +++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/web-console/src/druid-models/input-source/input-source.tsx b/web-console/src/druid-models/input-source/input-source.tsx index 4e0647320e50..4e40cd23a587 100644 --- a/web-console/src/druid-models/input-source/input-source.tsx +++ b/web-console/src/druid-models/input-source/input-source.tsx @@ -55,6 +55,9 @@ export interface InputSource { // inline data?: string; + // delta + tablePath?: string; + // hdfs paths?: string | string[]; @@ -111,6 +114,10 @@ export type InputSourceDesc = type: 'hdfs'; paths?: string | string[]; } + | { + type: 'delta'; + tablePath?: string; + } | { type: 'sql'; database: any; @@ -158,6 +165,12 @@ export function issueWithInputSource(inputSource: InputSource | undefined): stri } return; + case 'delta': + if (!inputSource.tablePath) { + return 'must have tablePath'; + } + return; + case 'hdfs': if (!inputSource.paths) { return 'must have paths'; @@ -169,7 +182,7 @@ export function issueWithInputSource(inputSource: InputSource | undefined): stri } } -const KNOWN_TYPES = ['inline', 'druid', 'http', 'local', 's3', 'azure', 'google', 'hdfs', 'sql']; +const KNOWN_TYPES = ['inline', 'druid', 'http', 'local', 's3', 'azure', 'delta', 'google', 'hdfs', 'sql']; export const INPUT_SOURCE_FIELDS: Field[] = [ // inline @@ -574,6 +587,16 @@ export const INPUT_SOURCE_FIELDS: Field[] = [ required: true, }, + // delta lake + { + name: 'tablePath', + label: 'Delta table path', + type: 'string', + placeholder: '/path/to/deltaTable', + defined: typeIsKnown(KNOWN_TYPES, 'delta'), + required: true, + }, + // sql { name: 'database.type', diff --git a/web-console/src/views/workbench-view/input-source-step/input-source-info.tsx b/web-console/src/views/workbench-view/input-source-step/input-source-info.tsx index 2a7e2361d5ea..c5dac4c12f9a 100644 --- a/web-console/src/views/workbench-view/input-source-step/input-source-info.tsx +++ b/web-console/src/views/workbench-view/input-source-step/input-source-info.tsx @@ -69,6 +69,9 @@ export const InputSourceInfo = React.memo(function InputSourceInfo(props: InputS case 'google': return

Load text based, avro, orc, or parquet data from the Google Blobstore.

; + case 'delta': + return

Load data from Delta Lake.

; + case 'hdfs': return

Load text based, avro, orc, or parquet data from HDFS.

; From a2502d246fe51822a05b01c234e7626c8ab541ef Mon Sep 17 00:00:00 2001 From: abhishekagarwal87 <1477457+abhishekagarwal87@users.noreply.github.com> Date: Wed, 20 Dec 2023 20:44:56 +0530 Subject: [PATCH 17/65] fix stuff --- .../druid/data/input/impl/delta/DeltaInputSource.java | 6 ++++++ .../data/input/impl/delta/DeltaInputSourceReader.java | 8 -------- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java index 04022afba2d1..50e9b159b086 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java @@ -31,6 +31,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -123,6 +124,11 @@ public InputSourceReader reader( public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException { + if (null != deltaSplit) { + // can't split a split + return Collections.singletonList(new InputSplit<>(deltaSplit)).stream(); + } + log.info("CREATE SPLITS Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); TableClient tableClient = DefaultTableClient.create(new Configuration()); final Snapshot latestSnapshot; diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java index 225683456efe..446eb979c8e8 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java @@ -125,16 +125,8 @@ public InputRow next() } Row dataRow = currentBatch.next(); - int numCols = dataRow.getSchema().length(); - - Object[] rowValues = IntStream.range(0, numCols) - .mapToObj(colOrdinal -> getValue(dataRow, colOrdinal)) - .toArray(); // TODO: construct schema? remove this after debugging - for (Object rowValue : rowValues) { - log.info("RowValue[%s]", rowValue); - } return new DeltaInputRow(dataRow, inputRowSchema); } From 73649b2761168b2f4bad78c2030816eb8d925985 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 20 Dec 2023 17:36:11 -0500 Subject: [PATCH 18/65] Use a different delta lake asset --- web-console/assets/delta.png | Bin 4338 -> 6527 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/web-console/assets/delta.png b/web-console/assets/delta.png index f7865be7e64a81074577ba5a0c39153417013cd0..db535506c7b9b89863ae59a33ef0404ca6dad11e 100644 GIT binary patch literal 6527 zcmZ`;3p~^7|5tvR$fT4LM$!rCkS>~O%%!kut|jHtMX62)vuRjGNkqq#l`b1YYbuv? zW#$$lG+J_<+oWwSTOo6c|MOkv{C?;6|G)mPmus*WS~3g}j!ZU&4w75*Sw`df+@29_~}w zEuewNrJ>F~1O)!Xl9!-n6d1i(zd~@EiLjy0Zg*4+gbp8Fk|{V%@ege0L14kXQSb#mb1s;o8Cuw zG@Brt&HoGF_F%*Y8{Gc}xG_CKGnHb_L!>hQfsKW=9@B#{6|V$S6ruFDg-CUMGdVGN zB4a2~ZspD$!g_0Y`cwa=tEo)qT`IX-YbRF7>>5z82Zaw><-c)Kd!R+a!c-DV&~N}T z2A7u=EGSy7kNhVtimBMPl@iNaWdFK$(Up9TVgrF6I|l!2u{Be16*Qp+!8RZz)wSyK znT-%pl`A-L#j3^u1`&h*YcT}ydh2avfGVy5yi@?Cm?fE}(u!-k!+l2;{xi538nkMB z83r`^s~sRz1&m@Yfo?`vjw6i`YeDBRqI&D6J8K_DY{NwTL(6*WubaNq|35OHr$_2i z;gCXd+0u`z?==1?g&ThqkBMu(MPk?i+^?sT>0gUtCLp84HG>%__0%#bH58}-*eGOr zqK;h4{#jxk)CQ;=%*+ZWF~;d1e5mjT=6EWcs7V55apQvo4j=)r4Hm{rLtWSmST2{ha~}`%3^h9Bm9ri-lgq$xgHEdg-1+*FdORL*}8MGAg{c<4MKETc4-(up#F~27B7S%wLz(LUKAmEcY z`?uFv#0t=d7y&3vlQanAf?ilyCsgoXW?TeS3lWoBo6h_B3{06=@)HqC3(=5M_$z}O z*kFm)Bly1oH-XL?OD2&qK)4tn(bH;olrmxx`8Ojd{y&15N+H^rMX@|P(bHN;QYfAQ z8Vl`?MW8njYd5}9n;y(Qehdx`{=Y1(sifs7V+62z%&*-4?gNzd3Y@H~jG)5h>p@kJ z^8be=ib^^tz<`YZWckxp#uA8OCMUMpP7JZc0KK5~Kl_RUYmQPBGaVvfnX$a1*T_S& zk~@U!c4AC0KBL4^1LXceP>MJ^SB_Xa;q-iq2_oUJz;YqO6|A#eLw#?)F{WAnaT!>C z^Gf=xlfovverdl43he)tCzpV!Q>By0wE@7VDP^#@JaJ zBNF@tBY5C;u*p~1%M7KMSkP0^!AGO`inlX+I2_QTnV{eE$KaaFys7X&XdjGhnz%6o zdoG88*@DyO!BIC<3a7ebAVyCN-bSEiws2}ZP|Z*GfFpI<0m#eP2dFfE>skXmMy{MS z1y&4kjzPn(_&1;aQD?1O+CMjY1GG|Q_r`!iJA6np4b1g~PLAT&PF&7*1r6LzJCBVQ z_-}UY7{G|s_8$O!+cEVg=$vW9aVF7Nq88KxG&et02)q;JaP7OH^l9~~JAVSR>_~VI zWf_LNV)IR}E3E8a8jBQr!aW_HSF1qRoe4nW`aRA6r zn)zxYgjIAM!s5_9P=M|EF2F*mw(G#6J-$`qhakVwN-A(ZIkp;jd*W*9J~1Hv+5;Q^ zYkt44Jb2$Hh=uXvdh1PirO7OavNW3t5+gsziCuc-Td|@oG`%~?n=^3;6>jmm&sK+%B76^olqM-c}0Q*6+oJr(tb56M38nA;>@_khTDnMjM`P< zAluedIkg-)wU56;rRn<1sf~D`G-a1By^~YRt_MO?y4h?7I`mk9eM+u+^b+W-8kN(a z0n|8IDydlEy?N7gq8cdlo*Zrbj2z8dj`l*1<^s@aO6{<6G#48i(DcI_m{p)QoLtT? zr@@jPny%)ypsVc(5b3%)xk8!pMl5;>c5z#%jR_(-s4^jt5-aFo7(hk!pY=dF=k%4{ z2vHG0ZHvNv4G-sI+ zvGH;TNFzACoFey@wh7{e=hqD`;;-vELRVj?S;8|pl{Se)UQ-+Nh>Q>RIiib&^>5=i zc%Y^p_G^M((zXt<+i=$tXaAEATOXIAx_ z-bon2hfZn_J%G#=dYz)9>G`$Q8Eju2E*%a3muK^Ozr4^PZ5%!ASJ4fnk0}N17&Emn zIT{YQzr`c851_5q;p#A}@TR}S0jsXnd65Oq6P;W^73`5F$H)W4*C1Xj?QnUtEsLsOHKCAr@?=UzPw{>uD z@rBG^wJ(m~8Of;d_JQV&6aAHLDCO={yiyf7ge6Dp+tmHu=p{dQ6a^o3b+Z1v3=2CE zq56Qh`6?lhqIgtuhl95TZ5_!1SVOpFG6l0b{buJb5iX_HQS45eoXvi4<_?Sqh z4Pll*~jU<9wHWpXMy|1{DB zm#Afahv3_yw^q4wrPn*52!=1;yW^yp#9J1EG zx8|>#qU3phJm0TV@8NShQScwwwn=2g%Tc_Cf>K1finbzB3m81+p>XJ`=OVzLJ2;y8 zW`*$464P=1UHgF+)*^Z=r4YQKE%2NE)D3Sx{Y*0X_79yM@#D4 zMtwh`8^>KC=id4jQgzFKY=${=_-^i1@l> zHNX*IX-4^+Y>Y^+XhZzsOjQ)qaAE)Z_rVm0OU-C$=bm)l6L6cpe9bW}dstfB@uVP< z@`P1X(m9IHSw1stsQ(5;zG`ODc^YNaORbsN%pz5AvHn)$*M5t%ZK+@T{p3rm{l85_ zSJ}kTc{g5Gm##V2)T$RBn7I;!{c3MCIbz*=JkMVcNawj9;GgK$i@zx_8=gc?DEqZ4 znqR~k5q(%h;&1q8!4ywE#Urp=&q?%5;^xL~INU;4!OBn@hN;1nag56)1uCg2we=QB zwk^}zOcNu%jXcBxC-PMUnYKq~&TaN>eL6e)ZukgjI%JxvPCWyi+`Ow>Ba_(n zGA%r)x^zToZ;MHHzCa@pg)o+4*X&0c)mvvKv19GUSImaMSxw978u0xXdi@o3F>@tb zsjO0k8hmYKctBxsmch^@(%$L2PZm+@MUJ&a`k&rr?kr&R_%9`vVQxL2Hb0?xX8)nV zUOCwAhRRGrF%Jv+_IK+Ek1e?{@o3w<-c=!zu4 zVhPBxp1y@GL)Osujo=flb+>?C{DQlS=BC@CXqD=HILV|8YpeJHnZfS zyIZeGx9TeGx_??b1?q0zvv-bH;rU~-xh7y$q>CM;T z9LH|qL|mOD-HSm+x=1q3RO>-7#gOr?>Y}iuPg(u9Zt%KaoN{JU_`ArTcKuNOPWNC_ZDPlFmXG%veWkxNkEt{Pt)9I* z>U8W!#V48J>}%^vSZ_gwpIedyQbOnBK0%wJynS<2e%7eKfHh>*o^*?8M6Fa7q4s~ z0Em}1d{Sef_XU~ofLO3c{lxiM^$G2l(s`?nfkOqY8WX0e`EIl8f5G;SjO!VGEm%s| zn|!yKTDC7Xv2iMzyz|3UV&2>#wWcYhllt8Bv9ig$Lo3Ff!`P)w`b?*Q$0yd^Cy7{Ucom#~3TS(i}GzSm^4M!V4+G zk$yb-Ui!?2Zig+GD!J5|H_Vjjzd#2|DG{RKw38RQT)QZokh!I*)8Bo1Ck?@mfMgZF zfN~RPqm&G-`_vZFIP)WNbQ zp4%MS7~njdEjjbN)^F^c%Jd^@)UgX$HgBXBSldyt3&;US#rW&X?p++I6n;ny_qYNZK{V{SNQ5ZR_cc zq|W@eq%6I=Xh~vI3#HsHQhg5p!QcyH;%b1M1@W9)RAm8UYBD6;b>WrD2)?8I@?y6h zYw*jAlnQr?5=j*ljKOdFxH#S-EQ{Fq`dhyn8yze>Qa)cNJHC4ATL$8C2*tC9lpw46 zFu!?nT>-~b`b25YWO|bPX7z zQ3IdzIi68_-WJn&qxhgwiaEb2ht9ht8%bH1#XJHhpSZ(7>m1PKW{RpcvunU6b?kTm z=XuuvRpxa|M?@tJmO2hrLm1xm*Fc&d6a&g0t3ppU+RS$l9H7|~`P2sXn4~|g+PAgj z)8bQEKOqv3H}92 z#s#s$fBO)Z|GY%9s-`n&@0S<2bz(A&Yc{lVMLVgB9sGrxh& z5ZbOP5N4T)rm<-R^Wn?s(X;i%Y2kKF`aOfbv04*TwMVah>{CJqh{JHh7msXMaIY)Q z2|9D3Ds}XuLY~$pR|l@$u=xECM`nYS$4i@?ivyk*1PqgH7FosTYUxVE*)h4oaw^;^ zzU}+vZ!REK$#9%XsEe9Ub8+Ar3@Zg_H&{vD&73^i`@@@tvw{_}YUdtx`*Ahye!t%9 zZ6R+4$;=q-b~Z{}GkL3FW!aQWnsTee$8Vaaz$lP%GvZrs8n?NS%= zp@3+~9yV=Y{mXzmlXvF}=5ba|q#GIU)u$7~Rc-pEg`~Rjw`4DwtCf?zsJB`;{)y3Gm7-6yf{n$R2{9L&w-V-u zYl6$hTAyu=&bRJabOSMQW0ho@DLtFZR#uxysFN+V4c$MB{%u99%`5oSs%Rhkp2Fzn z_?TIjJ=L%a2}R|?>r;%TKiAo`P45fjCFfnXktIysEBEp0#Cxt2gq2a>#v$%6@dNME zcKTZ+G7u>`{voA&^u>oavoF5ib$8SKbUO7Vz zQ`7rqpOLl@ePs!yAq}}jEBv#T%wW*JHM1xQxbbWN$eaXzES|le7CsWd&%nFY0X_?t zabL2MYY6j$A+2@%e^CbGiwfA7#?i0@eZd3YmO=+^xyKX;EOg?QuO%wd{NUfLj0oLT|2kBr7>D%mz0jjWch>FCCyo-`*9Z6`o5SkLO+jf5{#r*l?%J z!Cavimj#~R1QnHUlAua*jyKV4=SNEKC7~vBN2A^OHGHk%c1xi-II5frMkzJ~Ra(yR z<8<2td~j+NhI0d|eOqq$cj-sAf{1osp}rPL)pzFgAB#Ox#OtbcNeXc7VRw3vhI^$__W$YPed|2E2dX?k0?ev-K$&;f~@t z*Cww{IZb2{SjzbHf%sP4?GSR9hkKBGhxG=qY7YM7gfZ4Pv0JP*^ z%RLkwOHafq?)`cUp1@8cB%rEwGtGyyjJ`Z=umhpX2@yW=0H;RxX~Q+Ez7yS)T7@jL zFEMG~e@B(vsRDO|qBchfzC64DjD^z?jBpE>qhC<`67?;xysnN#?nZjm_sfsTETJ9dJBX?87Vp^q3i>VH+k91Gf$!7**atpM> zOkmHR;|42cee~_&%S#GADEMfF9eI)~)jVead(+$gei(DE{hDqae0FWvTGZ$kI8Ar* z{>S_$itkSAZ-et!$qWQBoM@}Q74S`IvY_=Bm61<(eGb^V5IPh+$71TVqh!5n7W#z? ec{@556kKy*1?PjeL;osKaM^`(%(p*s`F{XiGtH#{ literal 4338 zcmVD0ISpA-`_ViHvj(rG&D3qK|&-XBrq^ACnqN>D=U+elQ1qXt*xz!ii*O* z!n3opyu7?CDJlS|)5yrkKR-W5M@Q7u)c5!I@bK_sVq)gz=6!vA@caB(Sy}4&{MPdK zrlzKNczDI^^=)l!PsHG>aynYb&8X8YePm$Z~GrZlc z{-I!R>|RLXlEv>NizTd5A#VxK~#90?Oh8)s!9+RcLfru z>_J*)sAWE(l8)9n_y7N`yDuKXEMk_R@!fmwRe82stLE**#M_n1yeS31R=Xd%MTb_LU7n-C`qS7 z|L{u^GfCM+6qDybEK7H%Q`$kV9Is%{hJ*6jLgLxcftMskVC;e{&yUU|Q5{By-S|g> zbZC6j61pZ#c?lE8_>fl-CF(?LMMcmk&>@HHT4f@c%22co0L<0VL5J*fNtrDcl1l5& z2Ro(xlQ4=IabFVK9c-0-DXBNnuxxFP9>ZxXwhl97mz5vl?JgRtxhF;iB0kVNNdE?h zCL|t>g(igf5J}>kNhO%}4hC}NBGLjSxR9=8{S}Twa0i(m^)<4dE4G zjFW5NRH2`evK&6Rko^Pf@sTzvQ%?w1W4r^)(#D|r$iMHB1D3MF++md@Bx{5yB_?rT zl^@tDA{>PxoYKoSP(bGBpq|wk$O@}zBo?5t`S5V0n@sRxh09A|`@y+g8 zR+2c3RYb7+fmYWyO~`EZ0d@|DP?CUKlu6BLd zRm8D`cgB@pFk~koomKRZokby#>{%0Q(7z#+9{Uskvho8frDOs81ZLs8Bv!IYdiJMg zC8av0*daTMy#Vr=mxDAK_a)B27LG^uPfET6h)Smtg>Tif_s&}bl1eXX$gXWzBg~v% z$sRu~?keI0L$!phhbI)WE4;C}n8~R$;6z(41x;j4{RIu#Wov{)=kDlpXtSNN=&UcP zvKKUDhg%zbnJwhamTZt_pgxv8s5LEMDFzO+1CV&m!rm-f7rs5hU?5b;l7zI8&lj4N z)$wR-Z#)Ao02vD6)j{RVVvoq6A|GM&25;IZBLTt6{{pM-s%|y${wQ)LwyPN z3gAM$KE%%8*Z3ubxgUamlQr`Bl==9N5kVVXM>~+5u0X4ZDNoxVgfTHsX54GZ zCM!_knVoXl3G6noQkZ8E!KH2L?HFU^3LwRbxvpPyqp4C}cNZ~J1=84Fk@fv3h_-rpH9@&*?JuO=w!mkNZ3Q>}vK{}&A z;5p;D&3^Uef@m|?gN4ToK6MopuClXcxty76;%OUfx!z>hk8aoqd+4~-;#0c#B#nQl z2#tRZbydh)B%M^O3l-|joKFb#wD;b8`awY4!{y)DJSF~V0=ni*?YI5KXkyDZ)50w{ znHRvn4VEH?{E<<&l+5M1KdYnp|BKU}52v3q=fnG;aAu#&)z@Nd$y*jskwSJK_Fz2o zTx3}|ghe8n(QGQ6>7;;5JGem4>IfS4YUU|~GbAgkwTe1(W=Jw1Odn;+e>V4J7|gsN zD1-DwW&q2bZB<2)KH@Pd262k%2*C?}A#u1Q z#+q8(Np|NXc|TU}tBBmAzk8{=PU?$M&otLd9!W+;X#5+D(rp}4y5**YE>Xa>Yg~ZF zzeVPEIYs4GEH_(b4fg$G=HIkDpHZ6Ayg6%GJ)(2^Ox-AxhnJQlrg=K{<(4nFi3!gu z>>K2lyt6sKke&Y30C4`4ox1C+Dw~)OWe4Zy-~Q14k6oD4DbV5Rot(TaQ+Zw?yX{!4 za<&6;lxg``_JXjSRuOHHJFRIv6mbhfAoIE;l4WKvl^(aT8~od$ao&mPJKp*v>G=bw zE8c86k6nVoUe07EKuR7DncGmV>{~wOHDt`HqLRm?*AHX`OV^u?SE6D`&>svxM{Zh_ zki&`+(|25%B=cnHx1zXW35DN)j8@NbSH<=IIx~X5e?6Dh>C+Ck{_UOkEcqrz%$86- zoL~v1KNt+B|Sq0wZSnkNz;YkKj`93jhcXVdiLm^6woBIn7 zSY+HlyC}Ix!S2ro)9HAz_?%7$ z^RV>z3vry*Ek&dw_IEJXg8!`lRH8(Q5+zEMkU^_O(7lUpx~nC@X4Wp>IoPoS>=y1YJ^+hmvso%{7ucXY8>Vz<#M9aecJ(-rkU7Le-lb-HrUSfM;UQ`aN``_dULH)nDe&PPQfnDx2UWd(BD-7|2}!O`WyE7U8fNoOPMj*H*3M46)6)6(p3Ii{Rb<2;gbYSh z<(G|+_}x9_s#XnVV!IG0>>jjlBYy60GB3c(Zy?@16ZU94NQj@(3#h7ItyOj5H}hjUSZ;|bemaK!~N@T zZ0kimycM_G6Ysp~g1Bl%d{juhCPtFK^@1~CY$Hpf$2;*n5@L_dOG<|Oy-u0XgotTG&%8?2EXdXGdDfmcM^iQ;QbT{jgr%u zk(GGmKPLX!4Me=X74Zvp3#;a7<-uS|t3yIaiFdfW(R4dK3{ht4waSgbBn4+QDe)^#muw-^IlH{?B%;KPwR7%3oZ6?@KSG>cNZpQjO&ItC z8$T+$akWYUGU8$`(@%(jKZ<*BnIvAVUM7gE+_h{+yxZdYcf!cc&<*2^z%YB;5NA;| z3Aw0Cd*q>~38aWJ;x}TBexg?tBzjtsxL%FqFTQi*0#j6hdkqrjS%uyXrZeODCICN4 zHsT^LQ}z3&r>7ndJ@vO_jwU1iAUL$i;m^MMZlc6>J;9t$XL|c05Lu@1L;%CZWus9h zQa0j`bumcvGh#DBMdoGvWd2p+8)|+CK(uEBna)#$xPH$vAexy4%(o9<_!+gEHzT~Y zT%=^cm5F#X+zf{tkfTX%?@Kb{qsP}m;zqOE7P)xaw`W0ImI)?_S6j`~)6<8Z0VFBy z(-1FVl2WZcMK*Nf#4r5Hjw)Ou|B}qn#8uuRC%dZnjyw)NqXpU7ZA-}OGZ$l=vGE}p z4ga2j9xcNyc$Yo)Hf6&)M_{dh)5wioe5k=iHAS4vAah6R;XF`*_?4# zrzUc=o<;r%u^B#1Zp{6#Yy zj&pvHnwZg&bsqh#a_cMnQ5=5tESx+YgwfK|V}r|RK1*tDQsQwZV(2d-q&`h8Jpsho zRT*(Uml2O1&NP>GW(0{GEm^nHdwyOP3H3U{gy`unDOpv_L`~l|p5ZF1lz2lA%tF_V zhWQ-0pQfosLCd@iQC`;d^-U%p<7^jt{Npv!YgL_HF6s)K z*ZfM)^d7R3Gbeuc#2M%fVyG8pQZxzi+tx{d_S^Gg#0Ne`HVLr8e&;^1p+(qe53Lir zfed1;DfU<^3iF?;F+m=yiIY9w@ZVa|IK0+l=WLO|e8E47L38&YE~Z~qZrcgbs Date: Wed, 3 Jan 2024 09:21:02 -0800 Subject: [PATCH 19/65] Delta lake extension dependencies --- distribution/pom.xml | 2 + .../druid-deltalake-extensions/pom.xml | 451 ++++++++++++++++++ pom.xml | 1 + 3 files changed, 454 insertions(+) create mode 100644 extensions-contrib/druid-deltalake-extensions/pom.xml diff --git a/distribution/pom.xml b/distribution/pom.xml index ab8e082c9ab8..8636bad0bd42 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -450,6 +450,8 @@ org.apache.druid.extensions.contrib:opentelemetry-emitter -c org.apache.druid.extensions:druid-iceberg-extensions + -c + org.apache.druid.extensions:druid-delatalake-extensions diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml new file mode 100644 index 000000000000..b7035985f7dd --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -0,0 +1,451 @@ + + + + + org.apache.druid.extensions + druid-deltalake-extensions + druid-deltalake-extensions + Delta Lake connector for Druid + + + druid + org.apache.druid + 29.0.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + + + 3.0.0 + + + + + io.delta + delta-kernel-api + ${delta-kernel.version} + + + + io.delta + delta-kernel-defaults + ${delta-kernel.version} + + + + org.apache.hadoop + hadoop-client-runtime + ${hadoop.compile.version} + runtime + + + + org.apache.hadoop + hadoop-client-api + ${hadoop.compile.version} + provided + + + + org.apache.hadoop + hadoop-common + ${hadoop.compile.version} + compile + + + io.netty + netty-buffer + + + commons-cli + commons-cli + + + log4j + log4j + + + commons-codec + commons-codec + + + commons-logging + commons-logging + + + commons-io + commons-io + + + commons-lang + commons-lang + + + commons-net + commons-net + + + commons-collections + commons-collections + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.zookeeper + zookeeper + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + javax.ws.rs + jsr311-api + + + com.google.code.findbugs + jsr305 + + + org.mortbay.jetty + jetty-util + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-webapp + + + com.google.protobuf + protobuf-java + + + com.sun.jersey + jersey-core + + + org.apache.curator + curator-client + + + org.apache.curator + curator-recipes + + + org.apache.commons + commons-math3 + + + com.google.guava + guava + + + org.apache.avro + avro + + + net.java.dev.jets3t + jets3t + + + com.sun.jersey + jersey-json + + + com.github.pjfanning + jersey-json + + + com.jcraft + jsch + + + org.mortbay.jetty + jetty + + + com.sun.jersey + jersey-server + + + + commons-beanutils + commons-beanutils-core + + + commons-beanutils + commons-beanutils + + + ch.qos.reload4j + reload4j + + + com.sun.jersey + jersey-servlet + + + javax.servlet.jsp + jsp-api + + + org.slf4j + slf4j-reload4j + + + com.google.re2j + re2j + + + com.google.code.gson + gson + + + com.nimbusds + nimbus-jose-jwt + + + com.github.stephenc.jcip + jcip-annotations + + + org.apache.curator + curator-framework + + + io.dropwizard.metrics + metrics-core + + + dnsjava + dnsjava + + + org.xerial.snappy + snappy-java + + + + + + commons-cli + commons-cli + 1.5.0 + + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.apache.hadoop + hadoop-hdfs-client + runtime + + + org.jetbrains.kotlin + kotlin-stdlib + + + org.jetbrains.kotlin + kotlin-stdlib-common + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + runtime + ${hadoop.compile.version} + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-yarn-common + + + aopalliance + aopalliance + + + org.apache.commons + commons-compress + + + com.google.guava + guava + + + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + javax.inject + javax + + + io.netty + netty + + + slf4j-log4j12 + org.slf4j + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-reload4j + + + protobuf-java + com.google.protobuf + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.compile.version} + runtime + + + com.amazonaws + aws-java-sdk-bundle + + + + + + com.google.code.findbugs + jsr305 + provided + + + com.google.guava + guava + provided + + + com.google.inject + guice + provided + + + joda-time + joda-time + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + org.slf4j + slf4j-api + provided + + + + org.apache.parquet + parquet-column + 1.13.1 + test + + + + junit + junit + test + + + nl.jqno.equalsverifier + equalsverifier + test + + + + + + + org.owasp + dependency-check-maven + + true + + + + + + diff --git a/pom.xml b/pom.xml index 81cb00bb0cf5..94e2d6b3692e 100644 --- a/pom.xml +++ b/pom.xml @@ -229,6 +229,7 @@ extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions extensions-contrib/druid-iceberg-extensions + extensions-contrib/druid-deltalake-extensions distribution From b6fa446d8e8c5ae4e7874a1dea4ee009b52e200e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 3 Jan 2024 12:24:57 -0800 Subject: [PATCH 20/65] Cleanup --- .../apache/druid/data/input/InputSource.java | 3 +- .../data/input/impl/delta/DeltaInputRow.java | 181 ----------------- .../input/impl/delta/DeltaInputSource.java | 186 ----------------- .../impl/delta/DeltaInputSourceReader.java | 189 ------------------ .../data/input/impl/delta/DeltaSplit.java | 42 ---- .../druid/data/input/impl/delta/RowSerde.java | 130 ------------ 6 files changed, 1 insertion(+), 730 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java diff --git a/processing/src/main/java/org/apache/druid/data/input/InputSource.java b/processing/src/main/java/org/apache/druid/data/input/InputSource.java index 912dbbfd54e3..208dc4dd49cf 100644 --- a/processing/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/InputSource.java @@ -57,8 +57,7 @@ @Type(name = LocalInputSource.TYPE_KEY, value = LocalInputSource.class), @Type(name = HttpInputSource.TYPE_KEY, value = HttpInputSource.class), @Type(name = InlineInputSource.TYPE_KEY, value = InlineInputSource.class), - @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class), - @Type(name = DeltaInputSource.TYPE_KEY, value = DeltaInputSource.class) + @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class) }) public interface InputSource { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java deleted file mode 100644 index 4867b1ea52f7..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputRow.java +++ /dev/null @@ -1,181 +0,0 @@ -package org.apache.druid.data.input.impl.delta; - -import io.delta.kernel.types.ArrayType; -import io.delta.kernel.types.BinaryType; -import io.delta.kernel.types.BooleanType; -import io.delta.kernel.types.ByteType; -import io.delta.kernel.types.DataType; -import io.delta.kernel.types.DateType; -import io.delta.kernel.types.DecimalType; -import io.delta.kernel.types.DoubleType; -import io.delta.kernel.types.FloatType; -import io.delta.kernel.types.IntegerType; -import io.delta.kernel.types.LongType; -import io.delta.kernel.types.MapType; -import io.delta.kernel.types.ShortType; -import io.delta.kernel.types.StringType; -import io.delta.kernel.types.StructField; -import io.delta.kernel.types.StructType; -import io.delta.kernel.types.TimestampType; -import it.unimi.dsi.fastutil.objects.Object2IntMap; -import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.Rows; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.segment.column.ColumnHolder; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class DeltaInputRow implements InputRow -{ - private final io.delta.kernel.data.Row row; - private final StructType schema; - private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); - private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TIMEZONE HANDLING????? - private final InputRow delegateRow; - - public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) - { - this.row = row; - this.schema = row.getSchema(); - List fieldNames = this.schema.fieldNames(); - for (int i = 0; i < fieldNames.size(); ++i) { - fieldNameToOrdinal.put(fieldNames.get(i), i); - } - fieldNameToOrdinal.defaultReturnValue(-1); - - Map theMap = new HashMap<>(); - for (String fieldName : fieldNames) { - theMap.put(fieldName, _getRaw(fieldName)); - } - delegateRow = MapInputRowParser.parse(inputRowSchema, theMap); - - } - @Override - public List getDimensions() - { - return delegateRow.getDimensions(); - } - - @Override - public long getTimestampFromEpoch() - { - return delegateRow.getTimestampFromEpoch(); - } - - @Override - public DateTime getTimestamp() - { - return delegateRow.getTimestamp(); - } - - @Override - public List getDimension(String dimension) - { - return delegateRow.getDimension(dimension); - } - - @Nullable - @Override - public Object getRaw(String dimension) - { - return delegateRow.getRaw(dimension); - } - - @Nullable - public Object _getRaw(String dimension) - { - StructField field = schema.get(dimension); - if (field == null) { - return null; - } else if (field.isMetadataColumn()) { - return null; - } - - - - int ordinal = fieldNameToOrdinal.getInt(dimension); - if (ordinal < 0) { - return null; - } - return getValue(field.getDataType(), row, ordinal); - - } - - @Nullable - @Override - public Number getMetric(String metric) - { - return delegateRow.getMetric(metric); - } - - @Override - public int compareTo(Row o) - { - return this.getTimestamp().compareTo(o.getTimestamp()); - } - - @Nullable - private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataRow, int columnOrdinal) { - if (dataRow.isNullAt(columnOrdinal)) { - return null; - } else if (dataType instanceof BooleanType) { - return dataRow.getBoolean(columnOrdinal); - } else if (dataType instanceof ByteType) { - return dataRow.getByte(columnOrdinal); - } else if (dataType instanceof ShortType) { - return dataRow.getShort(columnOrdinal); - } else if (dataType instanceof IntegerType) { - return dataRow.getInt(columnOrdinal); - } else if (dataType instanceof DateType) { - // DateType data is stored internally as the number of days since 1970-01-01 - int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); - return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); - } else if (dataType instanceof LongType) { - return dataRow.getLong(columnOrdinal); - } else if (dataType instanceof TimestampType) { - // TimestampType data is stored internally as the number of microseconds since epoch - long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); - LocalDateTime dateTime = LocalDateTime.ofEpochSecond( - microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, - (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC); - return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); - } else if (dataType instanceof FloatType) { - return dataRow.getFloat(columnOrdinal); - } else if (dataType instanceof DoubleType) { - return dataRow.getDouble(columnOrdinal); - } else if (dataType instanceof StringType) { - return dataRow.getString(columnOrdinal); - } else if (dataType instanceof BinaryType) { - return new String(dataRow.getBinary(columnOrdinal)); - } else if (dataType instanceof DecimalType) { - return dataRow.getDecimal(columnOrdinal).longValue(); - } else if (dataType instanceof StructType) { - return "TODO: struct value"; - } else if (dataType instanceof ArrayType) { - return "TODO: list value"; - } else if (dataType instanceof MapType) { - return "TODO: map value"; - } else { - throw new UnsupportedOperationException("unsupported data type: " + dataType); - } - } - - public Map getRawRowAsMap() - { - return RowSerde.convertRowToJsonObject(row); - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java deleted file mode 100644 index 50e9b159b086..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSource.java +++ /dev/null @@ -1,186 +0,0 @@ -package org.apache.druid.data.input.impl.delta; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterators; -import com.google.common.primitives.Ints; -import io.delta.kernel.Scan; -import io.delta.kernel.Snapshot; -import io.delta.kernel.Table; -import io.delta.kernel.TableNotFoundException; -import io.delta.kernel.client.TableClient; -import io.delta.kernel.data.FilteredColumnarBatch; -import io.delta.kernel.data.Row; -import io.delta.kernel.defaults.client.DefaultTableClient; -import io.delta.kernel.internal.util.Utils; -import io.delta.kernel.utils.CloseableIterator; -import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.utils.Streams; -import org.apache.hadoop.conf.Configuration; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; - - -public class DeltaInputSource implements SplittableInputSource -{ - public static final String TYPE_KEY = "delta"; - - @JsonProperty - private final String tablePath; - - @JsonProperty - @Nullable - private final DeltaSplit deltaSplit; - - private static final Logger log = new Logger(DeltaInputSource.class); - - @JsonCreator - public DeltaInputSource( - @JsonProperty("tablePath") String tablePath, - @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit - ) - { - log.info("CONST Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); - this.tablePath = Preconditions.checkNotNull(tablePath, "tablePath cannot be null"); - this.deltaSplit = deltaSplit; - } - - @Override - public boolean needsFormat() - { - // Only support Parquet - return false; - } - - @Override - public InputSourceReader reader( - InputRowSchema inputRowSchema, - @Nullable InputFormat inputFormat, - File temporaryDirectory - ) - { - log.info("READER Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); - Configuration hadoopConf = new Configuration(); - TableClient tableClient = DefaultTableClient.create(hadoopConf); - try { - final Row scanState; - final List scanRowList; - - if (deltaSplit != null) { - scanState = deserialize(tableClient, deltaSplit.getStateRow()); - scanRowList = deltaSplit.getFile().stream().map(row -> deserialize(tableClient, row)).collect(Collectors.toList()); - } else { - Table table = Table.forPath(tableClient, tablePath); - Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); - - Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); - scanState = scan.getScanState(tableClient); - CloseableIterator scanFiles = scan.getScanFiles(tableClient); - scanRowList = new ArrayList<>(); - - while (scanFiles.hasNext()) { - FilteredColumnarBatch scanFileBatch = scanFiles.next(); - CloseableIterator scanFileRows = scanFileBatch.getRows(); - scanFileRows.forEachRemaining(scanRowList::add); - } - } - return new DeltaInputSourceReader( - Scan.readData( - tableClient, - scanState, - Utils.toCloseableIterator(scanRowList.iterator()), - Optional.empty() - ), - inputRowSchema - ); - } - catch (TableNotFoundException e) { - throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) - throws IOException - { - if (null != deltaSplit) { - // can't split a split - return Collections.singletonList(new InputSplit<>(deltaSplit)).stream(); - } - - log.info("CREATE SPLITS Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); - TableClient tableClient = DefaultTableClient.create(new Configuration()); - final Snapshot latestSnapshot; - final Table table; - try { - table = Table.forPath(tableClient, tablePath); - latestSnapshot = table.getLatestSnapshot(tableClient); - } - catch (TableNotFoundException e) { - throw new RuntimeException(e); - } - Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); - // scan files iterator for the current snapshot - CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient); - - Row scanState = scan.getScanState(tableClient); - String scanStateStr = RowSerde.serializeRowToJson(scanState); - - Iterator deltaSplitIterator = Iterators.transform( - scanFilesIterator, - scanFile -> { - CloseableIterator rows = scanFile.getRows(); - List fileRows = new ArrayList<>(); - while (rows.hasNext()) { - fileRows.add(RowSerde.serializeRowToJson(rows.next())); - } - return new DeltaSplit(scanStateStr, fileRows); - } - ); - - // TODO: account for the split spec as well -- getSplitHintSpecOrDefault(splitHintSpec).split() - return Streams.sequentialStreamFrom(deltaSplitIterator).map(InputSplit::new); - } - - @Override - public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException - { - return Ints.checkedCast(createSplits(inputFormat, splitHintSpec).count()); - } - - @Override - public InputSource withSplit(InputSplit split) - { - log.info("WITH SPLIT Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); - return new DeltaInputSource( - tablePath, - split.get() - ); - } - - private Row deserialize(TableClient myTableClient, String row) - { - return RowSerde.deserializeRowFromJson(myTableClient, row); - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java deleted file mode 100644 index 446eb979c8e8..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaInputSourceReader.java +++ /dev/null @@ -1,189 +0,0 @@ -package org.apache.druid.data.input.impl.delta; - -import com.google.common.collect.Iterators; -import io.delta.kernel.data.FilteredColumnarBatch; -import io.delta.kernel.data.Row; -import io.delta.kernel.types.ArrayType; -import io.delta.kernel.types.BinaryType; -import io.delta.kernel.types.BooleanType; -import io.delta.kernel.types.ByteType; -import io.delta.kernel.types.DataType; -import io.delta.kernel.types.DateType; -import io.delta.kernel.types.DecimalType; -import io.delta.kernel.types.DoubleType; -import io.delta.kernel.types.FloatType; -import io.delta.kernel.types.IntegerType; -import io.delta.kernel.types.LongType; -import io.delta.kernel.types.MapType; -import io.delta.kernel.types.ShortType; -import io.delta.kernel.types.StringType; -import io.delta.kernel.types.StructType; -import io.delta.kernel.types.TimestampType; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.InputStats; -import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.CloseableIterator; - -import java.io.IOException; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.Collections; -import java.util.NoSuchElementException; -import java.util.stream.IntStream; - -public class DeltaInputSourceReader implements InputSourceReader -{ - private static final Logger log = new Logger(DeltaInputSourceReader.class); - - private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; - private final InputRowSchema inputRowSchema; - - public DeltaInputSourceReader( - io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, - InputRowSchema inputRowSchema - ) - { - this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; - this.inputRowSchema = inputRowSchema; - } - - @Override - public CloseableIterator read() - { - return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); - } - - @Override - public CloseableIterator read(InputStats inputStats) throws IOException - { - return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); - } - - @Override - public CloseableIterator sample() throws IOException - { - CloseableIterator inner = read(); - return new CloseableIterator() - { - @Override - public void close() throws IOException - { - inner.close(); - } - - @Override - public boolean hasNext() - { - return inner.hasNext(); - } - - @Override - public InputRowListPlusRawValues next() - { - DeltaInputRow deltaInputRow = (DeltaInputRow) inner.next(); - return InputRowListPlusRawValues.of(deltaInputRow, deltaInputRow.getRawRowAsMap()); - } - }; - } - - private static class DeltaInputSourceIterator implements CloseableIterator - { - private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; - - private io.delta.kernel.utils.CloseableIterator currentBatch = null; - private final InputRowSchema inputRowSchema; - - public DeltaInputSourceIterator(io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, - InputRowSchema inputRowSchema - ) - { - this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; - this.inputRowSchema = inputRowSchema; - } - - @Override - public boolean hasNext() { - while (currentBatch == null || !currentBatch.hasNext()) { - if (!filteredColumnarBatchCloseableIterator.hasNext()) { - return false; // No more batches or records to read! - } - currentBatch = filteredColumnarBatchCloseableIterator.next().getRows(); - } - return true; - } - - @Override - public InputRow next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - Row dataRow = currentBatch.next(); - - // TODO: construct schema? remove this after debugging - return new DeltaInputRow(dataRow, inputRowSchema); - } - - @Override - public void close() throws IOException - { - filteredColumnarBatchCloseableIterator.close(); - } - } - - /** - * Derive value using the type mappings from row's schema pertaining to an ordinal - */ - private static String getValue(Row dataRow, int columnOrdinal) { - DataType dataType = dataRow.getSchema().at(columnOrdinal).getDataType(); - if (dataRow.isNullAt(columnOrdinal)) { - return null; - } else if (dataType instanceof BooleanType) { - return Boolean.toString(dataRow.getBoolean(columnOrdinal)); - } else if (dataType instanceof ByteType) { - return Byte.toString(dataRow.getByte(columnOrdinal)); - } else if (dataType instanceof ShortType) { - return Short.toString(dataRow.getShort(columnOrdinal)); - } else if (dataType instanceof IntegerType) { - return Integer.toString(dataRow.getInt(columnOrdinal)); - } else if (dataType instanceof DateType) { - // DateType data is stored internally as the number of days since 1970-01-01 - int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); - return LocalDate.ofEpochDay(daysSinceEpochUTC).toString(); - } else if (dataType instanceof LongType) { - return Long.toString(dataRow.getLong(columnOrdinal)); - } else if (dataType instanceof TimestampType) { - // TimestampType data is stored internally as the number of microseconds since epoch - long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); - LocalDateTime dateTime = LocalDateTime.ofEpochSecond( - microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, - (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC); - return dateTime.toString(); - } else if (dataType instanceof FloatType) { - return Float.toString(dataRow.getFloat(columnOrdinal)); - } else if (dataType instanceof DoubleType) { - return Double.toString(dataRow.getDouble(columnOrdinal)); - } else if (dataType instanceof StringType) { - return dataRow.getString(columnOrdinal); - } else if (dataType instanceof BinaryType) { - return new String(dataRow.getBinary(columnOrdinal)); - } else if (dataType instanceof DecimalType) { - return dataRow.getDecimal(columnOrdinal).toString(); - } else if (dataType instanceof StructType) { - return "TODO: struct value"; - } else if (dataType instanceof ArrayType) { - return "TODO: list value"; - } else if (dataType instanceof MapType) { - return "TODO: map value"; - } else { - throw new UnsupportedOperationException("unsupported data type: " + dataType); - } - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java deleted file mode 100644 index 856f7dc8aeb6..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/DeltaSplit.java +++ /dev/null @@ -1,42 +0,0 @@ -package org.apache.druid.data.input.impl.delta; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import io.delta.kernel.data.Row; - -import java.util.List; - -public class DeltaSplit -{ - private final String stateRow; - private final List fileRows; - - @JsonCreator - public DeltaSplit(@JsonProperty("state") String stateRow, @JsonProperty("file") List fileRows) - { - this.stateRow = stateRow; - this.fileRows = fileRows; - } - - @JsonProperty("state") - public String getStateRow() - { - return stateRow; - } - - @JsonProperty("file") - public List getFile() - { - return fileRows; - } - - - @Override - public String toString() - { - return "DeltaSplit{" + - "stateRow=" + stateRow + - ", file=" + fileRows + - "}"; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java b/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java deleted file mode 100644 index df769b31ea2b..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/delta/RowSerde.java +++ /dev/null @@ -1,130 +0,0 @@ -package org.apache.druid.data.input.impl.delta; - -/* - * Copyright (2023) The Delta Lake Project Authors. - * - * Licensed 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. - */ - -import java.io.UncheckedIOException; -import java.util.HashMap; -import java.util.Map; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; - -import io.delta.kernel.client.TableClient; -import io.delta.kernel.data.Row; -import io.delta.kernel.types.*; - -import io.delta.kernel.internal.types.TableSchemaSerDe; -import io.delta.kernel.internal.util.VectorUtils; - -import io.delta.kernel.defaults.internal.data.DefaultJsonRow; - -/** - * Utility class to serialize and deserialize {@link Row} object. - */ -public class RowSerde { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private RowSerde() { - } - - /** - * Utility method to serialize a {@link Row} as a JSON string - */ - public static String serializeRowToJson(Row row) { - Map rowObject = convertRowToJsonObject(row); - try { - Map rowWithSchema = new HashMap<>(); - rowWithSchema.put("schema", TableSchemaSerDe.toJson(row.getSchema())); - rowWithSchema.put("row", rowObject); - return OBJECT_MAPPER.writeValueAsString(rowWithSchema); - } catch (JsonProcessingException e) { - throw new UncheckedIOException(e); - } - } - - /** - * Utility method to deserialize a {@link Row} object from the JSON form. - */ - public static Row deserializeRowFromJson(TableClient tableClient, String jsonRowWithSchema) { - try { - JsonNode jsonNode = OBJECT_MAPPER.readTree(jsonRowWithSchema); - JsonNode schemaNode = jsonNode.get("schema"); - StructType schema = - TableSchemaSerDe.fromJson(tableClient.getJsonHandler(), schemaNode.asText()); - return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); - } catch (JsonProcessingException ex) { - throw new UncheckedIOException(ex); - } - } - - public static Map convertRowToJsonObject(Row row) { - StructType rowType = row.getSchema(); - Map rowObject = new HashMap<>(); - for (int fieldId = 0; fieldId < rowType.length(); fieldId++) { - StructField field = rowType.at(fieldId); - DataType fieldType = field.getDataType(); - String name = field.getName(); - - if (row.isNullAt(fieldId)) { - rowObject.put(name, null); - continue; - } - - Object value; - if (fieldType instanceof BooleanType) { - value = row.getBoolean(fieldId); - } else if (fieldType instanceof ByteType) { - value = row.getByte(fieldId); - } else if (fieldType instanceof ShortType) { - value = row.getShort(fieldId); - } else if (fieldType instanceof IntegerType) { - value = row.getInt(fieldId); - } else if (fieldType instanceof LongType) { - value = row.getLong(fieldId); - } else if (fieldType instanceof FloatType) { - value = row.getFloat(fieldId); - } else if (fieldType instanceof DoubleType) { - value = row.getDouble(fieldId); - } else if (fieldType instanceof DateType) { - value = row.getInt(fieldId); - } else if (fieldType instanceof TimestampType) { - value = row.getLong(fieldId); - } else if (fieldType instanceof StringType) { - value = row.getString(fieldId); - } else if (fieldType instanceof ArrayType) { - value = VectorUtils.toJavaList(row.getArray(fieldId)); - } else if (fieldType instanceof MapType) { - value = VectorUtils.toJavaMap(row.getMap(fieldId)); - } else if (fieldType instanceof StructType) { - Row subRow = row.getStruct(fieldId); - value = convertRowToJsonObject(subRow); - } else { - throw new UnsupportedOperationException("NYI"); - } - - rowObject.put(name, value); - } - - return rowObject; - } - - private static Row parseRowFromJsonWithSchema(ObjectNode rowJsonNode, StructType rowType) { - return new DefaultJsonRow(rowJsonNode, rowType); - } -} \ No newline at end of file From f2804809eb6bb201b0cc12854298ecb376966ebf Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 3 Jan 2024 12:25:09 -0800 Subject: [PATCH 21/65] Add InputSource, module init and helper code to process delta files. --- .../delta/common/DeltaLakeDruidModule.java | 70 ++++++ .../druid/delta/input/DeltaInputRow.java | 197 ++++++++++++++++ .../druid/delta/input/DeltaInputSource.java | 212 ++++++++++++++++++ .../delta/input/DeltaInputSourceReader.java | 207 +++++++++++++++++ .../delta/input/DeltaInputSourceTest.java | 25 +++ .../apache/druid/delta/input/DeltaSplit.java | 59 +++++ .../apache/druid/delta/input/RowSerde.java | 152 +++++++++++++ ...rg.apache.druid.initialization.DruidModule | 16 ++ 8 files changed, 938 insertions(+) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/common/DeltaLakeDruidModule.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/common/DeltaLakeDruidModule.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/common/DeltaLakeDruidModule.java new file mode 100644 index 000000000000..70c2b6ff655b --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/common/DeltaLakeDruidModule.java @@ -0,0 +1,70 @@ +/* + * 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.druid.delta.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import org.apache.druid.delta.input.DeltaInputSource; +import org.apache.druid.error.DruidException; +import org.apache.druid.initialization.DruidModule; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.util.Collections; +import java.util.List; + +public class DeltaLakeDruidModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule("DeltaLakeDruidModule") + .registerSubtypes( + new NamedType(DeltaInputSource.class, DeltaInputSource.TYPE_KEY) + ) + ); + } + + @Override + public void configure(Binder binder) + { + final Configuration conf = new Configuration(); + conf.setClassLoader(getClass().getClassLoader()); + + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + FileSystem.get(conf); + } + catch (Exception ex) { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(ex, "Problem during fileSystem class level initialization"); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + +} + diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java new file mode 100644 index 000000000000..36ddf609fc4d --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -0,0 +1,197 @@ +/* + * 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.druid.delta.input; + +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampType; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DeltaInputRow implements InputRow +{ + private final io.delta.kernel.data.Row row; + private final StructType schema; + private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); + private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TIMEZONE HANDLING????? + private final InputRow delegateRow; + + public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) + { + this.row = row; + this.schema = row.getSchema(); + List fieldNames = this.schema.fieldNames(); + for (int i = 0; i < fieldNames.size(); ++i) { + fieldNameToOrdinal.put(fieldNames.get(i), i); + } + fieldNameToOrdinal.defaultReturnValue(-1); + + Map theMap = new HashMap<>(); + for (String fieldName : fieldNames) { + theMap.put(fieldName, _getRaw(fieldName)); + } + delegateRow = MapInputRowParser.parse(inputRowSchema, theMap); + + } + + @Override + public List getDimensions() + { + return delegateRow.getDimensions(); + } + + @Override + public long getTimestampFromEpoch() + { + return delegateRow.getTimestampFromEpoch(); + } + + @Override + public DateTime getTimestamp() + { + return delegateRow.getTimestamp(); + } + + @Override + public List getDimension(String dimension) + { + return delegateRow.getDimension(dimension); + } + + @Nullable + @Override + public Object getRaw(String dimension) + { + return delegateRow.getRaw(dimension); + } + + @Nullable + public Object _getRaw(String dimension) + { + StructField field = schema.get(dimension); + if (field == null) { + return null; + } else if (field.isMetadataColumn()) { + return null; + } + + + int ordinal = fieldNameToOrdinal.getInt(dimension); + if (ordinal < 0) { + return null; + } + return getValue(field.getDataType(), row, ordinal); + + } + + @Nullable + @Override + public Number getMetric(String metric) + { + return delegateRow.getMetric(metric); + } + + @Override + public int compareTo(Row o) + { + return this.getTimestamp().compareTo(o.getTimestamp()); + } + + @Nullable + private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataRow, int columnOrdinal) + { + if (dataRow.isNullAt(columnOrdinal)) { + return null; + } else if (dataType instanceof BooleanType) { + return dataRow.getBoolean(columnOrdinal); + } else if (dataType instanceof ByteType) { + return dataRow.getByte(columnOrdinal); + } else if (dataType instanceof ShortType) { + return dataRow.getShort(columnOrdinal); + } else if (dataType instanceof IntegerType) { + return dataRow.getInt(columnOrdinal); + } else if (dataType instanceof DateType) { + // DateType data is stored internally as the number of days since 1970-01-01 + int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); + return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); + } else if (dataType instanceof LongType) { + return dataRow.getLong(columnOrdinal); + } else if (dataType instanceof TimestampType) { + // TimestampType data is stored internally as the number of microseconds since epoch + long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); + LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC); + return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); + } else if (dataType instanceof FloatType) { + return dataRow.getFloat(columnOrdinal); + } else if (dataType instanceof DoubleType) { + return dataRow.getDouble(columnOrdinal); + } else if (dataType instanceof StringType) { + return dataRow.getString(columnOrdinal); + } else if (dataType instanceof BinaryType) { + return new String(dataRow.getBinary(columnOrdinal)); + } else if (dataType instanceof DecimalType) { + return dataRow.getDecimal(columnOrdinal).longValue(); + } else if (dataType instanceof StructType) { + return "TODO: struct value"; + } else if (dataType instanceof ArrayType) { + return "TODO: list value"; + } else if (dataType instanceof MapType) { + return "TODO: map value"; + } else { + throw new UnsupportedOperationException("unsupported data type: " + dataType); + } + } + + public Map getRawRowAsMap() + { + return RowSerde.convertRowToJsonObject(row); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java new file mode 100644 index 000000000000..78b966f71769 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -0,0 +1,212 @@ +/* + * 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.druid.delta.input; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; +import com.google.common.primitives.Ints; +import io.delta.kernel.Scan; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.TableNotFoundException; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.internal.util.Utils; +import io.delta.kernel.utils.CloseableIterator; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.Streams; +import org.apache.hadoop.conf.Configuration; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Inputsource to ingest data managed by the Delta Lake table format. + * This inputsource talks to the configured catalog, executes any configured filters and retrieves the data file paths upto the latest snapshot associated with the iceberg table. + * The data file paths are then provided to a native {@link SplittableInputSource} implementation depending on the warehouse source defined. + */ +public class DeltaInputSource implements SplittableInputSource +{ + public static final String TYPE_KEY = "delta"; + + @JsonProperty + private final String tablePath; + + @JsonProperty + @Nullable + private final DeltaSplit deltaSplit; + + private static final Logger log = new Logger(DeltaInputSource.class); + + @JsonCreator + public DeltaInputSource( + @JsonProperty("tablePath") String tablePath, + @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit + ) + { + log.info("CONST Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); + this.tablePath = Preconditions.checkNotNull(tablePath, "tablePath cannot be null"); + this.deltaSplit = deltaSplit; + } + + @Override + public boolean needsFormat() + { + // Only support Parquet + return false; + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + File temporaryDirectory + ) + { + log.info("READER Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); + Configuration hadoopConf = new Configuration(); + TableClient tableClient = DefaultTableClient.create(hadoopConf); + try { + final Row scanState; + final List scanRowList; + + if (deltaSplit != null) { + scanState = deserialize(tableClient, deltaSplit.getStateRow()); + scanRowList = deltaSplit.getFile() + .stream() + .map(row -> deserialize(tableClient, row)) + .collect(Collectors.toList()); + } else { + Table table = Table.forPath(tableClient, tablePath); + Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); + + Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + scanState = scan.getScanState(tableClient); + CloseableIterator scanFiles = scan.getScanFiles(tableClient); + scanRowList = new ArrayList<>(); + + while (scanFiles.hasNext()) { + FilteredColumnarBatch scanFileBatch = scanFiles.next(); + CloseableIterator scanFileRows = scanFileBatch.getRows(); + scanFileRows.forEachRemaining(scanRowList::add); + } + } + return new DeltaInputSourceReader( + Scan.readData( + tableClient, + scanState, + Utils.toCloseableIterator(scanRowList.iterator()), + Optional.empty() + ), + inputRowSchema + ); + } + catch (TableNotFoundException e) { + throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + throws IOException + { + if (null != deltaSplit) { + // can't split a split + return Collections.singletonList(new InputSplit<>(deltaSplit)).stream(); + } + + log.info("CREATE SPLITS Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); + TableClient tableClient = DefaultTableClient.create(new Configuration()); + final Snapshot latestSnapshot; + final Table table; + try { + table = Table.forPath(tableClient, tablePath); + latestSnapshot = table.getLatestSnapshot(tableClient); + } + catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + // scan files iterator for the current snapshot + CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient); + + Row scanState = scan.getScanState(tableClient); + String scanStateStr = RowSerde.serializeRowToJson(scanState); + + Iterator deltaSplitIterator = Iterators.transform( + scanFilesIterator, + scanFile -> { + CloseableIterator rows = scanFile.getRows(); + List fileRows = new ArrayList<>(); + while (rows.hasNext()) { + fileRows.add(RowSerde.serializeRowToJson(rows.next())); + } + return new DeltaSplit(scanStateStr, fileRows); + } + ); + + // TODO: account for the split spec as well -- getSplitHintSpecOrDefault(splitHintSpec).split() + return Streams.sequentialStreamFrom(deltaSplitIterator).map(InputSplit::new); + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException + { + return Ints.checkedCast(createSplits(inputFormat, splitHintSpec).count()); + } + + @Override + public InputSource withSplit(InputSplit split) + { + log.info("WITH SPLIT Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); + return new DeltaInputSource( + tablePath, + split.get() + ); + } + + private Row deserialize(TableClient myTableClient, String row) + { + return RowSerde.deserializeRowFromJson(myTableClient, row); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java new file mode 100644 index 000000000000..8616ad11bb86 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -0,0 +1,207 @@ +/* + * 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.druid.delta.input; + +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampType; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.NoSuchElementException; + +public class DeltaInputSourceReader implements InputSourceReader +{ + private static final Logger log = new Logger(DeltaInputSourceReader.class); + + private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + private final InputRowSchema inputRowSchema; + + public DeltaInputSourceReader( + io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + InputRowSchema inputRowSchema + ) + { + this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.inputRowSchema = inputRowSchema; + } + + @Override + public CloseableIterator read() + { + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); + } + + @Override + public CloseableIterator read(InputStats inputStats) throws IOException + { + return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); + } + + @Override + public CloseableIterator sample() throws IOException + { + CloseableIterator inner = read(); + return new CloseableIterator() + { + @Override + public void close() throws IOException + { + inner.close(); + } + + @Override + public boolean hasNext() + { + return inner.hasNext(); + } + + @Override + public InputRowListPlusRawValues next() + { + DeltaInputRow deltaInputRow = (DeltaInputRow) inner.next(); + return InputRowListPlusRawValues.of(deltaInputRow, deltaInputRow.getRawRowAsMap()); + } + }; + } + + private static class DeltaInputSourceIterator implements CloseableIterator + { + private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; + + private io.delta.kernel.utils.CloseableIterator currentBatch = null; + private final InputRowSchema inputRowSchema; + + public DeltaInputSourceIterator( + io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator, + InputRowSchema inputRowSchema + ) + { + this.filteredColumnarBatchCloseableIterator = filteredColumnarBatchCloseableIterator; + this.inputRowSchema = inputRowSchema; + } + + @Override + public boolean hasNext() + { + while (currentBatch == null || !currentBatch.hasNext()) { + if (!filteredColumnarBatchCloseableIterator.hasNext()) { + return false; // No more batches or records to read! + } + currentBatch = filteredColumnarBatchCloseableIterator.next().getRows(); + } + return true; + } + + @Override + public InputRow next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + Row dataRow = currentBatch.next(); + + // TODO: construct schema? remove this after debugging + return new DeltaInputRow(dataRow, inputRowSchema); + } + + @Override + public void close() throws IOException + { + filteredColumnarBatchCloseableIterator.close(); + } + } + + /** + * Derive value using the type mappings from row's schema pertaining to an ordinal + */ + private static String getValue(Row dataRow, int columnOrdinal) + { + DataType dataType = dataRow.getSchema().at(columnOrdinal).getDataType(); + if (dataRow.isNullAt(columnOrdinal)) { + return null; + } else if (dataType instanceof BooleanType) { + return Boolean.toString(dataRow.getBoolean(columnOrdinal)); + } else if (dataType instanceof ByteType) { + return Byte.toString(dataRow.getByte(columnOrdinal)); + } else if (dataType instanceof ShortType) { + return Short.toString(dataRow.getShort(columnOrdinal)); + } else if (dataType instanceof IntegerType) { + return Integer.toString(dataRow.getInt(columnOrdinal)); + } else if (dataType instanceof DateType) { + // DateType data is stored internally as the number of days since 1970-01-01 + int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); + return LocalDate.ofEpochDay(daysSinceEpochUTC).toString(); + } else if (dataType instanceof LongType) { + return Long.toString(dataRow.getLong(columnOrdinal)); + } else if (dataType instanceof TimestampType) { + // TimestampType data is stored internally as the number of microseconds since epoch + long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); + LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC); + return dateTime.toString(); + } else if (dataType instanceof FloatType) { + return Float.toString(dataRow.getFloat(columnOrdinal)); + } else if (dataType instanceof DoubleType) { + return Double.toString(dataRow.getDouble(columnOrdinal)); + } else if (dataType instanceof StringType) { + return dataRow.getString(columnOrdinal); + } else if (dataType instanceof BinaryType) { + return new String(dataRow.getBinary(columnOrdinal)); + } else if (dataType instanceof DecimalType) { + return dataRow.getDecimal(columnOrdinal).toString(); + } else if (dataType instanceof StructType) { + return "TODO: struct value"; + } else if (dataType instanceof ArrayType) { + return "TODO: list value"; + } else if (dataType instanceof MapType) { + return "TODO: map value"; + } else { + throw new UnsupportedOperationException("unsupported data type: " + dataType); + } + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java new file mode 100644 index 000000000000..43e4061086a7 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -0,0 +1,25 @@ +/* + * 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.druid.delta.input; + +public class DeltaInputSourceTest +{ + +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java new file mode 100644 index 000000000000..178397a96e22 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java @@ -0,0 +1,59 @@ +/* + * 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.druid.delta.input; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class DeltaSplit +{ + private final String stateRow; + private final List fileRows; + + @JsonCreator + public DeltaSplit(@JsonProperty("state") String stateRow, @JsonProperty("file") List fileRows) + { + this.stateRow = stateRow; + this.fileRows = fileRows; + } + + @JsonProperty("state") + public String getStateRow() + { + return stateRow; + } + + @JsonProperty("file") + public List getFile() + { + return fileRows; + } + + @Override + public String toString() + { + return "DeltaSplit{" + + "stateRow=" + stateRow + + ", file=" + fileRows + + "}"; + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java new file mode 100644 index 000000000000..354aba5cb992 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -0,0 +1,152 @@ +/* + * 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.druid.delta.input; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.internal.data.DefaultJsonRow; +import io.delta.kernel.internal.types.TableSchemaSerDe; +import io.delta.kernel.internal.util.VectorUtils; +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampType; + +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Utility class to serialize and deserialize {@link Row} object. + */ +public class RowSerde +{ + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private RowSerde() + { + } + + /** + * Utility method to serialize a {@link Row} as a JSON string + */ + public static String serializeRowToJson(Row row) + { + Map rowObject = convertRowToJsonObject(row); + try { + Map rowWithSchema = new HashMap<>(); + rowWithSchema.put("schema", TableSchemaSerDe.toJson(row.getSchema())); + rowWithSchema.put("row", rowObject); + return OBJECT_MAPPER.writeValueAsString(rowWithSchema); + } + catch (JsonProcessingException e) { + throw new UncheckedIOException(e); + } + } + + /** + * Utility method to deserialize a {@link Row} object from the JSON form. + */ + public static Row deserializeRowFromJson(TableClient tableClient, String jsonRowWithSchema) + { + try { + JsonNode jsonNode = OBJECT_MAPPER.readTree(jsonRowWithSchema); + JsonNode schemaNode = jsonNode.get("schema"); + StructType schema = + TableSchemaSerDe.fromJson(tableClient.getJsonHandler(), schemaNode.asText()); + return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); + } + catch (JsonProcessingException ex) { + throw new UncheckedIOException(ex); + } + } + + public static Map convertRowToJsonObject(Row row) + { + StructType rowType = row.getSchema(); + Map rowObject = new HashMap<>(); + for (int fieldId = 0; fieldId < rowType.length(); fieldId++) { + StructField field = rowType.at(fieldId); + DataType fieldType = field.getDataType(); + String name = field.getName(); + + if (row.isNullAt(fieldId)) { + rowObject.put(name, null); + continue; + } + + Object value; + if (fieldType instanceof BooleanType) { + value = row.getBoolean(fieldId); + } else if (fieldType instanceof ByteType) { + value = row.getByte(fieldId); + } else if (fieldType instanceof ShortType) { + value = row.getShort(fieldId); + } else if (fieldType instanceof IntegerType) { + value = row.getInt(fieldId); + } else if (fieldType instanceof LongType) { + value = row.getLong(fieldId); + } else if (fieldType instanceof FloatType) { + value = row.getFloat(fieldId); + } else if (fieldType instanceof DoubleType) { + value = row.getDouble(fieldId); + } else if (fieldType instanceof DateType) { + value = row.getInt(fieldId); + } else if (fieldType instanceof TimestampType) { + value = row.getLong(fieldId); + } else if (fieldType instanceof StringType) { + value = row.getString(fieldId); + } else if (fieldType instanceof ArrayType) { + value = VectorUtils.toJavaList(row.getArray(fieldId)); + } else if (fieldType instanceof MapType) { + value = VectorUtils.toJavaMap(row.getMap(fieldId)); + } else if (fieldType instanceof StructType) { + Row subRow = row.getStruct(fieldId); + value = convertRowToJsonObject(subRow); + } else { + throw new UnsupportedOperationException("NYI"); + } + + rowObject.put(name, value); + } + + return rowObject; + } + + private static Row parseRowFromJsonWithSchema(ObjectNode rowJsonNode, StructType rowType) + { + return new DefaultJsonRow(rowJsonNode, rowType); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/druid-deltalake-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..b2752bd862c6 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# 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. + +org.apache.druid.delta.common.DeltaLakeDruidModule \ No newline at end of file From 68e8dcb9deead70a925cb9c77b8744eccf2b790e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 3 Jan 2024 12:49:38 -0800 Subject: [PATCH 22/65] Test init --- .../delta/input/DeltaInputSourceTest.java | 25 ------------------- .../delta/input/DeltaInputSourceTest.java | 16 ++++++++++++ 2 files changed, 16 insertions(+), 25 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java deleted file mode 100644 index 43e4061086a7..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.druid.delta.input; - -public class DeltaInputSourceTest -{ - -} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java new file mode 100644 index 000000000000..84c6925df2f8 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -0,0 +1,16 @@ +package org.apache.druid.delta.input; + +import org.junit.Test; + +import static org.junit.Assert.*; + +public class DeltaInputSourceTest +{ + + @Test + public void testInputSourceInit() + { + + } + +} \ No newline at end of file From 0bc238dd367f4a553358b6f58fa2b449097fcc0e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sat, 6 Jan 2024 17:34:19 -0800 Subject: [PATCH 23/65] Checkpoint changes --- distribution/pom.xml | 7 +- .../druid-deltalake-extensions/pom.xml | 13 ++ .../druid/delta/input/DeltaInputSource.java | 4 +- .../delta/input/DeltaInputSourceReader.java | 2 +- .../apache/druid/delta/input/RowSerde.java | 2 +- .../delta/input/DeltaInputSourceTest.java | 176 +++++++++++++++++- .../apache/druid/data/input/InputSource.java | 1 - 7 files changed, 196 insertions(+), 9 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index eeab25e97e42..1e749677bb98 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -234,9 +234,8 @@ -c org.apache.druid.extensions:postgresql-metadata-storage -c - - - + org.apache.druid.extensions:druid-kerberos + -c org.apache.druid.extensions:druid-s3-extensions -c org.apache.druid.extensions:druid-aws-rds-extensions @@ -258,6 +257,8 @@ org.apache.druid.extensions:druid-kubernetes-extensions -c org.apache.druid.extensions:druid-catalog + -c + org.apache.druid.extensions:druid-deltalake-extensions ${druid.distribution.pulldeps.opts} diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index b7035985f7dd..939bc8c824da 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -434,6 +434,19 @@ equalsverifier test
+ + + + + + + + + io.delta + delta-standalone_2.12 + 0.5.0 + + diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 78b966f71769..6c5c42760bb6 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -33,6 +33,7 @@ import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; import io.delta.kernel.internal.util.Utils; +import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; @@ -100,7 +101,7 @@ public InputSourceReader reader( File temporaryDirectory ) { - log.info("READER Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); + log.info("READER Delta input source reader for inputRowSchema[%s], tablePath[%s] and split[%s]", inputRowSchema, tablePath, deltaSplit); Configuration hadoopConf = new Configuration(); TableClient tableClient = DefaultTableClient.create(hadoopConf); try { @@ -116,6 +117,7 @@ public InputSourceReader reader( } else { Table table = Table.forPath(tableClient, tablePath); Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); + StructType schema = latestSnapshot.getSchema(tableClient); Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); scanState = scan.getScanState(tableClient); diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java index 8616ad11bb86..3bb58c2db0f9 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -142,7 +142,7 @@ public InputRow next() } Row dataRow = currentBatch.next(); - + System.out.println("Datarow" + dataRow); // TODO: construct schema? remove this after debugging return new DeltaInputRow(dataRow, inputRowSchema); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 354aba5cb992..ba6f9fb3e0f8 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -89,7 +89,7 @@ public static Row deserializeRowFromJson(TableClient tableClient, String jsonRow return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); } catch (JsonProcessingException ex) { - throw new UncheckedIOException(ex); + throw new UncheckedIOException(ex); // todo: ise? } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java index 84c6925df2f8..d12d359d6a83 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -1,16 +1,188 @@ +/* + * 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.druid.delta.input; +import io.delta.standalone.DeltaLog; +import io.delta.standalone.Operation; +import io.delta.standalone.OptimisticTransaction; +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.actions.Metadata; +import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.StringType; +import io.delta.standalone.types.StructType; + + +import com.google.common.collect.ImmutableList; +import org.apache.commons.io.FileUtils; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.*; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + public class DeltaInputSourceTest { @Test - public void testInputSourceInit() + public void testInputSourceInit() throws IOException { +// String deltaTablePath = "/Users/abhishek/Desktop/people-delta-table-oss"; + String deltaTablePath = "/var/folders/5g/jnsnl96j4wlfw404lyj89mxm0000gr/T/my_table9175802766602445708"; + + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null); + + Assert.assertNotNull(deltaInputSource); + + InputRowSchema someSchema = new InputRowSchema( + new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), + ColumnsFilter.all() + ); + + InputSourceReader inputSourceReader = deltaInputSource.reader(someSchema, null, null); + Assert.assertNotNull(inputSourceReader); + CloseableIterator rowReader = inputSourceReader.read(); + Assert.assertNotNull(rowReader); + int expectedCnt = 2; + int actualCnt = 0; + while(rowReader.hasNext()) { + InputRow row = rowReader.next(); + Assert.assertNotNull(row); + actualCnt += 1; + System.out.println("row:" + row); + Assert.assertNotNull(row.getDimensions()); + Assert.assertNotNull(row.getDimensions()); + if (actualCnt == 1) { + Assert.assertEquals(Collections.singletonList("Montclair"), row.getDimension("city")); + } else { + Assert.assertEquals(Collections.singletonList("Wildwood"), row.getDimension("city")); + } + } + Assert.assertEquals(expectedCnt, actualCnt); } + @Test + public void testWriteDeltaLake() throws IOException + { +// File tmpDir = Files.createTempDirectory("my_table").toFile(); +// File tmpDir = Files.createTempDirectory("my_table").toFile(); +// String tmpDirPath = tmpDir.getAbsolutePath(); + + String tmpDirPath = "/Users/abhishek/Desktop/imply/abhishek-agarwal-druid/druid/ut-table"; + + try { + final String engineInfo = "local"; + + DeltaLog log = DeltaLog.forTable(new Configuration(), tmpDirPath); + + StructType schema = new StructType() + .add("foo", new IntegerType()) + .add("bar", new IntegerType()) + .add("zip", new StringType()); + + List partitionColumns = Arrays.asList("foo", "bar"); + + Metadata metadata = Metadata.builder() + .schema(schema) + .partitionColumns(partitionColumns) + .build(); + + Operation op = new Operation(Operation.Name.WRITE); + + for (int i = 0; i < 10; i++) { + OptimisticTransaction txn = log.startTransaction(); + if (i == 0) { + txn.updateMetadata(metadata); + } + + Map partitionValues = new HashMap<>(); + partitionValues.put("foo", Integer.toString(i % 3)); + partitionValues.put("bar", Integer.toString(i % 2)); + + long now = System.currentTimeMillis(); + + AddFile addFile = AddFile.builder(Integer.toString(i+100), partitionValues, 100L, now, true) + .tags(Collections.singletonMap("someTagKey", "someTagVal")) + .build(); + + txn.commit(Collections.singletonList(addFile), op, engineInfo); + System.out.println(String.format("Committed version %d", i)); + } + + DeltaLog log2 = DeltaLog.forTable(new Configuration(), tmpDirPath); + Set pathVals = log2.update() + .getAllFiles() + .stream() + .map(addFile -> Integer.parseInt(addFile.getPath())) + .collect(Collectors.toSet()); +// +// for (int i = 0; i < 10; i++) { +// if (!pathVals.contains(i)) throw new RuntimeException(); +// System.out.println(String.format("Read version %d", i)); +// } + + } finally { + System.out.println("Written to " + tmpDirPath); +// FileUtils.deleteDirectory(tmpDir); + } + + // now try to read stuff + final DeltaInputSource deltaInputSource = new DeltaInputSource(tmpDirPath, null); + Assert.assertNotNull(deltaInputSource); + + InputRowSchema someSchema = new InputRowSchema( + new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar"))), + ColumnsFilter.all() + ); + + InputSourceReader inputSourceReader = deltaInputSource.reader(someSchema, null, null); + Assert.assertNotNull(inputSourceReader); + + CloseableIterator rowReader = inputSourceReader.read(); + Assert.assertNotNull(rowReader); + + while(rowReader.hasNext()) { + InputRow row = rowReader.next(); + Assert.assertNotNull(row); + System.out.println("row:" + row); + Assert.assertNotNull(row.getDimensions()); + } + } } \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/data/input/InputSource.java b/processing/src/main/java/org/apache/druid/data/input/InputSource.java index 208dc4dd49cf..be815742be16 100644 --- a/processing/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/InputSource.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.HttpInputSource; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.data.input.impl.delta.DeltaInputSource; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.UOE; From ffd8bb19caa4f4dddaa9479889a720069ee64d54 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 9 Jan 2024 16:56:10 -0800 Subject: [PATCH 24/65] Test resources and updates --- docs/ingestion/input-sources.md | 2 + .../druid/delta/input/DeltaInputSource.java | 4 +- .../delta/input/DeltaInputSourceReader.java | 51 ----- .../apache/druid/delta/input/DeltaSplit.java | 1 + .../apache/druid/delta/input/RowSerde.java | 1 + .../delta/input/DeltaInputSourceTest.java | 188 ------------------ .../delta/input/DeltaInputSourceTest.java | 164 +++++++++++++++ .../src/test/resources/README.md | 30 +++ .../src/test/resources/create_delta_table.py | 90 +++++++++ .../_delta_log/00000000000000000000.json | 13 ++ ...4299-af4d-3fdaa1c12001-c000.snappy.parquet | Bin 0 -> 2187 bytes ...4a77-96f6-e4a0abe939d3-c000.snappy.parquet | Bin 0 -> 2180 bytes ...4fbb-837b-9a38059e6ea1-c000.snappy.parquet | Bin 0 -> 2223 bytes ...4549-aa26-c7f0b03b051b-c000.snappy.parquet | Bin 0 -> 2263 bytes ...408b-acd9-21754be887de-c000.snappy.parquet | Bin 0 -> 2222 bytes ...4ac9-85f0-bf1b8be53719-c000.snappy.parquet | Bin 0 -> 2264 bytes ...4a9e-ad00-a705738e2aff-c000.snappy.parquet | Bin 0 -> 2151 bytes ...42f1-816c-6fd7be8b60e6-c000.snappy.parquet | Bin 0 -> 2207 bytes ...4c4c-a516-07728b69a7ef-c000.snappy.parquet | Bin 0 -> 2221 bytes ...4646-a224-f3d0187a34ee-c000.snappy.parquet | Bin 0 -> 2204 bytes .../src/test/resources/requirements.txt | 3 + 21 files changed, 306 insertions(+), 241 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md create mode 100755 extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00001-b3706429-b27b-4a77-96f6-e4a0abe939d3-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00002-dcd31c19-e96c-4fbb-837b-9a38059e6ea1-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00004-adfc1b3d-c6a2-408b-acd9-21754be887de-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00005-3d374415-de65-4ac9-85f0-bf1b8be53719-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00006-68588845-198d-4a9e-ad00-a705738e2aff-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 3e11734ee55f..63d5bc3f05c3 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1006,7 +1006,9 @@ This input source provides the following filters: `and`, `equals`, `interval`, a |type|Set this value to `not`.|yes| |filter|The iceberg filter on which logical NOT is applied|yes| +## DeltaLake input source +TODO: fill in details about the input source The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source. Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method: diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 6c5c42760bb6..320565316abe 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -207,8 +207,8 @@ public InputSource withSplit(InputSplit split) ); } - private Row deserialize(TableClient myTableClient, String row) + private Row deserialize(TableClient tableClient, String row) { - return RowSerde.deserializeRowFromJson(myTableClient, row); + return RowSerde.deserializeRowFromJson(tableClient, row); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java index 3bb58c2db0f9..9132b2638dd6 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -153,55 +153,4 @@ public void close() throws IOException filteredColumnarBatchCloseableIterator.close(); } } - - /** - * Derive value using the type mappings from row's schema pertaining to an ordinal - */ - private static String getValue(Row dataRow, int columnOrdinal) - { - DataType dataType = dataRow.getSchema().at(columnOrdinal).getDataType(); - if (dataRow.isNullAt(columnOrdinal)) { - return null; - } else if (dataType instanceof BooleanType) { - return Boolean.toString(dataRow.getBoolean(columnOrdinal)); - } else if (dataType instanceof ByteType) { - return Byte.toString(dataRow.getByte(columnOrdinal)); - } else if (dataType instanceof ShortType) { - return Short.toString(dataRow.getShort(columnOrdinal)); - } else if (dataType instanceof IntegerType) { - return Integer.toString(dataRow.getInt(columnOrdinal)); - } else if (dataType instanceof DateType) { - // DateType data is stored internally as the number of days since 1970-01-01 - int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); - return LocalDate.ofEpochDay(daysSinceEpochUTC).toString(); - } else if (dataType instanceof LongType) { - return Long.toString(dataRow.getLong(columnOrdinal)); - } else if (dataType instanceof TimestampType) { - // TimestampType data is stored internally as the number of microseconds since epoch - long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); - LocalDateTime dateTime = LocalDateTime.ofEpochSecond( - microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, - (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC); - return dateTime.toString(); - } else if (dataType instanceof FloatType) { - return Float.toString(dataRow.getFloat(columnOrdinal)); - } else if (dataType instanceof DoubleType) { - return Double.toString(dataRow.getDouble(columnOrdinal)); - } else if (dataType instanceof StringType) { - return dataRow.getString(columnOrdinal); - } else if (dataType instanceof BinaryType) { - return new String(dataRow.getBinary(columnOrdinal)); - } else if (dataType instanceof DecimalType) { - return dataRow.getDecimal(columnOrdinal).toString(); - } else if (dataType instanceof StructType) { - return "TODO: struct value"; - } else if (dataType instanceof ArrayType) { - return "TODO: list value"; - } else if (dataType instanceof MapType) { - return "TODO: map value"; - } else { - throw new UnsupportedOperationException("unsupported data type: " + dataType); - } - } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java index 178397a96e22..a5272fee82ce 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java @@ -42,6 +42,7 @@ public String getStateRow() return stateRow; } + // TODO: fileRows? @JsonProperty("file") public List getFile() { diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index ba6f9fb3e0f8..e42cc1bb7b57 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -72,6 +72,7 @@ public static String serializeRowToJson(Row row) return OBJECT_MAPPER.writeValueAsString(rowWithSchema); } catch (JsonProcessingException e) { + // todo: throw druid exception throw new UncheckedIOException(e); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java deleted file mode 100644 index d12d359d6a83..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/test/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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.druid.delta.input; - -import io.delta.standalone.DeltaLog; -import io.delta.standalone.Operation; -import io.delta.standalone.OptimisticTransaction; -import io.delta.standalone.actions.AddFile; -import io.delta.standalone.actions.Metadata; -import io.delta.standalone.types.IntegerType; -import io.delta.standalone.types.StringType; -import io.delta.standalone.types.StructType; - - -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.FileUtils; -import org.apache.druid.data.input.ColumnsFilter; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - - -public class DeltaInputSourceTest -{ - - @Test - public void testInputSourceInit() throws IOException - { -// String deltaTablePath = "/Users/abhishek/Desktop/people-delta-table-oss"; - String deltaTablePath = "/var/folders/5g/jnsnl96j4wlfw404lyj89mxm0000gr/T/my_table9175802766602445708"; - - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null); - - Assert.assertNotNull(deltaInputSource); - - InputRowSchema someSchema = new InputRowSchema( - new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), - ColumnsFilter.all() - ); - - InputSourceReader inputSourceReader = deltaInputSource.reader(someSchema, null, null); - Assert.assertNotNull(inputSourceReader); - CloseableIterator rowReader = inputSourceReader.read(); - Assert.assertNotNull(rowReader); - int expectedCnt = 2; - int actualCnt = 0; - - while(rowReader.hasNext()) { - InputRow row = rowReader.next(); - Assert.assertNotNull(row); - actualCnt += 1; - System.out.println("row:" + row); - Assert.assertNotNull(row.getDimensions()); - Assert.assertNotNull(row.getDimensions()); - if (actualCnt == 1) { - Assert.assertEquals(Collections.singletonList("Montclair"), row.getDimension("city")); - } else { - Assert.assertEquals(Collections.singletonList("Wildwood"), row.getDimension("city")); - } - } - Assert.assertEquals(expectedCnt, actualCnt); - } - - @Test - public void testWriteDeltaLake() throws IOException - { -// File tmpDir = Files.createTempDirectory("my_table").toFile(); -// File tmpDir = Files.createTempDirectory("my_table").toFile(); -// String tmpDirPath = tmpDir.getAbsolutePath(); - - String tmpDirPath = "/Users/abhishek/Desktop/imply/abhishek-agarwal-druid/druid/ut-table"; - - try { - final String engineInfo = "local"; - - DeltaLog log = DeltaLog.forTable(new Configuration(), tmpDirPath); - - StructType schema = new StructType() - .add("foo", new IntegerType()) - .add("bar", new IntegerType()) - .add("zip", new StringType()); - - List partitionColumns = Arrays.asList("foo", "bar"); - - Metadata metadata = Metadata.builder() - .schema(schema) - .partitionColumns(partitionColumns) - .build(); - - Operation op = new Operation(Operation.Name.WRITE); - - for (int i = 0; i < 10; i++) { - OptimisticTransaction txn = log.startTransaction(); - if (i == 0) { - txn.updateMetadata(metadata); - } - - Map partitionValues = new HashMap<>(); - partitionValues.put("foo", Integer.toString(i % 3)); - partitionValues.put("bar", Integer.toString(i % 2)); - - long now = System.currentTimeMillis(); - - AddFile addFile = AddFile.builder(Integer.toString(i+100), partitionValues, 100L, now, true) - .tags(Collections.singletonMap("someTagKey", "someTagVal")) - .build(); - - txn.commit(Collections.singletonList(addFile), op, engineInfo); - System.out.println(String.format("Committed version %d", i)); - } - - DeltaLog log2 = DeltaLog.forTable(new Configuration(), tmpDirPath); - Set pathVals = log2.update() - .getAllFiles() - .stream() - .map(addFile -> Integer.parseInt(addFile.getPath())) - .collect(Collectors.toSet()); -// -// for (int i = 0; i < 10; i++) { -// if (!pathVals.contains(i)) throw new RuntimeException(); -// System.out.println(String.format("Read version %d", i)); -// } - - } finally { - System.out.println("Written to " + tmpDirPath); -// FileUtils.deleteDirectory(tmpDir); - } - - // now try to read stuff - final DeltaInputSource deltaInputSource = new DeltaInputSource(tmpDirPath, null); - Assert.assertNotNull(deltaInputSource); - - InputRowSchema someSchema = new InputRowSchema( - new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar"))), - ColumnsFilter.all() - ); - - InputSourceReader inputSourceReader = deltaInputSource.reader(someSchema, null, null); - Assert.assertNotNull(inputSourceReader); - - CloseableIterator rowReader = inputSourceReader.read(); - Assert.assertNotNull(rowReader); - - while(rowReader.hasNext()) { - InputRow row = rowReader.next(); - Assert.assertNotNull(row); - System.out.println("row:" + row); - Assert.assertNotNull(row.getDimensions()); - } - } -} \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java new file mode 100644 index 000000000000..a3c3f747d62f --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -0,0 +1,164 @@ +/* + * 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.druid.delta.input; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class DeltaInputSourceTest +{ + private static final ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); + private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; + + @Test + public void testReadDeltaLakeFilesSample() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + Assert.assertNotNull(deltaInputSource); + + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), + ColumnsFilter.all() + ); + + InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + Assert.assertNotNull(inputSourceReader); + + List inputRowListPlusRawValues = sampleAllRows(inputSourceReader); + Assert.assertNotNull(inputRowListPlusRawValues); + + Assert.assertEquals(10, inputRowListPlusRawValues.size()); + + final String expectedJson = "{\n" + + " \"birthday\" : 1049418130358332,\n" + + " \"country\" : \"Panama\",\n" + + " \"city\" : \"Eastpointe\",\n" + + " \"surname\" : \"Francis\",\n" + + " \"name\" : \"Darren\",\n" + + " \"state\" : \"Minnesota\",\n" + + " \"email\" : \"rating1998@yandex.com\"\n" + + "}"; + Assert.assertEquals( + expectedJson, + DEFAULT_JSON_WRITER.writeValueAsString(inputRowListPlusRawValues.get(0).getRawValues()) + ); + } + + @Test + public void testReadDeltaLakeFilesRead() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + Assert.assertNotNull(deltaInputSource); + + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("birthday", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), + ColumnsFilter.all() + ); + + InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + Assert.assertNotNull(inputSourceReader); + + List rows = readAllRows(inputSourceReader); + Assert.assertNotNull(rows); + + Assert.assertEquals(10, rows.size()); + + final InputRow firstRow = rows.get(0); + Assert.assertNotNull(firstRow); + + Assert.assertEquals("2003-04-04T01:02:10.000Z", firstRow.getTimestamp().toString()); + Assert.assertEquals("Panama", firstRow.getDimension("country").get(0)); + Assert.assertEquals("Eastpointe", firstRow.getDimension("city").get(0)); + Assert.assertEquals("Francis", firstRow.getDimension("surname").get(0)); + Assert.assertEquals("Darren", firstRow.getDimension("name").get(0)); + Assert.assertEquals("Minnesota", firstRow.getDimension("state").get(0)); + Assert.assertEquals("rating1998@yandex.com", firstRow.getDimension("email").get(0)); + } + + + @Test + public void testReadDeltaLakeNoSplits() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + Assert.assertNotNull(deltaInputSource); + + Stream> splits = deltaInputSource.createSplits(null, null); + Assert.assertNotNull(splits); + Assert.assertEquals(1, splits.count()); + } + + @Test + public void testReadDeltaLakeWithSplits() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + Assert.assertNotNull(deltaInputSource); + + Stream> splits1 = deltaInputSource.createSplits(null, null); + List> splitsCollect1 = splits1.collect(Collectors.toList()); + Assert.assertEquals(1, splitsCollect1.size()); + + DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource(DELTA_TABLE_PATH, splitsCollect1.get(0).get()); + Assert.assertNotNull(deltaInputSourceWithSplit); + Stream> splits2 = deltaInputSourceWithSplit.createSplits(null, null); + List> splitsCollect2 = splits2.collect(Collectors.toList()); + Assert.assertEquals(1, splitsCollect2.size()); + + Assert.assertEquals(splitsCollect1.get(0).get(), splitsCollect2.get(0).get()); + } + + private List sampleAllRows(InputSourceReader reader) throws IOException + { + List rows = new ArrayList<>(); + try (CloseableIterator iterator = reader.sample()) { + iterator.forEachRemaining(rows::add); + } + return rows; + } + + private List readAllRows(InputSourceReader reader) throws IOException + { + List rows = new ArrayList<>(); + try (CloseableIterator iterator = reader.read()) { + iterator.forEachRemaining(rows::add); + } + return rows; + } +} \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md new file mode 100644 index 000000000000..164a970b9dc5 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -0,0 +1,30 @@ +/* +* 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. +*/ + +* In a conda environment, + +```shell +pip install -r requirements.txt +``` + +Once the dependencies are installed, you can then run + +```python +python3 create_delta_table.py +``` \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py new file mode 100755 index 000000000000..9e3230af4d57 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -0,0 +1,90 @@ +#!/usr/bin/env python3 + +# 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. + +import os +import sys + +import delta +import mimesis +import pyspark + +from pyspark.sql import SparkSession + + +def config_spark_with_delta_lake(): + builder = ( + pyspark.sql.SparkSession.builder.appName("DeltaLakeApp") + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.delta.catalog.DeltaCatalog", + ) + ) + spark = delta.configure_spark_with_delta_pip(builder).getOrCreate() + spark.sparkContext.setLogLevel("ERROR") + return spark + +def create_dataset(i): + fake = mimesis.Generic() + output = [] + + for _ in range(i): + data = { + "name": fake.person.name(), + "surname": fake.person.surname(), + "birthday": fake.datetime.datetime(1980, 2010), + "email": fake.person.email(), + "country": fake.address.country(), + "state": fake.address.state(), + "city": fake.address.city(), + } + output.append(data) + + return output + + +def main(): + save_mode = "append" + save_path = os.path.join(os.getcwd(), "people-delta-table4") + num_records = 10 + + if len(sys.argv) > 1: + save_mode = sys.argv[1] + + if len(sys.argv) > 2: + save_path = sys.argv[2] + + if len(sys.argv) > 3: + num_records = sys.argv[3] + + spark = config_spark_with_delta_lake() + + df = spark.createDataFrame(create_dataset(i=num_records)) + + df = df.select( + df.name, df.surname, df.birthday, df.email, df.country, df.state, df.city + ) + + df.write.format("delta").mode(save_mode).save(save_path) + + df.show() + + print(f"Generated delta records to {save_path} in {save_mode} mode with {num_records} records.") + + +if __name__ == "__main__": + main() diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..d58121dc0581 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json @@ -0,0 +1,13 @@ +{"commitInfo":{"timestamp":1704767217219,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"10","numOutputBytes":"22122"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"aa74cdbe-d130-4cf2-9935-520649df8a9b"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"8047ab6e-7d22-4bbb-9988-5619003ba238","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"surname\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"birthday\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"email\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"country\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"state\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"city\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1704767214974}} +{"add":{"path":"part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet","partitionValues":{},"size":2187,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Darren\",\"surname\":\"Francis\",\"birthday\":\"2003-04-03T17:02:10.358-08:00\",\"email\":\"rating1998@yandex.com\",\"country\":\"Panama\",\"state\":\"Minnesota\",\"city\":\"Eastpointe\"},\"maxValues\":{\"name\":\"Darren\",\"surname\":\"Francis\",\"birthday\":\"2003-04-03T17:02:10.358-08:00\",\"email\":\"rating1998@yandex.com\",\"country\":\"Panama\",\"state\":\"Minnesota\",\"city\":\"Eastpointe\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00001-b3706429-b27b-4a77-96f6-e4a0abe939d3-c000.snappy.parquet","partitionValues":{},"size":2180,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Madelene\",\"surname\":\"Berger\",\"birthday\":\"2010-09-05T20:29:23.753-07:00\",\"email\":\"invitations2036@duck.com\",\"country\":\"Aruba\",\"state\":\"New York\",\"city\":\"Wheaton\"},\"maxValues\":{\"name\":\"Madelene\",\"surname\":\"Berger\",\"birthday\":\"2010-09-05T20:29:23.753-07:00\",\"email\":\"invitations2036@duck.com\",\"country\":\"Aruba\",\"state\":\"New York\",\"city\":\"Wheaton\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00002-dcd31c19-e96c-4fbb-837b-9a38059e6ea1-c000.snappy.parquet","partitionValues":{},"size":2223,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Anibal\",\"surname\":\"Mccall\",\"birthday\":\"2002-02-06T19:36:55.543-08:00\",\"email\":\"modifications2025@yahoo.com\",\"country\":\"Anguilla\",\"state\":\"Oklahoma\",\"city\":\"Sahuarita\"},\"maxValues\":{\"name\":\"Anibal\",\"surname\":\"Mccall\",\"birthday\":\"2002-02-06T19:36:55.543-08:00\",\"email\":\"modifications2025@yahoo.com\",\"country\":\"Anguilla\",\"state\":\"Oklahoma\",\"city\":\"Sahuarita\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet","partitionValues":{},"size":2263,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Anibal\",\"surname\":\"Jackson\",\"birthday\":\"1988-01-18T20:20:22.313-08:00\",\"email\":\"medication1855@gmail.com\",\"country\":\"Luxembourg\",\"state\":\"New Hampshire\",\"city\":\"Santa Rosa\"},\"maxValues\":{\"name\":\"Anibal\",\"surname\":\"Jackson\",\"birthday\":\"1988-01-18T20:20:22.313-08:00\",\"email\":\"medication1855@gmail.com\",\"country\":\"Luxembourg\",\"state\":\"New Hampshire\",\"city\":\"Santa Rosa\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00004-adfc1b3d-c6a2-408b-acd9-21754be887de-c000.snappy.parquet","partitionValues":{},"size":2222,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Loree\",\"surname\":\"Tanner\",\"birthday\":\"1991-02-26T01:28:18.632-08:00\",\"email\":\"transport1961@duck.com\",\"country\":\"Anguilla\",\"state\":\"New Hampshire\",\"city\":\"Morristown\"},\"maxValues\":{\"name\":\"Loree\",\"surname\":\"Tanner\",\"birthday\":\"1991-02-26T01:28:18.632-08:00\",\"email\":\"transport1961@duck.com\",\"country\":\"Anguilla\",\"state\":\"New Hampshire\",\"city\":\"Morristown\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00005-3d374415-de65-4ac9-85f0-bf1b8be53719-c000.snappy.parquet","partitionValues":{},"size":2264,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Bernardo\",\"surname\":\"Gamble\",\"birthday\":\"1996-03-06T05:55:34.655-08:00\",\"email\":\"limitations1886@yandex.com\",\"country\":\"Panama\",\"state\":\"North Carolina\",\"city\":\"Greenville\"},\"maxValues\":{\"name\":\"Bernardo\",\"surname\":\"Gamble\",\"birthday\":\"1996-03-06T05:55:34.655-08:00\",\"email\":\"limitations1886@yandex.com\",\"country\":\"Panama\",\"state\":\"North Carolina\",\"city\":\"Greenville\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00006-68588845-198d-4a9e-ad00-a705738e2aff-c000.snappy.parquet","partitionValues":{},"size":2151,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Philip\",\"surname\":\"Cherry\",\"birthday\":\"2010-09-16T08:48:36.668-07:00\",\"email\":\"const1874@outlook.com\",\"country\":\"China\",\"state\":\"Nevada\",\"city\":\"Albert Lea\"},\"maxValues\":{\"name\":\"Philip\",\"surname\":\"Cherry\",\"birthday\":\"2010-09-16T08:48:36.668-07:00\",\"email\":\"const1874@outlook.com\",\"country\":\"China\",\"state\":\"Nevada\",\"city\":\"Albert Lea\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet","partitionValues":{},"size":2207,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Shaneka\",\"surname\":\"Beasley\",\"birthday\":\"2006-08-02T13:08:04.242-07:00\",\"email\":\"msg1894@example.com\",\"country\":\"Barbados\",\"state\":\"Montana\",\"city\":\"Mount Pleasant\"},\"maxValues\":{\"name\":\"Shaneka\",\"surname\":\"Beasley\",\"birthday\":\"2006-08-02T13:08:04.242-07:00\",\"email\":\"msg1894@example.com\",\"country\":\"Barbados\",\"state\":\"Montana\",\"city\":\"Mount Pleasant\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet","partitionValues":{},"size":2221,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Keneth\",\"surname\":\"Vinson\",\"birthday\":\"2002-10-15T08:32:10.678-07:00\",\"email\":\"questions2074@gmail.com\",\"country\":\"Honduras\",\"state\":\"Connecticut\",\"city\":\"Hutchinson\"},\"maxValues\":{\"name\":\"Keneth\",\"surname\":\"Vinson\",\"birthday\":\"2002-10-15T08:32:10.678-07:00\",\"email\":\"questions2074@gmail.com\",\"country\":\"Honduras\",\"state\":\"Connecticut\",\"city\":\"Hutchinson\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} +{"add":{"path":"part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet","partitionValues":{},"size":2204,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Hee\",\"surname\":\"Schwartz\",\"birthday\":\"2006-12-20T01:27:35.236-08:00\",\"email\":\"statements2016@protonmail.com\",\"country\":\"Senegal\",\"state\":\"New Jersey\",\"city\":\"Galt\"},\"maxValues\":{\"name\":\"Hee\",\"surname\":\"Schwartz\",\"birthday\":\"2006-12-20T01:27:35.236-08:00\",\"email\":\"statements2016@protonmail.com\",\"country\":\"Senegal\",\"state\":\"New Jersey\",\"city\":\"Galt\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1a038ef0eb87dea4f2ffb92206868ea1759e0f45 GIT binary patch literal 2187 zcmbVOUx?gf5dSut&E}4!ws`ZAK)@w<;VNtBZhC)Nq#jy|R5^}A%1Y7ivbi>NH@i); zZI8PHDMdtl5FZ2;d^l7JLMwe1UxYptPrAWcI;^hB#^iFz(Wiy%i&2MJD znfb`ltIJIwP=^=c-_P%TBWWPus{}G3huZ=nq+F8l|25|Oj)xW<47mH#^*_fs;E@vv zu!_KwKJ#oh01*_>VCR>g?^oJLDH2j^y!Y3QkGf?-9`_~*QXPf~_~*M%ZXSxL#3ZJy zB~+x0fLCAY`^fMF_bmviW&GMq`ZcSPt!A`*Ulsum2Hf8pK0 zKJQ=`4iTtL3W|s=aVE)WEkZ?#YB2J7orF4h)%C;kUACS;D(LUxEOAcuVC04VIOE1Ci@^>bnv3?-f`e#zP!!Nu3rY%r`%jn0%97 zE1}uAQPPwezi_+XFxLNAiQ2%06tN(zDtZw5Kn6;TTS8G&a{6)59vz~oz0Z*DqsA2W z9_oGY{83yR14=)Wz$eCbxk#y+wm_hg9+P7Q76MqpAe^`y$_lP4-W+)2}KMxWNrY8TjW+He|SdX1s~!CC3lkfu~d~(=wYxIr;=5xY#psX)lhP z(DVnF&At<|E(@9RM|ndJ*N2Y2pa-EpvO~S0uenaI8|Vw~Z0OMfqbYtgI3Qlz2=qq1 z9$W1p0$rVkp0Vl+8(UkA+&jGp(4mNQAcE}HC>TPw@8Gr2A3l8+@m*6eV6_LzA)U-; zCx~ycXLpaTyn}k5-6F~xY1?1!`}K5A>P@rRGF$bj)sfrl)@NrO>zF-vY~HfwW*mFE oWn1+aZU}?J@yhP$=QUEm{L0Xr+p?-sRM#HJM+yq zGw;m|YMh;|0D%u-9DaVZefy{i0)FN}BINjafe@0-ph!NoTCB~w3~eO@|MlkOt1JgS zR*#^0gq~yW0&_tG8B}<5^Xu)yb>tKYDVAS9Ib7>}K*-(IF@j9TU=((L_~yqW0h1U7 zmDGrdWD#&4F2GkWpMRg>LTR@mfQ+G>?uz4EzSHY^)d_RzOsj7%8FsIOt|vf)Cx865 z!xwaPCUVW8>shzov=A2*{`%g|1Ca~C7k-JYmOk&fO9<6Khu?qw<)=d&bmUA#SwPCq z7MbPuy2^;4$Wn4)oCtzK2?@>l=#msU7C4hUtSChuYU12;VPT3Kot!~RD#45rkuw*P z?Bzs?jVhJk3g%4|mCnIzEbV6CTr55mU?mo}ve1adz3jm2sR)+?EfU_oQ1q59cj=mE zFS3pWM+g+h1X=7_9X3WTssSoc6uiF6>+-0RnvUx)wyd=Xl0$!|z0b++^}D{i#_Mt( z_QyVI(RTa=7)I5c*hx zQF=cI9}|5pq8A8*x^x9i_2C^l+=7V= z?d#!Q{I&dpnbB|aaD(VuxRjEoEX3^)CU}uH&{&VhB0ey5G$!#kPkQGNlHM&qmFRmM zY1B_4WzZmdNI96MR6^o+XnFdrfPT9WbjVX&JxR`CbB62v=4N@|9WMcNNFyDHAlVoxgYR@0u7u9=|IZ@aGbsaBd!QWR zE_{E2aD%V0*S$e+Z|{!zy*#eGt$7?r9~EsZz(PP3HgB5Midn7Nr%#$%|+Rk>8)R~D| znqG|qghkNf8TSGtC+P&zN~{oe2WnfG3D z?$~%4BV5FL@t==ge9aUv2OkA2BDAy0A%wCS0sbFrS~Y4%z}K(t-QE5j;Az94wvBl# z;R0U$^xNBQRbb>1%9jSlKK}Y)J3^Od`Vlbg#C`a$=XYN2@R|59q*w?;qAUQ$@ELsT z*|#qo6RQ7C2-pKTHClCR(lV%LwVG~aurhRXj!w5)n$c>2_g*}J?=SuRNMO8op9sC7 zIH+i=466UXbKwzrl@17X-}8JWT~5#;TPljWGN9@b6gXcW05iUrgccE903YM#N$v0 zk_%)ak=8T#c%)q6a3>O%v-n~pF7k=E%HvDE76_M4=G|H9%>3dS(IBevS%J{8Hm5K?Biijy2F1m=q`uA+jdoKC+U-!$_S-Ipbmy z+6kZy=OBLiv>P=n?LR*4*MN2%R#uRQcmZ-v8WIq9$jywpo>7-G1u4%i-^Nc6sf!iK zC0I@#ih+v!2O>+om>_ip)-vu<*!dvn4blNqwdEHSEYXSW#Lmy0Qzw~o}Nm9$1 z(&#LOoz#N$)UKJ13wKt^>6)&Uk+g3U!o6y1u*p~3rl&cr>srI~Xq|dg`kb9tyt!FZ z8C6`*X&atWQYJ0auDirD7aD UxKHlD#ozpg_QQ{mbMRB;Um1TZ2><{9 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..bc5469663aed24f784b760aaeab851c3f7efe4bf GIT binary patch literal 2263 zcmbVO&ubf35Pn*%q>U?^1U#?1um`c!i&NAk(nhvpbcjnSCKzMX&=$3!d0MUQC0a?@ zT_rKLix0-Vw6yflLkp$!5c&gL@U1)-~?Ba%q?PwN+qaaO+-;?9j?dLNebSK%_9MV*u0;HdTj2Aa5FZK#b_4^KV8oI zUFO~T$hS6am%$W){EQ%rZRXlDWJN`pa3kmUJbo{WAgSSa!Df?fMUV`}JFSr*tJ7-- z-WI=?@dFl&P>YrmBoHbnsyPfBLvSDx(PxnSE6 zz?Py1=U|%tmVr5<*N2=*F(K_5y{^}*gO(T=NSw6EV2`K%x=($QfkmR9;2dX_ikQWG z2Zs`p@6i1ey7j}9D(Cru99^N-e|U*f!~qwvAgrqYAoTDYP@*3R!2?PQx-lZ`u+4Uyg`FqeSG3 zFChHeEdvMqcFztBuXERM?SM5|z~n#E+gh;IwY5df54@fgXhm(^v0F`FTl`>K>km7+ z!bgzYqSu7p>|PAShdCN-QD6KI=(b8V5oE^g5-u7G6arm;}zm||9=+Y zy)LfMJ*U+y%+K5A zC2Qf*71Lapv#mp;hc;c9IO;o!!pN zHijhNLmw1^6a=v^5`9ssq&{i|!S|vMLf?ea_RT)|(DuG85{kPBWyj({@o{&=IqhIe$6kjLg_Qg?xOvhma{`vm;t)meWpLmqi zgozXpa2n3StuOxkrO3D{ZzTYVu0r4Pf^NqTtFu$p)6JfJ*|0lpG&~7oaAW7muE-dk zIGY$^TI7tk+H+ei3!w(o;KAO<*VwdQT}q(C2tDttm(N-4Zm{P1QTk(0fv=z3{*=X! z%qRFGh<~@^`)&|+);;AFPLYN9+Oicm1>=qtVscN4oC=JJK0hx-9-v@Mr7SZ=-WM75 za9=ix+`nglN0>BZW|KlxD#1eZ0*XqPppiChH2_{AeeJH^Q{ZW7#(U)?*ERmS&$WB<=rxERa0j7xlsGx$V zO!6Yexqlfk@fN*SK(&6epvon-^ZIw_=|8kYsbFXFm<(nVErjm81eEA|T%J#I>#Y`eVjHQb#fU`}7j3>IWL?z!n|#H{1a|>+<)=<((mD5dAKn=M)!e>~WD^ zKuw)(PF6-=Oxv;=*2}}l%U2S-P3R{A<#dUDCyZcMS7f>4uNqd@!j?4R?li7Ce&BXI zSuhRLs1kYn8ANcUWnd#;={cd{ch-%z6Ix9xwB)~vn_9Thb+kDx2>qTNY87q8bz03p zo4dHFMNN&Wc&RWz*IPxP*TX@*+(HDJIu$Kr)#f(0w=4VB>2N@U+|!{5vPqINgl^lx zQPJ-H|0v@5lQUqphl(Ms*cT^=C) zrb*sP7^4rRFB{uy-x^W~9f7fjJ#2r#U{8ZVS-bUvF~XkqFv^a;tZSE>D}cjM))Ki$ItKFa!Oc$HwnuKz4g9;5X$F3MDxsZsn=|StqjRwOh$Kzy0&a zyE)F8Kj#v4X^R4I0#D+DpYA^{azJGzjxqvK7TaEMwfr9S+K%mp4&%6jfB)g(1DSIe zU5Fiq!wPtex$Cy$F!fDARphjIoQMULAQYLUxL~T12%PD}e12qrBA`J409>z~@K`p6UI`q1zPDw~s42xN(z)8%^B;7lTDyxaCn z3-y8Vfg}+kJIC-S`KEy1M*2#|ohB90FOp?_rJl}3`UL7^Qc?|1`(;S`cL7f${SYQ! zQl~@-&fG0bM7%|A<-n}(RuZCy- zIb?n~6c+v^M;s4VM0=Suz*O&Fv;KDo)3=EI2tMp#81}^wK9BTAVwO)d!BN!nPgX4L9jQ`KS1T5VVL*c_s8e^ z`%BtqR~tZ}1~0-VcfNVQYJh}K1yn*#wj@GGxuoEKyBCDPkPCkJ=k-sYJw)()&yV8` zkU;|jZaw~IzJ-7?A=UbO+dn_}-XY}9)fs|7Ct(^MetzT2;~9|5EUE^xNXjViGMs}S z{R>ES4Q)@ia|Xa9}vPb}(JQtmN_N8aoAs!U8~JJd%mAV}i??@Zj18Rbdr`4D+A75C8_dR>8-ShsN(6lI&@ zrO%yOp>1Tg_z>^|jk2NOZxQry2Kq>WCb9MeP}V!+s_h{ppPu5(%EsoYpop+Hom*@s8-iIEoz6%64Yuw<&m)(1$XWES27@OyjiHH7$xS z+V0TBQM2=@v#D- z4?QYOEFB4;*bozhH0b-d0{X-M??t{>CI}eqk$m8t`1Anz#(H}7Y`q;-`>idaz0S-2 zQa`ToKB+bAM$>NAX1k*xT&pcE`p#)@>GT=LSz7SD`KIT1>+XW*%yk$2b;oHooMzKI UGj~eAhKu~kAIUx*+W3j_FOnq{RsaA1 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..896da6b5ecb1517c23ff9578d6c10db4dfc31b40 GIT binary patch literal 2207 zcmbVOO=uit82)y4c6Y=jSey4_21Zy1Go-K%*`1_GmZh}8LlGk(mK@?A-_A~wwKKDu znOQX?fl@qpu;8I5g&cCMpi;$S@Fcx>?M?Jh5vqky0-hA#`JZf8F@6ZBG$|dJZPx@fV+dH4#AZ5vd?Yq@;)rEw}_n zw|{=f<+xrLL=NZiRJOO-SXi9BY<|e>fn{n&-^QqyU>c4N|NM*3MLqvY6oqM#CFXWm zx9`Qt{P^pm2YG^g+RJ^%XHJmz49vo#?>_%s$bpJ)MW7Q1y4)W+er?6V?3v?B=Qv5M z#Fv&yz)6&l$Qa)f8j~B#jOWKBCW+&;IYc4JOA=EtXRgG#AYqbKMmRBYd~rk(>2lRTbK4K~sT}>7>Z14(|rr1rsEM-WOn!-YG$gsOuSS zl28b}La(Xo2pu;;Dg@A@x;*s50Qx})eO!>CEY0y{Jac+6A^0BssGuGf)GrINSj`XL z={My14=a)CIGj8tgjod-LjM+k67?=lO>$RJ1wzs-YN4q;wq3_6k$zljkM+%8!R=>& z(t|R*N7RF2c9cq7`tUHTbPbK|Y$7rVY128$w0VL%0l~)=xJ1;G96{7aY4uJJJ{hT0 z2+9|aS*puJe;+{qPzgHY9`32+8pDPRk>8;mw6sSDar7N|Mn-kiBhPw2V7(#08lg8! zlv4$ISenFXu8U&d-PG8C;V5e1RBG>=uGi~3VoBF@twF>KPk4CSmWBhoJv4pI?SG)z zrq8;}XW~!AUB%xSm`Y3We0OO0N?qCLnO4_RT5s+u!7!sKeAIJ}J!cagouOs1j)fi= zYSxtmRcY<+@7L4lcqw2&#&jltWP@Y~e$O^>Mc9M?e~NJLWC)n;na&V*>eB;+8|~@C zgY|dT=(qQY_-b7Cmu8@k?X=*_0FID0|b!$tq(kGxnWWCcG>{s9frEDZnv literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..7f14a8a307aa3754a416e6f29e60e4df118fc413 GIT binary patch literal 2221 zcmbVO-)kII6uvtjhm8M36y+M}OaZQfVTiNJyo2?cM2Xe>4bryFE^j={StRKli@;A(j*qNl8s4Nf80( z-~v4U=GXlroYcur62J+}>3Zk{%s zSBA_jQiw_=Sj0MrqSBSVbx?qF>E@mQEH&>IVJS7A72!&1J`iC!GLi6$iv79m(>UrZHxq=&+nCSPg-?-w$ZXS6G#c;U2H8b$nJ(7^Edgu zQot-WKrPxX%OF%xRApPozNsiPWg;m_CLmCOzC{;xshPdh>`BQ)`kJ#HJQwT~ERGPm zFToi7z67&GU&%SMV*0czNSpLQ_Kau;cX-;T5$&Hb?Lk3>vN9up`RZCO7uM;uO~VAShPnOx|%~X1@pKv=I(~pRtz0q<)2I`9@;O9sy=~wi zeRH1Rh!@1*{SuT)5`$^a-~BNWH(99z}yc0x>%jw|IZ@6e{u$__E0&b zo%-Sg@y+();Zb)THu~xgkuRmM{dC8#rhQVa8})|Ks7|bgZo5^Tn{&)5`^?l?(>ycl q*fR~=wAZXz+nipVbJk3=Q8yb6`|R|jvW*x1^B*#ae?b=TkIKK7H808l literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0b2e287c7c8e07d95cbdac5bad4cb444b4210a60 GIT binary patch literal 2204 zcmbVOPiP!v6#sU1c6U;?sg=BsnKG@*V8&F|A-j`5ZCIp=Xb3Tcxa3eNGMRiy7H4+1 zGqZ-61WNGW!9(dm1u24hkXq`=gC2SidaMWAL(%F*@YIv`q6qrto7tVO6cSTdzIoq! zzxR9ZecyW@yZF*l4G5foQ*iV9PaYoALBeej6he-UON5YoE|2RhW2k->#^BTUul`gL zP*1;-pwNN5;cjd?LG&)jpn?v!?tJv~u?7;#gp{ir^6Mx5Iz`A_FiMc?DHwr2zPkMN zaLgqq5fwe*B6$QXz!H4;-8Z+yV9$M=08V1CFmfX1vvw5L$7@sPxPk*M z4SFeuw#EVNd4L&n+P}RI@E6bO@(`%>N)fK)FzI}jbSZ5Np7tVbqOmcUkb!`i^rvQ^ z6WPBU0!qIq!3;6?5{XUerFR;JmSZP#mdz3!f!IIZpAGynLHJX0iO5&xI` E2P%^wI{*Lx literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt new file mode 100644 index 000000000000..128c65d8eebe --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt @@ -0,0 +1,3 @@ +delta-spark==2.2.0 +mimesis==5.0.0 +pyspark==3.2.0 \ No newline at end of file From c4ec7f9491d6fb2c77314327ac69ca12cbd1fa00 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 9 Jan 2024 17:56:43 -0800 Subject: [PATCH 25/65] some fixes --- .../druid/delta/input/DeltaInputRow.java | 73 ++++++++++--------- .../druid/delta/input/DeltaInputSource.java | 2 +- .../delta/input/DeltaInputSourceReader.java | 24 ------ .../apache/druid/delta/input/DeltaSplit.java | 18 +++-- .../apache/druid/delta/input/RowSerde.java | 10 +-- 5 files changed, 53 insertions(+), 74 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 36ddf609fc4d..247fb5de07ac 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -19,7 +19,6 @@ package org.apache.druid.delta.input; -import io.delta.kernel.types.ArrayType; import io.delta.kernel.types.BinaryType; import io.delta.kernel.types.BooleanType; import io.delta.kernel.types.ByteType; @@ -30,7 +29,6 @@ import io.delta.kernel.types.FloatType; import io.delta.kernel.types.IntegerType; import io.delta.kernel.types.LongType; -import io.delta.kernel.types.MapType; import io.delta.kernel.types.ShortType; import io.delta.kernel.types.StringType; import io.delta.kernel.types.StructField; @@ -42,6 +40,7 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.error.InvalidInput; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -58,9 +57,10 @@ public class DeltaInputRow implements InputRow private final io.delta.kernel.data.Row row; private final StructType schema; private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); - private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TIMEZONE HANDLING????? private final InputRow delegateRow; + private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TODO: Timezone handling? + public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) { this.row = row; @@ -76,7 +76,6 @@ public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema theMap.put(fieldName, _getRaw(fieldName)); } delegateRow = MapInputRowParser.parse(inputRowSchema, theMap); - } @Override @@ -110,25 +109,6 @@ public Object getRaw(String dimension) return delegateRow.getRaw(dimension); } - @Nullable - public Object _getRaw(String dimension) - { - StructField field = schema.get(dimension); - if (field == null) { - return null; - } else if (field.isMetadataColumn()) { - return null; - } - - - int ordinal = fieldNameToOrdinal.getInt(dimension); - if (ordinal < 0) { - return null; - } - return getValue(field.getDataType(), row, ordinal); - - } - @Nullable @Override public Number getMetric(String metric) @@ -142,6 +122,37 @@ public int compareTo(Row o) return this.getTimestamp().compareTo(o.getTimestamp()); } + @Override + public String toString() + { + return "DeltaInputRow{" + + "row=" + row + + ", schema=" + schema + + ", fieldNameToOrdinal=" + fieldNameToOrdinal + + ", delegateRow=" + delegateRow + + '}'; + } + + public Map getRawRowAsMap() + { + return RowSerde.convertRowToJsonObject(row); + } + + @Nullable + private Object _getRaw(String dimension) + { + StructField field = schema.get(dimension); + if (field == null || field.isMetadataColumn()) { + return null; + } + + int ordinal = fieldNameToOrdinal.getInt(dimension); + if (ordinal < 0) { + return null; + } + return getValue(field.getDataType(), row, ordinal); + } + @Nullable private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataRow, int columnOrdinal) { @@ -167,7 +178,8 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR LocalDateTime dateTime = LocalDateTime.ofEpochSecond( microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC); + ZoneOffset.UTC + ); return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); } else if (dataType instanceof FloatType) { return dataRow.getFloat(columnOrdinal); @@ -179,19 +191,8 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR return new String(dataRow.getBinary(columnOrdinal)); } else if (dataType instanceof DecimalType) { return dataRow.getDecimal(columnOrdinal).longValue(); - } else if (dataType instanceof StructType) { - return "TODO: struct value"; - } else if (dataType instanceof ArrayType) { - return "TODO: list value"; - } else if (dataType instanceof MapType) { - return "TODO: map value"; } else { - throw new UnsupportedOperationException("unsupported data type: " + dataType); + throw InvalidInput.exception("Unsupported data type[%s]", dataType); } } - - public Map getRawRowAsMap() - { - return RowSerde.convertRowToJsonObject(row); - } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 320565316abe..d6fd8ff79557 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -110,7 +110,7 @@ public InputSourceReader reader( if (deltaSplit != null) { scanState = deserialize(tableClient, deltaSplit.getStateRow()); - scanRowList = deltaSplit.getFile() + scanRowList = deltaSplit.getFiles() .stream() .map(row -> deserialize(tableClient, row)) .collect(Collectors.toList()); diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java index 9132b2638dd6..659369ba1125 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -21,40 +21,18 @@ import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; -import io.delta.kernel.types.ArrayType; -import io.delta.kernel.types.BinaryType; -import io.delta.kernel.types.BooleanType; -import io.delta.kernel.types.ByteType; -import io.delta.kernel.types.DataType; -import io.delta.kernel.types.DateType; -import io.delta.kernel.types.DecimalType; -import io.delta.kernel.types.DoubleType; -import io.delta.kernel.types.FloatType; -import io.delta.kernel.types.IntegerType; -import io.delta.kernel.types.LongType; -import io.delta.kernel.types.MapType; -import io.delta.kernel.types.ShortType; -import io.delta.kernel.types.StringType; -import io.delta.kernel.types.StructType; -import io.delta.kernel.types.TimestampType; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputStats; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneOffset; import java.util.NoSuchElementException; public class DeltaInputSourceReader implements InputSourceReader { - private static final Logger log = new Logger(DeltaInputSourceReader.class); - private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; private final InputRowSchema inputRowSchema; @@ -142,8 +120,6 @@ public InputRow next() } Row dataRow = currentBatch.next(); - System.out.println("Datarow" + dataRow); - // TODO: construct schema? remove this after debugging return new DeltaInputRow(dataRow, inputRowSchema); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java index a5272fee82ce..36b8bf038bc0 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java @@ -27,13 +27,16 @@ public class DeltaSplit { private final String stateRow; - private final List fileRows; + private final List files; @JsonCreator - public DeltaSplit(@JsonProperty("state") String stateRow, @JsonProperty("file") List fileRows) + public DeltaSplit( + @JsonProperty("state") final String stateRow, + @JsonProperty("file") final List files + ) { this.stateRow = stateRow; - this.fileRows = fileRows; + this.files = files; } @JsonProperty("state") @@ -42,11 +45,10 @@ public String getStateRow() return stateRow; } - // TODO: fileRows? - @JsonProperty("file") - public List getFile() + @JsonProperty("files") + public List getFiles() { - return fileRows; + return files; } @Override @@ -54,7 +56,7 @@ public String toString() { return "DeltaSplit{" + "stateRow=" + stateRow + - ", file=" + fileRows + + ", files=" + files + "}"; } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index e42cc1bb7b57..013b5a0e8a78 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -43,6 +43,7 @@ import io.delta.kernel.types.StructField; import io.delta.kernel.types.StructType; import io.delta.kernel.types.TimestampType; +import org.apache.druid.error.InvalidInput; import java.io.UncheckedIOException; import java.util.HashMap; @@ -72,8 +73,7 @@ public static String serializeRowToJson(Row row) return OBJECT_MAPPER.writeValueAsString(rowWithSchema); } catch (JsonProcessingException e) { - // todo: throw druid exception - throw new UncheckedIOException(e); + throw new RuntimeException(e); } } @@ -89,8 +89,8 @@ public static Row deserializeRowFromJson(TableClient tableClient, String jsonRow TableSchemaSerDe.fromJson(tableClient.getJsonHandler(), schemaNode.asText()); return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema); } - catch (JsonProcessingException ex) { - throw new UncheckedIOException(ex); // todo: ise? + catch (JsonProcessingException e) { + throw new RuntimeException(e); } } @@ -137,7 +137,7 @@ public static Map convertRowToJsonObject(Row row) Row subRow = row.getStruct(fieldId); value = convertRowToJsonObject(subRow); } else { - throw new UnsupportedOperationException("NYI"); + throw InvalidInput.exception("Unsupported fieldType[%s] for fieldId[%s]", fieldType, fieldId); } rowObject.put(name, value); From 3b0b11ff940c2e1481803fb11989713e3f277e1e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 9 Jan 2024 18:00:05 -0800 Subject: [PATCH 26/65] move to the correct package --- .../org/apache/druid/delta/input/DeltaInputSourceTest.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename extensions-contrib/druid-deltalake-extensions/src/test/{org => java}/org/apache/druid/delta/input/DeltaInputSourceTest.java (100%) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java similarity index 100% rename from extensions-contrib/druid-deltalake-extensions/src/test/org/org/apache/druid/delta/input/DeltaInputSourceTest.java rename to extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java From dab76b12ac98b9cc6c5bc503ea67627cf10a627a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 08:40:50 -0800 Subject: [PATCH 27/65] More tests --- .../druid/delta/input/DeltaInputSource.java | 11 +- .../apache/druid/delta/input/RowSerde.java | 3 + .../druid/delta/input/DeltaInputRowTest.java | 179 ++++++++++++++++++ .../druid/delta/input/RowSerdeTest.java | 56 ++++++ 4 files changed, 239 insertions(+), 10 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index d6fd8ff79557..9e4f77703a4b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -43,7 +43,6 @@ import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.error.InvalidInput; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; @@ -51,7 +50,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -74,15 +72,12 @@ public class DeltaInputSource implements SplittableInputSource @Nullable private final DeltaSplit deltaSplit; - private static final Logger log = new Logger(DeltaInputSource.class); - @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") String tablePath, @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit ) { - log.info("CONST Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); this.tablePath = Preconditions.checkNotNull(tablePath, "tablePath cannot be null"); this.deltaSplit = deltaSplit; } @@ -101,7 +96,6 @@ public InputSourceReader reader( File temporaryDirectory ) { - log.info("READER Delta input source reader for inputRowSchema[%s], tablePath[%s] and split[%s]", inputRowSchema, tablePath, deltaSplit); Configuration hadoopConf = new Configuration(); TableClient tableClient = DefaultTableClient.create(hadoopConf); try { @@ -117,7 +111,6 @@ public InputSourceReader reader( } else { Table table = Table.forPath(tableClient, tablePath); Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); - StructType schema = latestSnapshot.getSchema(tableClient); Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); scanState = scan.getScanState(tableClient); @@ -154,10 +147,9 @@ public Stream> createSplits(InputFormat inputFormat, @Nul { if (null != deltaSplit) { // can't split a split - return Collections.singletonList(new InputSplit<>(deltaSplit)).stream(); + return Stream.of(new InputSplit<>(deltaSplit)); } - log.info("CREATE SPLITS Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); TableClient tableClient = DefaultTableClient.create(new Configuration()); final Snapshot latestSnapshot; final Table table; @@ -200,7 +192,6 @@ public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec sp @Override public InputSource withSplit(InputSplit split) { - log.info("WITH SPLIT Delta input source reader for tablePath[%s] and split[%s]", tablePath, deltaSplit); return new DeltaInputSource( tablePath, split.get() diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 013b5a0e8a78..6c9223f58126 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -51,6 +51,9 @@ /** * Utility class to serialize and deserialize {@link Row} object. + * Code borrowed from + * RowSerde.java. + * */ public class RowSerde { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java new file mode 100644 index 000000000000..4684961131dc --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -0,0 +1,179 @@ +package org.apache.druid.delta.input; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.delta.kernel.Scan; +import io.delta.kernel.ScanBuilder; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.TableNotFoundException; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.types.StructType; +import io.delta.kernel.utils.CloseableIterator; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class DeltaInputRowTest +{ + private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; + private static final List DIMENSIONS = ImmutableList.of("city", "state", "surname", "email", "country"); + private static final List> EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "birthday", 1049418130358332L, + "country", "Panama", + "city", "Eastpointe", + "surname", "Francis", + "name", "Darren", + "state", "Minnesota", + "email", "rating1998@yandex.com" + ), + ImmutableMap.of( + "birthday", 1283743763753323L, + "country", "Aruba", + "city", "Wheaton", + "surname", "Berger", + "name", "Madelene", + "state", "New York", + "email", "invitations2036@duck.com" + ), + ImmutableMap.of( + "birthday", 1013053015543401L, + "country", "Anguilla", + "city", "Sahuarita", + "surname", "Mccall", + "name", "Anibal", + "state", "Oklahoma", + "email", "modifications2025@yahoo.com" + ), + ImmutableMap.of( + "birthday", 569564422313618L, + "country", "Luxembourg", + "city", "Santa Rosa", + "surname", "Jackson", + "name", "Anibal", + "state", "New Hampshire", + "email", "medication1855@gmail.com" + ), + ImmutableMap.of( + "birthday", 667560498632507L, + "country", "Anguilla", + "city", "Morristown", + "surname", "Tanner", + "name", "Loree", + "state", "New Hampshire", + "email", "transport1961@duck.com" + ), + ImmutableMap.of( + "birthday", 826120534655077L, + "country", "Panama", + "city", "Greenville", + "surname", "Gamble", + "name", "Bernardo", + "state", "North Carolina", + "email", "limitations1886@yandex.com" + ), + ImmutableMap.of( + "birthday", 1284652116668688L, + "country", "China", + "city", "Albert Lea", + "surname", "Cherry", + "name", "Philip", + "state", "Nevada", + "email", "const1874@outlook.com" + ), + ImmutableMap.of( + "birthday", 1154549284242934L, + "country", "Barbados", + "city", "Mount Pleasant", + "surname", "Beasley", + "name", "Shaneka", + "state", "Montana", + "email", "msg1894@example.com" + ), + ImmutableMap.of( + "birthday", 1034695930678172L, + "country", "Honduras", + "city", "Hutchinson", + "surname", "Vinson", + "name", "Keneth", + "state", "Connecticut", + "email", "questions2074@gmail.com" + ), + ImmutableMap.of( + "birthday", 1166606855236945L, + "country", "Senegal", + "city", "Galt", + "surname", "Schwartz", + "name", "Hee", + "state", "New Jersey", + "email", "statements2016@protonmail.com" + ) + ) + ); + + @Test + public void testSerializeDeserializeRoundtrip() throws TableNotFoundException, IOException + { + TableClient tableClient = DefaultTableClient.create(new Configuration()); + Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); + Snapshot snapshot = table.getLatestSnapshot(tableClient); + StructType readSchema = snapshot.getSchema(tableClient); + ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) + .withReadSchema(tableClient, readSchema); + Scan scan = scanBuilder.build(); + Row scanState = scan.getScanState(tableClient); + + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("birthday", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), + ColumnsFilter.all() + ); + + CloseableIterator scanFileIter = scan.getScanFiles(tableClient); + int totalRecordCount = 0; + while (scanFileIter.hasNext()) { + try (CloseableIterator data = + Scan.readData( + tableClient, + scanState, + scanFileIter.next().getRows(), + Optional.empty() + )) { + while (data.hasNext()) { + FilteredColumnarBatch dataReadResult = data.next(); + Row next = dataReadResult.getRows().next(); + DeltaInputRow deltaInputRow = new DeltaInputRow( + next, + schema + ); + Assert.assertNotNull(deltaInputRow); + Assert.assertEquals(DIMENSIONS, deltaInputRow.getDimensions()); + + Map expectedRow = EXPECTED_ROWS.get(totalRecordCount); + Assert.assertEquals(expectedRow, deltaInputRow.getRawRowAsMap()); + + for (String dimension : DIMENSIONS) { + Assert.assertEquals(expectedRow.get(dimension), deltaInputRow.getDimension(dimension).get(0)); + } + totalRecordCount += 1; + } + } + } + Assert.assertEquals(EXPECTED_ROWS.size(), totalRecordCount); + } +} \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java new file mode 100644 index 000000000000..f4215465c60b --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -0,0 +1,56 @@ +/* + * 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.druid.delta.input; + +import io.delta.kernel.ScanBuilder; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.TableNotFoundException; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.client.DefaultTableClient; +import io.delta.kernel.types.StructType; +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; +import org.junit.Test; + +public class RowSerdeTest +{ + private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; + + @Test + public void testSerializeDeserializeRoundtrip() throws TableNotFoundException + { + TableClient tableClient = DefaultTableClient.create(new Configuration()); + Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); + Snapshot snapshot = table.getLatestSnapshot(tableClient); + StructType readSchema = snapshot.getSchema(tableClient); + ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) + .withReadSchema(tableClient, readSchema); + Row scanState = scanBuilder.build().getScanState(tableClient); + + String rowJson = RowSerde.serializeRowToJson(scanState); + Row row = RowSerde.deserializeRowFromJson(tableClient, rowJson); + + Assert.assertEquals(scanState.getSchema(), row.getSchema()); + } + + +} \ No newline at end of file From d81efe78f6054d44722fd19cdcd19dab1c9b9447 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 12:39:04 -0800 Subject: [PATCH 28/65] Test cleanup --- .../druid/delta/input/DeltaInputRowTest.java | 158 ++++-------------- .../delta/input/DeltaInputSourceTest.java | 92 ++++------ .../druid/delta/input/DeltaTestUtil.java | 154 +++++++++++++++++ .../druid/delta/input/RowSerdeTest.java | 22 +-- 4 files changed, 219 insertions(+), 207 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 4684961131dc..d028bf633518 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -1,148 +1,46 @@ +/* + * 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.druid.delta.input; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import io.delta.kernel.Scan; -import io.delta.kernel.ScanBuilder; -import io.delta.kernel.Snapshot; -import io.delta.kernel.Table; import io.delta.kernel.TableNotFoundException; import io.delta.kernel.client.TableClient; import io.delta.kernel.data.FilteredColumnarBatch; import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; -import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; -import org.apache.druid.data.input.ColumnsFilter; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.hadoop.conf.Configuration; import org.junit.Assert; import org.junit.Test; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import java.util.Optional; public class DeltaInputRowTest { - private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; - private static final List DIMENSIONS = ImmutableList.of("city", "state", "surname", "email", "country"); - private static final List> EXPECTED_ROWS = new ArrayList<>( - ImmutableList.of( - ImmutableMap.of( - "birthday", 1049418130358332L, - "country", "Panama", - "city", "Eastpointe", - "surname", "Francis", - "name", "Darren", - "state", "Minnesota", - "email", "rating1998@yandex.com" - ), - ImmutableMap.of( - "birthday", 1283743763753323L, - "country", "Aruba", - "city", "Wheaton", - "surname", "Berger", - "name", "Madelene", - "state", "New York", - "email", "invitations2036@duck.com" - ), - ImmutableMap.of( - "birthday", 1013053015543401L, - "country", "Anguilla", - "city", "Sahuarita", - "surname", "Mccall", - "name", "Anibal", - "state", "Oklahoma", - "email", "modifications2025@yahoo.com" - ), - ImmutableMap.of( - "birthday", 569564422313618L, - "country", "Luxembourg", - "city", "Santa Rosa", - "surname", "Jackson", - "name", "Anibal", - "state", "New Hampshire", - "email", "medication1855@gmail.com" - ), - ImmutableMap.of( - "birthday", 667560498632507L, - "country", "Anguilla", - "city", "Morristown", - "surname", "Tanner", - "name", "Loree", - "state", "New Hampshire", - "email", "transport1961@duck.com" - ), - ImmutableMap.of( - "birthday", 826120534655077L, - "country", "Panama", - "city", "Greenville", - "surname", "Gamble", - "name", "Bernardo", - "state", "North Carolina", - "email", "limitations1886@yandex.com" - ), - ImmutableMap.of( - "birthday", 1284652116668688L, - "country", "China", - "city", "Albert Lea", - "surname", "Cherry", - "name", "Philip", - "state", "Nevada", - "email", "const1874@outlook.com" - ), - ImmutableMap.of( - "birthday", 1154549284242934L, - "country", "Barbados", - "city", "Mount Pleasant", - "surname", "Beasley", - "name", "Shaneka", - "state", "Montana", - "email", "msg1894@example.com" - ), - ImmutableMap.of( - "birthday", 1034695930678172L, - "country", "Honduras", - "city", "Hutchinson", - "surname", "Vinson", - "name", "Keneth", - "state", "Connecticut", - "email", "questions2074@gmail.com" - ), - ImmutableMap.of( - "birthday", 1166606855236945L, - "country", "Senegal", - "city", "Galt", - "surname", "Schwartz", - "name", "Hee", - "state", "New Jersey", - "email", "statements2016@protonmail.com" - ) - ) - ); - @Test - public void testSerializeDeserializeRoundtrip() throws TableNotFoundException, IOException + public void testDeltaInputRow() throws TableNotFoundException, IOException { - TableClient tableClient = DefaultTableClient.create(new Configuration()); - Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); - Snapshot snapshot = table.getLatestSnapshot(tableClient); - StructType readSchema = snapshot.getSchema(tableClient); - ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) - .withReadSchema(tableClient, readSchema); - Scan scan = scanBuilder.build(); - Row scanState = scan.getScanState(tableClient); - - InputRowSchema schema = new InputRowSchema( - new TimestampSpec("birthday", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), - ColumnsFilter.all() - ); + final TableClient tableClient = DefaultTableClient.create(new Configuration()); + final Scan scan = DeltaTestUtil.getScan(tableClient); CloseableIterator scanFileIter = scan.getScanFiles(tableClient); int totalRecordCount = 0; @@ -150,7 +48,7 @@ public void testSerializeDeserializeRoundtrip() throws TableNotFoundException, I try (CloseableIterator data = Scan.readData( tableClient, - scanState, + scan.getScanState(tableClient), scanFileIter.next().getRows(), Optional.empty() )) { @@ -159,21 +57,21 @@ public void testSerializeDeserializeRoundtrip() throws TableNotFoundException, I Row next = dataReadResult.getRows().next(); DeltaInputRow deltaInputRow = new DeltaInputRow( next, - schema + DeltaTestUtil.SCHEMA ); Assert.assertNotNull(deltaInputRow); - Assert.assertEquals(DIMENSIONS, deltaInputRow.getDimensions()); + Assert.assertEquals(DeltaTestUtil.DIMENSIONS, deltaInputRow.getDimensions()); - Map expectedRow = EXPECTED_ROWS.get(totalRecordCount); + Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(totalRecordCount); Assert.assertEquals(expectedRow, deltaInputRow.getRawRowAsMap()); - for (String dimension : DIMENSIONS) { + for (String dimension : DeltaTestUtil.DIMENSIONS) { Assert.assertEquals(expectedRow.get(dimension), deltaInputRow.getDimension(dimension).get(0)); } totalRecordCount += 1; } } } - Assert.assertEquals(EXPECTED_ROWS.size(), totalRecordCount); + Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), totalRecordCount); } } \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index a3c3f747d62f..4c9277fcb12e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -19,18 +19,10 @@ package org.apache.druid.delta.input; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.junit.Assert; import org.junit.Test; @@ -38,86 +30,61 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; public class DeltaInputSourceTest { - private static final ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); - private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; - @Test public void testReadDeltaLakeFilesSample() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); Assert.assertNotNull(deltaInputSource); - InputRowSchema schema = new InputRowSchema( - new TimestampSpec("no_such_col!!", "auto", DateTimes.of("1970")), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), - ColumnsFilter.all() - ); - - InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); Assert.assertNotNull(inputSourceReader); - List inputRowListPlusRawValues = sampleAllRows(inputSourceReader); - Assert.assertNotNull(inputRowListPlusRawValues); - - Assert.assertEquals(10, inputRowListPlusRawValues.size()); - - final String expectedJson = "{\n" - + " \"birthday\" : 1049418130358332,\n" - + " \"country\" : \"Panama\",\n" - + " \"city\" : \"Eastpointe\",\n" - + " \"surname\" : \"Francis\",\n" - + " \"name\" : \"Darren\",\n" - + " \"state\" : \"Minnesota\",\n" - + " \"email\" : \"rating1998@yandex.com\"\n" - + "}"; - Assert.assertEquals( - expectedJson, - DEFAULT_JSON_WRITER.writeValueAsString(inputRowListPlusRawValues.get(0).getRawValues()) - ); + List actualSampledRows = sampleAllRows(inputSourceReader); + Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualSampledRows.size()); + + for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { + Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); + Assert.assertEquals( + expectedRow, + actualSampledRows.get(idx).getRawValues() + ); + } } @Test public void testReadDeltaLakeFilesRead() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); Assert.assertNotNull(deltaInputSource); - InputRowSchema schema = new InputRowSchema( - new TimestampSpec("birthday", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("city", "state"))), - ColumnsFilter.all() - ); - - InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); Assert.assertNotNull(inputSourceReader); - List rows = readAllRows(inputSourceReader); - Assert.assertNotNull(rows); + List actualReadRows = readAllRows(inputSourceReader); + Assert.assertNotNull(actualReadRows); - Assert.assertEquals(10, rows.size()); + Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualReadRows.size()); - final InputRow firstRow = rows.get(0); - Assert.assertNotNull(firstRow); - Assert.assertEquals("2003-04-04T01:02:10.000Z", firstRow.getTimestamp().toString()); - Assert.assertEquals("Panama", firstRow.getDimension("country").get(0)); - Assert.assertEquals("Eastpointe", firstRow.getDimension("city").get(0)); - Assert.assertEquals("Francis", firstRow.getDimension("surname").get(0)); - Assert.assertEquals("Darren", firstRow.getDimension("name").get(0)); - Assert.assertEquals("Minnesota", firstRow.getDimension("state").get(0)); - Assert.assertEquals("rating1998@yandex.com", firstRow.getDimension("email").get(0)); + for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { + Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); + InputRow actualInputRow = actualReadRows.get(idx); + for (String key : expectedRow.keySet()) { + Assert.assertEquals(expectedRow.get(key), actualInputRow.getDimension(key)); + } + } } - @Test public void testReadDeltaLakeNoSplits() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); Assert.assertNotNull(deltaInputSource); Stream> splits = deltaInputSource.createSplits(null, null); @@ -128,14 +95,17 @@ public void testReadDeltaLakeNoSplits() throws IOException @Test public void testReadDeltaLakeWithSplits() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); Assert.assertNotNull(deltaInputSource); Stream> splits1 = deltaInputSource.createSplits(null, null); List> splitsCollect1 = splits1.collect(Collectors.toList()); Assert.assertEquals(1, splitsCollect1.size()); - DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource(DELTA_TABLE_PATH, splitsCollect1.get(0).get()); + DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( + DeltaTestUtil.DELTA_TABLE_PATH, + splitsCollect1.get(0).get() + ); Assert.assertNotNull(deltaInputSourceWithSplit); Stream> splits2 = deltaInputSourceWithSplit.createSplits(null, null); List> splitsCollect2 = splits2.collect(Collectors.toList()); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java new file mode 100644 index 000000000000..4c2483bac6ae --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -0,0 +1,154 @@ +/* + * 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.druid.delta.input; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.delta.kernel.Scan; +import io.delta.kernel.ScanBuilder; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.TableNotFoundException; +import io.delta.kernel.client.TableClient; +import io.delta.kernel.types.StructType; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class DeltaTestUtil +{ + public static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; + public static final List DIMENSIONS = ImmutableList.of("city", "state", "surname", "email", "country"); + public static final List> EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "birthday", 1049418130358332L, + "country", "Panama", + "city", "Eastpointe", + "surname", "Francis", + "name", "Darren", + "state", "Minnesota", + "email", "rating1998@yandex.com" + ), + ImmutableMap.of( + "birthday", 1283743763753323L, + "country", "Aruba", + "city", "Wheaton", + "surname", "Berger", + "name", "Madelene", + "state", "New York", + "email", "invitations2036@duck.com" + ), + ImmutableMap.of( + "birthday", 1013053015543401L, + "country", "Anguilla", + "city", "Sahuarita", + "surname", "Mccall", + "name", "Anibal", + "state", "Oklahoma", + "email", "modifications2025@yahoo.com" + ), + ImmutableMap.of( + "birthday", 569564422313618L, + "country", "Luxembourg", + "city", "Santa Rosa", + "surname", "Jackson", + "name", "Anibal", + "state", "New Hampshire", + "email", "medication1855@gmail.com" + ), + ImmutableMap.of( + "birthday", 667560498632507L, + "country", "Anguilla", + "city", "Morristown", + "surname", "Tanner", + "name", "Loree", + "state", "New Hampshire", + "email", "transport1961@duck.com" + ), + ImmutableMap.of( + "birthday", 826120534655077L, + "country", "Panama", + "city", "Greenville", + "surname", "Gamble", + "name", "Bernardo", + "state", "North Carolina", + "email", "limitations1886@yandex.com" + ), + ImmutableMap.of( + "birthday", 1284652116668688L, + "country", "China", + "city", "Albert Lea", + "surname", "Cherry", + "name", "Philip", + "state", "Nevada", + "email", "const1874@outlook.com" + ), + ImmutableMap.of( + "birthday", 1154549284242934L, + "country", "Barbados", + "city", "Mount Pleasant", + "surname", "Beasley", + "name", "Shaneka", + "state", "Montana", + "email", "msg1894@example.com" + ), + ImmutableMap.of( + "birthday", 1034695930678172L, + "country", "Honduras", + "city", "Hutchinson", + "surname", "Vinson", + "name", "Keneth", + "state", "Connecticut", + "email", "questions2074@gmail.com" + ), + ImmutableMap.of( + "birthday", 1166606855236945L, + "country", "Senegal", + "city", "Galt", + "surname", "Schwartz", + "name", "Hee", + "state", "New Jersey", + "email", "statements2016@protonmail.com" + ) + ) + ); + + public static final InputRowSchema SCHEMA = new InputRowSchema( + new TimestampSpec("birthday", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), + ColumnsFilter.all() + ); + + public static Scan getScan(final TableClient tableClient) throws TableNotFoundException + { + final Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); + final Snapshot snapshot = table.getLatestSnapshot(tableClient); + final StructType readSchema = snapshot.getSchema(tableClient); + final ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) + .withReadSchema(tableClient, readSchema); + return scanBuilder.build(); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index f4215465c60b..2ec7cc467ef3 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -19,35 +19,25 @@ package org.apache.druid.delta.input; -import io.delta.kernel.ScanBuilder; -import io.delta.kernel.Snapshot; -import io.delta.kernel.Table; +import io.delta.kernel.Scan; import io.delta.kernel.TableNotFoundException; -import io.delta.kernel.client.TableClient; import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; -import io.delta.kernel.types.StructType; import org.apache.hadoop.conf.Configuration; import org.junit.Assert; import org.junit.Test; public class RowSerdeTest { - private static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; - @Test public void testSerializeDeserializeRoundtrip() throws TableNotFoundException { - TableClient tableClient = DefaultTableClient.create(new Configuration()); - Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); - Snapshot snapshot = table.getLatestSnapshot(tableClient); - StructType readSchema = snapshot.getSchema(tableClient); - ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) - .withReadSchema(tableClient, readSchema); - Row scanState = scanBuilder.build().getScanState(tableClient); + final DefaultTableClient tableClient = DefaultTableClient.create(new Configuration()); + final Scan scan = DeltaTestUtil.getScan(tableClient); + final Row scanState = scan.getScanState(tableClient); - String rowJson = RowSerde.serializeRowToJson(scanState); - Row row = RowSerde.deserializeRowFromJson(tableClient, rowJson); + final String rowJson = RowSerde.serializeRowToJson(scanState); + final Row row = RowSerde.deserializeRowFromJson(tableClient, rowJson); Assert.assertEquals(scanState.getSchema(), row.getSchema()); } From ff9e0ba321b557c3bf34d20fc10d8c4264272a9f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 14:49:02 -0800 Subject: [PATCH 29/65] TODOs --- .../test/java/org/apache/druid/delta/input/DeltaTestUtil.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 4c2483bac6ae..782b3a7c19a3 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -37,6 +37,9 @@ import java.util.List; import java.util.Map; +/** + * TODO: + */ public class DeltaTestUtil { public static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; From 633d32dcc670f0f2ead57366ebae5ad0fa86d764 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 15:52:17 -0800 Subject: [PATCH 30/65] Test updates --- .../druid/delta/input/DeltaInputRowTest.java | 2 ++ .../druid/delta/input/DeltaInputSourceTest.java | 15 +++++++++++---- .../apache/druid/delta/input/RowSerdeTest.java | 1 - 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index d028bf633518..c0c40157c6e6 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -68,6 +68,8 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException for (String dimension : DeltaTestUtil.DIMENSIONS) { Assert.assertEquals(expectedRow.get(dimension), deltaInputRow.getDimension(dimension).get(0)); } + long expectedMillis = ((Long) expectedRow.get(DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn()) / 1_000_000) * 1000; + Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); totalRecordCount += 1; } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 4c9277fcb12e..1d8e7cff31f2 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -50,10 +50,14 @@ public void testReadDeltaLakeFilesSample() throws IOException for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); + InputRowListPlusRawValues actualSampledRow = actualSampledRows.get(idx); + Assert.assertNull(actualSampledRow.getParseException()); Assert.assertEquals( expectedRow, - actualSampledRows.get(idx).getRawValues() + actualSampledRow.getRawValues() ); + Assert.assertNotNull(actualSampledRow.getRawValuesList()); + Assert.assertEquals(expectedRow, actualSampledRow.getRawValuesList().get(0)); } } @@ -67,8 +71,6 @@ public void testReadDeltaLakeFilesRead() throws IOException Assert.assertNotNull(inputSourceReader); List actualReadRows = readAllRows(inputSourceReader); - Assert.assertNotNull(actualReadRows); - Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualReadRows.size()); @@ -76,7 +78,12 @@ public void testReadDeltaLakeFilesRead() throws IOException Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); InputRow actualInputRow = actualReadRows.get(idx); for (String key : expectedRow.keySet()) { - Assert.assertEquals(expectedRow.get(key), actualInputRow.getDimension(key)); + if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + final long expectedMillis = ((Long) expectedRow.get(key) / 1_000_000) * 1000; + Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); + } else { + Assert.assertEquals(expectedRow.get(key), actualInputRow.getDimension(key).get(0)); + } } } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index 2ec7cc467ef3..a94a9b54f191 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -42,5 +42,4 @@ public void testSerializeDeserializeRoundtrip() throws TableNotFoundException Assert.assertEquals(scanState.getSchema(), row.getSchema()); } - } \ No newline at end of file From 8dd4ccd02cd381774e5ca1c78814fcee4a3152ed Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 21:51:52 -0800 Subject: [PATCH 31/65] requirements and javadocs --- .../druid/delta/input/DeltaInputRow.java | 5 +- .../druid/delta/input/DeltaInputSource.java | 12 ++-- .../delta/input/DeltaInputSourceReader.java | 5 ++ .../apache/druid/delta/input/DeltaSplit.java | 11 ++- .../druid/delta/input/DeltaTestUtil.java | 3 +- .../src/test/resources/README.md | 72 ++++++++++++------- .../src/test/resources/create_delta_table.py | 39 +++++----- .../src/test/resources/requirements.txt | 3 +- 8 files changed, 97 insertions(+), 53 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 247fb5de07ac..3726a870c277 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -52,6 +52,9 @@ import java.util.List; import java.util.Map; +/** + * Encodes the row and schema information from the Delta Lake. + */ public class DeltaInputRow implements InputRow { private final io.delta.kernel.data.Row row; @@ -59,7 +62,7 @@ public class DeltaInputRow implements InputRow private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); private final InputRow delegateRow; - private static final ZoneId ZONE_ID = ZoneId.systemDefault(); // TODO: Timezone handling? + private static final ZoneId ZONE_ID = ZoneId.systemDefault(); public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) { diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 9e4f77703a4b..0025e91d8976 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -57,9 +57,9 @@ import java.util.stream.Stream; /** - * Inputsource to ingest data managed by the Delta Lake table format. - * This inputsource talks to the configured catalog, executes any configured filters and retrieves the data file paths upto the latest snapshot associated with the iceberg table. - * The data file paths are then provided to a native {@link SplittableInputSource} implementation depending on the warehouse source defined. + * Input source to ingest data from a Delta Lake. + * This input source reads the latest snapshot from a Delta table specified by {@code tablePath} parameter. + * Note: the kernel table API only supports reading from the latest snapshot. */ public class DeltaInputSource implements SplittableInputSource { @@ -78,7 +78,10 @@ public DeltaInputSource( @JsonProperty("deltaSplit") @Nullable DeltaSplit deltaSplit ) { - this.tablePath = Preconditions.checkNotNull(tablePath, "tablePath cannot be null"); + if (tablePath == null) { + throw InvalidInput.exception("tablePath cannot be null"); + } + this.tablePath = tablePath; this.deltaSplit = deltaSplit; } @@ -179,7 +182,6 @@ public Stream> createSplits(InputFormat inputFormat, @Nul } ); - // TODO: account for the split spec as well -- getSplitHintSpecOrDefault(splitHintSpec).split() return Streams.sequentialStreamFrom(deltaSplitIterator).map(InputSplit::new); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java index 659369ba1125..21260d029772 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -31,6 +31,11 @@ import java.io.IOException; import java.util.NoSuchElementException; +/** + * A reader for the Delta Lake input source. It initializes an iterator {@link DeltaInputSourceIterator} + * for a subset of Delta records given by {@link FilteredColumnarBatch} and schema {@link InputRowSchema}. + * + */ public class DeltaInputSourceReader implements InputSourceReader { private final io.delta.kernel.utils.CloseableIterator filteredColumnarBatchCloseableIterator; diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java index 36b8bf038bc0..7ab52cef0897 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaSplit.java @@ -24,6 +24,15 @@ import java.util.List; +/** + * An input split of a Delta table containing the following information: + *
  • + * {@code stateRow} represents the canonical json representation of the latest snapshot of the Delta table. + *
  • + *
  • + * {@code files} represents the list of files from the latest snapshot. + *
  • + */ public class DeltaSplit { private final String stateRow; @@ -32,7 +41,7 @@ public class DeltaSplit @JsonCreator public DeltaSplit( @JsonProperty("state") final String stateRow, - @JsonProperty("file") final List files + @JsonProperty("files") final List files ) { this.stateRow = stateRow; diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 782b3a7c19a3..4219c530d34a 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -38,7 +38,8 @@ import java.util.Map; /** - * TODO: + * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the + * sample Delta Table used in the unit tests. */ public class DeltaTestUtil { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index 164a970b9dc5..bcef8457671f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -1,30 +1,54 @@ -/* -* 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. -*/ - -* In a conda environment, + +### Generate Delta Table for Unit Tests + +To test Delta Lake ingestion, use the Python script `create_delta_table.py` to generate a sample Delta table. +Set up the required dependencies by installing those specified in `requirements.txt`. ```shell pip install -r requirements.txt ``` -Once the dependencies are installed, you can then run - +Specifically, `pyspark` and `delta-spark` requirements are needed to generate Delta records. Once all the dependencies are installed, +you can run the python script: ```python -python3 create_delta_table.py -``` \ No newline at end of file +python3 create_delta_table.py +``` + +By default, the script uses `append` mode to generate 10 random records and writes the +Delta table to `resources/people-delta-table`. You can override the defaults by supplying the command line arguments: + +```shell +python3 create_delta_table.py -h + +usage: create_delta_table.py [-h] [--save_mode {append,overwrite}] [--save_path SAVE_PATH] [--num_records NUM_RECORDS] + +Script to write a Delta Lake table. + +optional arguments: + -h, --help show this help message and exit + --save_mode {append,overwrite} + Specify write mode (append/overwrite) (default: append) + --save_path SAVE_PATH + Save path for Delta table (default: /Users/abhishek/Desktop/opensource-druid/druid/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table4) + --num_records NUM_RECORDS + Specify number of Delta records to write (default: 10) +``` + +Check in the Delta table generated and update `DeltaTestUtil.java` as needed for validation. diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index 9e3230af4d57..41ff30eca38d 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -16,14 +16,12 @@ # limitations under the License. import os -import sys +import argparse import delta import mimesis import pyspark -from pyspark.sql import SparkSession - def config_spark_with_delta_lake(): builder = ( @@ -38,11 +36,12 @@ def config_spark_with_delta_lake(): spark.sparkContext.setLogLevel("ERROR") return spark -def create_dataset(i): + +def create_dataset(num_records): fake = mimesis.Generic() output = [] - for _ in range(i): + for _ in range(num_records): data = { "name": fake.person.name(), "surname": fake.person.surname(), @@ -58,32 +57,32 @@ def create_dataset(i): def main(): - save_mode = "append" - save_path = os.path.join(os.getcwd(), "people-delta-table4") - num_records = 10 + parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", + formatter_class=argparse.ArgumentDefaultsHelpFormatter) - if len(sys.argv) > 1: - save_mode = sys.argv[1] + parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append", + help="Specify write mode (append/overwrite)") + parser.add_argument('--save_path', default=os.path.join(os.getcwd(), "people-delta-table4"), + help="Save path for Delta table") + parser.add_argument('--num_records', type=int, default=10, + help="Specify number of Delta records to write") - if len(sys.argv) > 2: - save_path = sys.argv[2] + args = parser.parse_args() - if len(sys.argv) > 3: - num_records = sys.argv[3] + save_mode = args.save_mode + save_path = args.save_path + num_records = args.num_records spark = config_spark_with_delta_lake() - df = spark.createDataFrame(create_dataset(i=num_records)) - - df = df.select( - df.name, df.surname, df.birthday, df.email, df.country, df.state, df.city - ) + df = spark.createDataFrame(create_dataset(num_records=num_records)) + df = df.select(df.name, df.surname, df.birthday, df.email, df.country, df.state, df.city) df.write.format("delta").mode(save_mode).save(save_path) df.show() - print(f"Generated delta records to {save_path} in {save_mode} mode with {num_records} records.") + print(f"Generated Delta records to {save_path} in {save_mode} mode with {num_records} records.") if __name__ == "__main__": diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt index 128c65d8eebe..9b137987bbd2 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt @@ -1,3 +1,4 @@ delta-spark==2.2.0 +pyspark==3.2.0 mimesis==5.0.0 -pyspark==3.2.0 \ No newline at end of file +argparse==1.4.0 \ No newline at end of file From 8993ed331522339490c3289bede0182081ba8328 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 22:26:54 -0800 Subject: [PATCH 32/65] Adjust dependencies --- .../druid-deltalake-extensions/pom.xml | 13 ------------ pom.xml | 2 +- processing/pom.xml | 20 ------------------- 3 files changed, 1 insertion(+), 34 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index 939bc8c824da..b7035985f7dd 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -434,19 +434,6 @@ equalsverifier test - - - - - - - - - io.delta - delta-standalone_2.12 - 0.5.0 - - diff --git a/pom.xml b/pom.xml index 76069fcc7a85..94e2d6b3692e 100644 --- a/pom.xml +++ b/pom.xml @@ -175,7 +175,7 @@ extensions-core/azure-extensions extensions-core/datasketches extensions-core/druid-bloom-filter - + extensions-core/druid-kerberos extensions-core/druid-pac4j extensions-core/hdfs-storage extensions-core/histogram diff --git a/processing/pom.xml b/processing/pom.xml index 76b950e1cb1e..ed379bdaf8a5 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -342,26 +342,6 @@ ${oshi.version} - - io.delta - delta-kernel-api - 3.0.0 - - - - - io.delta - delta-kernel-defaults - 3.0.0 - - - - org.apache.hadoop - hadoop-client-api - 3.3.6 - - - junit From 001eeb7886450dce93d6035d232466d5a7492ffa Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 10 Jan 2024 23:48:04 -0800 Subject: [PATCH 33/65] Update readme --- .../extensions-contrib/delta-lake.md | 38 +++++++++++++++++++ docs/ingestion/input-sources.md | 35 ++++++++++++++--- 2 files changed, 67 insertions(+), 6 deletions(-) create mode 100644 docs/development/extensions-contrib/delta-lake.md diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md new file mode 100644 index 000000000000..bdb7f5b84050 --- /dev/null +++ b/docs/development/extensions-contrib/delta-lake.md @@ -0,0 +1,38 @@ +--- +id: delta-lake +title: "Delta Lake extension" +--- + + + +## Delta Lake Ingest extension + + +Delta Lake is an open source storage layer that brings reliability to data lakes. [DeltaLakeInputSource](../../ingestion/input-sources.md#deltalake-input-source) lets +you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta Lake extension, add the `druid-deltalake-extensions` to the list of loaded extensions. +See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. + +The Delta input source reads the configured Delta Lake table and extracts all the underlying delta files in the table's latest snapshot. +Delta Lake files are versioned Parquet format. + +## Known limitations + +- This extension only reads from the latest Delta table snapshot as the kernel API only supports that. +- Column filtering isn't supported yet. So the extension will reads all the columns in the configured table. \ No newline at end of file diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 63d5bc3f05c3..00ef5833bb8e 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -715,6 +715,13 @@ rolled-up datasource `wikipedia_rollup` by grouping on hour, "countryName", and to `true` to enable a compatibility mode where the timestampSpec is ignored. ::: +The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source. +Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method: +- `range` or `single_dim` partitioning: greater than or equal to 1 +- `hashed` or `dynamic` partitioning: greater than or equal to 2 + +For more information on the `maxNumConcurrentSubTasks` field, see [Implementation considerations](native-batch.md#implementation-considerations). + ## SQL input source The SQL input source is used to read data directly from RDBMS. @@ -1008,11 +1015,27 @@ This input source provides the following filters: `and`, `equals`, `interval`, a ## DeltaLake input source -TODO: fill in details about the input source +:::info To use the Delta Lake input source, add the druid-deltalake-extensions extension. ::: -The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source. -Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method: -- `range` or `single_dim` partitioning: greater than or equal to 1 -- `hashed` or `dynamic` partitioning: greater than or equal to 2 +You can use the Delta input source to read data stored in a Delta Lake table. For a given table, the input source scans +the latest snapshot from the configured table. Druid ingests the underlying delta files from the table. + +The following is a sample spec: + +```json +... + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "delta", + "tablePath": "/delta-table/directory" + }, + } +} +``` + +| Property|Description|Required| +|---------|-----------|--------| +| type|Set this value to `delta`.|yes| +| tablePath|The location of the Delta table.|yes| -For more information on the `maxNumConcurrentSubTasks` field, see [Implementation considerations](native-batch.md#implementation-considerations). From eb6f12af7dfcad71d27f2953586a3ff17203ccd8 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 24 Jan 2024 14:49:13 -0800 Subject: [PATCH 34/65] Bump up version --- extensions-contrib/druid-deltalake-extensions/pom.xml | 2 +- .../druid-deltalake-extensions/src/test/resources/README.md | 2 +- .../src/test/resources/requirements.txt | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index b7035985f7dd..2b8a6470c72c 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -29,7 +29,7 @@ druid org.apache.druid - 29.0.0-SNAPSHOT + 30.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index bcef8457671f..ca81db81a829 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -46,7 +46,7 @@ optional arguments: --save_mode {append,overwrite} Specify write mode (append/overwrite) (default: append) --save_path SAVE_PATH - Save path for Delta table (default: /Users/abhishek/Desktop/opensource-druid/druid/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table4) + Save path for Delta table (default: /druid/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table) --num_records NUM_RECORDS Specify number of Delta records to write (default: 10) ``` diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt index 9b137987bbd2..41bd50d04e2e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt @@ -1,4 +1,4 @@ -delta-spark==2.2.0 +delta-spark==3.0.0 pyspark==3.2.0 mimesis==5.0.0 argparse==1.4.0 \ No newline at end of file From 2eb2af3f4749f5329cb8cf77738cddc507abbf37 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 24 Jan 2024 23:00:59 -0800 Subject: [PATCH 35/65] fixup typo in deps --- distribution/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index 2b5b0de65407..dea41117e70f 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -455,7 +455,7 @@ -c org.apache.druid.extensions:druid-iceberg-extensions -c - org.apache.druid.extensions:druid-delatalake-extensions + org.apache.druid.extensions:druid-deltalake-extensions -c org.apache.druid.extensions.contrib:druid-spectator-histogram From 2b6ba3ed0b4b7f3cdda71989160e66301156e9d5 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 24 Jan 2024 23:28:03 -0800 Subject: [PATCH 36/65] forbidden api and checkstyle checks --- .../java/org/apache/druid/delta/input/DeltaInputRow.java | 7 ++++++- .../org/apache/druid/delta/input/DeltaInputSource.java | 2 -- .../main/java/org/apache/druid/delta/input/RowSerde.java | 1 - .../org/apache/druid/delta/input/DeltaInputRowTest.java | 5 +++-- .../org/apache/druid/delta/input/DeltaInputSourceTest.java | 2 +- .../java/org/apache/druid/delta/input/DeltaTestUtil.java | 2 +- .../java/org/apache/druid/delta/input/RowSerdeTest.java | 2 +- 7 files changed, 12 insertions(+), 9 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 3726a870c277..9e3204ef5108 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -191,7 +191,12 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR } else if (dataType instanceof StringType) { return dataRow.getString(columnOrdinal); } else if (dataType instanceof BinaryType) { - return new String(dataRow.getBinary(columnOrdinal)); + final byte[] arr = dataRow.getBinary(columnOrdinal); + final char[] charArray = new char[arr.length]; + for (int i = 0; i < arr.length; i++) { + charArray[i] = (char) (arr[i] & 0xff); + } + return String.valueOf(charArray); } else if (dataType instanceof DecimalType) { return dataRow.getDecimal(columnOrdinal).longValue(); } else { diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 0025e91d8976..140ab73cf447 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.google.common.primitives.Ints; import io.delta.kernel.Scan; @@ -33,7 +32,6 @@ import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; import io.delta.kernel.internal.util.Utils; -import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 6c9223f58126..23fb9e3b6a15 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -45,7 +45,6 @@ import io.delta.kernel.types.TimestampType; import org.apache.druid.error.InvalidInput; -import java.io.UncheckedIOException; import java.util.HashMap; import java.util.Map; diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index c0c40157c6e6..8533df12fe6f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -68,7 +68,8 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException for (String dimension : DeltaTestUtil.DIMENSIONS) { Assert.assertEquals(expectedRow.get(dimension), deltaInputRow.getDimension(dimension).get(0)); } - long expectedMillis = ((Long) expectedRow.get(DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn()) / 1_000_000) * 1000; + long expectedMillis = ((Long) expectedRow.get(DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn()) + / 1_000_000) * 1000; Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); totalRecordCount += 1; } @@ -76,4 +77,4 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException } Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), totalRecordCount); } -} \ No newline at end of file +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 1d8e7cff31f2..019d80b7c3a1 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -138,4 +138,4 @@ private List readAllRows(InputSourceReader reader) throws IOException } return rows; } -} \ No newline at end of file +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 4219c530d34a..fa5f46fea486 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -152,7 +152,7 @@ public static Scan getScan(final TableClient tableClient) throws TableNotFoundEx final Snapshot snapshot = table.getLatestSnapshot(tableClient); final StructType readSchema = snapshot.getSchema(tableClient); final ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient) - .withReadSchema(tableClient, readSchema); + .withReadSchema(tableClient, readSchema); return scanBuilder.build(); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index a94a9b54f191..53eab8383b05 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -42,4 +42,4 @@ public void testSerializeDeserializeRoundtrip() throws TableNotFoundException Assert.assertEquals(scanState.getSchema(), row.getSchema()); } -} \ No newline at end of file +} From 8891fbb3b427617f5e77b51f4ad9e610baf4ef0f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 24 Jan 2024 23:56:15 -0800 Subject: [PATCH 37/65] Trim down dependencies --- .../druid-deltalake-extensions/pom.xml | 336 +----------------- 1 file changed, 7 insertions(+), 329 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index 2b8a6470c72c..b72dd0e16148 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -51,13 +51,6 @@ ${delta-kernel.version} - - org.apache.hadoop - hadoop-client-runtime - ${hadoop.compile.version} - runtime - - org.apache.hadoop hadoop-client-api @@ -65,214 +58,6 @@ provided - - org.apache.hadoop - hadoop-common - ${hadoop.compile.version} - compile - - - io.netty - netty-buffer - - - commons-cli - commons-cli - - - log4j - log4j - - - commons-codec - commons-codec - - - commons-logging - commons-logging - - - commons-io - commons-io - - - commons-lang - commons-lang - - - commons-net - commons-net - - - commons-collections - commons-collections - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - org.apache.zookeeper - zookeeper - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - - javax.ws.rs - jsr311-api - - - com.google.code.findbugs - jsr305 - - - org.mortbay.jetty - jetty-util - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-webapp - - - com.google.protobuf - protobuf-java - - - com.sun.jersey - jersey-core - - - org.apache.curator - curator-client - - - org.apache.curator - curator-recipes - - - org.apache.commons - commons-math3 - - - com.google.guava - guava - - - org.apache.avro - avro - - - net.java.dev.jets3t - jets3t - - - com.sun.jersey - jersey-json - - - com.github.pjfanning - jersey-json - - - com.jcraft - jsch - - - org.mortbay.jetty - jetty - - - com.sun.jersey - jersey-server - - - - commons-beanutils - commons-beanutils-core - - - commons-beanutils - commons-beanutils - - - ch.qos.reload4j - reload4j - - - com.sun.jersey - jersey-servlet - - - javax.servlet.jsp - jsp-api - - - org.slf4j - slf4j-reload4j - - - com.google.re2j - re2j - - - com.google.code.gson - gson - - - com.nimbusds - nimbus-jose-jwt - - - com.github.stephenc.jcip - jcip-annotations - - - org.apache.curator - curator-framework - - - io.dropwizard.metrics - metrics-core - - - dnsjava - dnsjava - - - org.xerial.snappy - snappy-java - - - - - - commons-cli - commons-cli - 1.5.0 - - org.apache.druid druid-processing @@ -285,107 +70,6 @@ provided - - org.apache.hadoop - hadoop-hdfs-client - runtime - - - org.jetbrains.kotlin - kotlin-stdlib - - - org.jetbrains.kotlin - kotlin-stdlib-common - - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - runtime - ${hadoop.compile.version} - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.hadoop - hadoop-yarn-common - - - aopalliance - aopalliance - - - org.apache.commons - commons-compress - - - com.google.guava - guava - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - javax.inject - javax - - - io.netty - netty - - - slf4j-log4j12 - org.slf4j - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-reload4j - - - protobuf-java - com.google.protobuf - - - - - org.apache.hadoop - hadoop-aws - ${hadoop.compile.version} - runtime - - - com.amazonaws - aws-java-sdk-bundle - - - - com.google.code.findbugs jsr305 @@ -412,16 +96,15 @@ provided - org.slf4j - slf4j-api - provided + com.fasterxml.jackson.core + jackson-databind + 2.12.7.1 - - org.apache.parquet - parquet-column - 1.13.1 - test + it.unimi.dsi + fastutil-core + 8.5.4 + provided @@ -429,11 +112,6 @@ junit test - - nl.jqno.equalsverifier - equalsverifier - test - From 597e66458088c8c75e1da10e6969e83810039056 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 24 Jan 2024 23:58:08 -0800 Subject: [PATCH 38/65] new lines --- extensions-contrib/druid-deltalake-extensions/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index b72dd0e16148..36c2b4940c50 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -126,4 +126,3 @@ - From ce6f56c5555e6acfc94cc62f2dd1d7881327b5c4 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 00:42:38 -0800 Subject: [PATCH 39/65] Fixup Intellij inspections. --- .../java/org/apache/druid/delta/input/DeltaInputSource.java | 1 - .../org/apache/druid/delta/input/DeltaInputSourceReader.java | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 140ab73cf447..06944bd6d19d 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -144,7 +144,6 @@ public InputSourceReader reader( @Override public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) - throws IOException { if (null != deltaSplit) { // can't split a split diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java index 21260d029772..d0fc4780d001 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSourceReader.java @@ -57,14 +57,15 @@ public CloseableIterator read() } @Override - public CloseableIterator read(InputStats inputStats) throws IOException + public CloseableIterator read(InputStats inputStats) { return new DeltaInputSourceIterator(filteredColumnarBatchCloseableIterator, inputRowSchema); } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() { + CloseableIterator inner = read(); return new CloseableIterator() { From 9a3723a82129e4d39b423e06d674b91fb0ef39f9 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 00:46:19 -0800 Subject: [PATCH 40/65] Add equals() and hashCode() --- .../org/apache/druid/delta/input/DeltaInputRow.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 9e3204ef5108..88d9727b5e6a 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -51,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; /** * Encodes the row and schema information from the Delta Lake. @@ -125,6 +126,18 @@ public int compareTo(Row o) return this.getTimestamp().compareTo(o.getTimestamp()); } + @Override + public boolean equals(Object o) + { + return o instanceof DeltaInputRow && compareTo((DeltaInputRow) o) == 0; + } + + @Override + public int hashCode() + { + return Objects.hash(row, schema, fieldNameToOrdinal, delegateRow); + } + @Override public String toString() { From ca566feecd1be57aaece957d96db735b0d4d310e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 08:36:22 -0800 Subject: [PATCH 41/65] chain splits, intellij inspections --- .../druid/delta/input/DeltaInputSource.java | 2 +- .../delta/input/DeltaInputSourceTest.java | 47 +++++++------------ 2 files changed, 18 insertions(+), 31 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 06944bd6d19d..d5cb1de927f2 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -183,7 +183,7 @@ public Stream> createSplits(InputFormat inputFormat, @Nul } @Override - public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { return Ints.checkedCast(createSplits(inputFormat, splitHintSpec).count()); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 019d80b7c3a1..bcaac9dee6e6 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -37,13 +37,10 @@ public class DeltaInputSourceTest { @Test - public void testReadDeltaLakeFilesSample() throws IOException + public void testSampleDeltaTable() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - Assert.assertNotNull(deltaInputSource); - - InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); - Assert.assertNotNull(inputSourceReader); + final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualSampledRows.size()); @@ -62,18 +59,14 @@ public void testReadDeltaLakeFilesSample() throws IOException } @Test - public void testReadDeltaLakeFilesRead() throws IOException + public void testReadAllDeltaTable() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - Assert.assertNotNull(deltaInputSource); - - InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); - Assert.assertNotNull(inputSourceReader); + final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualReadRows.size()); - for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); InputRow actualInputRow = actualReadRows.get(idx); @@ -89,36 +82,30 @@ public void testReadDeltaLakeFilesRead() throws IOException } @Test - public void testReadDeltaLakeNoSplits() throws IOException + public void testReadDeltaTableWithNoSplits() { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - Assert.assertNotNull(deltaInputSource); - - Stream> splits = deltaInputSource.createSplits(null, null); + final Stream> splits = deltaInputSource.createSplits(null, null); Assert.assertNotNull(splits); Assert.assertEquals(1, splits.count()); } @Test - public void testReadDeltaLakeWithSplits() throws IOException + public void testReadDeltaLakeWithSplits() { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - Assert.assertNotNull(deltaInputSource); - - Stream> splits1 = deltaInputSource.createSplits(null, null); - List> splitsCollect1 = splits1.collect(Collectors.toList()); - Assert.assertEquals(1, splitsCollect1.size()); + final List> splits1 = deltaInputSource.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertEquals(1, splits1.size()); - DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( + final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( DeltaTestUtil.DELTA_TABLE_PATH, - splitsCollect1.get(0).get() + splits1.get(0).get() ); - Assert.assertNotNull(deltaInputSourceWithSplit); - Stream> splits2 = deltaInputSourceWithSplit.createSplits(null, null); - List> splitsCollect2 = splits2.collect(Collectors.toList()); - Assert.assertEquals(1, splitsCollect2.size()); - - Assert.assertEquals(splitsCollect1.get(0).get(), splitsCollect2.get(0).get()); + final List> splits2 = deltaInputSourceWithSplit.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertEquals(1, splits2.size()); + Assert.assertEquals(splits1.get(0).get(), splits2.get(0).get()); } private List sampleAllRows(InputSourceReader reader) throws IOException @@ -132,7 +119,7 @@ private List sampleAllRows(InputSourceReader reader) private List readAllRows(InputSourceReader reader) throws IOException { - List rows = new ArrayList<>(); + final List rows = new ArrayList<>(); try (CloseableIterator iterator = reader.read()) { iterator.forEachRemaining(rows::add); } From 027791da5cb2c9361018b37eb53b431b5e14d1e8 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 10:11:53 -0800 Subject: [PATCH 42/65] review comments and todo placeholder --- .../druid/delta/input/DeltaInputRow.java | 8 +++- .../druid/delta/input/DeltaInputSource.java | 42 ++++++++++++------- .../apache/druid/delta/input/RowSerde.java | 4 +- 3 files changed, 35 insertions(+), 19 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 88d9727b5e6a..4c3f5f8eb44d 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -207,13 +207,17 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR final byte[] arr = dataRow.getBinary(columnOrdinal); final char[] charArray = new char[arr.length]; for (int i = 0; i < arr.length; i++) { - charArray[i] = (char) (arr[i] & 0xff); + charArray[i] = (char) (arr[i]); } return String.valueOf(charArray); } else if (dataType instanceof DecimalType) { return dataRow.getDecimal(columnOrdinal).longValue(); } else { - throw InvalidInput.exception("Unsupported data type[%s]", dataType); + throw InvalidInput.exception( + "Unsupported data type[%s] for fieldName[%s].", + dataType, + dataRow.getSchema().fieldNames().get(columnOrdinal) + ); } } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index d5cb1de927f2..4724d76bf712 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -90,6 +90,15 @@ public boolean needsFormat() return false; } + /** + * Instantiates a {@link DeltaInputSourceReader} to read the Delta table rows. If a {@link DeltaSplit} is supplied, + * the Delta files and schema are obtained from it to instantiate the reader. Otherwise, a Delta table client is + * instantiated with the supplied configuration to read the table. + * @param inputRowSchema schema for {@link org.apache.druid.data.input.InputRow} + * @param inputFormat unused parameter. The input format is always parquet + * @param temporaryDirectory unused parameter + * @return + */ @Override public InputSourceReader reader( InputRowSchema inputRowSchema, @@ -97,8 +106,9 @@ public InputSourceReader reader( File temporaryDirectory ) { - Configuration hadoopConf = new Configuration(); - TableClient tableClient = DefaultTableClient.create(hadoopConf); + // TODO: allow hadoop configurations such as credentials to be set here. + final Configuration conf = new Configuration(); + final TableClient tableClient = DefaultTableClient.create(conf); try { final Row scanState; final List scanRowList; @@ -110,17 +120,17 @@ public InputSourceReader reader( .map(row -> deserialize(tableClient, row)) .collect(Collectors.toList()); } else { - Table table = Table.forPath(tableClient, tablePath); - Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); + final Table table = Table.forPath(tableClient, tablePath); + final Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); + final Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + final CloseableIterator scanFiles = scan.getScanFiles(tableClient); - Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); scanState = scan.getScanState(tableClient); - CloseableIterator scanFiles = scan.getScanFiles(tableClient); scanRowList = new ArrayList<>(); while (scanFiles.hasNext()) { - FilteredColumnarBatch scanFileBatch = scanFiles.next(); - CloseableIterator scanFileRows = scanFileBatch.getRows(); + final FilteredColumnarBatch scanFileBatch = scanFiles.next(); + final CloseableIterator scanFileRows = scanFileBatch.getRows(); scanFileRows.forEachRemaining(scanRowList::add); } } @@ -145,12 +155,12 @@ public InputSourceReader reader( @Override public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { - if (null != deltaSplit) { + if (deltaSplit != null) { // can't split a split return Stream.of(new InputSplit<>(deltaSplit)); } - TableClient tableClient = DefaultTableClient.create(new Configuration()); + final TableClient tableClient = DefaultTableClient.create(new Configuration()); final Snapshot latestSnapshot; final Table table; try { @@ -160,18 +170,18 @@ public Stream> createSplits(InputFormat inputFormat, @Nul catch (TableNotFoundException e) { throw new RuntimeException(e); } - Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + final Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); // scan files iterator for the current snapshot - CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient); + final CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient); - Row scanState = scan.getScanState(tableClient); - String scanStateStr = RowSerde.serializeRowToJson(scanState); + final Row scanState = scan.getScanState(tableClient); + final String scanStateStr = RowSerde.serializeRowToJson(scanState); Iterator deltaSplitIterator = Iterators.transform( scanFilesIterator, scanFile -> { - CloseableIterator rows = scanFile.getRows(); - List fileRows = new ArrayList<>(); + final CloseableIterator rows = scanFile.getRows(); + final List fileRows = new ArrayList<>(); while (rows.hasNext()) { fileRows.add(RowSerde.serializeRowToJson(rows.next())); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 23fb9e3b6a15..f7d693757d0f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -51,7 +51,9 @@ /** * Utility class to serialize and deserialize {@link Row} object. * Code borrowed from - * RowSerde.java. + * RowSerde.java. The only differences between the two classes are the code style and exception handling in + * {@link #convertRowToJsonObject}, where we use {@link org.apache.druid.error.DruidException} instead of + * {@link UnsupportedOperationException}. * */ public class RowSerde From 66d69b4aa42fe958748b3fc3ce1a7352c8b80286 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 11:32:13 -0800 Subject: [PATCH 43/65] fix up some docs --- docs/development/extensions-contrib/delta-lake.md | 2 +- docs/ingestion/input-sources.md | 8 +++++--- website/.spelling | 4 ++++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index bdb7f5b84050..48730c081e15 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -30,7 +30,7 @@ you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. The Delta input source reads the configured Delta Lake table and extracts all the underlying delta files in the table's latest snapshot. -Delta Lake files are versioned Parquet format. +These Delta Lake files are in versioned Parquet format ## Known limitations diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 00ef5833bb8e..7d26fd0f2f4d 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -824,7 +824,7 @@ The following is an example of a Combining input source spec: ## Iceberg input source :::info - To use the Iceberg input source, add the `druid-iceberg-extensions` extension. +You need to include the [`druid-iceberg-extensions`](../development/extensions-contrib/iceberg.md) as an extension to use the Iceberg input source. ::: You use the Iceberg input source to read data stored in the Iceberg table format. For a given table, the input source scans up to the latest Iceberg snapshot from the configured Hive catalog. Druid ingests the underlying live data files using the existing input source formats. @@ -1013,9 +1013,11 @@ This input source provides the following filters: `and`, `equals`, `interval`, a |type|Set this value to `not`.|yes| |filter|The iceberg filter on which logical NOT is applied|yes| -## DeltaLake input source +## Delta Lake input source -:::info To use the Delta Lake input source, add the druid-deltalake-extensions extension. ::: +:::info +You need to include the [`druid-deltalake-extensions`](../development/extensions-contrib/delta-lake.md) as an extension to use the Delta Lake input source. +::: You can use the Delta input source to read data stored in a Delta Lake table. For a given table, the input source scans the latest snapshot from the configured table. Druid ingests the underlying delta files from the table. diff --git a/website/.spelling b/website/.spelling index 67be33c2d999..d29287256595 100644 --- a/website/.spelling +++ b/website/.spelling @@ -63,6 +63,7 @@ DRUIDVERSION DataSketches DateTime DateType +DeltaLakeInputSource dimensionsSpec DimensionSpec DimensionSpecs @@ -79,6 +80,7 @@ downsamples downsampling Dropwizard dropwizard +druid-deltalake-extensions DruidInputSource DruidSQL DynamicConfigProvider @@ -525,6 +527,7 @@ SVG symlink syntaxes systemFields +tablePath tiering timeseries Timeseries @@ -571,6 +574,7 @@ varchar vectorizable vectorize vectorizeVirtualColumns +versioned versioning virtualColumns w.r.t. From 024ed4c74c90aa5a9e0448f140888a7fa5bdf53b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 11:51:03 -0800 Subject: [PATCH 44/65] null table path and test dependencies. Fixup broken link. --- .../extensions-contrib/delta-lake.md | 2 +- .../druid-deltalake-extensions/pom.xml | 17 +++++++++++++++++ .../druid/delta/input/DeltaInputSource.java | 4 +++- .../druid/delta/input/DeltaInputSourceTest.java | 17 +++++++++++++++++ 4 files changed, 38 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index 48730c081e15..bd407cba0d81 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -25,7 +25,7 @@ title: "Delta Lake extension" ## Delta Lake Ingest extension -Delta Lake is an open source storage layer that brings reliability to data lakes. [DeltaLakeInputSource](../../ingestion/input-sources.md#deltalake-input-source) lets +Delta Lake is an open source storage layer that brings reliability to data lakes. [DeltaLakeInputSource](../../ingestion/input-sources.md#delta-lake-input-source) lets you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta Lake extension, add the `druid-deltalake-extensions` to the list of loaded extensions. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index 36c2b4940c50..fea958e75eea 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -112,6 +112,23 @@ junit test + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.hamcrest + hamcrest-all + test + + + org.hamcrest + hamcrest-core + test + diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 4724d76bf712..6b0e2321d488 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -57,7 +57,9 @@ /** * Input source to ingest data from a Delta Lake. * This input source reads the latest snapshot from a Delta table specified by {@code tablePath} parameter. - * Note: the kernel table API only supports reading from the latest snapshot. + * We leverage the Delta Kernel APIs to interact with a Delta table. The Kernel API abstracts away the + * complexities of the Delta protocol itself. + * Note: currently, the Kernel table API only supports reading from the latest snapshot. */ public class DeltaInputSource implements SplittableInputSource { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index bcaac9dee6e6..7678f1d09deb 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -23,7 +23,10 @@ import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -108,6 +111,20 @@ public void testReadDeltaLakeWithSplits() Assert.assertEquals(splits1.get(0).get(), splits2.get(0).get()); } + @Test + public void testNullTablePath() + { + MatcherAssert.assertThat( + Assert.assertThrows( + DruidException.class, + () -> new DeltaInputSource(null, null) + ), + DruidExceptionMatcher.invalidInput().expectMessageIs( + "tablePath cannot be null" + ) + ); + } + private List sampleAllRows(InputSourceReader reader) throws IOException { List rows = new ArrayList<>(); From 3874dd453bd5c6f5a5e5d06b16c9a89a4c056d30 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 25 Jan 2024 14:31:40 -0800 Subject: [PATCH 45/65] run prettify --- .../src/druid-models/input-source/input-source.tsx | 13 ++++++++++++- .../src/views/load-data-view/load-data-view.tsx | 4 +--- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/web-console/src/druid-models/input-source/input-source.tsx b/web-console/src/druid-models/input-source/input-source.tsx index 4e40cd23a587..01b111b957f9 100644 --- a/web-console/src/druid-models/input-source/input-source.tsx +++ b/web-console/src/druid-models/input-source/input-source.tsx @@ -182,7 +182,18 @@ export function issueWithInputSource(inputSource: InputSource | undefined): stri } } -const KNOWN_TYPES = ['inline', 'druid', 'http', 'local', 's3', 'azure', 'delta', 'google', 'hdfs', 'sql']; +const KNOWN_TYPES = [ + 'inline', + 'druid', + 'http', + 'local', + 's3', + 'azure', + 'delta', + 'google', + 'hdfs', + 'sql', +]; export const INPUT_SOURCE_FIELDS: Field[] = [ // inline diff --git a/web-console/src/views/load-data-view/load-data-view.tsx b/web-console/src/views/load-data-view/load-data-view.tsx index 4df08a1e7c3e..18fdf045a5bc 100644 --- a/web-console/src/views/load-data-view/load-data-view.tsx +++ b/web-console/src/views/load-data-view/load-data-view.tsx @@ -926,9 +926,7 @@ export class LoadDataView extends React.PureComponent

    Load data from Delta Lake.

    -

    - Data must be stored in the Delta Lake format. -

    +

    Data must be stored in the Delta Lake format.

    ); From 482673175663e5cbfc7c04bc71bca8434d9a0bce Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 18:57:05 -0800 Subject: [PATCH 46/65] Different test; fixes --- .../druid-deltalake-extensions/pom.xml | 3 +- .../apache/druid/delta/input/RowSerde.java | 20 +- .../druid/delta/input/DeltaInputRowTest.java | 14 +- .../delta/input/DeltaInputSourceTest.java | 53 +++-- .../druid/delta/input/DeltaTestUtil.java | 215 ++++++++++++------ .../src/test/resources/create_delta_table.py | 77 +++++-- ...-9ad7-31ec84e39880-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-8a8b-f599ea586597-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-9c70-7424d71cb267-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-80a9-5c012b9f1a4c-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-b9db-bb48174d98dd-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-90e4-6cfec3897d78-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-91df-a700e6a6ead9-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-b556-c2f8d414faa1-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-a91c-9e6c1b04d884-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-adfb-02377478da38-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-982b-acdffa9be0de-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-ac19-64e8ec608096-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-9f34-ccb270f44d70-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...-96ca-f160d4bf9d0a-c000.snappy.parquet.crc | Bin 0 -> 28 bytes .../_delta_log/.00000000000000000000.json.crc | Bin 0 -> 68 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 0 -> 36 bytes .../_delta_log/00000000000000000000.json | 13 ++ .../_delta_log/00000000000000000001.json | 6 + ...4209-9ad7-31ec84e39880-c000.snappy.parquet | Bin 0 -> 2317 bytes ...4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet | Bin 0 -> 979 bytes ...429a-8a8b-f599ea586597-c000.snappy.parquet | Bin 0 -> 2302 bytes ...4b4f-9c70-7424d71cb267-c000.snappy.parquet | Bin 0 -> 2302 bytes ...4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet | Bin 0 -> 2316 bytes ...49f0-b9db-bb48174d98dd-c000.snappy.parquet | Bin 0 -> 2317 bytes ...4866-90e4-6cfec3897d78-c000.snappy.parquet | Bin 0 -> 2302 bytes ...423d-91df-a700e6a6ead9-c000.snappy.parquet | Bin 0 -> 2317 bytes ...42c7-b556-c2f8d414faa1-c000.snappy.parquet | Bin 0 -> 2317 bytes ...4137-a91c-9e6c1b04d884-c000.snappy.parquet | Bin 0 -> 2316 bytes ...46a0-adfb-02377478da38-c000.snappy.parquet | Bin 0 -> 2454 bytes ...46f0-982b-acdffa9be0de-c000.snappy.parquet | Bin 0 -> 2317 bytes ...4e66-ac19-64e8ec608096-c000.snappy.parquet | Bin 0 -> 2455 bytes ...410a-9f34-ccb270f44d70-c000.snappy.parquet | Bin 0 -> 2301 bytes ...4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet | Bin 0 -> 2317 bytes ...4df0-96ca-f160d4bf9d0a-c000.snappy.parquet | Bin 0 -> 2309 bytes .../src/test/resources/requirements.txt | 1 - 43 files changed, 274 insertions(+), 128 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000000.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000001.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index fea958e75eea..ed4b025a9f48 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -50,12 +50,11 @@ delta-kernel-defaults ${delta-kernel.version} - org.apache.hadoop hadoop-client-api ${hadoop.compile.version} - provided + compile diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index f7d693757d0f..5652194ef018 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -44,7 +44,12 @@ import io.delta.kernel.types.StructType; import io.delta.kernel.types.TimestampType; import org.apache.druid.error.InvalidInput; +import org.apache.druid.jackson.DefaultObjectMapper; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.HashMap; import java.util.Map; @@ -58,7 +63,7 @@ */ public class RowSerde { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private RowSerde() { @@ -128,9 +133,16 @@ public static Map convertRowToJsonObject(Row row) } else if (fieldType instanceof DoubleType) { value = row.getDouble(fieldId); } else if (fieldType instanceof DateType) { - value = row.getInt(fieldId); + final int daysSinceEpochUTC = row.getInt(fieldId); + value = LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZoneId.systemDefault()).toEpochSecond(); } else if (fieldType instanceof TimestampType) { - value = row.getLong(fieldId); + final long microSecsSinceEpochUTC = row.getLong(fieldId); + LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC + ); + value = dateTime.atZone(ZoneId.systemDefault()).toInstant().toEpochMilli(); } else if (fieldType instanceof StringType) { value = row.getString(fieldId); } else if (fieldType instanceof ArrayType) { @@ -141,7 +153,7 @@ public static Map convertRowToJsonObject(Row row) Row subRow = row.getStruct(fieldId); value = convertRowToJsonObject(subRow); } else { - throw InvalidInput.exception("Unsupported fieldType[%s] for fieldId[%s]", fieldType, fieldId); + throw InvalidInput.exception("Unsupported fieldType[%s] for fieldName[%s]", fieldType, name); } rowObject.put(name, value); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 8533df12fe6f..91df20b1f60e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -63,14 +63,14 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException Assert.assertEquals(DeltaTestUtil.DIMENSIONS, deltaInputRow.getDimensions()); Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(totalRecordCount); - Assert.assertEquals(expectedRow, deltaInputRow.getRawRowAsMap()); - - for (String dimension : DeltaTestUtil.DIMENSIONS) { - Assert.assertEquals(expectedRow.get(dimension), deltaInputRow.getDimension(dimension).get(0)); + for (String key : expectedRow.keySet()) { + if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + final long expectedMillis = ((Long) expectedRow.get(key)) * 1000; + Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); + } else { + Assert.assertEquals(expectedRow.get(key), deltaInputRow.getRaw(key)); + } } - long expectedMillis = ((Long) expectedRow.get(DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn()) - / 1_000_000) * 1000; - Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); totalRecordCount += 1; } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 7678f1d09deb..d4d8446f058c 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -19,12 +19,15 @@ package org.apache.druid.delta.input; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.hamcrest.MatcherAssert; import org.junit.Assert; @@ -52,12 +55,23 @@ public void testSampleDeltaTable() throws IOException Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); InputRowListPlusRawValues actualSampledRow = actualSampledRows.get(idx); Assert.assertNull(actualSampledRow.getParseException()); - Assert.assertEquals( - expectedRow, - actualSampledRow.getRawValues() - ); + + Map actualSampledRawVals = actualSampledRow.getRawValues(); + Assert.assertNotNull(actualSampledRawVals); Assert.assertNotNull(actualSampledRow.getRawValuesList()); - Assert.assertEquals(expectedRow, actualSampledRow.getRawValuesList().get(0)); + Assert.assertEquals(1, actualSampledRow.getRawValuesList().size()); + ObjectMapper objectMapper = new DefaultObjectMapper(); + objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL); + System.out.println("JSON:" + objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(actualSampledRawVals)); + for (String key : expectedRow.keySet()) { + if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + final long expectedMillis = (Long) expectedRow.get(key); + Assert.assertEquals(expectedMillis, actualSampledRawVals.get(key)); + + } else { + Assert.assertEquals(expectedRow.get(key), actualSampledRawVals.get(key)); + } + } } } @@ -75,10 +89,10 @@ public void testReadAllDeltaTable() throws IOException InputRow actualInputRow = actualReadRows.get(idx); for (String key : expectedRow.keySet()) { if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { - final long expectedMillis = ((Long) expectedRow.get(key) / 1_000_000) * 1000; + final long expectedMillis = (Long) expectedRow.get(key) * 1000; Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); } else { - Assert.assertEquals(expectedRow.get(key), actualInputRow.getDimension(key).get(0)); + Assert.assertEquals(expectedRow.get(key), actualInputRow.getRaw(key)); } } } @@ -90,25 +104,28 @@ public void testReadDeltaTableWithNoSplits() final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); final Stream> splits = deltaInputSource.createSplits(null, null); Assert.assertNotNull(splits); - Assert.assertEquals(1, splits.count()); + Assert.assertEquals(2, splits.count()); } @Test public void testReadDeltaLakeWithSplits() { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - final List> splits1 = deltaInputSource.createSplits(null, null) + final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(1, splits1.size()); + Assert.assertEquals(2, splits.size()); - final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( - DeltaTestUtil.DELTA_TABLE_PATH, - splits1.get(0).get() - ); - final List> splits2 = deltaInputSourceWithSplit.createSplits(null, null) - .collect(Collectors.toList()); - Assert.assertEquals(1, splits2.size()); - Assert.assertEquals(splits1.get(0).get(), splits2.get(0).get()); + for (int idx = 0; idx < splits.size(); idx++) { + final DeltaSplit split = splits.get(idx).get(); + final DeltaInputSource deltaInputSourceWithSplitx = new DeltaInputSource( + DeltaTestUtil.DELTA_TABLE_PATH, + split + ); + List> splitsResult = deltaInputSourceWithSplitx.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertEquals(1, splitsResult.size()); + Assert.assertEquals(split, splitsResult.get(0).get()); + } } @Test diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index fa5f46fea486..25206f78912e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -31,6 +31,10 @@ import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.DoubleDimensionSchema; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import java.util.ArrayList; @@ -43,106 +47,169 @@ */ public class DeltaTestUtil { - public static final String DELTA_TABLE_PATH = "src/test/resources/people-delta-table"; - public static final List DIMENSIONS = ImmutableList.of("city", "state", "surname", "email", "country"); + public static final String DELTA_TABLE_PATH = "src/test/resources/employee-delta-table"; + public static final List DIMENSIONS = ImmutableList.of( + "id", + "birthday", + "name", + "age", + "salary", + "bonus", + "yoe", + "is_fulltime", + "last_vacation_time" + ); public static final List> EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( - "birthday", 1049418130358332L, - "country", "Panama", - "city", "Eastpointe", - "surname", "Francis", - "name", "Darren", - "state", "Minnesota", - "email", "rating1998@yandex.com" + "birthday", 944265600L, + "id", 4662990160L, + "name", "Employee1", + "age", (short) 24, + "salary", 63807.010056944906, + "yoe", 5 + ), + ImmutableMap.of( + "id", 7436924672L, + "birthday", 909878400L, + "name", "Employee2", + "age", (short) 25, + "salary", 86178.47568217944, + "yoe", 10, + "is_fulltime", false + ), + ImmutableMap.of( + "id", 1127302418L, + "birthday", 901238400L, + "name", "Employee3", + "age", (short) 25, + "salary", 59001.92470779706, + "yoe", 10, + "is_fulltime", false + ), + ImmutableMap.of( + "id", 1810758014L, + "birthday", 789177600L, + "name", "Employee4", + "age", (short) 29, + "salary", 97151.7200456219, + "bonus", 2880.2966f, + "yoe", 10, + "is_fulltime", true, + "last_vacation_time", 1706309461000L + ), + ImmutableMap.of( + "id", 2675583494L, + "birthday", 950400000L, + "name", "Employee5", + "age", (short) 23, + "salary", 84092.96929134917, + "yoe", 6, + "is_fulltime", false + ), + ImmutableMap.of( + "id", 8109925563L, + "birthday", 1030320000L, + "name", "Employee1", + "age", (short) 21, + "salary", 98126.11562963494, + "yoe", 6 ), ImmutableMap.of( - "birthday", 1283743763753323L, - "country", "Aruba", - "city", "Wheaton", - "surname", "Berger", - "name", "Madelene", - "state", "New York", - "email", "invitations2036@duck.com" + "id", 348540417L, + "birthday", 1028764800L, + "name", "Employee2", + "salary", 88318.68501168216, + "age", (short) 21, + "yoe", 6 ), ImmutableMap.of( - "birthday", 1013053015543401L, - "country", "Anguilla", - "city", "Sahuarita", - "surname", "Mccall", - "name", "Anibal", - "state", "Oklahoma", - "email", "modifications2025@yahoo.com" + "birthday", 772675200L, + "is_fulltime", false, + "name", "Employee3", + "id", 644036573L, + "salary", 70031.88789434545, + "age", (short) 29, + "yoe", 14 ), ImmutableMap.of( - "birthday", 569564422313618L, - "country", "Luxembourg", - "city", "Santa Rosa", - "surname", "Jackson", - "name", "Anibal", - "state", "New Hampshire", - "email", "medication1855@gmail.com" + "birthday", 940118400L, + "name", "Employee4", + "id", 8451974441L, + "salary", 90127.23134932564, + "age", (short) 24, + "yoe", 4 ), ImmutableMap.of( - "birthday", 667560498632507L, - "country", "Anguilla", - "city", "Morristown", - "surname", "Tanner", - "name", "Loree", - "state", "New Hampshire", - "email", "transport1961@duck.com" + "birthday", 872294400L, + "is_fulltime", false, + "name", "Employee5", + "id", 1257915386L, + "salary", 55170.21435756755, + "age", (short) 26, + "yoe", 5 ), ImmutableMap.of( - "birthday", 826120534655077L, - "country", "Panama", - "city", "Greenville", - "surname", "Gamble", - "name", "Bernardo", - "state", "North Carolina", - "email", "limitations1886@yandex.com" + "birthday", 1023148800L, + "is_fulltime", false, + "name", "Employee6", + "id", 2034724452L, + "salary", 97643.72021601905, + "age", (short) 21, + "yoe", 1 ), ImmutableMap.of( - "birthday", 1284652116668688L, - "country", "China", - "city", "Albert Lea", - "surname", "Cherry", - "name", "Philip", - "state", "Nevada", - "email", "const1874@outlook.com" + "birthday", 1090627200L, + "is_fulltime", true, + "bonus", 3610.4019f, + "name", "Employee7", + "id", 1124457317L, + "salary", 60433.78056730033, + "last_vacation_time", 1706478632000L, + "age", (short) 19, + "yoe", 4 ), ImmutableMap.of( - "birthday", 1154549284242934L, - "country", "Barbados", - "city", "Mount Pleasant", - "surname", "Beasley", - "name", "Shaneka", - "state", "Montana", - "email", "msg1894@example.com" + "birthday", 867542400L, + "is_fulltime", false, + "name", "Employee8", + "id", 8289790572L, + "salary", 89266.9066406803, + "age", (short) 26, + "yoe", 1 ), ImmutableMap.of( - "birthday", 1034695930678172L, - "country", "Honduras", - "city", "Hutchinson", - "surname", "Vinson", - "name", "Keneth", - "state", "Connecticut", - "email", "questions2074@gmail.com" + "birthday", 777945600L, + "name", "Employee9", + "id", 4197550591L, + "salary", 82030.03829290869, + "age", (short) 29, + "yoe", 9 ), ImmutableMap.of( - "birthday", 1166606855236945L, - "country", "Senegal", - "city", "Galt", - "surname", "Schwartz", - "name", "Hee", - "state", "New Jersey", - "email", "statements2016@protonmail.com" + "birthday", 1105747200L, + "name", "Employee10", + "id", 1628304468L, + "salary", 87309.74810429095, + "age", (short) 19, + "yoe", 1 ) ) ); public static final InputRowSchema SCHEMA = new InputRowSchema( - new TimestampSpec("birthday", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), + new TimestampSpec("birthday", "posix", null), + new DimensionsSpec(ImmutableList.of( + new LongDimensionSchema("id"), + new LongDimensionSchema("birthday"), + new StringDimensionSchema("name"), + new LongDimensionSchema("age"), + new DoubleDimensionSchema("salary"), + new FloatDimensionSchema("bonus"), + new LongDimensionSchema("yoe"), + new StringDimensionSchema("is_fulltime"), + new LongDimensionSchema("last_vacation_time") + )), ColumnsFilter.all() ); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index 41ff30eca38d..e78f35879bbd 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -19,8 +19,10 @@ import argparse import delta -import mimesis import pyspark +from pyspark.sql.types import StructType, StructField, ShortType, StringType, TimestampType, LongType, IntegerType, DoubleType, FloatType, DateType, BooleanType +from datetime import datetime, timedelta +import random def config_spark_with_delta_lake(): @@ -38,31 +40,63 @@ def config_spark_with_delta_lake(): def create_dataset(num_records): - fake = mimesis.Generic() - output = [] - - for _ in range(num_records): - data = { - "name": fake.person.name(), - "surname": fake.person.surname(), - "birthday": fake.datetime.datetime(1980, 2010), - "email": fake.person.email(), - "country": fake.address.country(), - "state": fake.address.state(), - "city": fake.address.city(), - } - output.append(data) - - return output + """ + Generate a mock employee dataset with different datatypes for testing purposes. + + Parameters: + - num_records (int): Number of records to generate. + + Returns: + - Tuple: A tuple containing a list of records and the corresponding schema. + - List of Records: Each record is a tuple representing a row of data. + - StructType: The schema defining the structure of the records. + + Example: + ```python + data, schema = create_dataset(10) + ``` + """ + schema = StructType([ + StructField("id", LongType(), False), + StructField("birthday", DateType(), False), + StructField("name", StringType(), True), + StructField("age", ShortType(), True), + StructField("salary", DoubleType(), True), + StructField("bonus", FloatType(), True), + StructField("yoe", IntegerType(), True), + StructField("is_fulltime", BooleanType(), True), + StructField("last_vacation_time", TimestampType(), True) + ]) + + data = [] + current_date = datetime.now() + + for i in range(num_records): + birthday = current_date - timedelta(days=random.randint(365 * 18, 365 * 30)) + age = (current_date - birthday).days // 365 + is_fulltime = random.choice([True, False, None]) + record = ( + random.randint(1, 10000000000), + birthday, + f"Employee{i+1}", + age, + random.uniform(50000, 100000), + random.uniform(1000, 5000) if is_fulltime else None, + random.randint(1, min(20, age - 15)), + is_fulltime, + datetime.now() - timedelta(hours=random.randint(1, 90)) if is_fulltime else None, + ) + data.append(record) + return data, schema def main(): parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append", + parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="overwrite", help="Specify write mode (append/overwrite)") - parser.add_argument('--save_path', default=os.path.join(os.getcwd(), "people-delta-table4"), + parser.add_argument('--save_path', default=os.path.join(os.getcwd(), "employee-delta-table2"), help="Save path for Delta table") parser.add_argument('--num_records', type=int, default=10, help="Specify number of Delta records to write") @@ -75,9 +109,8 @@ def main(): spark = config_spark_with_delta_lake() - df = spark.createDataFrame(create_dataset(num_records=num_records)) - df = df.select(df.name, df.surname, df.birthday, df.email, df.country, df.state, df.city) - + data, schema = create_dataset(num_records=num_records) + df = spark.createDataFrame(data, schema=schema) df.write.format("delta").mode(save_mode).save(save_path) df.show() diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..358c33c316f07eb89d79aed963bf9ec77d84d1d7 GIT binary patch literal 28 kcmYc;N@ieSU}Bj0?>Y0wX?pj0TEEBcT~^3^mtCP10Gh`Ol>h($ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f1674bd05857f979d665194a21b777d52223f6b8 GIT binary patch literal 16 XcmYc;N@ieSU}7*_?!0BS0gnv;Au9we literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..cc96624a63532810675b8c3692b384252f89ee68 GIT binary patch literal 28 kcmYc;N@ieSU}6xt>KuQE^@5k((^b literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..a990561349e2b924808710395eaf5e0303f781d7 GIT binary patch literal 28 kcmYc;N@ieSU}9kFN&Ti_zVy}8lE|OFE&uMb=IuET0FDR@Z2$lO literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..45bedc7821f00be95686d155fe3df2544fa7bf80 GIT binary patch literal 28 kcmYc;N@ieSU}A`vden6H_QL)-mn&py8KwmBJOA8bDx0G*5segFUf literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b6d82bdc3f3fa47a8faa39b4817f4ff3185e8c23 GIT binary patch literal 28 kcmYc;N@ieSU}89YdAE^NVV{(Dcj)=O%L literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..ea61c7dfe913a6c0a2a392d8893292f16e04a9ff GIT binary patch literal 28 kcmYc;N@ieSU}EUi>-EfDbLoNTjPe6Ap)-zk#yw*O0En3j7ytkO literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000000.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..5326e6da9be6538a356471b49accfb1fc3731668 GIT binary patch literal 68 zcmV-K0K5NVa$^7h00IEM882Bu*U{{`Z=R|GaZREGeQR5Xg^j9q5VkT!fVJV>#vtM?hf_vQ>0N}t4-T(jq literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..f8f3c26d89e7 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json @@ -0,0 +1,13 @@ +{"commitInfo":{"timestamp":1706482237568,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"10","numOutputRows":"10","numOutputBytes":"23251"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"7859f5be-e2de-4f11-846b-12339efe101b"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"0cf39a1a-a30a-4f85-9921-22c480708b3a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"birthday\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"age\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"salary\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bonus\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"yoe\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"is_fulltime\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_vacation_time\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1706482234826}} +{"add":{"path":"part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8109925563,\"birthday\":\"2002-08-26\",\"name\":\"Employee1\",\"age\":21,\"salary\":98126.11562963494,\"yoe\":6},\"maxValues\":{\"id\":8109925563,\"birthday\":\"2002-08-26\",\"name\":\"Employee1\",\"age\":21,\"salary\":98126.11562963494,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":348540417,\"birthday\":\"2002-08-08\",\"name\":\"Employee2\",\"age\":21,\"salary\":88318.68501168216,\"yoe\":6},\"maxValues\":{\"id\":348540417,\"birthday\":\"2002-08-08\",\"name\":\"Employee2\",\"age\":21,\"salary\":88318.68501168216,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":644036573,\"birthday\":\"1994-06-27\",\"name\":\"Employee3\",\"age\":29,\"salary\":70031.88789434545,\"yoe\":14},\"maxValues\":{\"id\":644036573,\"birthday\":\"1994-06-27\",\"name\":\"Employee3\",\"age\":29,\"salary\":70031.88789434545,\"yoe\":14},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8451974441,\"birthday\":\"1999-10-17\",\"name\":\"Employee4\",\"age\":24,\"salary\":90127.23134932564,\"yoe\":4},\"maxValues\":{\"id\":8451974441,\"birthday\":\"1999-10-17\",\"name\":\"Employee4\",\"age\":24,\"salary\":90127.23134932564,\"yoe\":4},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1257915386,\"birthday\":\"1997-08-23\",\"name\":\"Employee5\",\"age\":26,\"salary\":55170.21435756755,\"yoe\":5},\"maxValues\":{\"id\":1257915386,\"birthday\":\"1997-08-23\",\"name\":\"Employee5\",\"age\":26,\"salary\":55170.21435756755,\"yoe\":5},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2034724452,\"birthday\":\"2002-06-04\",\"name\":\"Employee6\",\"age\":21,\"salary\":97643.72021601905,\"yoe\":1},\"maxValues\":{\"id\":2034724452,\"birthday\":\"2002-06-04\",\"name\":\"Employee6\",\"age\":21,\"salary\":97643.72021601905,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet","partitionValues":{},"size":2454,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1124457317,\"birthday\":\"2004-07-24\",\"name\":\"Employee7\",\"age\":19,\"salary\":60433.78056730033,\"bonus\":3610.4019,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T13:50:32.885-08:00\"},\"maxValues\":{\"id\":1124457317,\"birthday\":\"2004-07-24\",\"name\":\"Employee7\",\"age\":19,\"salary\":60433.78056730033,\"bonus\":3610.4019,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T13:50:32.885-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8289790572,\"birthday\":\"1997-06-29\",\"name\":\"Employee8\",\"age\":26,\"salary\":89266.9066406803,\"yoe\":1},\"maxValues\":{\"id\":8289790572,\"birthday\":\"1997-06-29\",\"name\":\"Employee8\",\"age\":26,\"salary\":89266.9066406803,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet","partitionValues":{},"size":2301,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4197550591,\"birthday\":\"1994-08-27\",\"name\":\"Employee9\",\"age\":29,\"salary\":82030.03829290869,\"yoe\":9},\"maxValues\":{\"id\":4197550591,\"birthday\":\"1994-08-27\",\"name\":\"Employee9\",\"age\":29,\"salary\":82030.03829290869,\"yoe\":9},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet","partitionValues":{},"size":2309,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1628304468,\"birthday\":\"2005-01-15\",\"name\":\"Employee10\",\"age\":19,\"salary\":87309.74810429095,\"yoe\":1},\"maxValues\":{\"id\":1628304468,\"birthday\":\"2005-01-15\",\"name\":\"Employee10\",\"age\":19,\"salary\":87309.74810429095,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..0bb51c210744 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json @@ -0,0 +1,6 @@ +{"commitInfo":{"timestamp":1706482267226,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"5","numOutputBytes":"12687"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"4f4db9e8-86c1-42f7-b035-ce817fc9186a"}} +{"add":{"path":"part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482267206,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4662990160,\"birthday\":\"1999-12-04\",\"name\":\"Employee1\",\"age\":24,\"salary\":63807.010056944906,\"yoe\":5},\"maxValues\":{\"id\":4662990160,\"birthday\":\"1999-12-04\",\"name\":\"Employee1\",\"age\":24,\"salary\":63807.010056944906,\"yoe\":5},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7436924672,\"birthday\":\"1998-11-01\",\"name\":\"Employee2\",\"age\":25,\"salary\":86178.47568217944,\"yoe\":10},\"maxValues\":{\"id\":7436924672,\"birthday\":\"1998-11-01\",\"name\":\"Employee2\",\"age\":25,\"salary\":86178.47568217944,\"yoe\":10},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267206,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1127302418,\"birthday\":\"1998-07-24\",\"name\":\"Employee3\",\"age\":25,\"salary\":59001.92470779706,\"yoe\":10},\"maxValues\":{\"id\":1127302418,\"birthday\":\"1998-07-24\",\"name\":\"Employee3\",\"age\":25,\"salary\":59001.92470779706,\"yoe\":10},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet","partitionValues":{},"size":2455,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1810758014,\"birthday\":\"1995-01-04\",\"name\":\"Employee4\",\"age\":29,\"salary\":97151.7200456219,\"bonus\":2880.2966,\"yoe\":10,\"last_vacation_time\":\"2024-01-26T14:51:01.826-08:00\"},\"maxValues\":{\"id\":1810758014,\"birthday\":\"1995-01-04\",\"name\":\"Employee4\",\"age\":29,\"salary\":97151.7200456219,\"bonus\":2880.2966,\"yoe\":10,\"last_vacation_time\":\"2024-01-26T14:51:01.826-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2675583494,\"birthday\":\"2000-02-13\",\"name\":\"Employee5\",\"age\":23,\"salary\":84092.96929134917,\"yoe\":6},\"maxValues\":{\"id\":2675583494,\"birthday\":\"2000-02-13\",\"name\":\"Employee5\",\"age\":23,\"salary\":84092.96929134917,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..857421fbf04d6bf51aedb7520db305ae76b95b05 GIT binary patch literal 2317 zcmb7G-D}%c6u*{bSxHUTrny1_GmKb7J2FvQY0?H0vcgI!p_DN~VT^Ds%ZZq5xsu#r zLSFV%MqyCc!?w_;QpV^Xuw}5p9zq6r*xOdvMnj?OZ7U7wLtEInS6`Q9A1(;#+;e~D zcRubp*S_-R+jWd^71!{mzdpNNsA2(@5|$AmObeImn5NF=Px_-s<)}f9N>WE2aDL*~bU3aM+X%#~x{5d*A)^ zbBTkbf$a=#-{0GRx(G}ad<-8v*?l<9!K6v%HH~}cv)_LybK9vSDRy)ePvAdx?mysD zE30wbl;XcUXvYOjmD4(LA{I0SA+{~j`e-WqaO$zF9-5lV96+Rn06SKXoK8-Y2XCpE zS`{=EED&7^iF2AHkg!~Hk^<-o3rA1gDpgQak*qpXwWWI}jvc3}aZpJi@klU4ScXNZ<@W|*f*tw}oE~V8xZz4? z;CYeT1&sun36F%)%5`Fs$n|?GY(o*Ws}WC`j6!C^rWluOEGLvH4$oJ$Hi znVo&Eg)&0dKokfmw`4r2d{n{<$XJFBOqjE58UK0Zg0b9;`y+k>FYA_g#q-^zd^;s< zB4ZOKwvrICN-?r`l&i4QKayy9Z@{%9C;V|A zF#36A0k)d6Fb?^tJf~)N%J@}eY}4B+%@JBA%W=C1Int?#-!#X3#gp9{1G4YSG{>7F z_(;h3u4AN4Wf1^P7&~}?j(vHeFDTKUWvnA(7l@>U=pULj%WPNtTV~$^2U30$6~T~| z4@;9U)6=Tz2dgIO6Bs>{O`&<+2}0<6wPcx=Sx4%rLx}L2XTrc<8#s{}_#c>ECn9YU z5p_>o*P~(I(dYFr3I=whPwO4m@!Fw2|L(d@M;5l+HmDPr=l51&)q@cvEf03V%$e3x zMty#LV`DnAOfQ1m9^5*k+p_>N-w>kvGTx!fbI0MDAJFthj6A-Ha`U!7py`fm&n~ar zyyN-gCA>rL=C*Dxa#o#Sbl=<+nX|Nf-*ZTBbe6-{YEDKk*)SsA{{PXvw}I44N$b7Q z4QffR*6L=xVK!$H9L^f}8qJL}kI8n$J3$gFM6wC0?SWi{$nqhX($ UIi+pF&&R(#=yRCqEAU6?FYHSg9RL6T literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..024e6cc74f81d70838454065adde0405de745e92 GIT binary patch literal 979 zcmb7D!D`z;5M3N6gw%&NK}Cif#Dbv#cZsxVocNYoDL#Z=LolP&%3iY4I=ia`WBe!m zg#1Upqkqzo>qLc6s(=o=Z|2Q=qnW+F`W^?J{?pf>2ODN=kOpa@y&f0;Q97L0+qxT1p9c|O-)(3 zZSK-Y5)ee+DL2h4t-UG`*+b+|gY(k|5a5+5r<=e>((m<$VQB1(!2-}G+c;(q(mKov zcRAxMCgI7e40o>?Hg{i1&#ZZ5h4hethwxW-Mg4Li>6kiiE8*#o=1S_!(ecfS*6m?P zWe9K6v?*tJE72~b8ly;)L;75ij#umT@P+yO(;7a?%J%$?ffsGGsNSNE!)S!v+}OIk zmd^i7t)XY8s@rYN-we8;=i0y<)NwT$t}4BpN!woguXSpwq{e6oW&0|xSFkBoFQoHO zEdE#Sc1^->+tvGA*l4?}QOx3mCDEs}QaX!9Bgs$2#pxyI7w1xZNdy-;oC|)Ij%3bx Z67wVxmuDyaANbV%^^u=Dgk0lu{tc%REcgHb literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6f1916b88dc1628e344e001f21a0df4a94876f10 GIT binary patch literal 2302 zcmb7GU1%It6uvtfN35u^`3NFSw09z+lp38LVW4}GZE2k}7=(>$n%=g!>uv8i?oVRG)d z=X~co=YIDl*Is+GjuF;z4L|tm?;pxK7GNo18KLLr1cXqzR9YQPgMq+5jaB^H_xC=j z6vC!nO2Teiafpdu#(&)U>-`rBA*Zh-A(MdeMyKcb1469IvhiqY{Hx%e%*LUqxm^el*&HA*9!Cx*r@ce9)J(HVnhF_+Ed^ej z^#~wws~qItQMw}IRp5#=U0K7IxwTiq*SUFE%DnqR7P?c$%RFFI&H{#F<~`shkhy!M z8um!=?t18K(hk9=5Ef?yRd8EC6%4uQ2GM4V3}&h;9y^mNIK?jM&>6I%GY5+*lMOmk zTe*As#7X9w0+$pLj|2mRWmr_2ezza;*nv;s^uha_8?Lweo)@_t@JQe}<&iL2zeyYt zxqf#&-cSTh*Y-?V$D(##mSAE)4hxE)Tvb+0c_}~R+)~h)?BsJVj1fc$ED%zLGM-j; zDtH;0Yal<4oOK)HUs5icYfFhbk}u%KT1i$#ypI{)sDdqI?v-?0<$&>|Vz9TA>#(y_ z&{M$#++1e`f3d=6GFFts3ciHQL#U8aW~0y89zuawl*yOTWfDzRMX&<~c28n-_XQnK zak~7_vgz&%Fy2(jxy8nN+*`zd!0;bRY`i=0NXQ&N(LoHqs4TSqcB$72jm5XO40cUn z%WZ+1hk1T?16EzQcBJXSE~L>}BSkb8x3;%uv&{7J=lM{Uj-MYB$Z7*c_gSigvh%{> zrXR5O#$=wnK6!e}@3VHt&W|6gJiYDt~-F?MB^hG@OfbXSF@}lks0aXczABHTcixAAS4@#{d8T literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c75d9f8fbf8fcfc4cd855951c7d20bf6a8eaecc2 GIT binary patch literal 2302 zcmb7G-)kII6uvt~{k&j1JqF)8~L^cjp$?Zab$malg(KvEAIqe;)rDmE{QWeNRd?_Twd5-`R zwVHrD8Kn^!Ujwd4)yOKooLD<0d?PVOrOdl4WTD$-yp#m&m9v1Mn0faT6Uf}VQVn|4 zzquCJ8>~a|353OIK@pr5Pz6J7I)1p(qJ!z`vdhn;3Qn<0J8T*)Yuv%3%5{ zJa(MBCcz~I)TRCaVHp;crq}HUN$kL5aQfhV$qClleb)`04tON+oN#FnuHB|K4IQt$ z7Htqg)zlqB*089amt>e2ki!BI$W?O9kQeha&MksYWhb9|VT>S3V1W=B%6N)wSMU-t zRzZFVa@K8xe~DZ+Ru^M+#9zRTv=Xn1cpr1Ty$Uvwu~X7;H35t!6@k4)ZotlBK~Dt} zaC4m%{KX2N%b1W+1z$qO2r8tM`RFsYBPbAwGX8S9T%z%+2)56`zK}TGT|vW>3Ekw- zvgz&$Fy2(jxy8qO*jvPZ!0~^T_;|PB&5;xQSO*dO0$GBs#W}bP`KdhSX?Ks{myxl@ zpQ4X44J%JnSy&fqbQ!gZqgow>t;qb&%Dk20MxEjEYmXUNI88d zBDm$6a7l0VS!nv+9kati+M*#(Lje4tmXVoo>k1nLmE*!n2s8G(FE{v^ze^;iohQ z;}2^P(oXMraPMp(}MY3+FCc)`dA{pKsWf m-KKN4HQStLZOdxZtwzJXID1;%fj=4l_k-S-5L$u%eEtPoxCWm9 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9097c721bf1eb7591787fd48a49b9dcab9202e34 GIT binary patch literal 2316 zcmb7GU1%It6uvt)gb^&*E=B`SizvU~>FoY%&TcFLe8!zwS6z0{6zdUwy3zwkh0GaYUBb$G^XR zuOgVFiEX|A<;mOsqV~qo#TPw0HM40VItx9>*ES6ZrA{Z+{RQ zFrvn3Glc&NpdFJmRY}_Q&HG@IY)SCBa#d0K#j{K?d=l`_-qB3?)g z_KNs?BHonoVj}L#u>quW=VB%7)8Lhr&|Y)86i*;5Pf4oewjl`+l$IMrYi+tdRax@b zu~a}Q_h{FdLQ7SqU{*!C>P*#^?i@dQjH$*zRSu~~gLQ-zSd?0RZxAN7>%If02ihZU zxY8MTUgUN`QwGh1N5g342DNGA`n?stK_soJ?HIsESv#$ip+ArumWV_yk;{hC%uhL& z2s)9SeXfNz0&5@(gvc!gPm=8tUO>h&Ffc*Rs^$3S$OU7$8S^8411@Jvyb|%=W_Wug zY$9U^#;lS6@=kHEm&jGvX$p)IM8M6NCg{ZoA1IiRgAzW6i~}~vNd(sVEZYMJz-bx3 zS-Om*@k#{yoPpgfvvhZ*DjrL~l0(b7yDLF^Q!eKcYwuyLi2sP;KPt2KZWpUqN$_JH zaQJz$09(yj7>E2+{$Orzjo_z|u?KEhjX)Xhv`Egw2Gq!=CVsOT^OXp8YZSmf8euiw zlp#ivgUC7#+9Zo$(1fvr2jKWu1bT-7{XBwoWb6Y_IRW|yvSyX-%YUovL%=}fJDEs^ zLbghiFw@hj>IbVP?Nb;%lTV>}!wEuQzFM+O%d8{yV# z8d3FuyrDjt%itF^jWZ4t6D9Xe}UGp#wNV_A*5)o9pf VXHIH6@bmF+4|=nV&=vR}^cVBY2vq<8 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ebbc17ed05c835ceb516b30663188995dc2bcc4a GIT binary patch literal 2317 zcmb7G-)q}e6uy>aMR85mrny1_GmN%q&B#P;rAZr1$O;Rkgi^)`g)zdlEhl2K}7r|V}t$y#e*^S5IV@iVDw=t8yg#a7=x`W?Ze2!&XumD%d$}ygmmt? z-}%mud(O45zW!zvBV5K6{NURMPvkNdU@2i4p{WG{A(YRdn`@uvKmz>?R`KKCKe|)m zV8xeX*h_N}#K5oOzu!CBdx3+Tz7#_yf#r=(&-DhjU1N0Le=vF|aoB`Ti#^i7KKuIU zVV;8|fqnT~Yxqj>$tp0F@JamiuY+HT988*tye4tq`}K#%Q`~mqs2Dq%!qfPV```U^ zf`d$`tOC#Qpq&ylRZi-}iCEASgraSc)<;vL52v0?>!GQc%mGAN2(XLlkWfjOs10!fr>Mv?WVGMFu`yYyH} zpcK2LW6z@XGF4Dio^08(mG!%)PoAQxBB-Q*xWpeIEW@JE@VfngVF#WK4FK8`PO#bT zyKd-oKqG->+9g4_d4n`b=y=`DXhRXSvNkke3`FgmBtv~5I4mfFa#^`z$o1@$Gf6>b z(zDOBP)6t)hyo#HPsTIKhXuTXj16cn#++sw@n2NdjE#ESAMqP_qi%^;Jl`jj?|uQB z$Qb6zxWovfN=0OEDc4}9&UF+g0&mu7oSyeElCh#ZE#QmDI0O$#$h7onw1?mzYRmXd z!=+6cuXwU=DA_%UhC33HTf_p+{{Ba*d z^vlW$Y}FTG9I{h6qGop{@QcXUr`HQJ%R|egIqv5nM>;j}o939Uc(T1oAp2^9=6G8K zAB>Fex`?!{tOB48V+Rk=MPHuiQ%dyb1lEyp07Mca`j2K!GdmFfmDv}-fs`LbMKEOL z{lW~)^t`Hi{+3C41V+z{rqH}$`vG*mTChyZtRi*p7$UgtnlP}}`*vvh-c7S(honV9 zqW&!I=;5Ge>&to&`u%38&+Bc+c3XkI{PvDcM;5l67N{A_^}1WI>cR+;h6}r3X3y&h zqrSYeyE~s+CKo|w4{q)8?V|uv-w>knJl>(pGsnTU=hO7Yj6A-HGV_+#r|FJuA6;IV zdE52Kb9jf|&1{`+Xm8p6_`bO-GGl3Yo@&d_JU=dU$mR&YE7%zCW}pLp|NDQEvr_wYPIHt Vg*j~qKOg`1ppPYluEHOoe*nSf0J#7F literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..51d47f4e0eeaaaf0b90556c364d926e937412723 GIT binary patch literal 2302 zcmb7G-)kII6uvt1loCo22^EoWclK^FbarPu zGuyI+fW?;x3;qEHsRhM{fDek)Jo&W3;y<8>R8;yP*oTJr;EU(Z&ue4bEriLr=brPO z@0|PHn_PS2tr|vH!&Us*?)@*0YFL0}8A}Kqn-dU1#lp-V!@mn)fqx1s`0noL!b-g~Na|zs|ho66WjEha-l(-@z_RGVc z?(lFav2Wk`_^Ze0;B`Q%;N$q<@ejXEa!BQ=sA|gl)9-)$fJaFSiE*LlAUtbZJt3%y zl#Yorv7pKbMcX19kE%q!IQK+04pqr*EUC4%JdK%_^u0WFWQ_lH#mK z0Et>nK>i!0$r8Q_T#>5DtN2o4?G*6!#N01r-aR1;-7ezgBw$#~0tRB{9VI4^*}Yr| zy2QV+7T6oKP4G#C#c4qioF-5OLuxpFxX~p2>B@@B&ZGiPu|wK)8m(x|!J>*}olaL* zc26BY!CaHzDhI?Reji~87Nv&Q=>JINf55xA~p#f zud^0y$bzb=JBFlTQ9UP>VPZfI3$h?zk*^xkQhvs{C8N{X$>&}eBZv}MAS4eYJSA_J z@G>%1L7fS5)@_7;QNCoXF2(AIzknNQC0_A(cNyNWgiU1Z6f|5(0HaAoU~kLUVP}cc z6K4W$t~1WhSNK}Ovbr?nV&gsRv33w9@(9VzpTf6#e)qd0PMRXtjB=}H4+ZI>M|w+@d+mPSn^*HlMy7B zljQ#qXofL*65~E#Ao*ue77R&#zcdB+cvew7f88Wq0@uxqZk~CI`T?k2DOsjv){ruD zC?dG&ns7;P_GoDO-fgo@L((K6QGOLS^|0Ti`l23$e$Ni|S-s^@w;AY*?{4bsn!=XT z1a}hVdYyGxb>P~Oh6}rpMrZXDQD5BL+M3NW)61XdLs>e0encRv4HTUhsSe7{3kMsX z&)OT4dHnk1=}oW4+8sMTda&~Jmg|uhs1AC~bDd5|*Qq~#ZSGmjQ5v4-Qqmb8OP(|*uLWrVK5e?I>LmQfAX literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3f3aad4af9aa0d598eeead82b08002a11cc3e964 GIT binary patch literal 2317 zcmb7G-D@0G6u&z=yP2e+v}x`zBa1A98C!`%cPAfhSVF8AN(iNhgn~%8yL&eooZZ>Z z%vP3=mp(|*K1dOKFb_UxtB^lH7WAQFsE`Mrgo28tP|ybr7%CJWJonDm#rUy>VdmU( zf9H2T?m2h&@=LE)Fv2BV#vgt0-fp3UC0G`*g3$392_clvJ$d=b3J3`FlUT#oZ~yS~ zr~n&#E`oKR=MWRWfWP|e&bxU5a_V9P83mSC+FjS{Q#u>dy*>EgKo+nG-7}F#7T9-p ze!VLRkR-5gTJrbH2fr)<(>OkXAOF7pa7=&|C%M-o?)N|W`VTR^#8Dw~bR19M-|pT2 z>6pMUqD65tivRMUpOkb>N$MnsSkhI5__oaIqig&ls7KR!=vpQV0GSm6?07vgIweUS zyd`2{mD4q_Kx`?+&RLQ`!gI|?a-hp`9PE}<1z&<_nY4zNV`(Rcuf*bEF4YdC6nZ0% z7h}L)J_X#A@lveamm>g#vwLwo=o0_U^`Nmy+XPP_EKfo_(f;GD6ot6bPxe6g;WEQ^1SJT!Z$CnbT}J{{?l~T&qR>5xs$zcT2Pq`EE15 zdj)JEb0=5A<1rzx6eoL4y#hNmp`!#5c(YCu^rDA>f>rf#0bfAoA$UkaW~EP~Jp>25 zEu%LJmo;g$63M=0WSXW zmg7yh&QMGyb{!|JsY?K;!PvnAbo?t4eaeUqMzDd*eIP2vME}sNX=eNK-!l6WIFS0I ztV*V$UN20-Oiyc?=Wke~OJMXYK84mb>IcyITEVt#tAeyMM-ahP*Mfn)+M}W6dv957 z8j>aniS|I=GQxhB8Vg1c`n^VIOdBnSy3N2?czw%YBMVzj6Vx%x^*S4{>c9w+x(mBt zMyHL0(OB5p-kwe^lZzm;2e)*1dmcdQ8$xuR#yfO*<~Z2&e3ssjkw-UCX5RFAEZw2) z`Q??Fw_J}rg?H%P%+~3Ibc6cC`xdUqjHT{*E+w7eS&m+-85z4|gOIel|3~-6Hqut& z)_bw-m*ZY7SFB3as+LdJdyd;I&(Bl)Y~$S7^R|6%jy7hi4ZG1Ia}9f@K2KY=U9H&F YYUBLO8GQ$SKK|`N9~Ke141a|F0$TnGNB{r; literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..2764351d1526971dc9db47568e43ea169a713c81 GIT binary patch literal 2317 zcmb7GO=ufO6dtWs>$Req#PKY3D3584qyQ{|*ioQMTYK`7i7S$#A${BY{=v>uw8$s9mrg#f#-9vPjSBoE#a zF|o>NDp(-46k_KrNgxSx%}8>f%Y``DEh{p<2+=ZW6`zZx?Hs-wi~G4$I}}pr^*mmR z0Xz8=a6`n)v35_401(ct^CiDWyw}(K<_7H$Jc+P4EvSOsf*?SU8@3m0w8&t(wBoR1 zDS=Y#k`A3lD`lo&s61Jx)0LH5M-Lxisv@W)pE$%DAS}b8&~UqbKgJGR3a1C!L$<%x z?mJFkcR(Y7X3`;kuy&0!NnpF(wQxfbw6eBs$Ym^QCnXu`1HoZI5tIwcMMJJ+Z(TeB$kn!yn zu!)Rqn6pw$7*;AIdquenJ9Vz3I1zZWPUG~vhoOuW9t>EaQTVml3g)%P2WbvV;)g1~@-lUf^iIw-Dmgj%S_#aBFyw~$(EXVv& zAB6Oa$`WkV=V2VOQ~8sbeK3ZfMaB-8r8z>&q&e>7AxAbf(VOL%t$4DV<3M(IjOBPk z1Rrq>eAk7fb!8aU!%Y=@A$`Gn_*68ufhWe6?VimRUvWu>%PIs$;^yUhUJs^xQYh4h=|) z1VsHw+|+|XkLruMA9(#{pwH@Un>sCDUwmy-XCn(+b_>)o%yGNxupGNlM`t;Bt!8BGlJx`9>HQzwl`W)Rh+FUZ zj#r6$wNf>!HM3TkY4mNURasb|)`{k+6Q?cf)I4pTtTnA>o6I+@xyAx*TUM=V)oRVt VbH}u8`1$y^2fZgDbP4_l{RNMH09pV5 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..5c2c23ddaf44d165a323fcf995b68030c07e5976 GIT binary patch literal 2316 zcmb7`UuYaf7{KTD_AZD{JD8;=kJ)=-X8Bo76VFn70?EbiT2 zclRnshzLR~eKGX0Ek20&AjKD<1oXj&w1gl&3W5)cc~*#0K~el>_TOOqbA{vXn{U4F z_x+h~Cb{;~t5u9}8CUSVPamsgEWxUX6@*UCN(iBRuHAcO2m%8A6jt&6tw#^@g4x7# zk=e4zO$_`z-ru=%_>5q3W+gHi2g=LczUvKbdoIR({QG-PMB4<1qlhf8_uskmhZr$Q z?5!7b+n;^?>k^Pk_!NHf^OrwN2vLhuJZh5nqjx{~^MtUS1jGTF~0CyA)3%EKf_SCOOg$azwX|{NbZZXb-{XOXeTo|o7y4&Q3&P%#Gm!?3269ms#H>~mls@~uoHLm_V$ zrr=J`sH*3$o3u~i>Y4l&nm27ffXP=2mT8$)q@Fuw5!`T1xUe?{c4+$En`YMzX`6;r z{a)VG!(rdn7xW=`{F z>I<7&TQjL=@(^VH5Y`^upC^#Uh7_Hri4Ieq1r9bmpOrUa=Fv-(xwpLmD|cjn{&;2X z9oM5z5gleX^L2Wmy>9!X#}=N*jMDTx*QUMEDaW7H48|VWAf(;?|KVQWLh6-x^j_%t zm3USwRkK<%Yn8Ljf#bF-^YgZKzIE~ZCCj=vXSXiYT2`w==UUcmbKdS)R;_B)YOPDN U=d}Cq=i}c#=w1<_Yw&;2Ull+8)&Kwi literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0436637b71ca2c88c9fad9fd5f40739115dac5b6 GIT binary patch literal 2454 zcmb7`U1%It6oBu}&Tb~@8rmfHG6VatEbh?CI&^o^CJl?k*v3*qEh3>H67KHKW(Q|? zwmY+xCFZ5BdFWFKK_LXum#Q=%6dwXUS;4*pm4c#(q98tL)qpumEcS!&-*5$06Ev{Sa#nqW&+yg;PW?+aJC_C(htb;`suH? ze@k*EVP4I}=AX0Iex46VAAj@Nry0&HB(-nSbAicQcfWe&kq4NFez0*ofj_$b-HjAy zG9vrAC$I=-&*E?2-}@y~fg*uVrufc>rNZE8gl=Ciu<{&*0v+7C@!mKCh<+8NtPem* z<(QzzQdmt+#DYQ)^0q}*Mn(2soO(1WnXi^2urX?Rqb}q@rlX4ZNPT~?Ge*iY4j}1HCv!b zgJ!~_j=OS|)~IXR?G2ufDjQKc*vl~Yn0Y6i(+fe7RRxu{B&_-n@!LC2#Wh_z5g z=pV=eA#z{BQ{;;jegdh>&`1F}I<3c_Cuh~=ir**x6S!X2_$!`wZ4`K0X{;f28?I$0 z0D6`3V9$}uuv6hWixYtxcO0kZBfK_>3AvNT3rIa+7e9!=O78_&Ad9d8>9Y3ppDf*Y z#lvE*U<^R}63!!a3&vTRfkfp&P3-YrCW8bRs`tdKoZ*MsKUWaxKX#E) zgrO%w4?`of7b4q`1|m08L{KI2NooqlMNyXR?y5#R6mGTVU2yHH(RJWb$thjawGxs~ z9I|k(SQ^ZND?P*2y7o(2%W!F(x>WvNT+6%tj**|sJ8rjEbMwV~!!)eAlb?HjEzhO_ zY?*aX2WFPtUWHW~Za1x3unTTRF&`51b8GAC#mF<9F|j{{HHP>15{P0$isr*ahYJ)3 zcA9pV9dF3Y{UH~-*X^x#QKIY1MNxGlqV$Hhc1nuFsw} rYNyIIz1E{hp{{0hUE9A#{9NKnUe?&N=Pp97v#_zzY84_WeJ`IJ2== zBeVI_p^1Ud;ctH2d0661PF;>nMuGB1yX&}pO6OwShl5X_@=+7q%TYv@*n_=aKYoeB zC5d_4pS^E>|N9b;|MN0a_iC9o&gu-o+^+#30A5J}*_Cr-NnKKYsBfu~0M~0J<;vrfhCSEyJ z0T0BMLL8hG36QYX43Yy~F0_MPO_uOg$d*Yf_);uwYV?U^kx{ z+!gUstlbwQ14w7*a>?%!@10e@u}<3rk0UHj3W{JiAqfzqy6pw)P12t%Ej#S|OP~}x zq)jK$vc?q5Do@tvWO;e#)QOW!H3q7pPaNX)5td+4sJoq>AKUg_3a1C!W46EA>N!qe zw?R_`&A3DSVD%Phkid33tKo($sG7QENE#N^3sMpK1Ib}Q7UV1PRYR&}r<_SLI+LD# zriC`b)Ib&p$@e5YA%9fBi^y1k35>~UwITnae92g;Me`AT12>$OXvOosWW2osHj%NF z({L#UhMfvwZ^_qTr^ZbbCjvK{G)~V)7)V%_4-5DrG7ca@k}_+3n(YAu2*)z|X6dpa zjaEGD0fT*AWa$nB4Ufg?#)p=6cOXD}lUdFr*50F9p8tgLpA=bp@8mTs#r$XvLi~cf z2wSxoxDMH={Kee97{RX~V;9`g8ewG88h7(hBfB-xH>)vQ@vwWN0J}fJYP<{28H#h@ zr!Iun(0>eTTG_t%ugbm!29kdgWx7Lf_0~+%C2;l3@D`f4sOQ7vD+SB6%nDM@9`wsp8KxZrU7Y^fGCf| z4L#_0slK55f!AvU`jpyRxulH$fenIc{eSRvoy4r0&2jxX~#+A?gbo zo10UqXYvqa{t%W9?;j?R#)cH_7l{s2o(1;TU5}MFWaiOJl({$E9xHcf|M2n3+*^)I zULZQmZsu!u0=h=M;bU`8WJamGu0u&@c*@adHG{E7)(=R#`+vB%H<5BB9=+GvUOArC za>cAv&1(5fy=Obk^6V_N&Nt@H&s)~q3~gMfHmpXA%rva&`Ydf(R<&YPtBv{Tv+5T7 Q`FPd`eO^T98vG;l4+z}@H2?qr literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b004cba7ac0fd4e938414adce026972823ce61b3 GIT binary patch literal 2455 zcmb7GU1%It6uvt$qkx&o`v%9m&(Ak~s z%xqXbs+{QBZsU zl$7^!vLu9bGA8C^5h1$G6B=cSz8LptM4K$dHWR=Tz95g%7Q+c4M^G&^*(|~hWKoLZ zES~TIZc0n+P9$V0kqFWiQNRn(HIfx^5g!YzK?0u)%$-E!-QuFq^3Azmy@+ zfRrf;_N;gkc4~~W7!z<~#W8-i!t0}06tAc8EGq6mg|KBpebns^6rl3-pQK$PR)58U zT_#}XMo7C`oPsk!yTM0`*xllQ-cZxAMd;n_W$~{Q{0~P6z3VV9gaF^yLDcSg651vA z!he!>;}r{clfd1f?QWCt6TlHEMPS(v2l5A&L)w?>}YL#Aa^^D=TaH*t}rfO;# zNe6aCxTh@@X2I#6;i-=Og4#Aby{UV;^drBT_xfEUKb?0yr`PcErF_dYtfrfver`2S zrU7i3O>hTcmfcx~RR?ajUbkQu(u`6*M9fdGuC0}#%y7oU`B2svKA$!a)rJ<$2dNGh zC@$=-*bX_~ke&NOE>3USJ#yTk^XXiS(_5CUKR|Vmf1GP}JY(5#hOf;`-x#HC+m@ks zhDX^wv12eYzgSI>pjzI=BB0$?O@~Z!6TY> t_>j?J2R;e_OR1e4l_{;QfKlDc$p%d^g+21N^IL!b6 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b29c26d395fda57a12839a07f3aaf6162528fab2 GIT binary patch literal 2301 zcmb7GPiP!f7=JrEJDtQu`X_IgfmN2#EUm0VcPCAn5P}vgp)93{gn~$TyZbg7oZZ>Z z%vP2V@Dy-8)JqS_QhE@)^i)JJ2TwWV;6-m%5VamXw7Pguu>Ic5o0(^$b_-$hz3+YB z@B4k<`@NT3dHu~AM!1B__|scoeo`!936=_05qf?>LI~w^k3OjV2?hfH9M`qoLY}VW&q`lZr}BWlrD3)4?p|nTUCfn;9iI;GGg!F z*f)iUgx9w6-S|gk@)`gY@p1ge?_b}a6~dIcsAxj_m+v3k6T3_5FiuZL_`d?i3CYma zWJ-dGB|}3f+LqaL3_bcKxM$L776z!$kS&f%-voa9pPzJwdxyO+mn+&svq0V6r}9&!`N+*vOM zed52n8ML-(m*A5K%kz>hIc=Z{hT3%eaJx;0^TmeC&ZG!VxktKm9yLnL!J_hHi_VuD zcTOEY!CbT8QUc-%ANsEMz z*V~LXG|4C#;|hpTHqNOEObp0jNt3io+U1J6nw@cGY3Ou%@|hRL2$BL82x%h~&uM!F zyoM?ppiYjQb{pYe&@NUsR%3Oz==5kv@UzW<@v8x=#H7)UFq7%eHDr zKXW7^xaL}LNv{oPX!+iIR+omPO+upoB=4BvuushuGYI`bD>SQS$DwXJFjwB*G1)bR zEvF4`9_D(zEm-y7+L5LUyO2h!W`bz0?CkDV)6C@ZXZcW;PM;qUNNWQ{=UJ+Qva`a$ zw&%0zBD!%QdT3x9a87&4J^#%S%htKHEBfcGnE%XnNJoLdr%tJt5r7DV0iZ3e0;)4&Kd*|!Yw7P|1=G=3C z=XXBtId^yY)z_;S;WDn^2Y=lBx>&{nEF~->baYxk2<3Cqdp^CykrX*PipTJ8d*A(Z zgkvbGQQVZ`zdUHi1x=NcI&mTvGzB5HEzE*qw4KA3WAPxDYIlSbdLxe) zV!&=b1#F3UG1l&j5dgxud!ZEch<{}@Xs+8Gg2xaRCj?b+S`Y*Xa>Mb%^%faSl$Kn2 zEG1BiUDB~9&{CNyC@N3Z?1{?K-4n--Q`IP_q=2}@A0RBlqR{ZV{UF8;JR43Av`3s^ zwcU5!(CL6i0?n99f^hX3X_C?_o#Ait@OC&mrRgJR~90(x=fLfCJW+ z(VK=#n>1SSWZzPa#Eo*{S?P%|0#SmyoebZ>!iW4=t1CxSNL@>C{ATnq#)&$!?7R*;hrH z&Vy#A}J<%O0%Y!?Tb%jb`Lm^@}sB- zhOAsKjKfS%sjBC%nWRTx^h`E|<~7?7p!3y&Wm;wxsizJhf~&3x1ADb^hokPs-tzi1-J$K- z<&~MYU5`A6cj(>B*6D`!n(Ytoo4X=2mWJoKHt7z}a`;-!$mk^-grw8^Kf1Rzk$N$1 zy;nMZCGOQq)vVUcTIFP;@3^hX+?;KlZk{=P*0Rpb+Rd3-(`vTKY}1-<%-L1}7nb$_OsZfIl}%#ek#$xLUH-7o|d46=kJh~%(~gqoh3q_Jmu zw!3F#47rFP$rkH+57#jq+3Q zhnFyWZ@%@{_iK+{0i+T>k0*Z|{aob2q;gm`A^z2epFcjWP%bL;6omiV)-DK|DkpQ| zOe|;$LgBW^=A)_MFV0=e=Ao(Ff(sGZBp@>!NCqe;-9x>^OtNyC3K@tkg}6NH557bR09&N#$~s<&t%DrC8Jm+_>OBVhn-3ZoMNB!=nPubnS({;$tInttlqzL z{sMCq!6o^`A>IgK85V`M+aLOI?8v2X{@^`p`y1V%;{=I>Zk)?huOvw%gwb zHxxnBwS$_hV^O;zOE57YhXqAYt}8ccaw|LK%u>*7diI$Y#t5W(GA=8-1-y)E z>!8mVIqf#Ye@$7bt+yh5L|?!S)e^0Eyw4ckxPT2*JILv{6a$8n3c=Quo3PX31;yKd z%t?*k;j4ThV?~)1@CvF;ph`lUjXz~Pfg+(Wqc5Y(WE!n_ux}XHp~NU33OX*vl;eX- zCwwTth!Ztu78~(NFOUB#!~acUBi_yHSdQ@{Erjq($}()V7T`i;C-XN;yLSe^h-za< zOM3*_q&<%F&?CDy(UMGpxrw5o*L7c;%%`_QWTc++)edA(JCWmg3}7 zM4Dn8J&ExUFp%=2s0cM#dB0GGyIfaQ*V{D6fWVbA!rN!up`H(lR|}?L8V#ggJ`v&H zb_}?vw}&(^JojCrM+4F!0a1Svx2wTuK&wktKk$ZDP_0+HHg!6Fb?Mf2m0eZXvOD08 z!yLE239CL_J<@hy7t(0Gnjlt}ws&^wX=d^OWcg5*PM;qVNNWQ{`&p`kva`bemg}+h zreq$yKv{an9kO<(&JQ20EWPWvpGP5r$;&YoMvF` zarFbz8$1o}?H#0EkEQo=&#S~*tu%~A(`Z&^+e6#wR2CPhIcHs+yJnhK7pQfmX_;1+ kELi4zdy#fcv)M44P3zkHW$gg|Z2Z>``ba`(4gLZ82e5?+K>z>% literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt index 41bd50d04e2e..7424dac98ed9 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt @@ -1,4 +1,3 @@ delta-spark==3.0.0 pyspark==3.2.0 -mimesis==5.0.0 argparse==1.4.0 \ No newline at end of file From 69a654f2330e9e3a2ec72915cddda25ca443482a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 19:29:38 -0800 Subject: [PATCH 47/65] Upgrade pyspark and delta-spark to latest (3.5.0 and 3.0.0) and regenerate tests --- .../delta/input/DeltaInputSourceTest.java | 11 +- .../druid/delta/input/DeltaTestUtil.java | 178 +++++++++--------- ...-9ad7-31ec84e39880-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-8a8b-f599ea586597-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-9c70-7424d71cb267-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-80a9-5c012b9f1a4c-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-b9db-bb48174d98dd-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-90e4-6cfec3897d78-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-91df-a700e6a6ead9-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-b556-c2f8d414faa1-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-a91c-9e6c1b04d884-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-adfb-02377478da38-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-982b-acdffa9be0de-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-ac19-64e8ec608096-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-9f34-ccb270f44d70-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc | Bin 28 -> 0 bytes ...-96ca-f160d4bf9d0a-c000.snappy.parquet.crc | Bin 28 -> 0 bytes .../_delta_log/.00000000000000000000.json.crc | Bin 68 -> 72 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 36 -> 36 bytes .../_delta_log/00000000000000000000.json | 24 +-- .../_delta_log/00000000000000000001.json | 12 +- ...4209-9ad7-31ec84e39880-c000.snappy.parquet | Bin 2317 -> 0 bytes ...4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet | Bin 979 -> 0 bytes ...429a-8a8b-f599ea586597-c000.snappy.parquet | Bin 2302 -> 0 bytes ...4b4f-9c70-7424d71cb267-c000.snappy.parquet | Bin 2302 -> 0 bytes ...4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet | Bin 2316 -> 0 bytes ...49f0-b9db-bb48174d98dd-c000.snappy.parquet | Bin 2317 -> 0 bytes ...4866-90e4-6cfec3897d78-c000.snappy.parquet | Bin 2302 -> 0 bytes ...423d-91df-a700e6a6ead9-c000.snappy.parquet | Bin 2317 -> 0 bytes ...42c7-b556-c2f8d414faa1-c000.snappy.parquet | Bin 2317 -> 0 bytes ...4137-a91c-9e6c1b04d884-c000.snappy.parquet | Bin 2316 -> 0 bytes ...46a0-adfb-02377478da38-c000.snappy.parquet | Bin 2454 -> 0 bytes ...46f0-982b-acdffa9be0de-c000.snappy.parquet | Bin 2317 -> 0 bytes ...4e66-ac19-64e8ec608096-c000.snappy.parquet | Bin 2455 -> 0 bytes ...410a-9f34-ccb270f44d70-c000.snappy.parquet | Bin 2301 -> 0 bytes ...4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet | Bin 2317 -> 0 bytes ...4df0-96ca-f160d4bf9d0a-c000.snappy.parquet | Bin 2309 -> 0 bytes .../src/test/resources/requirements.txt | 3 +- 39 files changed, 114 insertions(+), 114 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index d4d8446f058c..c080064e1696 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -19,15 +19,12 @@ package org.apache.druid.delta.input; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.hamcrest.MatcherAssert; import org.junit.Assert; @@ -60,14 +57,10 @@ public void testSampleDeltaTable() throws IOException Assert.assertNotNull(actualSampledRawVals); Assert.assertNotNull(actualSampledRow.getRawValuesList()); Assert.assertEquals(1, actualSampledRow.getRawValuesList().size()); - ObjectMapper objectMapper = new DefaultObjectMapper(); - objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL); - System.out.println("JSON:" + objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(actualSampledRawVals)); for (String key : expectedRow.keySet()) { if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = (Long) expectedRow.get(key); Assert.assertEquals(expectedMillis, actualSampledRawVals.get(key)); - } else { Assert.assertEquals(expectedRow.get(key), actualSampledRawVals.get(key)); } @@ -112,7 +105,7 @@ public void testReadDeltaLakeWithSplits() { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) - .collect(Collectors.toList()); + .collect(Collectors.toList()); Assert.assertEquals(2, splits.size()); for (int idx = 0; idx < splits.size(); idx++) { @@ -122,7 +115,7 @@ public void testReadDeltaLakeWithSplits() split ); List> splitsResult = deltaInputSourceWithSplitx.createSplits(null, null) - .collect(Collectors.toList()); + .collect(Collectors.toList()); Assert.assertEquals(1, splitsResult.size()); Assert.assertEquals(split, splitsResult.get(0).get()); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 25206f78912e..705797520951 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -62,137 +62,145 @@ public class DeltaTestUtil public static final List> EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( - "birthday", 944265600L, - "id", 4662990160L, + "birthday", 1057881600L, "name", "Employee1", - "age", (short) 24, - "salary", 63807.010056944906, - "yoe", 5 + "id", 867799346L, + "salary", 87642.55209817083, + "age", (short) 20, + "yoe", 4 ), ImmutableMap.of( - "id", 7436924672L, - "birthday", 909878400L, + "birthday", 1035417600L, + "is_fulltime", false, "name", "Employee2", - "age", (short) 25, - "salary", 86178.47568217944, - "yoe", 10, - "is_fulltime", false + "id", 9963151889L, + "salary", 79404.63969727767, + "age", (short) 21, + "yoe", 2 ), ImmutableMap.of( - "id", 1127302418L, - "birthday", 901238400L, + "birthday", 890179200L, "name", "Employee3", + "id", 2766777393L, + "salary", 92418.21424435009, "age", (short) 25, - "salary", 59001.92470779706, - "yoe", 10, - "is_fulltime", false + "yoe", 9 ), ImmutableMap.of( - "id", 1810758014L, - "birthday", 789177600L, + "birthday", 1073001600L, "name", "Employee4", - "age", (short) 29, - "salary", 97151.7200456219, - "bonus", 2880.2966f, - "yoe", 10, - "is_fulltime", true, - "last_vacation_time", 1706309461000L + "id", 6320361986L, + "salary", 97907.76612488469, + "age", (short) 20, + "yoe", 3 ), ImmutableMap.of( - "id", 2675583494L, - "birthday", 950400000L, + "birthday", 823996800L, + "is_fulltime", true, + "bonus", 4982.215f, "name", "Employee5", - "age", (short) 23, - "salary", 84092.96929134917, - "yoe", 6, - "is_fulltime", false + "id", 7068152260L, + "salary", 79037.77202099308, + "last_vacation_time", 1706256972000L, + "age", (short) 27, + "yoe", 9 ), ImmutableMap.of( - "id", 8109925563L, - "birthday", 1030320000L, + "birthday", 937526400L, + "is_fulltime", false, "name", "Employee1", - "age", (short) 21, - "salary", 98126.11562963494, - "yoe", 6 + "id", 4693651733L, + "salary", 83845.11357786917, + "age", (short) 24, + "yoe", 3 ), ImmutableMap.of( - "id", 348540417L, - "birthday", 1028764800L, + "birthday", 810777600L, + "is_fulltime", false, "name", "Employee2", - "salary", 88318.68501168216, - "age", (short) 21, - "yoe", 6 + "id", 7132772589L, + "salary", 90140.44051385639, + "age", (short) 28, + "yoe", 8 ), ImmutableMap.of( - "birthday", 772675200L, - "is_fulltime", false, + "birthday", 1104969600L, + "is_fulltime", true, + "bonus", 3699.0881f, "name", "Employee3", - "id", 644036573L, - "salary", 70031.88789434545, - "age", (short) 29, - "yoe", 14 + "id", 6627278510L, + "salary", 58857.27649436368, + "last_vacation_time", 1706458554000L, + "age", (short) 19, + "yoe", 4 ), ImmutableMap.of( - "birthday", 940118400L, + "birthday", 763257600L, + "is_fulltime", true, + "bonus", 2334.6675f, "name", "Employee4", - "id", 8451974441L, - "salary", 90127.23134932564, - "age", (short) 24, - "yoe", 4 + "id", 4786204912L, + "salary", 93646.81222022788, + "last_vacation_time", 1706390154000L, + "age", (short) 29, + "yoe", 5 ), ImmutableMap.of( - "birthday", 872294400L, - "is_fulltime", false, + "birthday", 1114646400L, "name", "Employee5", - "id", 1257915386L, - "salary", 55170.21435756755, - "age", (short) 26, - "yoe", 5 + "id", 2773939764L, + "salary", 66300.05339373322, + "age", (short) 18, + "yoe", 3 ), ImmutableMap.of( - "birthday", 1023148800L, + "birthday", 913334400L, "is_fulltime", false, "name", "Employee6", - "id", 2034724452L, - "salary", 97643.72021601905, - "age", (short) 21, - "yoe", 1 + "id", 8333438088L, + "salary", 59219.5257906128, + "age", (short) 25, + "yoe", 4 ), ImmutableMap.of( - "birthday", 1090627200L, - "is_fulltime", true, - "bonus", 3610.4019f, + "birthday", 893894400L, + "is_fulltime", false, "name", "Employee7", - "id", 1124457317L, - "salary", 60433.78056730033, - "last_vacation_time", 1706478632000L, - "age", (short) 19, - "yoe", 4 + "id", 8397454007L, + "salary", 61909.733851830584, + "age", (short) 25, + "yoe", 8 ), ImmutableMap.of( - "birthday", 867542400L, - "is_fulltime", false, + "birthday", 1038873600L, + "is_fulltime", true, + "bonus", 3000.0154f, "name", "Employee8", - "id", 8289790572L, - "salary", 89266.9066406803, - "age", (short) 26, + "id", 8925359945L, + "salary", 76588.05471316943, + "last_vacation_time", 1706195754000L, + "age", (short) 21, "yoe", 1 ), ImmutableMap.of( - "birthday", 777945600L, + "birthday", 989798400L, + "is_fulltime", true, + "bonus", 4463.3833f, "name", "Employee9", - "id", 4197550591L, - "salary", 82030.03829290869, - "age", (short) 29, - "yoe", 9 + "id", 8154788551L, + "salary", 59787.98539015684, + "last_vacation_time", 1706181354000L, + "age", (short) 22, + "yoe", 4 ), ImmutableMap.of( - "birthday", 1105747200L, + "birthday", 912297600L, + "is_fulltime", false, "name", "Employee10", - "id", 1628304468L, - "salary", 87309.74810429095, - "age", (short) 19, - "yoe", 1 + "id", 5884382356L, + "salary", 51565.91965119349, + "age", (short) 25, + "yoe", 9 ) ) ); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet.crc deleted file mode 100644 index 358c33c316f07eb89d79aed963bf9ec77d84d1d7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}Bj0?>Y0wX?pj0TEEBcT~^3^mtCP10Gh`Ol>h($ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet.crc deleted file mode 100644 index f1674bd05857f979d665194a21b777d52223f6b8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}7*_?!0BS0gnv;Au9we diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet.crc deleted file mode 100644 index cc96624a63532810675b8c3692b384252f89ee68..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}6xt>KuQE^@5k((^b diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet.crc deleted file mode 100644 index a990561349e2b924808710395eaf5e0303f781d7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}9kFN&Ti_zVy}8lE|OFE&uMb=IuET0FDR@Z2$lO diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet.crc deleted file mode 100644 index 45bedc7821f00be95686d155fe3df2544fa7bf80..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}A`vden6H_QL)-mn&py8KwmBJOA8bDx0G*5segFUf diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet.crc deleted file mode 100644 index b6d82bdc3f3fa47a8faa39b4817f4ff3185e8c23..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}89YdAE^NVV{(Dcj)=O%L diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet.crc deleted file mode 100644 index ea61c7dfe913a6c0a2a392d8893292f16e04a9ff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28 kcmYc;N@ieSU}EUi>-EfDbLoNTjPe6Ap)-zk#yw*O0En3j7ytkO diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000000.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/.00000000000000000000.json.crc index 5326e6da9be6538a356471b49accfb1fc3731668..75a1c6db2e3587f055b5bec92ac0da35ff333065 100644 GIT binary patch literal 72 zcmV-O0Jr~Ra$^7h00IEXk6fO$;Y^bBVC3<-Dv0A?4^>nR+6=oOz_cS)zo(cs*1cI7RFRNl`sR3j+V>#vtM?hf_vQ>0N}t4-T(jq diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json index f8f3c26d89e7..4a07c503c436 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000000.json @@ -1,13 +1,13 @@ -{"commitInfo":{"timestamp":1706482237568,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"10","numOutputRows":"10","numOutputBytes":"23251"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"7859f5be-e2de-4f11-846b-12339efe101b"}} +{"commitInfo":{"timestamp":1706498159640,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"10","numOutputRows":"10","numOutputBytes":"23710"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"5557f0e2-6cdc-4249-bd19-368bec96751d"}} +{"metaData":{"id":"c29741f1-4649-43a8-b2d9-9d46c42c58e1","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"birthday\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"age\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"salary\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bonus\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"yoe\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"is_fulltime\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_vacation_time\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1706498156768}} {"protocol":{"minReaderVersion":1,"minWriterVersion":2}} -{"metaData":{"id":"0cf39a1a-a30a-4f85-9921-22c480708b3a","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"birthday\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"age\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"salary\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bonus\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"yoe\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"is_fulltime\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_vacation_time\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1706482234826}} -{"add":{"path":"part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8109925563,\"birthday\":\"2002-08-26\",\"name\":\"Employee1\",\"age\":21,\"salary\":98126.11562963494,\"yoe\":6},\"maxValues\":{\"id\":8109925563,\"birthday\":\"2002-08-26\",\"name\":\"Employee1\",\"age\":21,\"salary\":98126.11562963494,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":348540417,\"birthday\":\"2002-08-08\",\"name\":\"Employee2\",\"age\":21,\"salary\":88318.68501168216,\"yoe\":6},\"maxValues\":{\"id\":348540417,\"birthday\":\"2002-08-08\",\"name\":\"Employee2\",\"age\":21,\"salary\":88318.68501168216,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":644036573,\"birthday\":\"1994-06-27\",\"name\":\"Employee3\",\"age\":29,\"salary\":70031.88789434545,\"yoe\":14},\"maxValues\":{\"id\":644036573,\"birthday\":\"1994-06-27\",\"name\":\"Employee3\",\"age\":29,\"salary\":70031.88789434545,\"yoe\":14},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8451974441,\"birthday\":\"1999-10-17\",\"name\":\"Employee4\",\"age\":24,\"salary\":90127.23134932564,\"yoe\":4},\"maxValues\":{\"id\":8451974441,\"birthday\":\"1999-10-17\",\"name\":\"Employee4\",\"age\":24,\"salary\":90127.23134932564,\"yoe\":4},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1257915386,\"birthday\":\"1997-08-23\",\"name\":\"Employee5\",\"age\":26,\"salary\":55170.21435756755,\"yoe\":5},\"maxValues\":{\"id\":1257915386,\"birthday\":\"1997-08-23\",\"name\":\"Employee5\",\"age\":26,\"salary\":55170.21435756755,\"yoe\":5},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2034724452,\"birthday\":\"2002-06-04\",\"name\":\"Employee6\",\"age\":21,\"salary\":97643.72021601905,\"yoe\":1},\"maxValues\":{\"id\":2034724452,\"birthday\":\"2002-06-04\",\"name\":\"Employee6\",\"age\":21,\"salary\":97643.72021601905,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet","partitionValues":{},"size":2454,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1124457317,\"birthday\":\"2004-07-24\",\"name\":\"Employee7\",\"age\":19,\"salary\":60433.78056730033,\"bonus\":3610.4019,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T13:50:32.885-08:00\"},\"maxValues\":{\"id\":1124457317,\"birthday\":\"2004-07-24\",\"name\":\"Employee7\",\"age\":19,\"salary\":60433.78056730033,\"bonus\":3610.4019,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T13:50:32.885-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} -{"add":{"path":"part-00007-54b67894-6c52-46f0-982b-acdffa9be0de-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8289790572,\"birthday\":\"1997-06-29\",\"name\":\"Employee8\",\"age\":26,\"salary\":89266.9066406803,\"yoe\":1},\"maxValues\":{\"id\":8289790572,\"birthday\":\"1997-06-29\",\"name\":\"Employee8\",\"age\":26,\"salary\":89266.9066406803,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet","partitionValues":{},"size":2301,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4197550591,\"birthday\":\"1994-08-27\",\"name\":\"Employee9\",\"age\":29,\"salary\":82030.03829290869,\"yoe\":9},\"maxValues\":{\"id\":4197550591,\"birthday\":\"1994-08-27\",\"name\":\"Employee9\",\"age\":29,\"salary\":82030.03829290869,\"yoe\":9},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00009-75e9ed6c-f99e-4df0-96ca-f160d4bf9d0a-c000.snappy.parquet","partitionValues":{},"size":2309,"modificationTime":1706482236883,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1628304468,\"birthday\":\"2005-01-15\",\"name\":\"Employee10\",\"age\":19,\"salary\":87309.74810429095,\"yoe\":1},\"maxValues\":{\"id\":1628304468,\"birthday\":\"2005-01-15\",\"name\":\"Employee10\",\"age\":19,\"salary\":87309.74810429095,\"yoe\":1},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4693651733,\"birthday\":\"1999-09-17\",\"name\":\"Employee1\",\"age\":24,\"salary\":83845.11357786917,\"yoe\":3},\"maxValues\":{\"id\":4693651733,\"birthday\":\"1999-09-17\",\"name\":\"Employee1\",\"age\":24,\"salary\":83845.11357786917,\"yoe\":3},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7132772589,\"birthday\":\"1995-09-11\",\"name\":\"Employee2\",\"age\":28,\"salary\":90140.44051385639,\"yoe\":8},\"maxValues\":{\"id\":7132772589,\"birthday\":\"1995-09-11\",\"name\":\"Employee2\",\"age\":28,\"salary\":90140.44051385639,\"yoe\":8},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00002-b2249397-0232-4a5c-b504-62c7c27702c1-c000.snappy.parquet","partitionValues":{},"size":2455,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6627278510,\"birthday\":\"2005-01-06\",\"name\":\"Employee3\",\"age\":19,\"salary\":58857.27649436368,\"bonus\":3699.0881,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T08:15:54.648-08:00\"},\"maxValues\":{\"id\":6627278510,\"birthday\":\"2005-01-06\",\"name\":\"Employee3\",\"age\":19,\"salary\":58857.27649436368,\"bonus\":3699.0881,\"yoe\":4,\"last_vacation_time\":\"2024-01-28T08:15:54.648-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet","partitionValues":{},"size":2454,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4786204912,\"birthday\":\"1994-03-10\",\"name\":\"Employee4\",\"age\":29,\"salary\":93646.81222022788,\"bonus\":2334.6675,\"yoe\":5,\"last_vacation_time\":\"2024-01-27T13:15:54.648-08:00\"},\"maxValues\":{\"id\":4786204912,\"birthday\":\"1994-03-10\",\"name\":\"Employee4\",\"age\":29,\"salary\":93646.81222022788,\"bonus\":2334.6675,\"yoe\":5,\"last_vacation_time\":\"2024-01-27T13:15:54.648-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2773939764,\"birthday\":\"2005-04-28\",\"name\":\"Employee5\",\"age\":18,\"salary\":66300.05339373322,\"yoe\":3},\"maxValues\":{\"id\":2773939764,\"birthday\":\"2005-04-28\",\"name\":\"Employee5\",\"age\":18,\"salary\":66300.05339373322,\"yoe\":3},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet","partitionValues":{},"size":2316,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8333438088,\"birthday\":\"1998-12-11\",\"name\":\"Employee6\",\"age\":25,\"salary\":59219.5257906128,\"yoe\":4},\"maxValues\":{\"id\":8333438088,\"birthday\":\"1998-12-11\",\"name\":\"Employee6\",\"age\":25,\"salary\":59219.5257906128,\"yoe\":4},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00006-78cd057e-faaa-477d-b5fd-d00a857f7e54-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8397454007,\"birthday\":\"1998-04-30\",\"name\":\"Employee7\",\"age\":25,\"salary\":61909.733851830584,\"yoe\":8},\"maxValues\":{\"id\":8397454007,\"birthday\":\"1998-04-30\",\"name\":\"Employee7\",\"age\":25,\"salary\":61909.733851830584,\"yoe\":8},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00007-00eb0d30-e71e-4092-8ea2-0ee576ca7327-c000.snappy.parquet","partitionValues":{},"size":2455,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8925359945,\"birthday\":\"2002-12-03\",\"name\":\"Employee8\",\"age\":21,\"salary\":76588.05471316943,\"bonus\":3000.0154,\"yoe\":1,\"last_vacation_time\":\"2024-01-25T07:15:54.648-08:00\"},\"maxValues\":{\"id\":8925359945,\"birthday\":\"2002-12-03\",\"name\":\"Employee8\",\"age\":21,\"salary\":76588.05471316943,\"bonus\":3000.0154,\"yoe\":1,\"last_vacation_time\":\"2024-01-25T07:15:54.648-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00008-9f526a56-2392-4f1c-8c07-3dac19b12e91-c000.snappy.parquet","partitionValues":{},"size":2455,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8154788551,\"birthday\":\"2001-05-14\",\"name\":\"Employee9\",\"age\":22,\"salary\":59787.98539015684,\"bonus\":4463.3833,\"yoe\":4,\"last_vacation_time\":\"2024-01-25T03:15:54.648-08:00\"},\"maxValues\":{\"id\":8154788551,\"birthday\":\"2001-05-14\",\"name\":\"Employee9\",\"age\":22,\"salary\":59787.98539015684,\"bonus\":4463.3833,\"yoe\":4,\"last_vacation_time\":\"2024-01-25T03:15:54.648-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} +{"add":{"path":"part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet","partitionValues":{},"size":2324,"modificationTime":1706498158981,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5884382356,\"birthday\":\"1998-11-29\",\"name\":\"Employee10\",\"age\":25,\"salary\":51565.91965119349,\"yoe\":9},\"maxValues\":{\"id\":5884382356,\"birthday\":\"1998-11-29\",\"name\":\"Employee10\",\"age\":25,\"salary\":51565.91965119349,\"yoe\":9},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json index 0bb51c210744..188f2417e6a9 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/_delta_log/00000000000000000001.json @@ -1,6 +1,6 @@ -{"commitInfo":{"timestamp":1706482267226,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"5","numOutputBytes":"12687"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"4f4db9e8-86c1-42f7-b035-ce817fc9186a"}} -{"add":{"path":"part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706482267206,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4662990160,\"birthday\":\"1999-12-04\",\"name\":\"Employee1\",\"age\":24,\"salary\":63807.010056944906,\"yoe\":5},\"maxValues\":{\"id\":4662990160,\"birthday\":\"1999-12-04\",\"name\":\"Employee1\",\"age\":24,\"salary\":63807.010056944906,\"yoe\":5},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7436924672,\"birthday\":\"1998-11-01\",\"name\":\"Employee2\",\"age\":25,\"salary\":86178.47568217944,\"yoe\":10},\"maxValues\":{\"id\":7436924672,\"birthday\":\"1998-11-01\",\"name\":\"Employee2\",\"age\":25,\"salary\":86178.47568217944,\"yoe\":10},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267206,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1127302418,\"birthday\":\"1998-07-24\",\"name\":\"Employee3\",\"age\":25,\"salary\":59001.92470779706,\"yoe\":10},\"maxValues\":{\"id\":1127302418,\"birthday\":\"1998-07-24\",\"name\":\"Employee3\",\"age\":25,\"salary\":59001.92470779706,\"yoe\":10},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} -{"add":{"path":"part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet","partitionValues":{},"size":2455,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1810758014,\"birthday\":\"1995-01-04\",\"name\":\"Employee4\",\"age\":29,\"salary\":97151.7200456219,\"bonus\":2880.2966,\"yoe\":10,\"last_vacation_time\":\"2024-01-26T14:51:01.826-08:00\"},\"maxValues\":{\"id\":1810758014,\"birthday\":\"1995-01-04\",\"name\":\"Employee4\",\"age\":29,\"salary\":97151.7200456219,\"bonus\":2880.2966,\"yoe\":10,\"last_vacation_time\":\"2024-01-26T14:51:01.826-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} -{"add":{"path":"part-00009-29baf0b7-5ccc-4d01-a6d7-c8a93c8c1d76-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706482267205,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2675583494,\"birthday\":\"2000-02-13\",\"name\":\"Employee5\",\"age\":23,\"salary\":84092.96929134917,\"yoe\":6},\"maxValues\":{\"id\":2675583494,\"birthday\":\"2000-02-13\",\"name\":\"Employee5\",\"age\":23,\"salary\":84092.96929134917,\"yoe\":6},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"commitInfo":{"timestamp":1706498178162,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"5","numOutputBytes":"12655"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"38049998-65f7-4ed6-8d48-e51c5da502c6"}} +{"add":{"path":"part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet","partitionValues":{},"size":2301,"modificationTime":1706498176452,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":867799346,\"birthday\":\"2003-07-11\",\"name\":\"Employee1\",\"age\":20,\"salary\":87642.55209817083,\"yoe\":4},\"maxValues\":{\"id\":867799346,\"birthday\":\"2003-07-11\",\"name\":\"Employee1\",\"age\":20,\"salary\":87642.55209817083,\"yoe\":4},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet","partitionValues":{},"size":2317,"modificationTime":1706498176452,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9963151889,\"birthday\":\"2002-10-24\",\"name\":\"Employee2\",\"age\":21,\"salary\":79404.63969727767,\"yoe\":2},\"maxValues\":{\"id\":9963151889,\"birthday\":\"2002-10-24\",\"name\":\"Employee2\",\"age\":21,\"salary\":79404.63969727767,\"yoe\":2},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00005-9ff9b585-5a9e-415f-b28a-a85d960ccb04-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706498176452,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2766777393,\"birthday\":\"1998-03-18\",\"name\":\"Employee3\",\"age\":25,\"salary\":92418.21424435009,\"yoe\":9},\"maxValues\":{\"id\":2766777393,\"birthday\":\"1998-03-18\",\"name\":\"Employee3\",\"age\":25,\"salary\":92418.21424435009,\"yoe\":9},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet","partitionValues":{},"size":2302,"modificationTime":1706498176452,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6320361986,\"birthday\":\"2004-01-02\",\"name\":\"Employee4\",\"age\":20,\"salary\":97907.76612488469,\"yoe\":3},\"maxValues\":{\"id\":6320361986,\"birthday\":\"2004-01-02\",\"name\":\"Employee4\",\"age\":20,\"salary\":97907.76612488469,\"yoe\":3},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":1,\"yoe\":0,\"is_fulltime\":1,\"last_vacation_time\":1}}"}} +{"add":{"path":"part-00009-e21486a5-e177-4c02-b256-bc890fadce7e-c000.snappy.parquet","partitionValues":{},"size":2454,"modificationTime":1706498176452,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7068152260,\"birthday\":\"1996-02-11\",\"name\":\"Employee5\",\"age\":27,\"salary\":79037.77202099308,\"bonus\":4982.215,\"yoe\":9,\"last_vacation_time\":\"2024-01-26T00:16:12.196-08:00\"},\"maxValues\":{\"id\":7068152260,\"birthday\":\"1996-02-11\",\"name\":\"Employee5\",\"age\":27,\"salary\":79037.77202099308,\"bonus\":4982.215,\"yoe\":9,\"last_vacation_time\":\"2024-01-26T00:16:12.196-08:00\"},\"nullCount\":{\"id\":0,\"birthday\":0,\"name\":0,\"age\":0,\"salary\":0,\"bonus\":0,\"yoe\":0,\"is_fulltime\":0,\"last_vacation_time\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-66d42216-962e-4209-9ad7-31ec84e39880-c000.snappy.parquet deleted file mode 100644 index 857421fbf04d6bf51aedb7520db305ae76b95b05..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2317 zcmb7G-D}%c6u*{bSxHUTrny1_GmKb7J2FvQY0?H0vcgI!p_DN~VT^Ds%ZZq5xsu#r zLSFV%MqyCc!?w_;QpV^Xuw}5p9zq6r*xOdvMnj?OZ7U7wLtEInS6`Q9A1(;#+;e~D zcRubp*S_-R+jWd^71!{mzdpNNsA2(@5|$AmObeImn5NF=Px_-s<)}f9N>WE2aDL*~bU3aM+X%#~x{5d*A)^ zbBTkbf$a=#-{0GRx(G}ad<-8v*?l<9!K6v%HH~}cv)_LybK9vSDRy)ePvAdx?mysD zE30wbl;XcUXvYOjmD4(LA{I0SA+{~j`e-WqaO$zF9-5lV96+Rn06SKXoK8-Y2XCpE zS`{=EED&7^iF2AHkg!~Hk^<-o3rA1gDpgQak*qpXwWWI}jvc3}aZpJi@klU4ScXNZ<@W|*f*tw}oE~V8xZz4? z;CYeT1&sun36F%)%5`Fs$n|?GY(o*Ws}WC`j6!C^rWluOEGLvH4$oJ$Hi znVo&Eg)&0dKokfmw`4r2d{n{<$XJFBOqjE58UK0Zg0b9;`y+k>FYA_g#q-^zd^;s< zB4ZOKwvrICN-?r`l&i4QKayy9Z@{%9C;V|A zF#36A0k)d6Fb?^tJf~)N%J@}eY}4B+%@JBA%W=C1Int?#-!#X3#gp9{1G4YSG{>7F z_(;h3u4AN4Wf1^P7&~}?j(vHeFDTKUWvnA(7l@>U=pULj%WPNtTV~$^2U30$6~T~| z4@;9U)6=Tz2dgIO6Bs>{O`&<+2}0<6wPcx=Sx4%rLx}L2XTrc<8#s{}_#c>ECn9YU z5p_>o*P~(I(dYFr3I=whPwO4m@!Fw2|L(d@M;5l+HmDPr=l51&)q@cvEf03V%$e3x zMty#LV`DnAOfQ1m9^5*k+p_>N-w>kvGTx!fbI0MDAJFthj6A-Ha`U!7py`fm&n~ar zyyN-gCA>rL=C*Dxa#o#Sbl=<+nX|Nf-*ZTBbe6-{YEDKk*)SsA{{PXvw}I44N$b7Q z4QffR*6L=xVK!$H9L^f}8qJL}kI8n$J3$gFM6wC0?SWi{$nqhX($ UIi+pF&&R(#=yRCqEAU6?FYHSg9RL6T diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-d242b4c2-9dfa-4e99-9e0b-1b7be05e3d3e-c000.snappy.parquet deleted file mode 100644 index 024e6cc74f81d70838454065adde0405de745e92..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 979 zcmb7D!D`z;5M3N6gw%&NK}Cif#Dbv#cZsxVocNYoDL#Z=LolP&%3iY4I=ia`WBe!m zg#1Upqkqzo>qLc6s(=o=Z|2Q=qnW+F`W^?J{?pf>2ODN=kOpa@y&f0;Q97L0+qxT1p9c|O-)(3 zZSK-Y5)ee+DL2h4t-UG`*+b+|gY(k|5a5+5r<=e>((m<$VQB1(!2-}G+c;(q(mKov zcRAxMCgI7e40o>?Hg{i1&#ZZ5h4hethwxW-Mg4Li>6kiiE8*#o=1S_!(ecfS*6m?P zWe9K6v?*tJE72~b8ly;)L;75ij#umT@P+yO(;7a?%J%$?ffsGGsNSNE!)S!v+}OIk zmd^i7t)XY8s@rYN-we8;=i0y<)NwT$t}4BpN!woguXSpwq{e6oW&0|xSFkBoFQoHO zEdE#Sc1^->+tvGA*l4?}QOx3mCDEs}QaX!9Bgs$2#pxyI7w1xZNdy-;oC|)Ij%3bx Z67wVxmuDyaANbV%^^u=Dgk0lu{tc%REcgHb diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-52e22cc5-86d9-429a-8a8b-f599ea586597-c000.snappy.parquet deleted file mode 100644 index 6f1916b88dc1628e344e001f21a0df4a94876f10..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2302 zcmb7GU1%It6uvtfN35u^`3NFSw09z+lp38LVW4}GZE2k}7=(>$n%=g!>uv8i?oVRG)d z=X~co=YIDl*Is+GjuF;z4L|tm?;pxK7GNo18KLLr1cXqzR9YQPgMq+5jaB^H_xC=j z6vC!nO2Teiafpdu#(&)U>-`rBA*Zh-A(MdeMyKcb1469IvhiqY{Hx%e%*LUqxm^el*&HA*9!Cx*r@ce9)J(HVnhF_+Ed^ej z^#~wws~qItQMw}IRp5#=U0K7IxwTiq*SUFE%DnqR7P?c$%RFFI&H{#F<~`shkhy!M z8um!=?t18K(hk9=5Ef?yRd8EC6%4uQ2GM4V3}&h;9y^mNIK?jM&>6I%GY5+*lMOmk zTe*As#7X9w0+$pLj|2mRWmr_2ezza;*nv;s^uha_8?Lweo)@_t@JQe}<&iL2zeyYt zxqf#&-cSTh*Y-?V$D(##mSAE)4hxE)Tvb+0c_}~R+)~h)?BsJVj1fc$ED%zLGM-j; zDtH;0Yal<4oOK)HUs5icYfFhbk}u%KT1i$#ypI{)sDdqI?v-?0<$&>|Vz9TA>#(y_ z&{M$#++1e`f3d=6GFFts3ciHQL#U8aW~0y89zuawl*yOTWfDzRMX&<~c28n-_XQnK zak~7_vgz&%Fy2(jxy8nN+*`zd!0;bRY`i=0NXQ&N(LoHqs4TSqcB$72jm5XO40cUn z%WZ+1hk1T?16EzQcBJXSE~L>}BSkb8x3;%uv&{7J=lM{Uj-MYB$Z7*c_gSigvh%{> zrXR5O#$=wnK6!e}@3VHt&W|6gJiYDt~-F?MB^hG@OfbXSF@}lks0aXczABHTcixAAS4@#{d8T diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-7c7e82c7-0754-4b4f-9c70-7424d71cb267-c000.snappy.parquet deleted file mode 100644 index c75d9f8fbf8fcfc4cd855951c7d20bf6a8eaecc2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2302 zcmb7G-)kII6uvt~{k&j1JqF)8~L^cjp$?Zab$malg(KvEAIqe;)rDmE{QWeNRd?_Twd5-`R zwVHrD8Kn^!Ujwd4)yOKooLD<0d?PVOrOdl4WTD$-yp#m&m9v1Mn0faT6Uf}VQVn|4 zzquCJ8>~a|353OIK@pr5Pz6J7I)1p(qJ!z`vdhn;3Qn<0J8T*)Yuv%3%5{ zJa(MBCcz~I)TRCaVHp;crq}HUN$kL5aQfhV$qClleb)`04tON+oN#FnuHB|K4IQt$ z7Htqg)zlqB*089amt>e2ki!BI$W?O9kQeha&MksYWhb9|VT>S3V1W=B%6N)wSMU-t zRzZFVa@K8xe~DZ+Ru^M+#9zRTv=Xn1cpr1Ty$Uvwu~X7;H35t!6@k4)ZotlBK~Dt} zaC4m%{KX2N%b1W+1z$qO2r8tM`RFsYBPbAwGX8S9T%z%+2)56`zK}TGT|vW>3Ekw- zvgz&$Fy2(jxy8qO*jvPZ!0~^T_;|PB&5;xQSO*dO0$GBs#W}bP`KdhSX?Ks{myxl@ zpQ4X44J%JnSy&fqbQ!gZqgow>t;qb&%Dk20MxEjEYmXUNI88d zBDm$6a7l0VS!nv+9kati+M*#(Lje4tmXVoo>k1nLmE*!n2s8G(FE{v^ze^;iohQ z;}2^P(oXMraPMp(}MY3+FCc)`dA{pKsWf m-KKN4HQStLZOdxZtwzJXID1;%fj=4l_k-S-5L$u%eEtPoxCWm9 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-17dea9b2-e614-4ebf-80a9-5c012b9f1a4c-c000.snappy.parquet deleted file mode 100644 index 9097c721bf1eb7591787fd48a49b9dcab9202e34..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2316 zcmb7GU1%It6uvt)gb^&*E=B`SizvU~>FoY%&TcFLe8!zwS6z0{6zdUwy3zwkh0GaYUBb$G^XR zuOgVFiEX|A<;mOsqV~qo#TPw0HM40VItx9>*ES6ZrA{Z+{RQ zFrvn3Glc&NpdFJmRY}_Q&HG@IY)SCBa#d0K#j{K?d=l`_-qB3?)g z_KNs?BHonoVj}L#u>quW=VB%7)8Lhr&|Y)86i*;5Pf4oewjl`+l$IMrYi+tdRax@b zu~a}Q_h{FdLQ7SqU{*!C>P*#^?i@dQjH$*zRSu~~gLQ-zSd?0RZxAN7>%If02ihZU zxY8MTUgUN`QwGh1N5g342DNGA`n?stK_soJ?HIsESv#$ip+ArumWV_yk;{hC%uhL& z2s)9SeXfNz0&5@(gvc!gPm=8tUO>h&Ffc*Rs^$3S$OU7$8S^8411@Jvyb|%=W_Wug zY$9U^#;lS6@=kHEm&jGvX$p)IM8M6NCg{ZoA1IiRgAzW6i~}~vNd(sVEZYMJz-bx3 zS-Om*@k#{yoPpgfvvhZ*DjrL~l0(b7yDLF^Q!eKcYwuyLi2sP;KPt2KZWpUqN$_JH zaQJz$09(yj7>E2+{$Orzjo_z|u?KEhjX)Xhv`Egw2Gq!=CVsOT^OXp8YZSmf8euiw zlp#ivgUC7#+9Zo$(1fvr2jKWu1bT-7{XBwoWb6Y_IRW|yvSyX-%YUovL%=}fJDEs^ zLbghiFw@hj>IbVP?Nb;%lTV>}!wEuQzFM+O%d8{yV# z8d3FuyrDjt%itF^jWZ4t6D9Xe}UGp#wNV_A*5)o9pf VXHIH6@bmF+4|=nV&=vR}^cVBY2vq<8 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-200cb9f6-1bab-49f0-b9db-bb48174d98dd-c000.snappy.parquet deleted file mode 100644 index ebbc17ed05c835ceb516b30663188995dc2bcc4a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2317 zcmb7G-)q}e6uy>aMR85mrny1_GmN%q&B#P;rAZr1$O;Rkgi^)`g)zdlEhl2K}7r|V}t$y#e*^S5IV@iVDw=t8yg#a7=x`W?Ze2!&XumD%d$}ygmmt? z-}%mud(O45zW!zvBV5K6{NURMPvkNdU@2i4p{WG{A(YRdn`@uvKmz>?R`KKCKe|)m zV8xeX*h_N}#K5oOzu!CBdx3+Tz7#_yf#r=(&-DhjU1N0Le=vF|aoB`Ti#^i7KKuIU zVV;8|fqnT~Yxqj>$tp0F@JamiuY+HT988*tye4tq`}K#%Q`~mqs2Dq%!qfPV```U^ zf`d$`tOC#Qpq&ylRZi-}iCEASgraSc)<;vL52v0?>!GQc%mGAN2(XLlkWfjOs10!fr>Mv?WVGMFu`yYyH} zpcK2LW6z@XGF4Dio^08(mG!%)PoAQxBB-Q*xWpeIEW@JE@VfngVF#WK4FK8`PO#bT zyKd-oKqG->+9g4_d4n`b=y=`DXhRXSvNkke3`FgmBtv~5I4mfFa#^`z$o1@$Gf6>b z(zDOBP)6t)hyo#HPsTIKhXuTXj16cn#++sw@n2NdjE#ESAMqP_qi%^;Jl`jj?|uQB z$Qb6zxWovfN=0OEDc4}9&UF+g0&mu7oSyeElCh#ZE#QmDI0O$#$h7onw1?mzYRmXd z!=+6cuXwU=DA_%UhC33HTf_p+{{Ba*d z^vlW$Y}FTG9I{h6qGop{@QcXUr`HQJ%R|egIqv5nM>;j}o939Uc(T1oAp2^9=6G8K zAB>Fex`?!{tOB48V+Rk=MPHuiQ%dyb1lEyp07Mca`j2K!GdmFfmDv}-fs`LbMKEOL z{lW~)^t`Hi{+3C41V+z{rqH}$`vG*mTChyZtRi*p7$UgtnlP}}`*vvh-c7S(honV9 zqW&!I=;5Ge>&to&`u%38&+Bc+c3XkI{PvDcM;5l67N{A_^}1WI>cR+;h6}r3X3y&h zqrSYeyE~s+CKo|w4{q)8?V|uv-w>knJl>(pGsnTU=hO7Yj6A-HGV_+#r|FJuA6;IV zdE52Kb9jf|&1{`+Xm8p6_`bO-GGl3Yo@&d_JU=dU$mR&YE7%zCW}pLp|NDQEvr_wYPIHt Vg*j~qKOg`1ppPYluEHOoe*nSf0J#7F diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-33f770a9-7c99-4866-90e4-6cfec3897d78-c000.snappy.parquet deleted file mode 100644 index 51d47f4e0eeaaaf0b90556c364d926e937412723..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2302 zcmb7G-)kII6uvt1loCo22^EoWclK^FbarPu zGuyI+fW?;x3;qEHsRhM{fDek)Jo&W3;y<8>R8;yP*oTJr;EU(Z&ue4bEriLr=brPO z@0|PHn_PS2tr|vH!&Us*?)@*0YFL0}8A}Kqn-dU1#lp-V!@mn)fqx1s`0noL!b-g~Na|zs|ho66WjEha-l(-@z_RGVc z?(lFav2Wk`_^Ze0;B`Q%;N$q<@ejXEa!BQ=sA|gl)9-)$fJaFSiE*LlAUtbZJt3%y zl#Yorv7pKbMcX19kE%q!IQK+04pqr*EUC4%JdK%_^u0WFWQ_lH#mK z0Et>nK>i!0$r8Q_T#>5DtN2o4?G*6!#N01r-aR1;-7ezgBw$#~0tRB{9VI4^*}Yr| zy2QV+7T6oKP4G#C#c4qioF-5OLuxpFxX~p2>B@@B&ZGiPu|wK)8m(x|!J>*}olaL* zc26BY!CaHzDhI?Reji~87Nv&Q=>JINf55xA~p#f zud^0y$bzb=JBFlTQ9UP>VPZfI3$h?zk*^xkQhvs{C8N{X$>&}eBZv}MAS4eYJSA_J z@G>%1L7fS5)@_7;QNCoXF2(AIzknNQC0_A(cNyNWgiU1Z6f|5(0HaAoU~kLUVP}cc z6K4W$t~1WhSNK}Ovbr?nV&gsRv33w9@(9VzpTf6#e)qd0PMRXtjB=}H4+ZI>M|w+@d+mPSn^*HlMy7B zljQ#qXofL*65~E#Ao*ue77R&#zcdB+cvew7f88Wq0@uxqZk~CI`T?k2DOsjv){ruD zC?dG&ns7;P_GoDO-fgo@L((K6QGOLS^|0Ti`l23$e$Ni|S-s^@w;AY*?{4bsn!=XT z1a}hVdYyGxb>P~Oh6}rpMrZXDQD5BL+M3NW)61XdLs>e0encRv4HTUhsSe7{3kMsX z&)OT4dHnk1=}oW4+8sMTda&~Jmg|uhs1AC~bDd5|*Qq~#ZSGmjQ5v4-Qqmb8OP(|*uLWrVK5e?I>LmQfAX diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-ac5afcdd-66c4-423d-91df-a700e6a6ead9-c000.snappy.parquet deleted file mode 100644 index 3f3aad4af9aa0d598eeead82b08002a11cc3e964..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2317 zcmb7G-D@0G6u&z=yP2e+v}x`zBa1A98C!`%cPAfhSVF8AN(iNhgn~%8yL&eooZZ>Z z%vP3=mp(|*K1dOKFb_UxtB^lH7WAQFsE`Mrgo28tP|ybr7%CJWJonDm#rUy>VdmU( zf9H2T?m2h&@=LE)Fv2BV#vgt0-fp3UC0G`*g3$392_clvJ$d=b3J3`FlUT#oZ~yS~ zr~n&#E`oKR=MWRWfWP|e&bxU5a_V9P83mSC+FjS{Q#u>dy*>EgKo+nG-7}F#7T9-p ze!VLRkR-5gTJrbH2fr)<(>OkXAOF7pa7=&|C%M-o?)N|W`VTR^#8Dw~bR19M-|pT2 z>6pMUqD65tivRMUpOkb>N$MnsSkhI5__oaIqig&ls7KR!=vpQV0GSm6?07vgIweUS zyd`2{mD4q_Kx`?+&RLQ`!gI|?a-hp`9PE}<1z&<_nY4zNV`(Rcuf*bEF4YdC6nZ0% z7h}L)J_X#A@lveamm>g#vwLwo=o0_U^`Nmy+XPP_EKfo_(f;GD6ot6bPxe6g;WEQ^1SJT!Z$CnbT}J{{?l~T&qR>5xs$zcT2Pq`EE15 zdj)JEb0=5A<1rzx6eoL4y#hNmp`!#5c(YCu^rDA>f>rf#0bfAoA$UkaW~EP~Jp>25 zEu%LJmo;g$63M=0WSXW zmg7yh&QMGyb{!|JsY?K;!PvnAbo?t4eaeUqMzDd*eIP2vME}sNX=eNK-!l6WIFS0I ztV*V$UN20-Oiyc?=Wke~OJMXYK84mb>IcyITEVt#tAeyMM-ahP*Mfn)+M}W6dv957 z8j>aniS|I=GQxhB8Vg1c`n^VIOdBnSy3N2?czw%YBMVzj6Vx%x^*S4{>c9w+x(mBt zMyHL0(OB5p-kwe^lZzm;2e)*1dmcdQ8$xuR#yfO*<~Z2&e3ssjkw-UCX5RFAEZw2) z`Q??Fw_J}rg?H%P%+~3Ibc6cC`xdUqjHT{*E+w7eS&m+-85z4|gOIel|3~-6Hqut& z)_bw-m*ZY7SFB3as+LdJdyd;I&(Bl)Y~$S7^R|6%jy7hi4ZG1Ia}9f@K2KY=U9H&F YYUBLO8GQ$SKK|`N9~Ke141a|F0$TnGNB{r; diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-640d557d-f469-42c7-b556-c2f8d414faa1-c000.snappy.parquet deleted file mode 100644 index 2764351d1526971dc9db47568e43ea169a713c81..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2317 zcmb7GO=ufO6dtWs>$Req#PKY3D3584qyQ{|*ioQMTYK`7i7S$#A${BY{=v>uw8$s9mrg#f#-9vPjSBoE#a zF|o>NDp(-46k_KrNgxSx%}8>f%Y``DEh{p<2+=ZW6`zZx?Hs-wi~G4$I}}pr^*mmR z0Xz8=a6`n)v35_401(ct^CiDWyw}(K<_7H$Jc+P4EvSOsf*?SU8@3m0w8&t(wBoR1 zDS=Y#k`A3lD`lo&s61Jx)0LH5M-Lxisv@W)pE$%DAS}b8&~UqbKgJGR3a1C!L$<%x z?mJFkcR(Y7X3`;kuy&0!NnpF(wQxfbw6eBs$Ym^QCnXu`1HoZI5tIwcMMJJ+Z(TeB$kn!yn zu!)Rqn6pw$7*;AIdquenJ9Vz3I1zZWPUG~vhoOuW9t>EaQTVml3g)%P2WbvV;)g1~@-lUf^iIw-Dmgj%S_#aBFyw~$(EXVv& zAB6Oa$`WkV=V2VOQ~8sbeK3ZfMaB-8r8z>&q&e>7AxAbf(VOL%t$4DV<3M(IjOBPk z1Rrq>eAk7fb!8aU!%Y=@A$`Gn_*68ufhWe6?VimRUvWu>%PIs$;^yUhUJs^xQYh4h=|) z1VsHw+|+|XkLruMA9(#{pwH@Un>sCDUwmy-XCn(+b_>)o%yGNxupGNlM`t;Bt!8BGlJx`9>HQzwl`W)Rh+FUZ zj#r6$wNf>!HM3TkY4mNURasb|)`{k+6Q?cf)I4pTtTnA>o6I+@xyAx*TUM=V)oRVt VbH}u8`1$y^2fZgDbP4_l{RNMH09pV5 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-ac378b3f-8fd6-4137-a91c-9e6c1b04d884-c000.snappy.parquet deleted file mode 100644 index 5c2c23ddaf44d165a323fcf995b68030c07e5976..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2316 zcmb7`UuYaf7{KTD_AZD{JD8;=kJ)=-X8Bo76VFn70?EbiT2 zclRnshzLR~eKGX0Ek20&AjKD<1oXj&w1gl&3W5)cc~*#0K~el>_TOOqbA{vXn{U4F z_x+h~Cb{;~t5u9}8CUSVPamsgEWxUX6@*UCN(iBRuHAcO2m%8A6jt&6tw#^@g4x7# zk=e4zO$_`z-ru=%_>5q3W+gHi2g=LczUvKbdoIR({QG-PMB4<1qlhf8_uskmhZr$Q z?5!7b+n;^?>k^Pk_!NHf^OrwN2vLhuJZh5nqjx{~^MtUS1jGTF~0CyA)3%EKf_SCOOg$azwX|{NbZZXb-{XOXeTo|o7y4&Q3&P%#Gm!?3269ms#H>~mls@~uoHLm_V$ zrr=J`sH*3$o3u~i>Y4l&nm27ffXP=2mT8$)q@Fuw5!`T1xUe?{c4+$En`YMzX`6;r z{a)VG!(rdn7xW=`{F z>I<7&TQjL=@(^VH5Y`^upC^#Uh7_Hri4Ieq1r9bmpOrUa=Fv-(xwpLmD|cjn{&;2X z9oM5z5gleX^L2Wmy>9!X#}=N*jMDTx*QUMEDaW7H48|VWAf(;?|KVQWLh6-x^j_%t zm3USwRkK<%Yn8Ljf#bF-^YgZKzIE~ZCCj=vXSXiYT2`w==UUcmbKdS)R;_B)YOPDN U=d}Cq=i}c#=w1<_Yw&;2Ull+8)&Kwi diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-238f32ef-3df8-46a0-adfb-02377478da38-c000.snappy.parquet deleted file mode 100644 index 0436637b71ca2c88c9fad9fd5f40739115dac5b6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2454 zcmb7`U1%It6oBu}&Tb~@8rmfHG6VatEbh?CI&^o^CJl?k*v3*qEh3>H67KHKW(Q|? zwmY+xCFZ5BdFWFKK_LXum#Q=%6dwXUS;4*pm4c#(q98tL)qpumEcS!&-*5$06Ev{Sa#nqW&+yg;PW?+aJC_C(htb;`suH? ze@k*EVP4I}=AX0Iex46VAAj@Nry0&HB(-nSbAicQcfWe&kq4NFez0*ofj_$b-HjAy zG9vrAC$I=-&*E?2-}@y~fg*uVrufc>rNZE8gl=Ciu<{&*0v+7C@!mKCh<+8NtPem* z<(QzzQdmt+#DYQ)^0q}*Mn(2soO(1WnXi^2urX?Rqb}q@rlX4ZNPT~?Ge*iY4j}1HCv!b zgJ!~_j=OS|)~IXR?G2ufDjQKc*vl~Yn0Y6i(+fe7RRxu{B&_-n@!LC2#Wh_z5g z=pV=eA#z{BQ{;;jegdh>&`1F}I<3c_Cuh~=ir**x6S!X2_$!`wZ4`K0X{;f28?I$0 z0D6`3V9$}uuv6hWixYtxcO0kZBfK_>3AvNT3rIa+7e9!=O78_&Ad9d8>9Y3ppDf*Y z#lvE*U<^R}63!!a3&vTRfkfp&P3-YrCW8bRs`tdKoZ*MsKUWaxKX#E) zgrO%w4?`of7b4q`1|m08L{KI2NooqlMNyXR?y5#R6mGTVU2yHH(RJWb$thjawGxs~ z9I|k(SQ^ZND?P*2y7o(2%W!F(x>WvNT+6%tj**|sJ8rjEbMwV~!!)eAlb?HjEzhO_ zY?*aX2WFPtUWHW~Za1x3unTTRF&`51b8GAC#mF<9F|j{{HHP>15{P0$isr*ahYJ)3 zcA9pV9dF3Y{UH~-*X^x#QKIY1MNxGlqV$Hhc1nuFsw} rYNyIIz1E{hp{{0hUE9A#{9NKnUe?&N=Pp97v#_zzY84_WeJ`IJ2== zBeVI_p^1Ud;ctH2d0661PF;>nMuGB1yX&}pO6OwShl5X_@=+7q%TYv@*n_=aKYoeB zC5d_4pS^E>|N9b;|MN0a_iC9o&gu-o+^+#30A5J}*_Cr-NnKKYsBfu~0M~0J<;vrfhCSEyJ z0T0BMLL8hG36QYX43Yy~F0_MPO_uOg$d*Yf_);uwYV?U^kx{ z+!gUstlbwQ14w7*a>?%!@10e@u}<3rk0UHj3W{JiAqfzqy6pw)P12t%Ej#S|OP~}x zq)jK$vc?q5Do@tvWO;e#)QOW!H3q7pPaNX)5td+4sJoq>AKUg_3a1C!W46EA>N!qe zw?R_`&A3DSVD%Phkid33tKo($sG7QENE#N^3sMpK1Ib}Q7UV1PRYR&}r<_SLI+LD# zriC`b)Ib&p$@e5YA%9fBi^y1k35>~UwITnae92g;Me`AT12>$OXvOosWW2osHj%NF z({L#UhMfvwZ^_qTr^ZbbCjvK{G)~V)7)V%_4-5DrG7ca@k}_+3n(YAu2*)z|X6dpa zjaEGD0fT*AWa$nB4Ufg?#)p=6cOXD}lUdFr*50F9p8tgLpA=bp@8mTs#r$XvLi~cf z2wSxoxDMH={Kee97{RX~V;9`g8ewG88h7(hBfB-xH>)vQ@vwWN0J}fJYP<{28H#h@ zr!Iun(0>eTTG_t%ugbm!29kdgWx7Lf_0~+%C2;l3@D`f4sOQ7vD+SB6%nDM@9`wsp8KxZrU7Y^fGCf| z4L#_0slK55f!AvU`jpyRxulH$fenIc{eSRvoy4r0&2jxX~#+A?gbo zo10UqXYvqa{t%W9?;j?R#)cH_7l{s2o(1;TU5}MFWaiOJl({$E9xHcf|M2n3+*^)I zULZQmZsu!u0=h=M;bU`8WJamGu0u&@c*@adHG{E7)(=R#`+vB%H<5BB9=+GvUOArC za>cAv&1(5fy=Obk^6V_N&Nt@H&s)~q3~gMfHmpXA%rva&`Ydf(R<&YPtBv{Tv+5T7 Q`FPd`eO^T98vG;l4+z}@H2?qr diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-a15dd5ab-2986-4e66-ac19-64e8ec608096-c000.snappy.parquet deleted file mode 100644 index b004cba7ac0fd4e938414adce026972823ce61b3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2455 zcmb7GU1%It6uvt$qkx&o`v%9m&(Ak~s z%xqXbs+{QBZsU zl$7^!vLu9bGA8C^5h1$G6B=cSz8LptM4K$dHWR=Tz95g%7Q+c4M^G&^*(|~hWKoLZ zES~TIZc0n+P9$V0kqFWiQNRn(HIfx^5g!YzK?0u)%$-E!-QuFq^3Azmy@+ zfRrf;_N;gkc4~~W7!z<~#W8-i!t0}06tAc8EGq6mg|KBpebns^6rl3-pQK$PR)58U zT_#}XMo7C`oPsk!yTM0`*xllQ-cZxAMd;n_W$~{Q{0~P6z3VV9gaF^yLDcSg651vA z!he!>;}r{clfd1f?QWCt6TlHEMPS(v2l5A&L)w?>}YL#Aa^^D=TaH*t}rfO;# zNe6aCxTh@@X2I#6;i-=Og4#Aby{UV;^drBT_xfEUKb?0yr`PcErF_dYtfrfver`2S zrU7i3O>hTcmfcx~RR?ajUbkQu(u`6*M9fdGuC0}#%y7oU`B2svKA$!a)rJ<$2dNGh zC@$=-*bX_~ke&NOE>3USJ#yTk^XXiS(_5CUKR|Vmf1GP}JY(5#hOf;`-x#HC+m@ks zhDX^wv12eYzgSI>pjzI=BB0$?O@~Z!6TY> t_>j?J2R;e_OR1e4l_{;QfKlDc$p%d^g+21N^IL!b6 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-aaf68aea-4b35-410a-9f34-ccb270f44d70-c000.snappy.parquet deleted file mode 100644 index b29c26d395fda57a12839a07f3aaf6162528fab2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2301 zcmb7GPiP!f7=JrEJDtQu`X_IgfmN2#EUm0VcPCAn5P}vgp)93{gn~$TyZbg7oZZ>Z z%vP2V@Dy-8)JqS_QhE@)^i)JJ2TwWV;6-m%5VamXw7Pguu>Ic5o0(^$b_-$hz3+YB z@B4k<`@NT3dHu~AM!1B__|scoeo`!936=_05qf?>LI~w^k3OjV2?hfH9M`qoLY}VW&q`lZr}BWlrD3)4?p|nTUCfn;9iI;GGg!F z*f)iUgx9w6-S|gk@)`gY@p1ge?_b}a6~dIcsAxj_m+v3k6T3_5FiuZL_`d?i3CYma zWJ-dGB|}3f+LqaL3_bcKxM$L776z!$kS&f%-voa9pPzJwdxyO+mn+&svq0V6r}9&!`N+*vOM zed52n8ML-(m*A5K%kz>hIc=Z{hT3%eaJx;0^TmeC&ZG!VxktKm9yLnL!J_hHi_VuD zcTOEY!CbT8QUc-%ANsEMz z*V~LXG|4C#;|hpTHqNOEObp0jNt3io+U1J6nw@cGY3Ou%@|hRL2$BL82x%h~&uM!F zyoM?ppiYjQb{pYe&@NUsR%3Oz==5kv@UzW<@v8x=#H7)UFq7%eHDr zKXW7^xaL}LNv{oPX!+iIR+omPO+upoB=4BvuushuGYI`bD>SQS$DwXJFjwB*G1)bR zEvF4`9_D(zEm-y7+L5LUyO2h!W`bz0?CkDV)6C@ZXZcW;PM;qUNNWQ{=UJ+Qva`a$ zw&%0zBD!%QdT3x9a87&4J^#%S%htKHEBfcGnE%XnNJoLdr%tJt5r7DV0iZ3e0;)4&Kd*|!Yw7P|1=G=3C z=XXBtId^yY)z_;S;WDn^2Y=lBx>&{nEF~->baYxk2<3Cqdp^CykrX*PipTJ8d*A(Z zgkvbGQQVZ`zdUHi1x=NcI&mTvGzB5HEzE*qw4KA3WAPxDYIlSbdLxe) zV!&=b1#F3UG1l&j5dgxud!ZEch<{}@Xs+8Gg2xaRCj?b+S`Y*Xa>Mb%^%faSl$Kn2 zEG1BiUDB~9&{CNyC@N3Z?1{?K-4n--Q`IP_q=2}@A0RBlqR{ZV{UF8;JR43Av`3s^ zwcU5!(CL6i0?n99f^hX3X_C?_o#Ait@OC&mrRgJR~90(x=fLfCJW+ z(VK=#n>1SSWZzPa#Eo*{S?P%|0#SmyoebZ>!iW4=t1CxSNL@>C{ATnq#)&$!?7R*;hrH z&Vy#A}J<%O0%Y!?Tb%jb`Lm^@}sB- zhOAsKjKfS%sjBC%nWRTx^h`E|<~7?7p!3y&Wm;wxsizJhf~&3x1ADb^hokPs-tzi1-J$K- z<&~MYU5`A6cj(>B*6D`!n(Ytoo4X=2mWJoKHt7z}a`;-!$mk^-grw8^Kf1Rzk$N$1 zy;nMZCGOQq)vVUcTIFP;@3^hX+?;KlZk{=P*0Rpb+Rd3-(`vTKY}1-<%-L1}7nb$_OsZfIl}%#ek#$xLUH-7o|d46=kJh~%(~gqoh3q_Jmu zw!3F#47rFP$rkH+57#jq+3Q zhnFyWZ@%@{_iK+{0i+T>k0*Z|{aob2q;gm`A^z2epFcjWP%bL;6omiV)-DK|DkpQ| zOe|;$LgBW^=A)_MFV0=e=Ao(Ff(sGZBp@>!NCqe;-9x>^OtNyC3K@tkg}6NH557bR09&N#$~s<&t%DrC8Jm+_>OBVhn-3ZoMNB!=nPubnS({;$tInttlqzL z{sMCq!6o^`A>IgK85V`M+aLOI?8v2X{@^`p`y1V%;{=I>Zk)?huOvw%gwb zHxxnBwS$_hV^O;zOE57YhXqAYt}8ccaw|LK%u>*7diI$Y#t5W(GA=8-1-y)E z>!8mVIqf#Ye@$7bt+yh5L|?!S)e^0Eyw4ckxPT2*JILv{6a$8n3c=Quo3PX31;yKd z%t?*k;j4ThV?~)1@CvF;ph`lUjXz~Pfg+(Wqc5Y(WE!n_ux}XHp~NU33OX*vl;eX- zCwwTth!Ztu78~(NFOUB#!~acUBi_yHSdQ@{Erjq($}()V7T`i;C-XN;yLSe^h-za< zOM3*_q&<%F&?CDy(UMGpxrw5o*L7c;%%`_QWTc++)edA(JCWmg3}7 zM4Dn8J&ExUFp%=2s0cM#dB0GGyIfaQ*V{D6fWVbA!rN!up`H(lR|}?L8V#ggJ`v&H zb_}?vw}&(^JojCrM+4F!0a1Svx2wTuK&wktKk$ZDP_0+HHg!6Fb?Mf2m0eZXvOD08 z!yLE239CL_J<@hy7t(0Gnjlt}ws&^wX=d^OWcg5*PM;qVNNWQ{`&p`kva`bemg}+h zreq$yKv{an9kO<(&JQ20EWPWvpGP5r$;&YoMvF` zarFbz8$1o}?H#0EkEQo=&#S~*tu%~A(`Z&^+e6#wR2CPhIcHs+yJnhK7pQfmX_;1+ kELi4zdy#fcv)M44P3zkHW$gg|Z2Z>``ba`(4gLZ82e5?+K>z>% diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt index 7424dac98ed9..8a846d26385d 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/requirements.txt @@ -1,3 +1,2 @@ delta-spark==3.0.0 -pyspark==3.2.0 -argparse==1.4.0 \ No newline at end of file +pyspark==3.5.0 \ No newline at end of file From f6a7658e7f71ad0cf685a681820dfe5ce0351f7f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 19:31:55 -0800 Subject: [PATCH 48/65] yank the old test resource. --- .../_delta_log/00000000000000000000.json | 13 ------------- ...42-4299-af4d-3fdaa1c12001-c000.snappy.parquet | Bin 2187 -> 0 bytes ...7b-4a77-96f6-e4a0abe939d3-c000.snappy.parquet | Bin 2180 -> 0 bytes ...6c-4fbb-837b-9a38059e6ea1-c000.snappy.parquet | Bin 2223 -> 0 bytes ...a0-4549-aa26-c7f0b03b051b-c000.snappy.parquet | Bin 2263 -> 0 bytes ...a2-408b-acd9-21754be887de-c000.snappy.parquet | Bin 2222 -> 0 bytes ...65-4ac9-85f0-bf1b8be53719-c000.snappy.parquet | Bin 2264 -> 0 bytes ...8d-4a9e-ad00-a705738e2aff-c000.snappy.parquet | Bin 2151 -> 0 bytes ...9f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet | Bin 2207 -> 0 bytes ...7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet | Bin 2221 -> 0 bytes ...e4-4646-a224-f3d0187a34ee-c000.snappy.parquet | Bin 2204 -> 0 bytes 11 files changed, 13 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00001-b3706429-b27b-4a77-96f6-e4a0abe939d3-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00002-dcd31c19-e96c-4fbb-837b-9a38059e6ea1-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00004-adfc1b3d-c6a2-408b-acd9-21754be887de-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00005-3d374415-de65-4ac9-85f0-bf1b8be53719-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00006-68588845-198d-4a9e-ad00-a705738e2aff-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json deleted file mode 100644 index d58121dc0581..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,13 +0,0 @@ -{"commitInfo":{"timestamp":1704767217219,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"10","numOutputBytes":"22122"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"aa74cdbe-d130-4cf2-9935-520649df8a9b"}} -{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} -{"metaData":{"id":"8047ab6e-7d22-4bbb-9988-5619003ba238","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"surname\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"birthday\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"email\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"country\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"state\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"city\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1704767214974}} -{"add":{"path":"part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet","partitionValues":{},"size":2187,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Darren\",\"surname\":\"Francis\",\"birthday\":\"2003-04-03T17:02:10.358-08:00\",\"email\":\"rating1998@yandex.com\",\"country\":\"Panama\",\"state\":\"Minnesota\",\"city\":\"Eastpointe\"},\"maxValues\":{\"name\":\"Darren\",\"surname\":\"Francis\",\"birthday\":\"2003-04-03T17:02:10.358-08:00\",\"email\":\"rating1998@yandex.com\",\"country\":\"Panama\",\"state\":\"Minnesota\",\"city\":\"Eastpointe\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00001-b3706429-b27b-4a77-96f6-e4a0abe939d3-c000.snappy.parquet","partitionValues":{},"size":2180,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Madelene\",\"surname\":\"Berger\",\"birthday\":\"2010-09-05T20:29:23.753-07:00\",\"email\":\"invitations2036@duck.com\",\"country\":\"Aruba\",\"state\":\"New York\",\"city\":\"Wheaton\"},\"maxValues\":{\"name\":\"Madelene\",\"surname\":\"Berger\",\"birthday\":\"2010-09-05T20:29:23.753-07:00\",\"email\":\"invitations2036@duck.com\",\"country\":\"Aruba\",\"state\":\"New York\",\"city\":\"Wheaton\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00002-dcd31c19-e96c-4fbb-837b-9a38059e6ea1-c000.snappy.parquet","partitionValues":{},"size":2223,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Anibal\",\"surname\":\"Mccall\",\"birthday\":\"2002-02-06T19:36:55.543-08:00\",\"email\":\"modifications2025@yahoo.com\",\"country\":\"Anguilla\",\"state\":\"Oklahoma\",\"city\":\"Sahuarita\"},\"maxValues\":{\"name\":\"Anibal\",\"surname\":\"Mccall\",\"birthday\":\"2002-02-06T19:36:55.543-08:00\",\"email\":\"modifications2025@yahoo.com\",\"country\":\"Anguilla\",\"state\":\"Oklahoma\",\"city\":\"Sahuarita\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet","partitionValues":{},"size":2263,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Anibal\",\"surname\":\"Jackson\",\"birthday\":\"1988-01-18T20:20:22.313-08:00\",\"email\":\"medication1855@gmail.com\",\"country\":\"Luxembourg\",\"state\":\"New Hampshire\",\"city\":\"Santa Rosa\"},\"maxValues\":{\"name\":\"Anibal\",\"surname\":\"Jackson\",\"birthday\":\"1988-01-18T20:20:22.313-08:00\",\"email\":\"medication1855@gmail.com\",\"country\":\"Luxembourg\",\"state\":\"New Hampshire\",\"city\":\"Santa Rosa\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00004-adfc1b3d-c6a2-408b-acd9-21754be887de-c000.snappy.parquet","partitionValues":{},"size":2222,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Loree\",\"surname\":\"Tanner\",\"birthday\":\"1991-02-26T01:28:18.632-08:00\",\"email\":\"transport1961@duck.com\",\"country\":\"Anguilla\",\"state\":\"New Hampshire\",\"city\":\"Morristown\"},\"maxValues\":{\"name\":\"Loree\",\"surname\":\"Tanner\",\"birthday\":\"1991-02-26T01:28:18.632-08:00\",\"email\":\"transport1961@duck.com\",\"country\":\"Anguilla\",\"state\":\"New Hampshire\",\"city\":\"Morristown\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00005-3d374415-de65-4ac9-85f0-bf1b8be53719-c000.snappy.parquet","partitionValues":{},"size":2264,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Bernardo\",\"surname\":\"Gamble\",\"birthday\":\"1996-03-06T05:55:34.655-08:00\",\"email\":\"limitations1886@yandex.com\",\"country\":\"Panama\",\"state\":\"North Carolina\",\"city\":\"Greenville\"},\"maxValues\":{\"name\":\"Bernardo\",\"surname\":\"Gamble\",\"birthday\":\"1996-03-06T05:55:34.655-08:00\",\"email\":\"limitations1886@yandex.com\",\"country\":\"Panama\",\"state\":\"North Carolina\",\"city\":\"Greenville\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00006-68588845-198d-4a9e-ad00-a705738e2aff-c000.snappy.parquet","partitionValues":{},"size":2151,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Philip\",\"surname\":\"Cherry\",\"birthday\":\"2010-09-16T08:48:36.668-07:00\",\"email\":\"const1874@outlook.com\",\"country\":\"China\",\"state\":\"Nevada\",\"city\":\"Albert Lea\"},\"maxValues\":{\"name\":\"Philip\",\"surname\":\"Cherry\",\"birthday\":\"2010-09-16T08:48:36.668-07:00\",\"email\":\"const1874@outlook.com\",\"country\":\"China\",\"state\":\"Nevada\",\"city\":\"Albert Lea\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet","partitionValues":{},"size":2207,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Shaneka\",\"surname\":\"Beasley\",\"birthday\":\"2006-08-02T13:08:04.242-07:00\",\"email\":\"msg1894@example.com\",\"country\":\"Barbados\",\"state\":\"Montana\",\"city\":\"Mount Pleasant\"},\"maxValues\":{\"name\":\"Shaneka\",\"surname\":\"Beasley\",\"birthday\":\"2006-08-02T13:08:04.242-07:00\",\"email\":\"msg1894@example.com\",\"country\":\"Barbados\",\"state\":\"Montana\",\"city\":\"Mount Pleasant\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet","partitionValues":{},"size":2221,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Keneth\",\"surname\":\"Vinson\",\"birthday\":\"2002-10-15T08:32:10.678-07:00\",\"email\":\"questions2074@gmail.com\",\"country\":\"Honduras\",\"state\":\"Connecticut\",\"city\":\"Hutchinson\"},\"maxValues\":{\"name\":\"Keneth\",\"surname\":\"Vinson\",\"birthday\":\"2002-10-15T08:32:10.678-07:00\",\"email\":\"questions2074@gmail.com\",\"country\":\"Honduras\",\"state\":\"Connecticut\",\"city\":\"Hutchinson\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} -{"add":{"path":"part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet","partitionValues":{},"size":2204,"modificationTime":1704767217107,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"name\":\"Hee\",\"surname\":\"Schwartz\",\"birthday\":\"2006-12-20T01:27:35.236-08:00\",\"email\":\"statements2016@protonmail.com\",\"country\":\"Senegal\",\"state\":\"New Jersey\",\"city\":\"Galt\"},\"maxValues\":{\"name\":\"Hee\",\"surname\":\"Schwartz\",\"birthday\":\"2006-12-20T01:27:35.236-08:00\",\"email\":\"statements2016@protonmail.com\",\"country\":\"Senegal\",\"state\":\"New Jersey\",\"city\":\"Galt\"},\"nullCount\":{\"name\":0,\"surname\":0,\"birthday\":0,\"email\":0,\"country\":0,\"state\":0,\"city\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00000-5532668e-8942-4299-af4d-3fdaa1c12001-c000.snappy.parquet deleted file mode 100644 index 1a038ef0eb87dea4f2ffb92206868ea1759e0f45..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2187 zcmbVOUx?gf5dSut&E}4!ws`ZAK)@w<;VNtBZhC)Nq#jy|R5^}A%1Y7ivbi>NH@i); zZI8PHDMdtl5FZ2;d^l7JLMwe1UxYptPrAWcI;^hB#^iFz(Wiy%i&2MJD znfb`ltIJIwP=^=c-_P%TBWWPus{}G3huZ=nq+F8l|25|Oj)xW<47mH#^*_fs;E@vv zu!_KwKJ#oh01*_>VCR>g?^oJLDH2j^y!Y3QkGf?-9`_~*QXPf~_~*M%ZXSxL#3ZJy zB~+x0fLCAY`^fMF_bmviW&GMq`ZcSPt!A`*Ulsum2Hf8pK0 zKJQ=`4iTtL3W|s=aVE)WEkZ?#YB2J7orF4h)%C;kUACS;D(LUxEOAcuVC04VIOE1Ci@^>bnv3?-f`e#zP!!Nu3rY%r`%jn0%97 zE1}uAQPPwezi_+XFxLNAiQ2%06tN(zDtZw5Kn6;TTS8G&a{6)59vz~oz0Z*DqsA2W z9_oGY{83yR14=)Wz$eCbxk#y+wm_hg9+P7Q76MqpAe^`y$_lP4-W+)2}KMxWNrY8TjW+He|SdX1s~!CC3lkfu~d~(=wYxIr;=5xY#psX)lhP z(DVnF&At<|E(@9RM|ndJ*N2Y2pa-EpvO~S0uenaI8|Vw~Z0OMfqbYtgI3Qlz2=qq1 z9$W1p0$rVkp0Vl+8(UkA+&jGp(4mNQAcE}HC>TPw@8Gr2A3l8+@m*6eV6_LzA)U-; zCx~ycXLpaTyn}k5-6F~xY1?1!`}K5A>P@rRGF$bj)sfrl)@NrO>zF-vY~HfwW*mFE oWn1+aZU}?J@yhP$=QUEm{L0Xr+p?-sRM#HJM+yq zGw;m|YMh;|0D%u-9DaVZefy{i0)FN}BINjafe@0-ph!NoTCB~w3~eO@|MlkOt1JgS zR*#^0gq~yW0&_tG8B}<5^Xu)yb>tKYDVAS9Ib7>}K*-(IF@j9TU=((L_~yqW0h1U7 zmDGrdWD#&4F2GkWpMRg>LTR@mfQ+G>?uz4EzSHY^)d_RzOsj7%8FsIOt|vf)Cx865 z!xwaPCUVW8>shzov=A2*{`%g|1Ca~C7k-JYmOk&fO9<6Khu?qw<)=d&bmUA#SwPCq z7MbPuy2^;4$Wn4)oCtzK2?@>l=#msU7C4hUtSChuYU12;VPT3Kot!~RD#45rkuw*P z?Bzs?jVhJk3g%4|mCnIzEbV6CTr55mU?mo}ve1adz3jm2sR)+?EfU_oQ1q59cj=mE zFS3pWM+g+h1X=7_9X3WTssSoc6uiF6>+-0RnvUx)wyd=Xl0$!|z0b++^}D{i#_Mt( z_QyVI(RTa=7)I5c*hx zQF=cI9}|5pq8A8*x^x9i_2C^l+=7V= z?d#!Q{I&dpnbB|aaD(VuxRjEoEX3^)CU}uH&{&VhB0ey5G$!#kPkQGNlHM&qmFRmM zY1B_4WzZmdNI96MR6^o+XnFdrfPT9WbjVX&JxR`CbB62v=4N@|9WMcNNFyDHAlVoxgYR@0u7u9=|IZ@aGbsaBd!QWR zE_{E2aD%V0*S$e+Z|{!zy*#eGt$7?r9~EsZz(PP3HgB5Midn7Nr%#$%|+Rk>8)R~D| znqG|qghkNf8TSGtC+P&zN~{oe2WnfG3D z?$~%4BV5FL@t==ge9aUv2OkA2BDAy0A%wCS0sbFrS~Y4%z}K(t-QE5j;Az94wvBl# z;R0U$^xNBQRbb>1%9jSlKK}Y)J3^Od`Vlbg#C`a$=XYN2@R|59q*w?;qAUQ$@ELsT z*|#qo6RQ7C2-pKTHClCR(lV%LwVG~aurhRXj!w5)n$c>2_g*}J?=SuRNMO8op9sC7 zIH+i=466UXbKwzrl@17X-}8JWT~5#;TPljWGN9@b6gXcW05iUrgccE903YM#N$v0 zk_%)ak=8T#c%)q6a3>O%v-n~pF7k=E%HvDE76_M4=G|H9%>3dS(IBevS%J{8Hm5K?Biijy2F1m=q`uA+jdoKC+U-!$_S-Ipbmy z+6kZy=OBLiv>P=n?LR*4*MN2%R#uRQcmZ-v8WIq9$jywpo>7-G1u4%i-^Nc6sf!iK zC0I@#ih+v!2O>+om>_ip)-vu<*!dvn4blNqwdEHSEYXSW#Lmy0Qzw~o}Nm9$1 z(&#LOoz#N$)UKJ13wKt^>6)&Uk+g3U!o6y1u*p~3rl&cr>srI~Xq|dg`kb9tyt!FZ z8C6`*X&atWQYJ0auDirD7aD UxKHlD#ozpg_QQ{mbMRB;Um1TZ2><{9 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00003-878e86d2-dca0-4549-aa26-c7f0b03b051b-c000.snappy.parquet deleted file mode 100644 index bc5469663aed24f784b760aaeab851c3f7efe4bf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2263 zcmbVO&ubf35Pn*%q>U?^1U#?1um`c!i&NAk(nhvpbcjnSCKzMX&=$3!d0MUQC0a?@ zT_rKLix0-Vw6yflLkp$!5c&gL@U1)-~?Ba%q?PwN+qaaO+-;?9j?dLNebSK%_9MV*u0;HdTj2Aa5FZK#b_4^KV8oI zUFO~T$hS6am%$W){EQ%rZRXlDWJN`pa3kmUJbo{WAgSSa!Df?fMUV`}JFSr*tJ7-- z-WI=?@dFl&P>YrmBoHbnsyPfBLvSDx(PxnSE6 zz?Py1=U|%tmVr5<*N2=*F(K_5y{^}*gO(T=NSw6EV2`K%x=($QfkmR9;2dX_ikQWG z2Zs`p@6i1ey7j}9D(Cru99^N-e|U*f!~qwvAgrqYAoTDYP@*3R!2?PQx-lZ`u+4Uyg`FqeSG3 zFChHeEdvMqcFztBuXERM?SM5|z~n#E+gh;IwY5df54@fgXhm(^v0F`FTl`>K>km7+ z!bgzYqSu7p>|PAShdCN-QD6KI=(b8V5oE^g5-u7G6arm;}zm||9=+Y zy)LfMJ*U+y%+K5A zC2Qf*71Lapv#mp;hc;c9IO;o!!pN zHijhNLmw1^6a=v^5`9ssq&{i|!S|vMLf?ea_RT)|(DuG85{kPBWyj({@o{&=IqhIe$6kjLg_Qg?xOvhma{`vm;t)meWpLmqi zgozXpa2n3StuOxkrO3D{ZzTYVu0r4Pf^NqTtFu$p)6JfJ*|0lpG&~7oaAW7muE-dk zIGY$^TI7tk+H+ei3!w(o;KAO<*VwdQT}q(C2tDttm(N-4Zm{P1QTk(0fv=z3{*=X! z%qRFGh<~@^`)&|+);;AFPLYN9+Oicm1>=qtVscN4oC=JJK0hx-9-v@Mr7SZ=-WM75 za9=ix+`nglN0>BZW|KlxD#1eZ0*XqPppiChH2_{AeeJH^Q{ZW7#(U)?*ERmS&$WB<=rxERa0j7xlsGx$V zO!6Yexqlfk@fN*SK(&6epvon-^ZIw_=|8kYsbFXFm<(nVErjm81eEA|T%J#I>#Y`eVjHQb#fU`}7j3>IWL?z!n|#H{1a|>+<)=<((mD5dAKn=M)!e>~WD^ zKuw)(PF6-=Oxv;=*2}}l%U2S-P3R{A<#dUDCyZcMS7f>4uNqd@!j?4R?li7Ce&BXI zSuhRLs1kYn8ANcUWnd#;={cd{ch-%z6Ix9xwB)~vn_9Thb+kDx2>qTNY87q8bz03p zo4dHFMNN&Wc&RWz*IPxP*TX@*+(HDJIu$Kr)#f(0w=4VB>2N@U+|!{5vPqINgl^lx zQPJ-H|0v@5lQUqphl(Ms*cT^=C) zrb*sP7^4rRFB{uy-x^W~9f7fjJ#2r#U{8ZVS-bUvF~XkqFv^a;tZSE>D}cjM))Ki$ItKFa!Oc$HwnuKz4g9;5X$F3MDxsZsn=|StqjRwOh$Kzy0&a zyE)F8Kj#v4X^R4I0#D+DpYA^{azJGzjxqvK7TaEMwfr9S+K%mp4&%6jfB)g(1DSIe zU5Fiq!wPtex$Cy$F!fDARphjIoQMULAQYLUxL~T12%PD}e12qrBA`J409>z~@K`p6UI`q1zPDw~s42xN(z)8%^B;7lTDyxaCn z3-y8Vfg}+kJIC-S`KEy1M*2#|ohB90FOp?_rJl}3`UL7^Qc?|1`(;S`cL7f${SYQ! zQl~@-&fG0bM7%|A<-n}(RuZCy- zIb?n~6c+v^M;s4VM0=Suz*O&Fv;KDo)3=EI2tMp#81}^wK9BTAVwO)d!BN!nPgX4L9jQ`KS1T5VVL*c_s8e^ z`%BtqR~tZ}1~0-VcfNVQYJh}K1yn*#wj@GGxuoEKyBCDPkPCkJ=k-sYJw)()&yV8` zkU;|jZaw~IzJ-7?A=UbO+dn_}-XY}9)fs|7Ct(^MetzT2;~9|5EUE^xNXjViGMs}S z{R>ES4Q)@ia|Xa9}vPb}(JQtmN_N8aoAs!U8~JJd%mAV}i??@Zj18Rbdr`4D+A75C8_dR>8-ShsN(6lI&@ zrO%yOp>1Tg_z>^|jk2NOZxQry2Kq>WCb9MeP}V!+s_h{ppPu5(%EsoYpop+Hom*@s8-iIEoz6%64Yuw<&m)(1$XWES27@OyjiHH7$xS z+V0TBQM2=@v#D- z4?QYOEFB4;*bozhH0b-d0{X-M??t{>CI}eqk$m8t`1Anz#(H}7Y`q;-`>idaz0S-2 zQa`ToKB+bAM$>NAX1k*xT&pcE`p#)@>GT=LSz7SD`KIT1>+XW*%yk$2b;oHooMzKI UGj~eAhKu~kAIUx*+W3j_FOnq{RsaA1 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00007-fdb228e6-459f-42f1-816c-6fd7be8b60e6-c000.snappy.parquet deleted file mode 100644 index 896da6b5ecb1517c23ff9578d6c10db4dfc31b40..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2207 zcmbVOO=uit82)y4c6Y=jSey4_21Zy1Go-K%*`1_GmZh}8LlGk(mK@?A-_A~wwKKDu znOQX?fl@qpu;8I5g&cCMpi;$S@Fcx>?M?Jh5vqky0-hA#`JZf8F@6ZBG$|dJZPx@fV+dH4#AZ5vd?Yq@;)rEw}_n zw|{=f<+xrLL=NZiRJOO-SXi9BY<|e>fn{n&-^QqyU>c4N|NM*3MLqvY6oqM#CFXWm zx9`Qt{P^pm2YG^g+RJ^%XHJmz49vo#?>_%s$bpJ)MW7Q1y4)W+er?6V?3v?B=Qv5M z#Fv&yz)6&l$Qa)f8j~B#jOWKBCW+&;IYc4JOA=EtXRgG#AYqbKMmRBYd~rk(>2lRTbK4K~sT}>7>Z14(|rr1rsEM-WOn!-YG$gsOuSS zl28b}La(Xo2pu;;Dg@A@x;*s50Qx})eO!>CEY0y{Jac+6A^0BssGuGf)GrINSj`XL z={My14=a)CIGj8tgjod-LjM+k67?=lO>$RJ1wzs-YN4q;wq3_6k$zljkM+%8!R=>& z(t|R*N7RF2c9cq7`tUHTbPbK|Y$7rVY128$w0VL%0l~)=xJ1;G96{7aY4uJJJ{hT0 z2+9|aS*puJe;+{qPzgHY9`32+8pDPRk>8;mw6sSDar7N|Mn-kiBhPw2V7(#08lg8! zlv4$ISenFXu8U&d-PG8C;V5e1RBG>=uGi~3VoBF@twF>KPk4CSmWBhoJv4pI?SG)z zrq8;}XW~!AUB%xSm`Y3We0OO0N?qCLnO4_RT5s+u!7!sKeAIJ}J!cagouOs1j)fi= zYSxtmRcY<+@7L4lcqw2&#&jltWP@Y~e$O^>Mc9M?e~NJLWC)n;na&V*>eB;+8|~@C zgY|dT=(qQY_-b7Cmu8@k?X=*_0FID0|b!$tq(kGxnWWCcG>{s9frEDZnv diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00008-9558a023-4c7b-4c4c-a516-07728b69a7ef-c000.snappy.parquet deleted file mode 100644 index 7f14a8a307aa3754a416e6f29e60e4df118fc413..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2221 zcmbVO-)kII6uvtjhm8M36y+M}OaZQfVTiNJyo2?cM2Xe>4bryFE^j={StRKli@;A(j*qNl8s4Nf80( z-~v4U=GXlroYcur62J+}>3Zk{%s zSBA_jQiw_=Sj0MrqSBSVbx?qF>E@mQEH&>IVJS7A72!&1J`iC!GLi6$iv79m(>UrZHxq=&+nCSPg-?-w$ZXS6G#c;U2H8b$nJ(7^Edgu zQot-WKrPxX%OF%xRApPozNsiPWg;m_CLmCOzC{;xshPdh>`BQ)`kJ#HJQwT~ERGPm zFToi7z67&GU&%SMV*0czNSpLQ_Kau;cX-;T5$&Hb?Lk3>vN9up`RZCO7uM;uO~VAShPnOx|%~X1@pKv=I(~pRtz0q<)2I`9@;O9sy=~wi zeRH1Rh!@1*{SuT)5`$^a-~BNWH(99z}yc0x>%jw|IZ@6e{u$__E0&b zo%-Sg@y+();Zb)THu~xgkuRmM{dC8#rhQVa8})|Ks7|bgZo5^Tn{&)5`^?l?(>ycl q*fR~=wAZXz+nipVbJk3=Q8yb6`|R|jvW*x1^B*#ae?b=TkIKK7H808l diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table/part-00009-f1cd65ba-c4e4-4646-a224-f3d0187a34ee-c000.snappy.parquet deleted file mode 100644 index 0b2e287c7c8e07d95cbdac5bad4cb444b4210a60..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2204 zcmbVOPiP!v6#sU1c6U;?sg=BsnKG@*V8&F|A-j`5ZCIp=Xb3Tcxa3eNGMRiy7H4+1 zGqZ-61WNGW!9(dm1u24hkXq`=gC2SidaMWAL(%F*@YIv`q6qrto7tVO6cSTdzIoq! zzxR9ZecyW@yZF*l4G5foQ*iV9PaYoALBeej6he-UON5YoE|2RhW2k->#^BTUul`gL zP*1;-pwNN5;cjd?LG&)jpn?v!?tJv~u?7;#gp{ir^6Mx5Iz`A_FiMc?DHwr2zPkMN zaLgqq5fwe*B6$QXz!H4;-8Z+yV9$M=08V1CFmfX1vvw5L$7@sPxPk*M z4SFeuw#EVNd4L&n+P}RI@E6bO@(`%>N)fK)FzI}jbSZ5Np7tVbqOmcUkb!`i^rvQ^ z6WPBU0!qIq!3;6?5{XUerFR;JmSZP#mdz3!f!IIZpAGynLHJX0iO5&xI` E2P%^wI{*Lx From d7590b6344f568f7a7a78c6b6c80a1d4b21d5a1d Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 19:41:53 -0800 Subject: [PATCH 49/65] add a couple of sad path tests --- .../druid/delta/input/DeltaInputSource.java | 4 +- .../delta/input/DeltaInputSourceTest.java | 44 ++++++++++++++++--- 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 6b0e2321d488..791c55adac61 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -79,7 +79,7 @@ public DeltaInputSource( ) { if (tablePath == null) { - throw InvalidInput.exception("tablePath cannot be null"); + throw InvalidInput.exception("tablePath cannot be null."); } this.tablePath = tablePath; this.deltaSplit = deltaSplit; @@ -170,7 +170,7 @@ public Stream> createSplits(InputFormat inputFormat, @Nul latestSnapshot = table.getLatestSnapshot(tableClient); } catch (TableNotFoundException e) { - throw new RuntimeException(e); + throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); } final Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); // scan files iterator for the current snapshot diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index c080064e1696..d60e4eaf2573 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -108,21 +108,21 @@ public void testReadDeltaLakeWithSplits() .collect(Collectors.toList()); Assert.assertEquals(2, splits.size()); - for (int idx = 0; idx < splits.size(); idx++) { - final DeltaSplit split = splits.get(idx).get(); + for (InputSplit split : splits) { + final DeltaSplit deltaSplit = split.get(); final DeltaInputSource deltaInputSourceWithSplitx = new DeltaInputSource( DeltaTestUtil.DELTA_TABLE_PATH, - split + deltaSplit ); List> splitsResult = deltaInputSourceWithSplitx.createSplits(null, null) .collect(Collectors.toList()); Assert.assertEquals(1, splitsResult.size()); - Assert.assertEquals(split, splitsResult.get(0).get()); + Assert.assertEquals(deltaSplit, splitsResult.get(0).get()); } } @Test - public void testNullTablePath() + public void testNullTable() { MatcherAssert.assertThat( Assert.assertThrows( @@ -130,7 +130,39 @@ public void testNullTablePath() () -> new DeltaInputSource(null, null) ), DruidExceptionMatcher.invalidInput().expectMessageIs( - "tablePath cannot be null" + "tablePath cannot be null." + ) + ); + } + + @Test + public void testSplitNonExistentTable() + { + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null); + + MatcherAssert.assertThat( + Assert.assertThrows( + DruidException.class, + () -> deltaInputSource.createSplits(null, null) + ), + DruidExceptionMatcher.invalidInput().expectMessageIs( + "tablePath[non-existent-table] not found." + ) + ); + } + + @Test + public void testReadNonExistentTable() + { + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null); + + MatcherAssert.assertThat( + Assert.assertThrows( + DruidException.class, + () -> deltaInputSource.reader(null, null, null) + ), + DruidExceptionMatcher.invalidInput().expectMessageIs( + "tablePath[non-existent-table] not found." ) ); } From 752fa2b9a929f064b93aa3a01b82451cdb7925e4 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 19:50:46 -0800 Subject: [PATCH 50/65] Updates to readme based on latest. --- .../extensions-contrib/delta-lake.md | 2 +- .../src/test/resources/README.md | 19 +++++++++++++------ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index bd407cba0d81..52621820799a 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -35,4 +35,4 @@ These Delta Lake files are in versioned Parquet format ## Known limitations - This extension only reads from the latest Delta table snapshot as the kernel API only supports that. -- Column filtering isn't supported yet. So the extension will reads all the columns in the configured table. \ No newline at end of file +- Column filtering isn't supported yet. So the extension will read all the columns in the configured table. \ No newline at end of file diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index ca81db81a829..de773a71b7cd 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -20,19 +20,26 @@ ### Generate Delta Table for Unit Tests To test Delta Lake ingestion, use the Python script `create_delta_table.py` to generate a sample Delta table. -Set up the required dependencies by installing those specified in `requirements.txt`. +Create a conda env `delta_test` with all the requirements specified in `requirements.txt` installed in the +environment: ```shell -pip install -r requirements.txt +conda create --name delta_test --file requirements.txt ``` -Specifically, `pyspark` and `delta-spark` requirements are needed to generate Delta records. Once all the dependencies are installed, -you can run the python script: +To activate the environment: + +```shell +conda activate delta_test +``` + +From the conda environment, you can run the python script: + ```python python3 create_delta_table.py ``` By default, the script uses `append` mode to generate 10 random records and writes the -Delta table to `resources/people-delta-table`. You can override the defaults by supplying the command line arguments: +Delta table to `resources/employee-delta-table`. You can override the defaults by supplying the command line arguments: ```shell python3 create_delta_table.py -h @@ -46,7 +53,7 @@ optional arguments: --save_mode {append,overwrite} Specify write mode (append/overwrite) (default: append) --save_path SAVE_PATH - Save path for Delta table (default: /druid/extensions-contrib/druid-deltalake-extensions/src/test/resources/people-delta-table) + Save path for Delta table (default: /druid/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table) --num_records NUM_RECORDS Specify number of Delta records to write (default: 10) ``` From 2da7b1a12094448e64128cb22489bb67c5a960b2 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 20:54:44 -0800 Subject: [PATCH 51/65] Version support --- docs/development/extensions-contrib/delta-lake.md | 7 ++++++- website/.spelling | 4 +++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index 52621820799a..10e1a80033cb 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -30,7 +30,12 @@ you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. The Delta input source reads the configured Delta Lake table and extracts all the underlying delta files in the table's latest snapshot. -These Delta Lake files are in versioned Parquet format +These Delta Lake files are versioned Parquet files. + +## Version support + +The Delta Lake extension uses the Delta Kernel introduced in Delta Lake 3.0.0, which is compatible with Apache Spark 3.5.x. +Older versions are unsupported, so consider upgrading to Delta Lake 3.0.x or higher to use this extension. ## Known limitations diff --git a/website/.spelling b/website/.spelling index 4bd4cf9706a5..e3c9863dbfdd 100644 --- a/website/.spelling +++ b/website/.spelling @@ -799,9 +799,11 @@ multi-server BasicDataSource LeaderLatch 2.x -28.x +28.x +3.0.x 3.5.x 3.4.x +3.5.x. AllowAll AuthenticationResult AuthorizationLoadingLookupTest From cb1047559112a79152e1255626231978157680fb Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 21:14:46 -0800 Subject: [PATCH 52/65] Extract Delta DateTime converstions to DeltaTimeUtils class and add test --- .../druid/delta/input/DeltaInputRow.java | 16 +--- .../druid/delta/input/DeltaTimeUtils.java | 59 +++++++++++++++ .../apache/druid/delta/input/RowSerde.java | 15 +--- .../druid/delta/input/DeltaTimeUtilsTest.java | 73 +++++++++++++++++++ 4 files changed, 136 insertions(+), 27 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 4c3f5f8eb44d..526f3936d925 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -44,10 +44,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.time.LocalDate; -import java.time.LocalDateTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -183,20 +180,11 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR } else if (dataType instanceof IntegerType) { return dataRow.getInt(columnOrdinal); } else if (dataType instanceof DateType) { - // DateType data is stored internally as the number of days since 1970-01-01 - int daysSinceEpochUTC = dataRow.getInt(columnOrdinal); - return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); + return DeltaTimeUtils.getDateTimeValue(dataRow.getInt(columnOrdinal)); } else if (dataType instanceof LongType) { return dataRow.getLong(columnOrdinal); } else if (dataType instanceof TimestampType) { - // TimestampType data is stored internally as the number of microseconds since epoch - long microSecsSinceEpochUTC = dataRow.getLong(columnOrdinal); - LocalDateTime dateTime = LocalDateTime.ofEpochSecond( - microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, - (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC - ); - return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); + return DeltaTimeUtils.getTimestampValue(dataRow.getLong(columnOrdinal)); } else if (dataType instanceof FloatType) { return dataRow.getFloat(columnOrdinal); } else if (dataType instanceof DoubleType) { diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java new file mode 100644 index 000000000000..c7ebf5ce858c --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java @@ -0,0 +1,59 @@ +/* + * 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.druid.delta.input; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; + +public class DeltaTimeUtils +{ + private static final ZoneId ZONE_ID = ZoneId.systemDefault(); + + /** + * TimestampType data in Delta Lake tables is stored internally as the number of + * microseconds since epoch. + * + * @param microSecsSinceEpochUTC microseconds since epoch + * @return Datetime millis correpsonding to {@code microSecsSinceEpochUTC} + */ + public static long getTimestampValue(final long microSecsSinceEpochUTC) + { + final LocalDateTime dateTime = LocalDateTime.ofEpochSecond( + microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, + (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, + ZoneOffset.UTC + ); + return dateTime.atZone(ZONE_ID).toInstant().toEpochMilli(); + } + + /** + * DateType data in Delta Lake tables is stored internally as the number of + * days since epoch. + * + * @param daysSinceEpochUTC number of days since epoch + * @return number of seconds corresponding to {@code daysSinceEpochUTC}. + */ + public static long getDateTimeValue(final int daysSinceEpochUTC) + { + return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 5652194ef018..98e27b23af13 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -46,10 +46,6 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.jackson.DefaultObjectMapper; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; import java.util.HashMap; import java.util.Map; @@ -133,16 +129,9 @@ public static Map convertRowToJsonObject(Row row) } else if (fieldType instanceof DoubleType) { value = row.getDouble(fieldId); } else if (fieldType instanceof DateType) { - final int daysSinceEpochUTC = row.getInt(fieldId); - value = LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZoneId.systemDefault()).toEpochSecond(); + value = DeltaTimeUtils.getDateTimeValue(row.getInt(fieldId)); } else if (fieldType instanceof TimestampType) { - final long microSecsSinceEpochUTC = row.getLong(fieldId); - LocalDateTime dateTime = LocalDateTime.ofEpochSecond( - microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, - (int) (1000 * microSecsSinceEpochUTC % 1_000_000) /* nanoOfSecond */, - ZoneOffset.UTC - ); - value = dateTime.atZone(ZoneId.systemDefault()).toInstant().toEpochMilli(); + value = DeltaTimeUtils.getTimestampValue(row.getLong(fieldId)); } else if (fieldType instanceof StringType) { value = row.getString(fieldId); } else if (fieldType instanceof ArrayType) { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java new file mode 100644 index 000000000000..55ef171ed04a --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java @@ -0,0 +1,73 @@ +/* + * 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.druid.delta.input; + +import org.apache.druid.java.util.common.Intervals; +import org.junit.Assert; +import org.junit.Test; + +import java.time.Instant; + +public class DeltaTimeUtilsTest +{ + @Test + public void getTimestampValue() + { + Assert.assertEquals( + Instant.parse("2018-02-02T00:28:02.000Z"), + Instant.ofEpochMilli( + DeltaTimeUtils.getTimestampValue( + Instant.parse("2018-02-02T00:28:02.000Z").toEpochMilli() * 1_000 + ) + ) + ); + + Assert.assertEquals( + Instant.parse("2024-01-31T00:58:03.000Z"), + Instant.ofEpochMilli( + DeltaTimeUtils.getTimestampValue( + Instant.parse("2024-01-31T00:58:03.002Z").toEpochMilli() * 1_000 + ) + ) + ); + } + + @Test + public void getDateTimeValue() + { + Assert.assertEquals( + Instant.parse("2020-02-01T00:00:00.000Z"), + Instant.ofEpochSecond( + DeltaTimeUtils.getDateTimeValue( + (int) Intervals.of("1970-01-01/2020-02-01").toDuration().getStandardDays() + ) + ) + ); + + Assert.assertEquals( + Instant.parse("2024-01-01T00:00:00.000Z"), + Instant.ofEpochSecond( + DeltaTimeUtils.getDateTimeValue( + (int) Intervals.of("1970-01-01/2024-01-01T02:23:00").toDuration().getStandardDays() + ) + ) + ); + } +} From ec7add5dcc903c7ededb9bb56617d56cc3284a51 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Sun, 28 Jan 2024 23:44:42 -0800 Subject: [PATCH 53/65] More comprehensive split tests. --- .../druid/delta/input/DeltaInputRow.java | 3 - .../delta/input/DeltaInputSourceTest.java | 85 +++++++++++++------ .../druid/delta/input/DeltaTestUtil.java | 23 ++++- 3 files changed, 78 insertions(+), 33 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 526f3936d925..2bffa2c871e1 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -44,7 +44,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.time.ZoneId; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -60,8 +59,6 @@ public class DeltaInputRow implements InputRow private final Object2IntMap fieldNameToOrdinal = new Object2IntOpenHashMap<>(); private final InputRow delegateRow; - private static final ZoneId ZONE_ID = ZoneId.systemDefault(); - public DeltaInputRow(io.delta.kernel.data.Row row, InputRowSchema inputRowSchema) { this.row = row; diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index d60e4eaf2573..a2c1282ea6c6 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.InputSplit; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.hamcrest.MatcherAssert; import org.junit.Assert; @@ -35,7 +36,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import java.util.stream.Stream; public class DeltaInputSourceTest { @@ -57,6 +57,7 @@ public void testSampleDeltaTable() throws IOException Assert.assertNotNull(actualSampledRawVals); Assert.assertNotNull(actualSampledRow.getRawValuesList()); Assert.assertEquals(1, actualSampledRow.getRawValuesList().size()); + for (String key : expectedRow.keySet()) { if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = (Long) expectedRow.get(key); @@ -72,52 +73,63 @@ public void testSampleDeltaTable() throws IOException public void testReadAllDeltaTable() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); - - List actualReadRows = readAllRows(inputSourceReader); + final InputSourceReader inputSourceReader = deltaInputSource.reader( + DeltaTestUtil.SCHEMA, + null, + null + ); + final List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualReadRows.size()); - for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { - Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); - InputRow actualInputRow = actualReadRows.get(idx); - for (String key : expectedRow.keySet()) { - if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { - final long expectedMillis = (Long) expectedRow.get(key) * 1000; - Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); - } else { - Assert.assertEquals(expectedRow.get(key), actualInputRow.getRaw(key)); - } - } - } + validateRows(DeltaTestUtil.EXPECTED_ROWS, actualReadRows); } @Test - public void testReadDeltaTableWithNoSplits() + public void testDeltaLakeWithCreateSplits() { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - final Stream> splits = deltaInputSource.createSplits(null, null); - Assert.assertNotNull(splits); - Assert.assertEquals(2, splits.count()); + final List> splits = deltaInputSource.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertEquals(DeltaTestUtil.SPLIT_TO_ROWS.size(), splits.size()); + + for (InputSplit split : splits) { + final DeltaSplit deltaSplit = split.get(); + final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( + DeltaTestUtil.DELTA_TABLE_PATH, + deltaSplit + ); + List> splitsResult = deltaInputSourceWithSplit.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertEquals(1, splitsResult.size()); + Assert.assertEquals(deltaSplit, splitsResult.get(0).get()); + } } @Test - public void testReadDeltaLakeWithSplits() + public void testDeltaLakeWithReadSplits() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(2, splits.size()); + Assert.assertEquals(DeltaTestUtil.SPLIT_TO_ROWS.size(), splits.size()); - for (InputSplit split : splits) { + for (int i = 0; i < splits.size(); i++) { + final InputSplit split = splits.get(i); final DeltaSplit deltaSplit = split.get(); - final DeltaInputSource deltaInputSourceWithSplitx = new DeltaInputSource( + final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( DeltaTestUtil.DELTA_TABLE_PATH, deltaSplit ); - List> splitsResult = deltaInputSourceWithSplitx.createSplits(null, null) - .collect(Collectors.toList()); - Assert.assertEquals(1, splitsResult.size()); - Assert.assertEquals(deltaSplit, splitsResult.get(0).get()); + final InputSourceReader inputSourceReader = deltaInputSourceWithSplit.reader( + DeltaTestUtil.SCHEMA, + null, + null + ); + final List actualRowsInSplit = readAllRows(inputSourceReader); + final List> expectedRowsInSplit = DeltaTestUtil.SPLIT_TO_ROWS.get(i); + Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); + + validateRows(expectedRowsInSplit, actualRowsInSplit); } } @@ -184,4 +196,21 @@ private List readAllRows(InputSourceReader reader) throws IOException } return rows; } + + private void validateRows(final List> expectedRows, final List actualReadRows) + { + for (int idx = 0; idx < expectedRows.size(); idx++) { + final Map expectedRow = expectedRows.get(idx); + final InputRow actualInputRow = actualReadRows.get(idx); + for (String key : expectedRow.keySet()) { + if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + final long expectedMillis = (Long) expectedRow.get(key) * 1000; + Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.utc(expectedMillis), actualInputRow.getTimestamp()); + } else { + Assert.assertEquals(expectedRow.get(key), actualInputRow.getRaw(key)); + } + } + } + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 705797520951..736a3615c631 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -38,8 +38,10 @@ import org.apache.druid.data.input.impl.TimestampSpec; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the @@ -59,7 +61,8 @@ public class DeltaTestUtil "is_fulltime", "last_vacation_time" ); - public static final List> EXPECTED_ROWS = new ArrayList<>( + + private static final List> SPLIT_ROWS_0 = new ArrayList<>( ImmutableList.of( ImmutableMap.of( "birthday", 1057881600L, @@ -104,7 +107,12 @@ public class DeltaTestUtil "last_vacation_time", 1706256972000L, "age", (short) 27, "yoe", 9 - ), + ) + ) + ); + + private static final List> SPLIT_ROWS_1 = new ArrayList<>( + ImmutableList.of( ImmutableMap.of( "birthday", 937526400L, "is_fulltime", false, @@ -205,6 +213,17 @@ public class DeltaTestUtil ) ); + public static final Map>> SPLIT_TO_ROWS = new HashMap<>( + ImmutableMap.of( + 0, SPLIT_ROWS_0, + 1, SPLIT_ROWS_1 + ) + ); + + public static final List> EXPECTED_ROWS = SPLIT_TO_ROWS.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + public static final InputRowSchema SCHEMA = new InputRowSchema( new TimestampSpec("birthday", "posix", null), new DimensionsSpec(ImmutableList.of( From 9eb5a7321862f2a80094e89c9a15e24233dd5930 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 00:37:43 -0800 Subject: [PATCH 54/65] Some test renames. --- .../delta/input/DeltaInputSourceTest.java | 6 +-- .../druid/delta/input/DeltaTestUtil.java | 40 ++++++++++--------- 2 files changed, 24 insertions(+), 22 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index a2c1282ea6c6..5b3524cbb32b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -90,7 +90,7 @@ public void testDeltaLakeWithCreateSplits() final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(DeltaTestUtil.SPLIT_TO_ROWS.size(), splits.size()); + Assert.assertEquals(DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); for (InputSplit split : splits) { final DeltaSplit deltaSplit = split.get(); @@ -111,7 +111,7 @@ public void testDeltaLakeWithReadSplits() throws IOException final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(DeltaTestUtil.SPLIT_TO_ROWS.size(), splits.size()); + Assert.assertEquals(DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); for (int i = 0; i < splits.size(); i++) { final InputSplit split = splits.get(i); @@ -126,7 +126,7 @@ public void testDeltaLakeWithReadSplits() throws IOException null ); final List actualRowsInSplit = readAllRows(inputSourceReader); - final List> expectedRowsInSplit = DeltaTestUtil.SPLIT_TO_ROWS.get(i); + final List> expectedRowsInSplit = DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.get(i); Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); validateRows(expectedRowsInSplit, actualRowsInSplit); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java index 736a3615c631..887a0eb26058 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java @@ -62,7 +62,7 @@ public class DeltaTestUtil "last_vacation_time" ); - private static final List> SPLIT_ROWS_0 = new ArrayList<>( + private static final List> SPLIT_0_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( "birthday", 1057881600L, @@ -111,7 +111,7 @@ public class DeltaTestUtil ) ); - private static final List> SPLIT_ROWS_1 = new ArrayList<>( + private static final List> SPLIT_1_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( "birthday", 937526400L, @@ -213,30 +213,32 @@ public class DeltaTestUtil ) ); - public static final Map>> SPLIT_TO_ROWS = new HashMap<>( + public static final Map>> SPLIT_TO_EXPECTED_ROWS = new HashMap<>( ImmutableMap.of( - 0, SPLIT_ROWS_0, - 1, SPLIT_ROWS_1 + 0, SPLIT_0_EXPECTED_ROWS, + 1, SPLIT_1_EXPECTED_ROWS ) ); - public static final List> EXPECTED_ROWS = SPLIT_TO_ROWS.values().stream() - .flatMap(List::stream) - .collect(Collectors.toList()); + public static final List> EXPECTED_ROWS = SPLIT_TO_EXPECTED_ROWS.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); public static final InputRowSchema SCHEMA = new InputRowSchema( new TimestampSpec("birthday", "posix", null), - new DimensionsSpec(ImmutableList.of( - new LongDimensionSchema("id"), - new LongDimensionSchema("birthday"), - new StringDimensionSchema("name"), - new LongDimensionSchema("age"), - new DoubleDimensionSchema("salary"), - new FloatDimensionSchema("bonus"), - new LongDimensionSchema("yoe"), - new StringDimensionSchema("is_fulltime"), - new LongDimensionSchema("last_vacation_time") - )), + new DimensionsSpec( + ImmutableList.of( + new LongDimensionSchema("id"), + new LongDimensionSchema("birthday"), + new StringDimensionSchema("name"), + new LongDimensionSchema("age"), + new DoubleDimensionSchema("salary"), + new FloatDimensionSchema("bonus"), + new LongDimensionSchema("yoe"), + new StringDimensionSchema("is_fulltime"), + new LongDimensionSchema("last_vacation_time") + ) + ), ColumnsFilter.all() ); From 0dccf146a6eeab0a9ed87d6f0597ca10720853aa Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 00:51:30 -0800 Subject: [PATCH 55/65] Cleanup and update instructions. --- .../druid/delta/input/DeltaTimeUtils.java | 4 +- .../druid/delta/input/DeltaInputRowTest.java | 12 +++--- .../delta/input/DeltaInputSourceTest.java | 42 +++++++++---------- ...DeltaTestUtil.java => DeltaTestUtils.java} | 28 ++++++++++++- .../druid/delta/input/DeltaTimeUtilsTest.java | 4 +- .../druid/delta/input/RowSerdeTest.java | 2 +- .../src/test/resources/README.md | 9 +++- 7 files changed, 66 insertions(+), 35 deletions(-) rename extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/{DeltaTestUtil.java => DeltaTestUtils.java} (90%) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java index c7ebf5ce858c..f9f251e82916 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java @@ -29,7 +29,7 @@ public class DeltaTimeUtils private static final ZoneId ZONE_ID = ZoneId.systemDefault(); /** - * TimestampType data in Delta Lake tables is stored internally as the number of + * {@link io.delta.kernel.types.TimestampType} data in Delta Lake tables is stored internally as the number of * microseconds since epoch. * * @param microSecsSinceEpochUTC microseconds since epoch @@ -46,7 +46,7 @@ public static long getTimestampValue(final long microSecsSinceEpochUTC) } /** - * DateType data in Delta Lake tables is stored internally as the number of + * {@link io.delta.kernel.types.DateType} data in Delta Lake tables is stored internally as the number of * days since epoch. * * @param daysSinceEpochUTC number of days since epoch diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 91df20b1f60e..e792d4f2af79 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -40,7 +40,7 @@ public class DeltaInputRowTest public void testDeltaInputRow() throws TableNotFoundException, IOException { final TableClient tableClient = DefaultTableClient.create(new Configuration()); - final Scan scan = DeltaTestUtil.getScan(tableClient); + final Scan scan = DeltaTestUtils.getScan(tableClient); CloseableIterator scanFileIter = scan.getScanFiles(tableClient); int totalRecordCount = 0; @@ -57,14 +57,14 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException Row next = dataReadResult.getRows().next(); DeltaInputRow deltaInputRow = new DeltaInputRow( next, - DeltaTestUtil.SCHEMA + DeltaTestUtils.SCHEMA ); Assert.assertNotNull(deltaInputRow); - Assert.assertEquals(DeltaTestUtil.DIMENSIONS, deltaInputRow.getDimensions()); + Assert.assertEquals(DeltaTestUtils.DIMENSIONS, deltaInputRow.getDimensions()); - Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(totalRecordCount); + Map expectedRow = DeltaTestUtils.EXPECTED_ROWS.get(totalRecordCount); for (String key : expectedRow.keySet()) { - if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = ((Long) expectedRow.get(key)) * 1000; Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); } else { @@ -75,6 +75,6 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException } } } - Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), totalRecordCount); + Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), totalRecordCount); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 5b3524cbb32b..4aff0b11b01f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -42,14 +42,14 @@ public class DeltaInputSourceTest @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); - final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtil.SCHEMA, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtils.SCHEMA, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); - Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualSampledRows.size()); + Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualSampledRows.size()); - for (int idx = 0; idx < DeltaTestUtil.EXPECTED_ROWS.size(); idx++) { - Map expectedRow = DeltaTestUtil.EXPECTED_ROWS.get(idx); + for (int idx = 0; idx < DeltaTestUtils.EXPECTED_ROWS.size(); idx++) { + Map expectedRow = DeltaTestUtils.EXPECTED_ROWS.get(idx); InputRowListPlusRawValues actualSampledRow = actualSampledRows.get(idx); Assert.assertNull(actualSampledRow.getParseException()); @@ -59,7 +59,7 @@ public void testSampleDeltaTable() throws IOException Assert.assertEquals(1, actualSampledRow.getRawValuesList().size()); for (String key : expectedRow.keySet()) { - if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = (Long) expectedRow.get(key); Assert.assertEquals(expectedMillis, actualSampledRawVals.get(key)); } else { @@ -72,30 +72,30 @@ public void testSampleDeltaTable() throws IOException @Test public void testReadAllDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final InputSourceReader inputSourceReader = deltaInputSource.reader( - DeltaTestUtil.SCHEMA, + DeltaTestUtils.SCHEMA, null, null ); final List actualReadRows = readAllRows(inputSourceReader); - Assert.assertEquals(DeltaTestUtil.EXPECTED_ROWS.size(), actualReadRows.size()); + Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtil.EXPECTED_ROWS, actualReadRows); + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows); } @Test public void testDeltaLakeWithCreateSplits() { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); + Assert.assertEquals(DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); for (InputSplit split : splits) { final DeltaSplit deltaSplit = split.get(); final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( - DeltaTestUtil.DELTA_TABLE_PATH, + DeltaTestUtils.DELTA_TABLE_PATH, deltaSplit ); List> splitsResult = deltaInputSourceWithSplit.createSplits(null, null) @@ -108,25 +108,25 @@ public void testDeltaLakeWithCreateSplits() @Test public void testDeltaLakeWithReadSplits() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtil.DELTA_TABLE_PATH, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final List> splits = deltaInputSource.createSplits(null, null) .collect(Collectors.toList()); - Assert.assertEquals(DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); + Assert.assertEquals(DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.size(), splits.size()); - for (int i = 0; i < splits.size(); i++) { - final InputSplit split = splits.get(i); + for (int idx = 0; idx < splits.size(); idx++) { + final InputSplit split = splits.get(idx); final DeltaSplit deltaSplit = split.get(); final DeltaInputSource deltaInputSourceWithSplit = new DeltaInputSource( - DeltaTestUtil.DELTA_TABLE_PATH, + DeltaTestUtils.DELTA_TABLE_PATH, deltaSplit ); final InputSourceReader inputSourceReader = deltaInputSourceWithSplit.reader( - DeltaTestUtil.SCHEMA, + DeltaTestUtils.SCHEMA, null, null ); final List actualRowsInSplit = readAllRows(inputSourceReader); - final List> expectedRowsInSplit = DeltaTestUtil.SPLIT_TO_EXPECTED_ROWS.get(i); + final List> expectedRowsInSplit = DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.get(idx); Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); validateRows(expectedRowsInSplit, actualRowsInSplit); @@ -203,7 +203,7 @@ private void validateRows(final List> expectedRows, final Li final Map expectedRow = expectedRows.get(idx); final InputRow actualInputRow = actualReadRows.get(idx); for (String key : expectedRow.keySet()) { - if (DeltaTestUtil.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = (Long) expectedRow.get(key) * 1000; Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); Assert.assertEquals(DateTimes.utc(expectedMillis), actualInputRow.getTimestamp()); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java similarity index 90% rename from extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java rename to extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java index 887a0eb26058..26e227ba9a37 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtil.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java @@ -45,11 +45,17 @@ /** * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the - * sample Delta Table used in the unit tests. + * sample Delta Lake table used in the unit tests. */ -public class DeltaTestUtil +public class DeltaTestUtils { + /** + * The Delta table path used by unit tests. + */ public static final String DELTA_TABLE_PATH = "src/test/resources/employee-delta-table"; + /** + * The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}. + */ public static final List DIMENSIONS = ImmutableList.of( "id", "birthday", @@ -62,6 +68,9 @@ public class DeltaTestUtil "last_vacation_time" ); + /** + * The expected set of rows from the first checkpoint file {@code DELTA_TABLE_PATH/_delta_log/00000000000000000000.json} + */ private static final List> SPLIT_0_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( @@ -111,6 +120,9 @@ public class DeltaTestUtil ) ); + /** + * The expected rows from second checkpoint file {@code DELTA_TABLE_PATH/_delta_log/00000000000000000001.json} + */ private static final List> SPLIT_1_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( @@ -213,6 +225,9 @@ public class DeltaTestUtil ) ); + /** + * Mapping of checkpoint file identifier to the list of expected rows in that checkpoint. + */ public static final Map>> SPLIT_TO_EXPECTED_ROWS = new HashMap<>( ImmutableMap.of( 0, SPLIT_0_EXPECTED_ROWS, @@ -220,10 +235,16 @@ public class DeltaTestUtil ) ); + /** + * Complete set of expected rows across all checkpoint files for {@link #DELTA_TABLE_PATH}. + */ public static final List> EXPECTED_ROWS = SPLIT_TO_EXPECTED_ROWS.values().stream() .flatMap(List::stream) .collect(Collectors.toList()); + /** + * The Druid schema used for ingestion of {@link #DELTA_TABLE_PATH}. + */ public static final InputRowSchema SCHEMA = new InputRowSchema( new TimestampSpec("birthday", "posix", null), new DimensionsSpec( @@ -242,6 +263,9 @@ public class DeltaTestUtil ColumnsFilter.all() ); + /** + * A simple wrapper that builds the table scan for {@link #DELTA_TABLE_PATH} meant to be used in tests. + */ public static Scan getScan(final TableClient tableClient) throws TableNotFoundException { final Table table = Table.forPath(tableClient, DELTA_TABLE_PATH); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java index 55ef171ed04a..490407f8bb2a 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java @@ -28,7 +28,7 @@ public class DeltaTimeUtilsTest { @Test - public void getTimestampValue() + public void testTimestampValue() { Assert.assertEquals( Instant.parse("2018-02-02T00:28:02.000Z"), @@ -50,7 +50,7 @@ public void getTimestampValue() } @Test - public void getDateTimeValue() + public void testDateTimeValue() { Assert.assertEquals( Instant.parse("2020-02-01T00:00:00.000Z"), diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index 53eab8383b05..eb06f532a021 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -33,7 +33,7 @@ public class RowSerdeTest public void testSerializeDeserializeRoundtrip() throws TableNotFoundException { final DefaultTableClient tableClient = DefaultTableClient.create(new Configuration()); - final Scan scan = DeltaTestUtil.getScan(tableClient); + final Scan scan = DeltaTestUtils.getScan(tableClient); final Row scanState = scan.getScanState(tableClient); final String rowJson = RowSerde.serializeRowToJson(scanState); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index de773a71b7cd..c524adec4b58 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -58,4 +58,11 @@ optional arguments: Specify number of Delta records to write (default: 10) ``` -Check in the Delta table generated and update `DeltaTestUtil.java` as needed for validation. +The test data in `resources/employee-delta-table` was generated by: +```shell +python3 create_delta_table.py +python3 create_delta_table.py --num_records=5 --save_mode=append +``` + +This creates a total of 15 Delta records across two transactional commits. The resulting Delta table is checked in +to the repo. The expectated rows `DeltaTestUtils.java` are updated accordingly. From e8f229b748be16a5a1f973b7308f16bd7297c362 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 01:55:19 -0800 Subject: [PATCH 56/65] add pruneSchema() optimization for table scans. --- .../druid/delta/input/DeltaInputSource.java | 34 +++++++++++- .../delta/input/DeltaInputSourceTest.java | 55 ++++++++++++++++--- .../druid/delta/input/DeltaTestUtils.java | 45 +++++++++++++++ 3 files changed, 124 insertions(+), 10 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 791c55adac61..d849060cde38 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -32,7 +32,10 @@ import io.delta.kernel.data.Row; import io.delta.kernel.defaults.client.DefaultTableClient; import io.delta.kernel.internal.util.Utils; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; import io.delta.kernel.utils.CloseableIterator; +import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; @@ -96,10 +99,10 @@ public boolean needsFormat() * Instantiates a {@link DeltaInputSourceReader} to read the Delta table rows. If a {@link DeltaSplit} is supplied, * the Delta files and schema are obtained from it to instantiate the reader. Otherwise, a Delta table client is * instantiated with the supplied configuration to read the table. + * * @param inputRowSchema schema for {@link org.apache.druid.data.input.InputRow} * @param inputFormat unused parameter. The input format is always parquet * @param temporaryDirectory unused parameter - * @return */ @Override public InputSourceReader reader( @@ -124,7 +127,11 @@ public InputSourceReader reader( } else { final Table table = Table.forPath(tableClient, tablePath); final Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); - final Scan scan = latestSnapshot.getScanBuilder(tableClient).build(); + final StructType prunedSchema = pruneSchema( + latestSnapshot.getSchema(tableClient), + inputRowSchema.getColumnsFilter() + ); + final Scan scan = latestSnapshot.getScanBuilder(tableClient).withReadSchema(tableClient, prunedSchema).build(); final CloseableIterator scanFiles = scan.getScanFiles(tableClient); scanState = scan.getScanState(tableClient); @@ -213,4 +220,27 @@ private Row deserialize(TableClient tableClient, String row) { return RowSerde.deserializeRowFromJson(tableClient, row); } + + /** + * Utility method to return a pruned schema that contains the given {@code columns} from + * {@code baseSchema} applied by {@code columnsFilter}. This will serve as an optimization + * for table scans if we're interested in reading only a subset of columns from the Delta Lake table. + */ + private StructType pruneSchema(final StructType baseSchema, final ColumnsFilter columnsFilter) + { + final List columnNames = baseSchema.fieldNames(); + final List fiteredColumnNames = columnNames + .stream() + .filter(columnsFilter::apply) + .collect(Collectors.toList()); + + if (fiteredColumnNames.equals(columnNames)) { + return baseSchema; + } + final List selectedFields = fiteredColumnNames + .stream() + .map(baseSchema::get) + .collect(Collectors.toList()); + return new StructType(selectedFields); + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 4aff0b11b01f..cc4b62890f96 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -21,6 +21,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.error.DruidException; @@ -81,7 +82,37 @@ public void testReadAllDeltaTable() throws IOException final List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows); + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA); + } + + @Test + public void testReadAllDeltaTableSubSchema1() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader( + DeltaTestUtils.SUB_SCHEMA_1, + null, + null + ); + final List actualReadRows = readAllRows(inputSourceReader); + Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); + + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SUB_SCHEMA_1); + } + + @Test + public void testReadAllDeltaTableInvalidSubSchema2() throws IOException + { + final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader( + DeltaTestUtils.SUB_SCHEMA_2, + null, + null + ); + final List actualReadRows = readAllRows(inputSourceReader); + Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); + + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SUB_SCHEMA_2); } @Test @@ -129,7 +160,7 @@ public void testDeltaLakeWithReadSplits() throws IOException final List> expectedRowsInSplit = DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.get(idx); Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); - validateRows(expectedRowsInSplit, actualRowsInSplit); + validateRows(expectedRowsInSplit, actualRowsInSplit, DeltaTestUtils.SCHEMA); } } @@ -197,18 +228,26 @@ private List readAllRows(InputSourceReader reader) throws IOException return rows; } - private void validateRows(final List> expectedRows, final List actualReadRows) + private void validateRows( + final List> expectedRows, + final List actualReadRows, + final InputRowSchema schema + ) { for (int idx = 0; idx < expectedRows.size(); idx++) { final Map expectedRow = expectedRows.get(idx); final InputRow actualInputRow = actualReadRows.get(idx); for (String key : expectedRow.keySet()) { - if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { - final long expectedMillis = (Long) expectedRow.get(key) * 1000; - Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); - Assert.assertEquals(DateTimes.utc(expectedMillis), actualInputRow.getTimestamp()); + if (!schema.getColumnsFilter().apply(key)) { + Assert.assertNull(actualInputRow.getRaw(key)); } else { - Assert.assertEquals(expectedRow.get(key), actualInputRow.getRaw(key)); + if (schema.getTimestampSpec().getTimestampColumn().equals(key)) { + final long expectedMillis = (Long) expectedRow.get(key) * 1000; + Assert.assertEquals(expectedMillis, actualInputRow.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.utc(expectedMillis), actualInputRow.getTimestamp()); + } else { + Assert.assertEquals(expectedRow.get(key), actualInputRow.getRaw(key)); + } } } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java index 26e227ba9a37..ec48df1d8d1f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.delta.kernel.Scan; import io.delta.kernel.ScanBuilder; import io.delta.kernel.Snapshot; @@ -263,6 +264,50 @@ public class DeltaTestUtils ColumnsFilter.all() ); + + /** + * Similar to {@link #SCHEMA}, but with a smaller set of columns with an inclusion filter applied. + */ + public static final InputRowSchema SUB_SCHEMA_1 = new InputRowSchema( + new TimestampSpec("birthday", "posix", null), + new DimensionsSpec( + ImmutableList.of( + new LongDimensionSchema("id"), + new LongDimensionSchema("birthday"), + new StringDimensionSchema("name"), + new LongDimensionSchema("age"), + new DoubleDimensionSchema("salary"), + new FloatDimensionSchema("bonus"), + new LongDimensionSchema("yoe"), + new StringDimensionSchema("is_fulltime"), + new LongDimensionSchema("last_vacation_time") + ) + ), + ColumnsFilter.inclusionBased(ImmutableSet.of("id", "birthday", "name", "is_fulltime")) + ); + + /** + * Similar to {@link #SCHEMA}, but with a smaller set of columns with an exclusion filter applied. A non-existent + * column is added to the exclusion filter - it should silently get thrown away. + */ + public static final InputRowSchema SUB_SCHEMA_2 = new InputRowSchema( + new TimestampSpec("birthday", "posix", null), + new DimensionsSpec( + ImmutableList.of( + new LongDimensionSchema("id"), + new LongDimensionSchema("birthday"), + new StringDimensionSchema("name"), + new LongDimensionSchema("age"), + new DoubleDimensionSchema("salary"), + new FloatDimensionSchema("bonus"), + new LongDimensionSchema("yoe"), + new StringDimensionSchema("is_fulltime"), + new LongDimensionSchema("last_vacation_time") + ) + ), + ColumnsFilter.exclusionBased(ImmutableSet.of("last_vacation_time", "bonus", "non_existent_column")) + ); + /** * A simple wrapper that builds the table scan for {@link #DELTA_TABLE_PATH} meant to be used in tests. */ From 803f73a55a7e60f4acf2b31e6b1e3da15972973b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 09:25:16 -0800 Subject: [PATCH 57/65] Oops, missed the parquet files. --- ...e49-b8e7-46132a57d039-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...0f2-9185-45237f51b9bf-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...189-8927-97fe1720df8d-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...a5c-b504-62c7c27702c1-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...75a-afbb-44ae0b9cba30-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...145-ac40-6ccbc04acc79-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...3b7-87db-448c67a315df-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...44c-8984-6baecf6987ee-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...15f-b28a-a85d960ccb04-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...77d-b5fd-d00a857f7e54-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...092-8ea2-0ee576ca7327-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...5ab-92d4-24d725cc07e1-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...f1c-8c07-3dac19b12e91-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...c02-b256-bc890fadce7e-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...f80-bd63-e369c6335699-c000.snappy.parquet.crc | Bin 0 -> 28 bytes ...50-4e49-b8e7-46132a57d039-c000.snappy.parquet | Bin 0 -> 2316 bytes ...df-4dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet | Bin 0 -> 979 bytes ...98-40f2-9185-45237f51b9bf-c000.snappy.parquet | Bin 0 -> 2316 bytes ...b2-4189-8927-97fe1720df8d-c000.snappy.parquet | Bin 0 -> 2301 bytes ...32-4a5c-b504-62c7c27702c1-c000.snappy.parquet | Bin 0 -> 2455 bytes ...bd-475a-afbb-44ae0b9cba30-c000.snappy.parquet | Bin 0 -> 2317 bytes ...75-4145-ac40-6ccbc04acc79-c000.snappy.parquet | Bin 0 -> 2454 bytes ...cb-43b7-87db-448c67a315df-c000.snappy.parquet | Bin 0 -> 2302 bytes ...14-444c-8984-6baecf6987ee-c000.snappy.parquet | Bin 0 -> 2316 bytes ...9e-415f-b28a-a85d960ccb04-c000.snappy.parquet | Bin 0 -> 2302 bytes ...aa-477d-b5fd-d00a857f7e54-c000.snappy.parquet | Bin 0 -> 2317 bytes ...1e-4092-8ea2-0ee576ca7327-c000.snappy.parquet | Bin 0 -> 2455 bytes ...1a-45ab-92d4-24d725cc07e1-c000.snappy.parquet | Bin 0 -> 2302 bytes ...92-4f1c-8c07-3dac19b12e91-c000.snappy.parquet | Bin 0 -> 2455 bytes ...77-4c02-b256-bc890fadce7e-c000.snappy.parquet | Bin 0 -> 2454 bytes ...ec-4f80-bd63-e369c6335699-c000.snappy.parquet | Bin 0 -> 2324 bytes 32 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-f0224389-c0df-4dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00002-b2249397-0232-4a5c-b504-62c7c27702c1-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-9ff9b585-5a9e-415f-b28a-a85d960ccb04-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-78cd057e-faaa-477d-b5fd-d00a857f7e54-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-00eb0d30-e71e-4092-8ea2-0ee576ca7327-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00008-9f526a56-2392-4f1c-8c07-3dac19b12e91-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-e21486a5-e177-4c02-b256-bc890fadce7e-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-f0224389-c0df-4dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00002-b2249397-0232-4a5c-b504-62c7c27702c1-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-9ff9b585-5a9e-415f-b28a-a85d960ccb04-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00006-78cd057e-faaa-477d-b5fd-d00a857f7e54-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-00eb0d30-e71e-4092-8ea2-0ee576ca7327-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00008-9f526a56-2392-4f1c-8c07-3dac19b12e91-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-e21486a5-e177-4c02-b256-bc890fadce7e-c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..33d75bafdc5f0dce1b1e7204937d069cf06cb614 GIT binary patch literal 28 jcmYc;N@ieSU}C6UVHVP0p_ub7LQyf~nYGFc$)Dx`bd?D) literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-f0224389-c0df-4dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00000-f0224389-c0df-4dbc-90e5-de1d6a5b5ac6-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..3bdcd64c34e35288c050bd63be2aab50623d3b34 GIT binary patch literal 16 XcmYc;N@ieSU}CUOo7LDWq2dJqAYB9w literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f7e53b3258b13997a3c5fb4948aacc2c7f73b890 GIT binary patch literal 28 kcmYc;N@ieSU}DgmIYYnU^QHZBwwZ4Ud1kFLL-MCN0FY`6e*gdg literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..7de25e042fca269f43785a945016336797ffc0e9 GIT binary patch literal 28 kcmYc;N@ieSU}9+UpC81r&bDIq{ux=`HYbB5tHs{}0DM#l{r~^~ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00002-b2249397-0232-4a5c-b504-62c7c27702c1-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00002-b2249397-0232-4a5c-b504-62c7c27702c1-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..e6c6dc8bf9e7591fc9e3b7456392a0481d526ae5 GIT binary patch literal 28 kcmYc;N@ieSU}EUc)!aHiylc@F_rNJ``+n-Q2Dz{S0EsXPfdBvi literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..693057b9126484f24f699961d3bdf0c3ba8a7cda GIT binary patch literal 28 kcmYc;N@ieSU}E4_c)w<~Ri1FpzTmV~EVB}S>vMbt0D3M87ytkO literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..81f3070f0233e770026e6020cad6fc2ede6a7e46 GIT binary patch literal 28 kcmYc;N@ieSU}DJs5}k7O_2P&66EZC4A6ROt>^5a50GUAx@Bjb+ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..3a8e19d5f789085ebcadc0f75e8606978f20d648 GIT binary patch literal 28 kcmYc;N@ieSU}9k1qj5`BOZ4bWu_v26&Niw!I!?F+0C#5!6aWAK literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..16b0da7f9b254c868b42a33a46ab1dc5f933243a GIT binary patch literal 28 kcmYc;N@ieSU}9)FHeKJ~V6|%P{u$>(o>{BRko;*50EzVq`2YX_ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-9ff9b585-5a9e-415f-b28a-a85d960ccb04-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00005-9ff9b585-5a9e-415f-b28a-a85d960ccb04-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..2f8bfc23a9216ebe350acf7cc8da28f7eafa4b4c GIT binary patch literal 28 kcmYc;N@ieSU}AXrdi%z2*^jrGcx&GGINPY^=s4jP0H-?)pa1{> literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-78cd057e-faaa-477d-b5fd-d00a857f7e54-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00006-78cd057e-faaa-477d-b5fd-d00a857f7e54-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b11e9d379d1b6203b6c955e1f7cf5cbb88d93314 GIT binary patch literal 28 jcmYc;N@ieSU}CUH?2ij*lV4 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-00eb0d30-e71e-4092-8ea2-0ee576ca7327-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-00eb0d30-e71e-4092-8ea2-0ee576ca7327-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..a0daef3f2b65983d2fda754a83e7f85c0f68abaf GIT binary patch literal 28 kcmYc;N@ieSU}EUK*}d3fzHbcQSN#C9uV0t72Dz{S0E}k}$N&HU literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..96e2c291bf25561b4df0ce5e3149b97b9c2b25c8 GIT binary patch literal 28 kcmYc;N@ieSU}7kGvT~K?5(h)3NetIK&Niw!I!?F+0D+wfGynhq literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00008-9f526a56-2392-4f1c-8c07-3dac19b12e91-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00008-9f526a56-2392-4f1c-8c07-3dac19b12e91-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..ba21f2ab46031042272d0e33d2210f0f7ca0c64d GIT binary patch literal 28 kcmYc;N@ieSU}BgTs!@`vF!Mw9mFp?T{=`pc4RT=v0E*cQBme*a literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-e21486a5-e177-4c02-b256-bc890fadce7e-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-e21486a5-e177-4c02-b256-bc890fadce7e-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c55ee1b40bc97750924a04514e2830bfbe1a3ff7 GIT binary patch literal 28 kcmYc;N@ieSU}AW+O!@AwOIa>6ZY;WxaGR@I*=@>B0HX#Bod5s; literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/.part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..57a765a96ee6e017fad81239e00fd5206a3330e2 GIT binary patch literal 28 kcmYc;N@ieSU}9jI#IDb$esop(!;GXEYZt6o7F@Iy0CF-4Y5)KL literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00000-b17c520a-0c50-4e49-b8e7-46132a57d039-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d8882215890880e960b9f1e62636f338da444d06 GIT binary patch literal 2316 zcmb7GU1%It6uvtfI7CB^b{q)u0QAvO# zaXnp{|N6x@9xnq^8K1}I zi{fS!|K&kHCFz=ylt~b=q^k(=ZJE_a*Z4xNDTG)2%%x+DnKZ;&<#9k092H&jWl=yyz|f@OWKh!mX#hb2`~ud1u2(#%ddlT>sj zJ^M@xWrVJQC=gO_DR@eKtALk~xd!bOGpE^d{$=%&xz>#OBYFcb@0MsK^1aXa?iH|w z%-hh?<(QCHij%#pUWc8g&{2X2yjiCSdeOsB!K(VWfG;BR5gVG=11mjua8bPs8xSsQ z%jnI*%~m4WmyGPAA`5p2_se)p79Uz#-Jt~KO?o+#Sb2|YMg9kj|3Q(J_ayU%mEW1!dnNn~G7s2omE;I5ljeAkhaB0|L~oX3wi3y1jRD!GC6?o! z3_fBqvFkW#Q(Xo?6UGi6pyOYW=q@Arxr7a59s*G@Ci;hFO*1=`|CZS&z=70nWmPg2 z_07T*%=C<=dH$wFdIUz#;!|kdpnd?IuN4~BoK-{G*<*-c+qGa|xBE1-eD8IuLqlSd zkZ9k_J4QI@QDe~vLciY*jTvLZp{^Ymi?8e$Y-C}}u|XZfT(7$ct1gTnX}PcqW^~3# z7>&i9-QAhgGPwvcdvHrfx90(*z9B^CNxVasXO4p{&u8h47XL2XQr&& zA$Mbt1&q*_%FP#*)?O8aoDg!R!TI?ObA zNrDe)+LTMWm1Gw(O;MuZkUv!9)9r3Id}KcSw1!Wz*q*;P@Tg4|)pOKo7>%%78C$p4 z()pjMHFRmJy4}|N{h%BALK}F7I;}>-Ri(!z+V5d@0A-MLNQKe3`~_auMeeF!`9^ XD4s}}!Fm6dp4z`Yc7Md!6+P$Qzl|+^ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-e18dc7d0-db98-40f2-9185-45237f51b9bf-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..549b403d15355ed4b7325efea6e1b2a6e2bca1b3 GIT binary patch literal 2316 zcmb7GU1%It6uvtmQq3~A|W6W?(W7xCx8(1+rK=id3d7=N~~?4Emm zzVn@P&$;ZS)6X<8!WCS_w{Ly)U8#a4I2N&j(7|a5A(YSk`cmz^97v#_#2S9EhD)=I>WzrHp7fV|?yc~;rIea!2x24qe zS{~2G+MRqFc3s8`v36IE9DvTv^W~sV{1;Y&&N}T9Jb|!WlQhY(fdmLj+wsG7n+$8^ zMVDPm8I*F5bZHGOMhyT(o~+SYb@AqrLx-8qIH-yNafv@fSb;;K?ezvhgqRO<8(*;ctG!rff!j&teLqf;vt#E@X=@os;R4Q23XNpMC896Mel6pbCXezDj zmNQ93$I`pcw9rPF8eoBtdPBjJ>MI32kIW?)uNXP4mgApM&zVcDXg;EEz~$2tokYC1 z8Q$jwY$0Bg6qc6VEX_9nBONvys5wIcp~hJU}v+IuZu z!AguD%>jp>Q|H0dIsxmD-O582_GSq`j?5hhOKXIY;X&+1cJfdo+nVT`)tH?`up46l z_GyXLcpY9D6mt-#j)S(;1#oD=+QA3l_^$}`4g>nFgbifw0#Gpq`iEgnE8CU-R@ujZ zfztgf6XF&0;^~7EwrvsKY+>C3QcRqsw3_AK9AtCYr(=^9?;P8 zy%()64T()cqWvgu7~!x_jX5I-{Xr)*ri@jGx^`g9J-1=7l?9VygF1F|z1|v}da#0| z?Sd7;=#-Hl8gm<)n^S3I@(^V4kd}@f&k3Ztfui#$*}#HR_L^TX-Tfl(y%&l=MbN+5fC&VC<0%LelO3AKdes zNV^b^-jiLw8qaFAVKtgoqk618a9q1;x2GGk%|`u{J<~qfoTi}iU?hT|3QBNwTuK2 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00001-f5c4b19d-a2b2-4189-8927-97fe1720df8d-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..006b8482ac8e5adc85a3dfe0ed13c12b1790a357 GIT binary patch literal 2301 zcmb7GU1%It6uvtB!h1IwU?A{MV;z6`@#xmm zQrOh9aoBe!qL3%a3-$kCNJWD-zb>GWK0z!*~uATEc7jSNr=<*xeTasDuyUdw0J1Wl9PwPDe!(+8=%O!`n|v+esZYuJZ_l|JybW zD~7HmQ<6-q7zCkcTjkR+^yru5o=m4<=$S1AsC*2N7fmCB(~{nyT4E+yIYWmG#FvUF z&U*xqs8s>--zbx4_&jh$hDlcOSz+zw@I_&ca;f*Gf}6s-lgG=#+{>o{LpAm83lqp( zUnvDW>R(z5+MBFH@ezd88AVqd8>oVzwH!a(wCP}`)O7iol)$NWX@||ArpX;VDo;1q zOu2de=%K^hH3hC>Kwat&5Y}K(XnEa!AYum|gVP7^6Hc(U-gn*5>42vQo+B;|!nG^3 zO+&})u02p^`F(PN;v4T%3>u5{eC;F%vRPN=GsAJ@%60= zzoxL|*x(jnuGigwRTr)uZMm=uX>7KVAXXN)wzp@~%;fTC`B0XPpC1uOYXe2+ajJu| zv%GnAI+xr}zl zM>+VEW?=kb4MN)KJqqrPZKR(Q(tEz+mxWf#b*tX6>g8jtzT?_uyERu|Xw++`?fKTJ s#vE(UpKjFbg;TY*OuQ;p%?TKn$qF)p?bHxgB(jeYRb zukZ39{l=`1^Sk4p{5~BceS2qjFMkM6)e!pV<{0JYI54#L{;dxgKT<@atVaMSqn!{n zRqoTtnOM{mgxI!3Y1CBq;@pD?ZJL_eT!2LRf;>iB3McnDf@*z}&!XHw7NaQ5k|-bG zX0)VkLC{n|h|_gN#+RXMr0L2Wz7SiR0=^oX`$FQ~7L(BROyb>1&D#>5j?*@>c!ru# zX6s@h=#cKK^FedTX%Re(urwyBqHDu07;?kyhD$bC9xKdxG^GGesZCnW7@E}?1~@Wg z(HSewZjBs2L0x%p@>dut$P$ z{yJ%r(DmE%Y(o(>UE3_lI+nCr4#^sA4vUJYTv9HV-bN0asI&>!vJe9@rWn{u z%2n8@bIRgOz)cm$`S}WW2Cwg~k0HJw_N-h*Bq{{h9nKS1eS&*)f=@gp51?XG8_U3xE~Cv7)f z@o+y-xINbH4!|X62Pq|BI~0uGrszq*(iIQ(go53b>Dg~fpoq9PUX2X2u1rHf9j+t% z0FJ$Rpc{Dr+L5t=N*gfFa#~HwfjkcpF~fUZSrZ_z^ia~(0zcILxq?dnv5T}a3`q$z z3__+aq-;VPNO>=-h$UJ1COZn_qO7WZchMvr0=L>^7u>w=bOX3lYPM?D%nDLZ9*798 zc_z$)Ydt44yZ&ot%L$21LZbdGtr+2Q$1$diAnf*V!gjE}EH)(jV3u#W-=p!0aE32#JB(pzb(tIfE96g^kkkp12-KVJz7bq3aeDW04LH=p3+YX&Yr+f6;-1JRR8ouv2qe^K~8F$@R#Y|e(28}LRa8lvcCcMZz2}} literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-648766cd-8ebd-475a-afbb-44ae0b9cba30-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d76352414b3e574f236272898eed0e551875ef93 GIT binary patch literal 2317 zcmb7GPiP!f7=JrEyP2eGZPL791{PTcGqe(iT{lV7u!LB!loCo62?dexX6GdtoZZ>Z z%vP2V5WPeR2ssF1P9h?bgBPLX;Gu_5Bw#Nd>OmBtSA~Xv+JoPlc{B4c{@KDX^S$r= zzTfxfec#)?`SNQGjBpj#@b>%1KRi>#5-iJDLFnv^gb*qg#y@@dDw9Azi8cJu{?5IE z02_NgfjzKdh>2gof4uW#_gMjQ{#pVV1(sJjJ`k^<-oF%EXCs)BDov`kvTSGlxTz_++~T*$NsQU={F;zbTP zEM|Zm8830|k(>Y^oc(K+ut$QoR>SrN?GQYHuskJclIuVaASkvQL>mqnOjVXWcK#Jm z%3ac-Q)s!$6bw})>vXENynp`OI8%**svHuJ1OtQ>Sd?tP+YdQ*;8Qp~(4KL_)wRCo zMQ#T)WzbA`B#c(?kT!{2zq=Z5sFGgQ_e`aVWqr1c6rBZ!B~? z_PG|y2wekFAf)apcvAhKgcp&y0`0|_vutDjE9zBqrIqwY@&?|xTauN?_c`OcU&0nL z_h1w&oG`9bO!m5Z3wByUM+qYE=A9x1hWz{{9puA}>=MpRLX|2fri19xvv+`~it61UuNgu@Y z3+f_lwdP6-j%sv~z&mr>=%(5JzWwIO(i;yFmn&i!L%vU1W?kJGm8(}%_$l!y= zAa-3$+ESMQ(1Njp2k7FjNc06G`fUUo$UFj~GAH_nX3a7?lK+<3H^70^A7xcC74_ZH zB+T@*ruo6TMS28A&x)tex80FL5~^>Mi>SCc4SN&YcBPi&{%k5(_kYDTdo6Y4)gr(I;^@dg2eV<7tH9ikun+! zn_FAcnPqwrqUi zPoBa%^lonJb|bn@gW-J(S7gp&`@Tm>cX*c5*J@72F4-_5o!%8?|>;yI5 ztF?yJXj+Zh1-tKhPR+4r8uQIY{jxJ_Uuw?K_Uz?m-I>2sZ##rG<{EUSKJPd-xv1~K Q&&R(#=#w%+H{p-aU+4M)i~s-t literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00003-fa1d35b8-bb75-4145-ac40-6ccbc04acc79-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..19fc8efe4f42cc6772f04bb4b60e09d657f80d04 GIT binary patch literal 2454 zcmb7GU1%It6uvV%yP3o_ZPMIf2BxqK?$XLS?6SKy4NHl^)KWqzBB3A>?(R&I#o3+B z&TM530SSEw)rTTWA0$M z@0|P1cTaNundgfbVHIcan{WT|-XRt9uuNeQq0usr5K3_OK7Q^f2Nw7zu#Dfh^XIKJ z6E^aMAGWS}Avydc{^h5w2ahu$N6z>m!+mT0!N@QX~xTAhW+Sv8q zuU$u-u?EkpTE059U$#}`T3_0L%ROs*lZBefB&=Jk_=K;lz+4s1}-R7NrKVJci;d zfzknPkCoWXaf-}wLAol5_#AYN6jhqXrvqz~!xsW`pNqUZd=$Eoh`hV8*%$D1kk(J) z8EQhA?K5eoO*$_xIQ3<-N$@zrLY9~Lh5@@^h_yz?T{cKBo1U|1N*bI(i!{wFno~Ut za3si*na#{?A2~cmT_fO1Im9BJ9>O9lk~O>4b$rA`&o<#Zg7>k8v#{8;EVt1FPYOKa z7IEB#i=#a%~yS$_)oy~RO7E$y!Bzg>!+}ea+{or(*dBz zln3^LbOCm%jItOLaAU`?pX3M&1_E>iL^ z^aRi_2${Z+^cJ*%qz{r3pA)6s{s-r=@NsuME*fo)!bg&)TT7Y?R4v|me&>=re!$V)QhVcod&Sg zFu)yzS$1m)RxP;Qq-Mb`q?vgwMAW8M*Vgh;W;kQwd?;%Up6@jf)rJ-uk5U~jP+Ztq zwmbBA19t8Yxj5ahyY#pN=X-N8PG7Wa@(9&I{&8-j<(f-oXYkt0^o>z!wr!cDH8{$_ zi5-K{`R%x*+5SJcm)4McHjv&^%}yrJYNn_cOL{SLtk!Ki<6~d;iOTiohX&f pdgWxPU`(DU)D2=5pDvo^!lYr;$Z=&8{xbdB58Y28bRPaA`x|eTJLCWW literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00004-d580234a-54cb-43b7-87db-448c67a315df-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3490b5662134957f749ebd69f129dd9b1a1102c5 GIT binary patch literal 2302 zcmb7G-)q}O96!sls$^!gX--IBh7pTsN2b}c-KHUgt*}x`C}oUL7$cmmOCly)t|YgZ z5a`1m0yS)ZKww*8ltG@>F-8U>Y!Fzs$32xjY;5dpWzdl4eos0{w`Of!Fz4@m@B96H zf86JL_O-X)tz(2MxQhSzQ`BOAgrsJq--a8LI{ah?B4OC15JqO`g$Hpni(ABg} zf`}zULnuC$S$hmU{u0y^Svw3pmjwfv^#T0icI0qsT02Bb#nh`{=-`3aR^q`~jQ|pt z%0d2XRnb&@6>>#JMO(&~xU^ls*SUCD$h5msX1Z0xi`-zpm>G=ZOnbmZNM`49IqZ|* z!eVghb2wYu4q>+bs@jwT++~)?B;VVv=K-NSs*7_{lLkJL)GFe%=jH1a-1Uq10-zY5IU8#a6IT*jR zth>7sv^QmPF0u9=*NXVRGyK04*4{06M^uiV@F0et*B0Su;T#M@ek(`J?d}Qu3bOXu z^OM#Hl*ww`FG7uMW|EcFnD0ce(FB0)onSSNWQf7RMAl`QjN~UWxyzg%#+e*|v%;PK zhoBk8!IK!@0tV84lQqdwwcDj>nB!Sp_k%T?^a+fc9Z#NplLjHMT`x84mR&>o*<%*r z4bO%ly)mGX9r(BGE{%voBBK8xZR`>N-Y7l;mg&3)ZoMAv9Aer(}c%u(9D?@`hlALaN{nuD>2 zHH=8N|2()4Hj#dXQ}6X|P~}{$*6n)3u2;{r2d?K-op!T6*QnPnIIZ^iMw5107aBEZ j?tHD|5L!Q1r_I`&D+pbKe?E@^JhBDJ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00005-963e7ff5-c414-444c-8984-6baecf6987ee-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..bdf5e95f168468e554cd21d1dfd4541ea40b6810 GIT binary patch literal 2316 zcmb7`U1%It6oBu}&Q2!j+MhIcn1Mx>!3>S8!!DboX;?xm7(!i25eWq;;qJ~P8JykO z&djzfAs~pQ^eOmiUKH`EFZxh4_|S)-C8(fMq3VOhJV{W&;DaxE?w!Aj@n;Lm?z!il z^PQi2?oF;d|56PjT*761`1t<&VhIbdl(3A@$!P&0l+WF+-1{{L66hzeioZGh^MfL1 zHug+p_Tm&XG4Qka!S&u5rU-y1GTm(=NpT@uc_SuhPoS8Jif+lId{pq9EpW?QYIARp%Bp%1VJb3iO z3C?6hjnbxw|B9fU5j0g!+Qf-i&=iE&wn*EfsqBYSkEZR=)J*0KMA`@NW9`Ura#B1* zOT@%0r>Wq9=u(J-(;@-FYRw=y(B+sN?3NT6UxsX{w1SsnX(xxT#^O;f)$R(o8EbFl z@j@)_D*l|20h}xx*D`M><+=>2#b?~DmWG-0fO9e{BXk}gUMpk zrN>ePrPw7MdlEHER6(usWZj-DH}9T3eTJ&WKqUplCH??m85V_>*X;+f?ZC6)^gw&U z30Bwot{XZX&`6*ecS#Vg-XLufI$n2`Z770P(sm5Fghg#eLb67a!-66xmz2wf+{jKj zlN59=J^M@xZG@?TED%z*WjvwWF5m@ZtiX81U5d>gk z8GX}q>5xV%9`*%=-IHj#yFv+%#p%X}#(2w06ZW9J$t-6QZSQd{&p)921BtfxR=$Mg zm>yVwwAJpxg5&SeV_P{Nz5k@AhaW4-w(pwXK(;BlC58ECE*!>Y& z;~jWIQJe!mbqv~27Qvta*A6~_W4}D;Jqr4H1nbDy2aptl{$W_t%J#*-RrWD3kn+8# z2!^b@R+xZ0T~SrfUpGmQz|}L^Ei`Y~egKoN7V74VSw-s9F^k~3Yr=)S-nT>3_imaU zJ0unfiTYUF)Wbo~*5~yg^!x2lujp%z?OK68|H`ILuPkgi7N}!0*XypsstZ?;v|QK) zH@l)IM16j9YparaCJ#a84`J=${aFHOY)H|0lISqyS>Ry9^J#fQW*)snnY-onX}LrD zv&SoQUvoY31kqu3Ghe40+UvGId~EKC%qT6-b8XTco^t$I&0zGA4MNiC{U7ey7E&+8 zqxW3LFUPZ5u9>yESu3Aw^&QtLTdnEZT)kGkV9m77*Qf3F%!PW@nmb=@Tg0x-*6iu( YoMpAhl(qwZKK|{4-jWcy0{;j71uFvZp_a{|!!yrRznykcTnn`xEAw*qpS(dOYB4I%!)J#{B#-8cf z?w*w)1ok2X9MFTuIV_9E97QB25eyPMs0R-l5B}uhMFc|xz4)rT`p3=c&caah>eYMS z``)YfYUb8!Z`c^&8m{9%zx(@8s$mJX3RV$%c1}VFl}q;SuOEYeKtF{w{O!?SA3awv zn|v`bdu1jzG4V_I`!Bxyxl}N@a6K`Z0F>7|J3brYnohfSlE8dq?uGTmU)^_ATV zXU}ofB&d{-dNddytiqAkYgpFj6{PB%9F~YgZjhU%x|H8>E)g`7-F&WvGD1rM3xvo}#ZzRbf|rrG z3hgJ5vutDhtK^!wx|DQBvI1`0E6J{icc0_!SFnZ55e!*X0LGPy!QLdd;bf`MPX!Th z^FAx+ixECiF(Jnld=;6;e2hf|UiysfF$9R4GFdrY-lEB_2zJ21K2Yhf`XGj1B+GELbP0wbzm*g2_TUVD9+~^# zmgNX7ljXQyh8+3KBrDG`-xa}z69D%48J^=%h8TiFvFkD>L-`RV54iJhF_QyuR)q6o z8Z^T=cogF^z(C{&nMkHe?p3B>j%PK^4>l~?qcCn(JbBh#7KG64TBT{tTMeXLJY^By z@hlk9JAD>ef&Z@6VG(s`M75vfEh8HAn6YStQP6Kk#;mdKGS3N(#kaN$KBjQwI-nM2 zp5NVoT^GiVwmdimH#Tdeh{od9_V#S%nGS#M4`JE({xN|pHc)h*COUL>9yr|e1D@WP z%#+tAcX#|gPj_tp_`%BE*FB#;MRe%b+}G_!Y=Z^k$1XgJIZDg-Jx06Zqnv(9b1?p} zh7s-bo&@*yHqvg0)_b`V)J3n>ZOd+2c73MRcRi=>wC3!Erro&W%(pH#=U99GO0(fC iTyC@-%Ir%vn`cQ{5`S&nYw=m3n?|Z-R z_x*X__jWHo`+N-}T*77i)9~P~TEY@66|5q3d`?0L<#W4#ym6QV3G`D~$8SA&_tp~v zY~ra1_VNseRPfXI{#PHqlM^7PE<})VV0o_7bG-qj3o+eyuMfXh1Z+ZQL>^gS-~73I z?@@uh{q@INyNzF#fvJd3;D^8O-fBr}owi8E?DT5rxlla#=-~M<^ zfQ;!;+-(29JQyb>LsyeJ2_lvZ4I#cQv-%i1{|M^wv>t|@$pS!Tg#bHVkBm-Dk_T^z zm{{cu9V`%AO0jd6B#`i2Gm;$WavTS{B~8VbAX+A^;`6bzlfzeH@i3Qahf)f?p2tfu zU@xBnZpe5!*6zy@0K&O_p&0at|H@j>-lQFZClQvXC0%lC2m%DP<@n*IO$O7&6_*`L z5tMS5bm%l%DKP~@<;ez}F0b4^b>bvbO@K-Xh)etd!YV8ZEw9@TV(h@9aC)FU<^*f& zeb)`04rmn6Ou8fp*RGK^2_3Jy#y2#{C>c8y=r-AySCDG3;IO1g+C}YBMQvuMoJks* zNzXphLK&fJAPR)En<}2t-YnoHR9S`gikZ`FIsZBBd}Xy6^+)suUfwOyO60r6`0f<2 zg(^ERXT_M1SBjIppk0BTrqEG>2)tRR33}1PP{o?|uz=5@$|0NN*aItl8toxC;B6Vb zS-7l8qm@YZ1ta@JVc`y?5}t_3;zLWTJCvZjNiSy-EALUQ$bZ224-{73>-iE^WB#ZQ zIDJD~g01Em7>Dds{$ghDkKrd#We?2K9HC{>9QX2&Bb%D&&2r3EBH7JxAp3lb<#+>b z85GALb{!{eYRdp;sV!6Fs6?)6Dkee`NLIW_niFJ%7U@Jp!X=@hP;fQ9ppr*9&!P-l`(~^btgG)wN(?ul8wZ`QEEmhla!^ zA<^&3TV^=uQL|wNq2F(Z=B&ByP}dI3#*14f8(G+LY*5EA*XwS;stY4XS}yE@8J#r~ zMzgWCy*-;+CKo|w4{qt`_B?>pH-zXsj(6zt%yF>k`7FH=Bad#P%-r_+EZvdq`Q??F zue%<34DZmpnXS_e=?3*j_bptJ8B5FaTuQp5vmCuvGctC`1|jM6{*UgZZKPj}Tkn~U zUyggVT(fF*t5%+A^&QtP+pW3UV!c*9YtOe9>T|R`f3{w=7Z<8+o6y>s8l9^y+IEYa THg@3W?q(**;_S|L zX121#fI_LUFZQ7Xk$n(E5e+S9ks1(+rmrCnK8SsZuRfI!icpB2JHJ;_bqmYPx#ynq zo$sFc?ny4Z`bGsKtl~W0I{5zcAruPoHAK zhMx_>{?z@DB7P44aDQVr$ApZX4MMVja=O*A?4D^(hHzg$`uO3~Ol$;)gNl^cM_>MQ zFUy2P#Mb|KX7}#5d($D(&fUL$d>pY?-hb(I7;^8!n+Hc2q^KAd6dS>#_~&c8*V9bM zkQ{L5umFcw@dx+!zu@Z7B>|yaX{Y{r{!s;?Th|J-KS!ZMd*58YH9`S|ppBv$0HBm| zj8|kaYA0i2UXc*;w*}fpMfP8eI~%u6krSH<5NN+3&u?1-Cq_MjYLUri(cVCo-=ZW- zp#1>1-)^i{Md&g#s`0O#^np5F1U$TQ*3qkejt=N)DVto3zXVnpJ%a zaHPnRS;)_Bj2%5jUBlqYxWpn(4`C4&>4x3zx&dOUXPfXH!F#0X&M$T?%WJm4lL61D zMO<(GGFc#A({9iE8xpUmO1~(oSWs#iBr22~<|SS_C!H^f_2g?OmV}PS8c4j*M$iwi zKuCHZ;&JKMG=3Qs=Rimya(r4Je?~e}oT~>q37&xKt0q{ncvrH3x1PZ|D)u=Q=R!cg zQ$E<6(nZ**Gs`w zu&YCq?k2C|;gD|l(c|!_I$eA02iMfq!fc~Q!sj)f+qz_RxH>93ig>u&wfJyMTFc~HTs}+X&M6R za2??XaQ>SGx;hL%+alIbaUI54MyUxIkmqAW?Bl&BUEv_G_=}**Id-W1a|MlBRFC+QH zp$PYqrNb<^)HOZbvES8Orbi6o5&3svMe}+cQ=8IU&*?6BT1i`MnwH^eQ*W(kbQ-`` z(*SoEX4&l}SheAHlZFMmkY<*&2vM6_SzRr~nbC|%@}aCbc)m{{t_>8MPf{H&P*T`k zwjFxB0htFwE=f1+E^CI{ydv_9~Lkh0=Sz<>W)H<|}%os#o&I8{MX5SFE{`I literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00007-15147217-b81a-45ab-92d4-24d725cc07e1-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..858a8a10b1832ca5d0e3e83d7b96881a1cfad8d0 GIT binary patch literal 2302 zcmb7G&uiR96rR;;W!c@9#PJ9TL|H^FlHiJ6cAYq8F|DB?1QS9jCQyo*U5#x~?P?=w z(=5giS^_>S>7j=bFiq>TkEOIe^|S}m{sUs*VL%N?1ndwT6HYDi(z4?`{^r0{;wF@$E;Cf1JpL zO}!C^eNv4=Onesq@#u@c^GK&I#UYb`@=mAcdIL%u9PZxT&pv)0+)P}N5xaAD??EY7 zF(EcOTV8HH{PQh9D&rIQ>60IRo63brGf~xq_x}BRUlnrO3GZTD=miM>x2>HNG*wQ< zlryoQDF{W|A{&pUM!$0I$#fi=n%TJkk<9_}qH$z!a?(3gOUxvzpsA37*izudS&sk` zwaP*M9i=NWz5-m4rYo!XBDZ!6_!>8l3aR%%NJDpuc!>v$ifO=5OudKP1Ty=V%0Z9# zAFc)MP1+%N8ewr(PzA>Zs$j@1#}7AcGMFu|xa>^I;1s*0Lub*7&KxYNNH*wfWo7@= ziIdDV1uiKdF7XEl%djZ5yly|>u>+67>4WzbCsUPk5-yD_{18-2?52ns}^jK7R7lW4rkgB>!kZzM+dK+y3N2jhp9 zPWM27@g_>nEH>Wb-aP&<4F7?|#=BG0vCQ#f9YpYp$`Wif&%tHLPUR^}+nd0zBXb04 zX^$Y8w8v2qdSrJd{<0pkRUT|O31Ht(upWmZ)ZiTQs!N#+#b=o8vE<()CWnwL@#NrMfk5Rgrq;SVVB$ zwcwIo@6*uoy^pL84T()cqW&sw8R4KujYT5}{eC+%=8ScRx^`eJzQ1L#YYJPA4Q?Lh zdfg3Jb>Z5PmJ7R(M(2zK(OBHt-kwV{lgppwLs>e0encRx4HTW1sSe7{3J05>&)OT4 zdHnih>9*Hr?T(!vJy==#y6cgbs1AC~a-D8SH>f{;?cB4Np|m{DrKCGP%JHW(17iwNF+sFq{?b+BZ_NC2*CtXiZO&>M%uM4y3#7z zRf=#;p$);f7Z+mF1|9lyDKs=av=^TWHKZq#Ll0>okT#G*pkR#2rI)^$UF}YB+A1J+ z-@JM6dvD&pdG^9vZx=DbI!@!A508Er(y<831eOsRnG_L1G2#B}qfZ5pKtF+1eEikp zzfN&x!)JW6AM?y4hhN1zpY3dmoXP1+zDW`&7aA?w=~~uwfP23G<)c?P+dfY5Bhtiv zesbqdj5FycW*jbTeo*{#HXuEG_TAGF&a6)wTmSe%VDiQNUq5xT zjft8n_shwNSk&OSY+Isb)KvE3)XA`Hni|QRfkf*CeynT}PVUzTqV+}Ii`E9bScxJp ziPi(utSpgQ5HwW~0(V`J@f8+SR~GTbKp7JNt0o~pacD;?YR>J88&Kr?O= z*IT|pYQ(EM&1JTsh?=hTa=CmeeFn6(AlsDA}!K%=pV=e zA>~lU6UxIleiP*up%Vjgcv{AvQ!eHfD}JB&PvEky@mD{PhU;zZy^9mnbU2%jdgqUotBPQ{P`n$BA@wzQ5Gm~+>cBkfL@|l&oWt(nh=IVNeP6ODg zo1hNNY^S*jt0vrTQng_h+^l@2Pt44$Z*1g4&;E>w{2{D0xIarEj14K)UnV+SpeV4r z=CtYY2F%@)FUZ{UhIc)3a8s_TaI(=^Ih1j$>P-IXLCy z#ExKee!CuNwEhowZ3C&7gVuYk(M|`wnl2i}l2J^bt#<0RnKrAF#i>%UFm0Br=S!1T ptvp>Sm{aEqHIrDyH;UF|VahbC+AtWY*P>e|_W~5!)qARVe zU8M-um*Sd|&=OM9gA;mdOQ}oR<`A87ND2+q`II?89VKp3IHsY*&>`A)(E17BJag&172Q< zA}@*61JqtwBDEl>iXa5;8j<8iPorFx0?&z1`$e5xr*N>{u8)f*Z3=*_s%fzHWOG!xh+A% zsQ~Cz%7eW|=3%G8brvTAH|jV}&qw%Z7!z_gfoD)|4bnfOJ`V`cIZ_ zwBljg47NVR((Q>FP6p`)AFW^Ao(Sa)dpeR>dG~91{$0lZW{8z{1E!Q5@clmMr@Ik@ zblJV|pDf*I#p50^+^(1I4jXy_jiTHu+Bmi&i}B34?tsv$J25pd$hp-;Ex$ zLZ-o>0@o3Kfb-rwXgvv_9T{g(ZWG2?0zk5)KCOv8-rHnd0K?o5lBT5iq4v)eMEZ|i zq~>AhNzlX4$n1s47Nmj5r*R_YWbzFRC!Exlp7)Sxa^9!skkx7#u^QyIr?cWQ1XpINM%mf>WkUSG|y zX#iVw1Jr?;WjB{#)r8wks}}5no0-pq#LU#{+FHKv8P1r+iiBd0W6%Msd+^xY^o=N0+qO*F z9GtR$Vn;AGza5u0TK|W8V+|>1g4TPv(M|`wnl9?al3q+7s&?v@kv6Im#mQ2!aLg!I tkCrCPTKQP1U`!q@)C_7Cj}*;`!lYqT>0$K-{AK#LANnJK(0TZe>~DG`EDQhu literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/employee-delta-table/part-00009-ee9dd918-aaec-4f80-bd63-e369c6335699-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1f3145483862d148ec825b336eae4f776abc81af GIT binary patch literal 2324 zcmb7`O=ufO6o5yo)q0}{H*qz~E{GzQTG*g0T4X1V5w2UPDZzx$gAOUJ8EMz{(v?=( zU8M-)OKG8mkVEMq2Se}0&8-xQoA%IyD8Yvw3ULliX)l4477C)h_08>G%I7}1ee_liB+!?!iobgBy2#NY{a~i_rG`~amEQxi$c=OKKp&{ub9nOmgP)2VNh-6jN#Z|0`t78|Z6|@wM1f}U9R79u@bOvB zWJZk=r-=WGpq&>qRZbekiCEASgxI!78>Ff1hf^1`hG;6c;7ml?3h-nd$$)ZFKEzAJ z#4D$%;DP8;h{Mw|0m7=yAUV+Gm>uj^6d7NIe5usN8?khd!`EYRl1sHmLTY*|kJn;@ zaXvM;BjWW~dn`rn(x3_}suH7Se4qY~&z?t{6{?_Cd9q{AS2quzKX;z0ilCAL;u3#^undbr$LkM+*mmUEaDJdY>jYcd zL)Q(R9%v-c%()~8w{DUy2_3J$#WoZ{t7r!`xq?M)SwgZ#lf!}{C|8uLHMy0Yb0#Th zAwB&}3vGnCfh-VG?#j5V+%DiXRBOY4#pJYF#(zcGsI^iQT!)<&KT^C5WNz5_9X`s}GFFs73U~w6CJ-e_o3=mAcmhG#Xhz>O zWjd(QiidqqVfQ7P@{v%%#W>~o;L;8s3DDwXo->KI__UViKc)Pq5^eFVds%t9lUn!)~b8^=-#?tw3LWV^62o7PcG<)UlcC^><*^higbWF6@GvUDp$$ zzPh)+Ur#-g#~|~Eu=e!+EP*sOr06_LbeQriaIow7w7e-Zk6xq9-SURC+^PN91D3gO zyB>Ll=rFsPuhS3h9owHiHup?sl#b`QHtA1KIsLR|F#6C2A?XeN5BKIiQm@3LccJH3 z<5{hmhS@aC>OyDexK`EbESW1!vvJv4?p$gv+1=&K&4#sdsnNBFZC*6(rN)Y7b;zQ2 S0DnUM?Snp+5V{8c5B&wEGYj1S literal 0 HcmV?d00001 From da882700b36bd18d731b36e43ec4675b52714a06 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 13:04:49 -0800 Subject: [PATCH 58/65] Update default table and rename schema constants. --- .../druid/delta/input/DeltaInputRowTest.java | 4 ++-- .../delta/input/DeltaInputSourceTest.java | 22 +++++++++---------- .../druid/delta/input/DeltaTestUtils.java | 11 +++++----- .../src/test/resources/create_delta_table.py | 2 +- processing/pom.xml | 1 + 5 files changed, 20 insertions(+), 20 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index e792d4f2af79..6f68774dbd61 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -57,14 +57,14 @@ public void testDeltaInputRow() throws TableNotFoundException, IOException Row next = dataReadResult.getRows().next(); DeltaInputRow deltaInputRow = new DeltaInputRow( next, - DeltaTestUtils.SCHEMA + DeltaTestUtils.FULL_SCHEMA ); Assert.assertNotNull(deltaInputRow); Assert.assertEquals(DeltaTestUtils.DIMENSIONS, deltaInputRow.getDimensions()); Map expectedRow = DeltaTestUtils.EXPECTED_ROWS.get(totalRecordCount); for (String key : expectedRow.keySet()) { - if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + if (DeltaTestUtils.FULL_SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = ((Long) expectedRow.get(key)) * 1000; Assert.assertEquals(expectedMillis, deltaInputRow.getTimestampFromEpoch()); } else { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index cc4b62890f96..28c68a37831c 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -44,7 +44,7 @@ public class DeltaInputSourceTest public void testSampleDeltaTable() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); - final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtils.SCHEMA, null, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader(DeltaTestUtils.FULL_SCHEMA, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualSampledRows.size()); @@ -60,7 +60,7 @@ public void testSampleDeltaTable() throws IOException Assert.assertEquals(1, actualSampledRow.getRawValuesList().size()); for (String key : expectedRow.keySet()) { - if (DeltaTestUtils.SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { + if (DeltaTestUtils.FULL_SCHEMA.getTimestampSpec().getTimestampColumn().equals(key)) { final long expectedMillis = (Long) expectedRow.get(key); Assert.assertEquals(expectedMillis, actualSampledRawVals.get(key)); } else { @@ -75,14 +75,14 @@ public void testReadAllDeltaTable() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final InputSourceReader inputSourceReader = deltaInputSource.reader( - DeltaTestUtils.SCHEMA, + DeltaTestUtils.FULL_SCHEMA, null, null ); final List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA); + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.FULL_SCHEMA); } @Test @@ -90,29 +90,29 @@ public void testReadAllDeltaTableSubSchema1() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final InputSourceReader inputSourceReader = deltaInputSource.reader( - DeltaTestUtils.SUB_SCHEMA_1, + DeltaTestUtils.SCHEMA_1, null, null ); final List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SUB_SCHEMA_1); + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA_1); } @Test - public void testReadAllDeltaTableInvalidSubSchema2() throws IOException + public void testReadAllDeltaTableWithSubSchema2() throws IOException { final DeltaInputSource deltaInputSource = new DeltaInputSource(DeltaTestUtils.DELTA_TABLE_PATH, null); final InputSourceReader inputSourceReader = deltaInputSource.reader( - DeltaTestUtils.SUB_SCHEMA_2, + DeltaTestUtils.SCHEMA_2, null, null ); final List actualReadRows = readAllRows(inputSourceReader); Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SUB_SCHEMA_2); + validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA_2); } @Test @@ -152,7 +152,7 @@ public void testDeltaLakeWithReadSplits() throws IOException deltaSplit ); final InputSourceReader inputSourceReader = deltaInputSourceWithSplit.reader( - DeltaTestUtils.SCHEMA, + DeltaTestUtils.FULL_SCHEMA, null, null ); @@ -160,7 +160,7 @@ public void testDeltaLakeWithReadSplits() throws IOException final List> expectedRowsInSplit = DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.get(idx); Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); - validateRows(expectedRowsInSplit, actualRowsInSplit, DeltaTestUtils.SCHEMA); + validateRows(expectedRowsInSplit, actualRowsInSplit, DeltaTestUtils.FULL_SCHEMA); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java index ec48df1d8d1f..180adaefcfb5 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java @@ -246,7 +246,7 @@ public class DeltaTestUtils /** * The Druid schema used for ingestion of {@link #DELTA_TABLE_PATH}. */ - public static final InputRowSchema SCHEMA = new InputRowSchema( + public static final InputRowSchema FULL_SCHEMA = new InputRowSchema( new TimestampSpec("birthday", "posix", null), new DimensionsSpec( ImmutableList.of( @@ -264,11 +264,10 @@ public class DeltaTestUtils ColumnsFilter.all() ); - /** - * Similar to {@link #SCHEMA}, but with a smaller set of columns with an inclusion filter applied. + * Similar to {@link #FULL_SCHEMA}, but with a smaller set of columns with an inclusion filter applied. */ - public static final InputRowSchema SUB_SCHEMA_1 = new InputRowSchema( + public static final InputRowSchema SCHEMA_1 = new InputRowSchema( new TimestampSpec("birthday", "posix", null), new DimensionsSpec( ImmutableList.of( @@ -287,10 +286,10 @@ public class DeltaTestUtils ); /** - * Similar to {@link #SCHEMA}, but with a smaller set of columns with an exclusion filter applied. A non-existent + * Similar to {@link #FULL_SCHEMA}, but with a smaller set of columns with an exclusion filter applied. A non-existent * column is added to the exclusion filter - it should silently get thrown away. */ - public static final InputRowSchema SUB_SCHEMA_2 = new InputRowSchema( + public static final InputRowSchema SCHEMA_2 = new InputRowSchema( new TimestampSpec("birthday", "posix", null), new DimensionsSpec( ImmutableList.of( diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index e78f35879bbd..ab9ec87fb005 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -96,7 +96,7 @@ def main(): parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="overwrite", help="Specify write mode (append/overwrite)") - parser.add_argument('--save_path', default=os.path.join(os.getcwd(), "employee-delta-table2"), + parser.add_argument('--save_path', default=os.path.join(os.getcwd(), "employee-delta-table"), help="Save path for Delta table") parser.add_argument('--num_records', type=int, default=10, help="Specify number of Delta records to write") diff --git a/processing/pom.xml b/processing/pom.xml index 915ae28d5abb..9abe06439e41 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -338,6 +338,7 @@ ${oshi.version} + junit From 883a75c5940eb4a28ef6dc6c763603c0b193b750 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 29 Jan 2024 23:17:41 -0800 Subject: [PATCH 59/65] Test setup and misc changes. --- .../apache/druid/delta/input/DeltaInputRow.java | 4 ++-- .../apache/druid/delta/input/DeltaTimeUtils.java | 4 ++-- .../org/apache/druid/delta/input/RowSerde.java | 4 ++-- .../druid/delta/input/DeltaInputSourceTest.java | 7 +++++++ .../druid/delta/input/DeltaTimeUtilsTest.java | 15 +++++++++++---- 5 files changed, 24 insertions(+), 10 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java index 2bffa2c871e1..acf909452a0b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputRow.java @@ -177,11 +177,11 @@ private static Object getValue(DataType dataType, io.delta.kernel.data.Row dataR } else if (dataType instanceof IntegerType) { return dataRow.getInt(columnOrdinal); } else if (dataType instanceof DateType) { - return DeltaTimeUtils.getDateTimeValue(dataRow.getInt(columnOrdinal)); + return DeltaTimeUtils.getSecondsFromDate(dataRow.getInt(columnOrdinal)); } else if (dataType instanceof LongType) { return dataRow.getLong(columnOrdinal); } else if (dataType instanceof TimestampType) { - return DeltaTimeUtils.getTimestampValue(dataRow.getLong(columnOrdinal)); + return DeltaTimeUtils.getMillisFromTimestamp(dataRow.getLong(columnOrdinal)); } else if (dataType instanceof FloatType) { return dataRow.getFloat(columnOrdinal); } else if (dataType instanceof DoubleType) { diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java index f9f251e82916..f47428944027 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaTimeUtils.java @@ -35,7 +35,7 @@ public class DeltaTimeUtils * @param microSecsSinceEpochUTC microseconds since epoch * @return Datetime millis correpsonding to {@code microSecsSinceEpochUTC} */ - public static long getTimestampValue(final long microSecsSinceEpochUTC) + public static long getMillisFromTimestamp(final long microSecsSinceEpochUTC) { final LocalDateTime dateTime = LocalDateTime.ofEpochSecond( microSecsSinceEpochUTC / 1_000_000 /* epochSecond */, @@ -52,7 +52,7 @@ public static long getTimestampValue(final long microSecsSinceEpochUTC) * @param daysSinceEpochUTC number of days since epoch * @return number of seconds corresponding to {@code daysSinceEpochUTC}. */ - public static long getDateTimeValue(final int daysSinceEpochUTC) + public static long getSecondsFromDate(final int daysSinceEpochUTC) { return LocalDate.ofEpochDay(daysSinceEpochUTC).atStartOfDay(ZONE_ID).toEpochSecond(); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java index 98e27b23af13..f10ac0574e9f 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java @@ -129,9 +129,9 @@ public static Map convertRowToJsonObject(Row row) } else if (fieldType instanceof DoubleType) { value = row.getDouble(fieldId); } else if (fieldType instanceof DateType) { - value = DeltaTimeUtils.getDateTimeValue(row.getInt(fieldId)); + value = DeltaTimeUtils.getSecondsFromDate(row.getInt(fieldId)); } else if (fieldType instanceof TimestampType) { - value = DeltaTimeUtils.getTimestampValue(row.getLong(fieldId)); + value = DeltaTimeUtils.getMillisFromTimestamp(row.getLong(fieldId)); } else if (fieldType instanceof StringType) { value = row.getString(fieldId); } else if (fieldType instanceof ArrayType) { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 28c68a37831c..890d5ee082d7 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.hamcrest.MatcherAssert; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -40,6 +41,12 @@ public class DeltaInputSourceTest { + @Before + public void setUp() + { + System.setProperty("user.timezone", "UTC"); + } + @Test public void testSampleDeltaTable() throws IOException { diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java index 490407f8bb2a..de78ed97aa30 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTimeUtilsTest.java @@ -21,19 +21,26 @@ import org.apache.druid.java.util.common.Intervals; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.time.Instant; public class DeltaTimeUtilsTest { + @Before + public void setUp() + { + System.setProperty("user.timezone", "UTC"); + } + @Test public void testTimestampValue() { Assert.assertEquals( Instant.parse("2018-02-02T00:28:02.000Z"), Instant.ofEpochMilli( - DeltaTimeUtils.getTimestampValue( + DeltaTimeUtils.getMillisFromTimestamp( Instant.parse("2018-02-02T00:28:02.000Z").toEpochMilli() * 1_000 ) ) @@ -42,7 +49,7 @@ public void testTimestampValue() Assert.assertEquals( Instant.parse("2024-01-31T00:58:03.000Z"), Instant.ofEpochMilli( - DeltaTimeUtils.getTimestampValue( + DeltaTimeUtils.getMillisFromTimestamp( Instant.parse("2024-01-31T00:58:03.002Z").toEpochMilli() * 1_000 ) ) @@ -55,7 +62,7 @@ public void testDateTimeValue() Assert.assertEquals( Instant.parse("2020-02-01T00:00:00.000Z"), Instant.ofEpochSecond( - DeltaTimeUtils.getDateTimeValue( + DeltaTimeUtils.getSecondsFromDate( (int) Intervals.of("1970-01-01/2020-02-01").toDuration().getStandardDays() ) ) @@ -64,7 +71,7 @@ public void testDateTimeValue() Assert.assertEquals( Instant.parse("2024-01-01T00:00:00.000Z"), Instant.ofEpochSecond( - DeltaTimeUtils.getDateTimeValue( + DeltaTimeUtils.getSecondsFromDate( (int) Intervals.of("1970-01-01/2024-01-01T02:23:00").toDuration().getStandardDays() ) ) From f4c3a99581e33385bd2e888ed3e0872711d256aa Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 30 Jan 2024 01:13:06 -0800 Subject: [PATCH 60/65] Add class loader logic as the context class loader is unaware about extension classes --- .../druid/delta/input/DeltaInputSource.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index d849060cde38..41496b6ba30a 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -111,7 +111,6 @@ public InputSourceReader reader( File temporaryDirectory ) { - // TODO: allow hadoop configurations such as credentials to be set here. final Configuration conf = new Configuration(); final TableClient tableClient = DefaultTableClient.create(conf); try { @@ -125,7 +124,7 @@ public InputSourceReader reader( .map(row -> deserialize(tableClient, row)) .collect(Collectors.toList()); } else { - final Table table = Table.forPath(tableClient, tablePath); + final Table table = readTable(); final Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); final StructType prunedSchema = pruneSchema( latestSnapshot.getSchema(tableClient), @@ -171,9 +170,8 @@ public Stream> createSplits(InputFormat inputFormat, @Nul final TableClient tableClient = DefaultTableClient.create(new Configuration()); final Snapshot latestSnapshot; - final Table table; + final Table table = readTable(); try { - table = Table.forPath(tableClient, tablePath); latestSnapshot = table.getLatestSnapshot(tableClient); } catch (TableNotFoundException e) { @@ -243,4 +241,21 @@ private StructType pruneSchema(final StructType baseSchema, final ColumnsFilter .collect(Collectors.toList()); return new StructType(selectedFields); } + + private Table readTable() + { + final ClassLoader currCtxClassloader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + final Configuration conf = new Configuration(); + final TableClient tableClient = DefaultTableClient.create(conf); + return Table.forPath(tableClient, tablePath); + } + catch (TableNotFoundException e) { + throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxClassloader); + } + } } From 72cb1f4fbce698be25ab90d1112d04bcb6c98967 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 30 Jan 2024 01:57:08 -0800 Subject: [PATCH 61/65] change some table client creation logic. --- .../druid/delta/input/DeltaInputSource.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 41496b6ba30a..936e2dd70d6b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -111,8 +111,7 @@ public InputSourceReader reader( File temporaryDirectory ) { - final Configuration conf = new Configuration(); - final TableClient tableClient = DefaultTableClient.create(conf); + final TableClient tableClient = createTableClient(); try { final Row scanState; final List scanRowList; @@ -124,7 +123,7 @@ public InputSourceReader reader( .map(row -> deserialize(tableClient, row)) .collect(Collectors.toList()); } else { - final Table table = readTable(); + final Table table = Table.forPath(tableClient, tablePath); final Snapshot latestSnapshot = table.getLatestSnapshot(tableClient); final StructType prunedSchema = pruneSchema( latestSnapshot.getSchema(tableClient), @@ -168,10 +167,10 @@ public Stream> createSplits(InputFormat inputFormat, @Nul return Stream.of(new InputSplit<>(deltaSplit)); } - final TableClient tableClient = DefaultTableClient.create(new Configuration()); + final TableClient tableClient = createTableClient(); final Snapshot latestSnapshot; - final Table table = readTable(); try { + final Table table = Table.forPath(tableClient, tablePath); latestSnapshot = table.getLatestSnapshot(tableClient); } catch (TableNotFoundException e) { @@ -242,17 +241,18 @@ private StructType pruneSchema(final StructType baseSchema, final ColumnsFilter return new StructType(selectedFields); } - private Table readTable() + /** + * @return a table client where the client is initialized with {@link Configuration} class that uses the class's + * class loader instead of the context classloader. The latter by default doesn't know about the extension classes, + * so the table client cannot load runtime classes resulting in {@link ClassNotFoundException}. + */ + private TableClient createTableClient() { final ClassLoader currCtxClassloader = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); final Configuration conf = new Configuration(); - final TableClient tableClient = DefaultTableClient.create(conf); - return Table.forPath(tableClient, tablePath); - } - catch (TableNotFoundException e) { - throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); + return DefaultTableClient.create(conf); } finally { Thread.currentThread().setContextClassLoader(currCtxClassloader); From e2159e5e8d71960dfb481181a0f13e8d86cf0e26 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 30 Jan 2024 02:33:49 -0800 Subject: [PATCH 62/65] Add hadoop-aws, hadoop-common and related exclusions. --- .../druid-deltalake-extensions/pom.xml | 213 ++++++++++++++++++ .../delta/input/DeltaInputSourceTest.java | 10 +- 2 files changed, 215 insertions(+), 8 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index ed4b025a9f48..5e8debfdddb6 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -105,6 +105,219 @@ 8.5.4 provided + + org.apache.hadoop + hadoop-aws + ${hadoop.compile.version} + runtime + + + com.amazonaws + aws-java-sdk-bundle + + + + + org.apache.hadoop + hadoop-common + ${hadoop.compile.version} + compile + + + io.netty + netty-buffer + + + commons-cli + commons-cli + + + log4j + log4j + + + commons-codec + commons-codec + + + commons-logging + commons-logging + + + commons-io + commons-io + + + commons-lang + commons-lang + + + commons-net + commons-net + + + commons-collections + commons-collections + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.zookeeper + zookeeper + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + javax.ws.rs + jsr311-api + + + com.google.code.findbugs + jsr305 + + + org.mortbay.jetty + jetty-util + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-webapp + + + com.google.protobuf + protobuf-java + + + com.sun.jersey + jersey-core + + + org.apache.curator + curator-client + + + org.apache.curator + curator-recipes + + + org.apache.commons + commons-math3 + + + com.google.guava + guava + + + org.apache.avro + avro + + + net.java.dev.jets3t + jets3t + + + com.sun.jersey + jersey-json + + + com.github.pjfanning + jersey-json + + + com.jcraft + jsch + + + org.mortbay.jetty + jetty + + + com.sun.jersey + jersey-server + + + + commons-beanutils + commons-beanutils-core + + + commons-beanutils + commons-beanutils + + + ch.qos.reload4j + reload4j + + + com.sun.jersey + jersey-servlet + + + javax.servlet.jsp + jsp-api + + + org.slf4j + slf4j-reload4j + + + com.google.re2j + re2j + + + com.google.code.gson + gson + + + com.nimbusds + nimbus-jose-jwt + + + com.github.stephenc.jcip + jcip-annotations + + + org.apache.curator + curator-framework + + + io.dropwizard.metrics + metrics-core + + + dnsjava + dnsjava + + + org.xerial.snappy + snappy-java + + + junit diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index 890d5ee082d7..24b1096abe1e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -87,8 +87,6 @@ public void testReadAllDeltaTable() throws IOException null ); final List actualReadRows = readAllRows(inputSourceReader); - Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.FULL_SCHEMA); } @@ -102,8 +100,6 @@ public void testReadAllDeltaTableSubSchema1() throws IOException null ); final List actualReadRows = readAllRows(inputSourceReader); - Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA_1); } @@ -117,8 +113,6 @@ public void testReadAllDeltaTableWithSubSchema2() throws IOException null ); final List actualReadRows = readAllRows(inputSourceReader); - Assert.assertEquals(DeltaTestUtils.EXPECTED_ROWS.size(), actualReadRows.size()); - validateRows(DeltaTestUtils.EXPECTED_ROWS, actualReadRows, DeltaTestUtils.SCHEMA_2); } @@ -165,8 +159,6 @@ public void testDeltaLakeWithReadSplits() throws IOException ); final List actualRowsInSplit = readAllRows(inputSourceReader); final List> expectedRowsInSplit = DeltaTestUtils.SPLIT_TO_EXPECTED_ROWS.get(idx); - Assert.assertEquals(expectedRowsInSplit.size(), actualRowsInSplit.size()); - validateRows(expectedRowsInSplit, actualRowsInSplit, DeltaTestUtils.FULL_SCHEMA); } } @@ -241,6 +233,8 @@ private void validateRows( final InputRowSchema schema ) { + Assert.assertEquals(expectedRows.size(), actualReadRows.size()); + for (int idx = 0; idx < expectedRows.size(); idx++) { final Map expectedRow = expectedRows.get(idx); final InputRow actualInputRow = actualReadRows.get(idx); From c2b820a23a56de13924a4b7c3e90258e4210a873 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 30 Jan 2024 03:13:43 -0800 Subject: [PATCH 63/65] Remove org.apache.hadoop:hadoop-common --- .../druid-deltalake-extensions/pom.xml | 201 ------------------ 1 file changed, 201 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index 5e8debfdddb6..267d21936928 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -117,207 +117,6 @@ - - org.apache.hadoop - hadoop-common - ${hadoop.compile.version} - compile - - - io.netty - netty-buffer - - - commons-cli - commons-cli - - - log4j - log4j - - - commons-codec - commons-codec - - - commons-logging - commons-logging - - - commons-io - commons-io - - - commons-lang - commons-lang - - - commons-net - commons-net - - - commons-collections - commons-collections - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - org.apache.zookeeper - zookeeper - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - - javax.ws.rs - jsr311-api - - - com.google.code.findbugs - jsr305 - - - org.mortbay.jetty - jetty-util - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-webapp - - - com.google.protobuf - protobuf-java - - - com.sun.jersey - jersey-core - - - org.apache.curator - curator-client - - - org.apache.curator - curator-recipes - - - org.apache.commons - commons-math3 - - - com.google.guava - guava - - - org.apache.avro - avro - - - net.java.dev.jets3t - jets3t - - - com.sun.jersey - jersey-json - - - com.github.pjfanning - jersey-json - - - com.jcraft - jsch - - - org.mortbay.jetty - jetty - - - com.sun.jersey - jersey-server - - - - commons-beanutils - commons-beanutils-core - - - commons-beanutils - commons-beanutils - - - ch.qos.reload4j - reload4j - - - com.sun.jersey - jersey-servlet - - - javax.servlet.jsp - jsp-api - - - org.slf4j - slf4j-reload4j - - - com.google.re2j - re2j - - - com.google.code.gson - gson - - - com.nimbusds - nimbus-jose-jwt - - - com.github.stephenc.jcip - jcip-annotations - - - org.apache.curator - curator-framework - - - io.dropwizard.metrics - metrics-core - - - dnsjava - dnsjava - - - org.xerial.snappy - snappy-java - - - junit From 1a52b56c6520ed6cc28f6db6d34309b8037cfb03 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Tue, 30 Jan 2024 17:14:14 -0800 Subject: [PATCH 64/65] Apply suggestions from code review Co-authored-by: Victoria Lim --- docs/development/extensions-contrib/delta-lake.md | 9 +++++---- docs/ingestion/input-sources.md | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index 10e1a80033cb..8b1de4849718 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -22,10 +22,11 @@ title: "Delta Lake extension" ~ under the License. --> -## Delta Lake Ingest extension +## Delta Lake extension -Delta Lake is an open source storage layer that brings reliability to data lakes. [DeltaLakeInputSource](../../ingestion/input-sources.md#delta-lake-input-source) lets +Delta Lake is an open source storage framework that enables building a +Lakehouse architecture with various compute engines. [DeltaLakeInputSource](../../ingestion/input-sources.md#delta-lake-input-source) lets you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta Lake extension, add the `druid-deltalake-extensions` to the list of loaded extensions. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. @@ -39,5 +40,5 @@ Older versions are unsupported, so consider upgrading to Delta Lake 3.0.x or hig ## Known limitations -- This extension only reads from the latest Delta table snapshot as the kernel API only supports that. -- Column filtering isn't supported yet. So the extension will read all the columns in the configured table. \ No newline at end of file +- This extension relies on the Delta Kernel API and can only read from the latest Delta table snapshot. +- Column filtering isn't supported. The extension reads all columns in the configured table. \ No newline at end of file diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index a2bc77a41a96..635f0b7c5b1f 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -932,7 +932,7 @@ The following is an example of a Combining input source spec: ## Iceberg input source :::info -You need to include the [`druid-iceberg-extensions`](../development/extensions-contrib/iceberg.md) as an extension to use the Iceberg input source. +To use the Iceberg input source, load the extension [`druid-iceberg-extensions`](../development/extensions-contrib/iceberg.md). ::: You use the Iceberg input source to read data stored in the Iceberg table format. For a given table, the input source scans up to the latest Iceberg snapshot from the configured Hive catalog. Druid ingests the underlying live data files using the existing input source formats. @@ -1124,7 +1124,7 @@ This input source provides the following filters: `and`, `equals`, `interval`, a ## Delta Lake input source :::info -You need to include the [`druid-deltalake-extensions`](../development/extensions-contrib/delta-lake.md) as an extension to use the Delta Lake input source. +To use the Delta Lake input source, load the extension [`druid-deltalake-extensions`](../development/extensions-contrib/delta-lake.md). ::: You can use the Delta input source to read data stored in a Delta Lake table. For a given table, the input source scans From 8b9ecb9d3a24322bdec7bc3ca3b6fefd7e996dbf Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 30 Jan 2024 17:51:03 -0800 Subject: [PATCH 65/65] Add entry to .spelling to fix docs static check --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index e3c9863dbfdd..0d4aeaf09a8e 100644 --- a/website/.spelling +++ b/website/.spelling @@ -163,6 +163,7 @@ Kerberos KeyStores Kinesis Kubernetes +Lakehouse LDAPS LRU LZ4