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
+
+
+
+ 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]}
>
-
-
- Delta Lake
-
+
+
+ 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-l{wgq$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 extends Module> 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