diff --git a/server/app/src/main/java/io/whitefox/api/deltasharing/DeltaMappers.java b/server/app/src/main/java/io/whitefox/api/deltasharing/DeltaMappers.java index 6948065c7..47503f78e 100644 --- a/server/app/src/main/java/io/whitefox/api/deltasharing/DeltaMappers.java +++ b/server/app/src/main/java/io/whitefox/api/deltasharing/DeltaMappers.java @@ -33,17 +33,21 @@ public static ReadTableRequest api2ReadTableRequest(QueryRequest request) { throw new IllegalArgumentException("version cannot be negative."); } else if (request.getVersion() != null && request.getTimestamp() == null) { return new ReadTableRequest.ReadTableVersion( - request.getPredicateHints(), + Optional.ofNullable(request.getPredicateHints()), + Optional.ofNullable(request.getJsonPredicateHints()), Optional.ofNullable(request.getLimitHint()), request.getVersion()); } else if (request.getVersion() == null && request.getTimestamp() != null) { return new ReadTableRequest.ReadTableAsOfTimestamp( - request.getPredicateHints(), + Optional.ofNullable(request.getPredicateHints()), + Optional.ofNullable(request.getJsonPredicateHints()), Optional.ofNullable(request.getLimitHint()), CommonMappers.parseTimestamp(request.getTimestamp())); } else if (request.getVersion() == null && request.getTimestamp() == null) { return new ReadTableRequest.ReadTableCurrentVersion( - request.getPredicateHints(), Optional.ofNullable(request.getLimitHint())); + Optional.ofNullable(request.getPredicateHints()), + Optional.ofNullable(request.getJsonPredicateHints()), + Optional.ofNullable(request.getLimitHint())); } else { throw new IllegalArgumentException("Cannot specify both version and timestamp"); } diff --git a/server/app/src/test/java/io/whitefox/api/deltasharing/server/DeltaSharesApiImplTest.java b/server/app/src/test/java/io/whitefox/api/deltasharing/server/DeltaSharesApiImplTest.java index de67b1609..0b268b957 100644 --- a/server/app/src/test/java/io/whitefox/api/deltasharing/server/DeltaSharesApiImplTest.java +++ b/server/app/src/test/java/io/whitefox/api/deltasharing/server/DeltaSharesApiImplTest.java @@ -369,6 +369,54 @@ public void queryNotExistingTable() throws IOException { .statusCode(404); } + @DisabledOnOs(OS.WINDOWS) + @Test + public void queryTableCurrentVersionWithPredicates() throws IOException { + var responseBodyLines = given() + .when() + .filter(deltaFilter) + .body("{\"jsonPredicateHints\": \"{" + " \\\"op\\\": \\\"equal\\\"," + + " \\\"children\\\": [" + + " {\\\"op\\\": \\\"column\\\", \\\"name\\\":\\\"date\\\", \\\"valueType\\\":\\\"date\\\"}," + + " {\\\"op\\\":\\\"literal\\\",\\\"value\\\":\\\"2021-04-29\\\",\\\"valueType\\\":\\\"date\\\"}" + + " ]" + + "}\"}") + .header(new Header("Content-Type", "application/json")) + .post( + "delta-api/v1/shares/{share}/schemas/{schema}/tables/{table}/query", + "name", + "default", + "table1") + .then() + .statusCode(200) + .extract() + .body() + .asString() + .split("\n"); + + assertEquals( + deltaTable1Protocol, + objectMapper.reader().readValue(responseBodyLines[0], ProtocolObject.class)); + assertEquals( + deltaTable1Metadata, + objectMapper.reader().readValue(responseBodyLines[1], MetadataObject.class)); + var files = Arrays.stream(responseBodyLines) + .skip(2) + .map(line -> { + try { + return objectMapper + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .reader() + .readValue(line, FileObjectWithoutPresignedUrl.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + assertEquals(7, responseBodyLines.length); + assertEquals(deltaTable1FilesWithoutPresignedUrl, files); + } + @DisabledOnOs(OS.WINDOWS) @Test public void queryTableCurrentVersion() throws IOException { diff --git a/server/core/build.gradle.kts b/server/core/build.gradle.kts index df966d8e7..3dd83fa86 100644 --- a/server/core/build.gradle.kts +++ b/server/core/build.gradle.kts @@ -43,6 +43,9 @@ dependencies { compileOnly(String.format("com.amazonaws:aws-java-sdk-s3:%s", awsSdkVersion)) implementation(String.format("org.apache.hadoop:hadoop-aws:%s", hadoopVersion)) + //PREDICATE PARSER + implementation("com.github.jsqlparser:jsqlparser:4.8") + // TEST testImplementation("org.junit.jupiter:junit-jupiter") testImplementation("io.quarkus:quarkus-arc") diff --git a/server/core/src/main/java/io/whitefox/core/ColumnRange.java b/server/core/src/main/java/io/whitefox/core/ColumnRange.java new file mode 100644 index 000000000..f6f7b2702 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/ColumnRange.java @@ -0,0 +1,132 @@ +package io.whitefox.core; + +import io.whitefox.annotations.SkipCoverageGenerated; +import io.whitefox.core.types.*; +import io.whitefox.core.types.predicates.TypeNotSupportedException; +import java.sql.Date; +import java.sql.Timestamp; + +public class ColumnRange { + + String minVal; + String maxVal; + + DataType valueType; + + public ColumnRange(String minVal, String maxVal, DataType valueType) { + this.minVal = minVal; + this.maxVal = maxVal; + this.valueType = valueType; + } + + public ColumnRange(String onlyVal, DataType valueType) { + this.minVal = onlyVal; + this.maxVal = onlyVal; + this.valueType = valueType; + } + + public DataType getValueType() { + return valueType; + } + + public String getSingleValue() { + return minVal; + } + + private Boolean typedContains(String point) throws TypeNotSupportedException { + if (valueType instanceof IntegerType) { + var c1 = Integer.compare(Integer.parseInt(minVal), Integer.parseInt(point)); + var c2 = Integer.compare(Integer.parseInt(maxVal), Integer.parseInt(point)); + return (c1 <= 0 && c2 >= 0); + } else if (valueType instanceof LongType) { + var c1 = Long.compare(Long.parseLong(minVal), Long.parseLong(point)); + var c2 = Long.compare(Long.parseLong(maxVal), Long.parseLong(point)); + return (c1 <= 0 && c2 >= 0); + } else if (valueType instanceof TimestampType) { + var c1 = Timestamp.valueOf(minVal).before(Timestamp.valueOf(point)); + var c2 = Timestamp.valueOf(maxVal).after(Timestamp.valueOf(point)); + return (c1 && c2) || Timestamp.valueOf(minVal).equals(Timestamp.valueOf(point)); + } else if (valueType instanceof FloatType) { + var c1 = Float.compare(Float.parseFloat(minVal), Float.parseFloat(point)); + var c2 = Float.compare(Float.parseFloat(maxVal), Float.parseFloat(point)); + return (c1 <= 0 && c2 >= 0); + } else if (valueType instanceof DoubleType) { + var c1 = Double.compare(Double.parseDouble(minVal), Double.parseDouble(point)); + var c2 = Double.compare(Double.parseDouble(maxVal), Double.parseDouble(point)); + return (c1 <= 0 && c2 >= 0); + } else if (valueType instanceof DateType) { + var c1 = Date.valueOf(minVal).before(Date.valueOf(point)); + var c2 = Date.valueOf(maxVal).after(Date.valueOf(point)); + return (c1 && c2) || Date.valueOf(minVal).equals(Date.valueOf(point)); + } else if (valueType instanceof BooleanType) { + var c1 = Boolean.parseBoolean(minVal) == Boolean.parseBoolean(point); + var c2 = Boolean.parseBoolean(maxVal) == Boolean.parseBoolean(point); + return c1 || c2; + } else if (valueType instanceof StringType) { + var c1 = minVal.compareTo(point); + var c2 = maxVal.compareTo(point); + return (c1 <= 0 && c2 >= 0); + } else throw new TypeNotSupportedException(valueType); + } + + private Boolean typedLessThan(String point) throws TypeNotSupportedException { + if (valueType instanceof IntegerType) { + var c1 = Integer.compare(Integer.parseInt(minVal), Integer.parseInt(point)); + return (c1 < 0); + } else if (valueType instanceof LongType) { + var c1 = Long.compare(Long.parseLong(minVal), Long.parseLong(point)); + return (c1 < 0); + } else if (valueType instanceof TimestampType) { + return Timestamp.valueOf(minVal).before(Timestamp.valueOf(point)); + } else if (valueType instanceof FloatType) { + var c1 = Float.compare(Float.parseFloat(minVal), Float.parseFloat(point)); + return (c1 < 0); + } else if (valueType instanceof DoubleType) { + var c1 = Double.compare(Double.parseDouble(minVal), Double.parseDouble(point)); + return (c1 < 0); + } else if (valueType instanceof DateType) { + return Date.valueOf(minVal).before(Date.valueOf(point)); + } else if (valueType instanceof StringType) { + var c = minVal.compareTo(point); + return (c < 0); + } else throw new TypeNotSupportedException(valueType); + } + + // not used currently + @SkipCoverageGenerated + private Boolean typedGreaterThan(String point) { + if (valueType instanceof IntegerType) { + var c = Integer.compare(Integer.parseInt(point), Integer.parseInt(maxVal)); + return (c < 0); + } else if (valueType instanceof LongType) { + var c = Long.compare(Long.parseLong(point), Long.parseLong(maxVal)); + return (c < 0); + } else if (valueType instanceof TimestampType) { + return Timestamp.valueOf(point).before(Timestamp.valueOf(maxVal)); + } else if (valueType instanceof FloatType) { + var c = Float.compare(Float.parseFloat(maxVal), Float.parseFloat(point)); + return (c < 0); + } else if (valueType instanceof DoubleType) { + var c = Double.compare(Double.parseDouble(maxVal), Double.parseDouble(point)); + return (c < 0); + } else if (valueType instanceof DateType) { + return Date.valueOf(point).before(Date.valueOf(maxVal)); + + } else { + var c = point.compareTo(maxVal); + return (c < 0); + } + } + + public Boolean contains(String point) throws TypeNotSupportedException { + return typedContains(point); + } + + public Boolean canBeLess(String point) throws TypeNotSupportedException { + return typedLessThan(point); + } + + public Boolean canBeGreater(String point) { + return typedGreaterThan(point); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/DeltaObjectMapper.java b/server/core/src/main/java/io/whitefox/core/DeltaObjectMapper.java new file mode 100644 index 000000000..12d00580c --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/DeltaObjectMapper.java @@ -0,0 +1,25 @@ +package io.whitefox.core; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import io.whitefox.core.types.DataType; +import io.whitefox.core.types.predicates.DataTypeDeserializer; + +public class DeltaObjectMapper { + + private static final ObjectMapper objectMapper = newInstance(); + + private static ObjectMapper newInstance() { + ObjectMapper mapper = new ObjectMapper(); + mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL); + var customSerializersModule = new SimpleModule(); + customSerializersModule.addDeserializer(DataType.class, new DataTypeDeserializer()); + mapper.registerModule(customSerializersModule); + return mapper; + } + + public static ObjectMapper getInstance() { + return objectMapper; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/FileStats.java b/server/core/src/main/java/io/whitefox/core/FileStats.java new file mode 100644 index 000000000..2c5b648ae --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/FileStats.java @@ -0,0 +1,50 @@ +package io.whitefox.core; + +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; + +public class FileStats { + // {"numRecords":1,"minValues":{"id":0},"maxValues":{"id":0},"nullCount":{"id":0}} + @JsonProperty("numRecords") + String numRecords; + + @JsonProperty("minValues") + Map minValues; + + @JsonProperty("maxValues") + Map maxValues; + + @JsonProperty("nullCount") + Map nullCount; + + public FileStats() { + super(); + } + + public String getNumRecords() { + return numRecords; + } + + public Map getMinValues() { + return minValues; + } + + public Map getMaxValues() { + return maxValues; + } + + public Map getNullCount() { + return nullCount; + } + + public FileStats( + String numRecords, + Map minValues, + Map maxValues, + Map nullCount) { + this.numRecords = numRecords; + this.minValues = minValues; + this.maxValues = maxValues; + this.nullCount = nullCount; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/PredicateUtils.java b/server/core/src/main/java/io/whitefox/core/PredicateUtils.java new file mode 100644 index 000000000..c1945b971 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/PredicateUtils.java @@ -0,0 +1,129 @@ +package io.whitefox.core; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.delta.standalone.actions.AddFile; +import io.whitefox.core.types.DataType; +import io.whitefox.core.types.predicates.*; +import java.util.List; +import java.util.Optional; +import net.sf.jsqlparser.JSQLParserException; +import net.sf.jsqlparser.expression.BinaryExpression; +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.StringValue; +import net.sf.jsqlparser.expression.operators.relational.IsNullExpression; +import net.sf.jsqlparser.parser.CCJSqlParserUtil; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.log4j.Logger; + +public class PredicateUtils { + + private static final Logger logger = Logger.getLogger(PredicateUtils.class); + + private static final ObjectMapper objectMapper = DeltaObjectMapper.getInstance(); + + public static BaseOp parseJsonPredicate(String predicate) throws PredicateParsingException { + try { + return objectMapper.readValue(predicate, BaseOp.class); + } catch (JsonProcessingException e) { + throw new PredicateParsingException(e); + } + } + + public static boolean evaluateJsonPredicate( + Optional predicate, EvalContext ctx, AddFile f) { + try { + if (predicate.isEmpty()) return true; + else { + var parsedPredicate = PredicateUtils.parseJsonPredicate(predicate.get()); + return parsedPredicate.evalExpectBoolean(ctx); + } + } catch (PredicateException e) { + logger.debug("Caught exception for predicate: " + predicate + " - " + e.getMessage()); + logger.info("File: " + f.getPath() + + " will be used in processing due to failure in parsing or processing the predicate: " + + predicate); + return true; + } + } + + public static boolean evaluateSqlPredicate( + String predicate, EvalContext ctx, AddFile f, Metadata metadata) { + try { + var parsedPredicate = PredicateUtils.parseSqlPredicate(predicate, ctx, metadata); + return parsedPredicate.evalExpectBoolean(ctx); + } catch (PredicateException e) { + logger.debug("Caught exception for predicate: " + predicate + " - " + e.getMessage()); + logger.info("File: " + f.getPath() + + " will be used in processing due to failure in parsing or processing the predicate: " + + predicate); + return true; + } + } + + public static BaseOp parseSqlPredicate(String predicate, EvalContext ctx, Metadata metadata) + throws PredicateException { + try { + var expression = CCJSqlParserUtil.parseCondExpression(predicate); + if (expression instanceof IsNullExpression) { + var isNullExpression = (IsNullExpression) expression; + String column = + isNullExpression.getLeftExpression().getASTNode().jjtGetFirstToken().toString(); + var dataType = metadata.tableSchema().structType().get(column).getDataType(); + var colOp = new ColumnOp(column, dataType); + var children = List.of((LeafOp) colOp); + var operator = "isnull"; + return NonLeafOp.createPartitionFilter(children, operator); + } else if (expression instanceof BinaryExpression) { + BinaryExpression binaryExpression = (BinaryExpression) expression; + String column = binaryExpression.getLeftExpression().toString(); + String operator = binaryExpression.getStringExpression(); + Expression value = binaryExpression.getRightExpression(); + if (value instanceof StringValue) { + StringValue stringValue = (StringValue) value; + var dataType = metadata.tableSchema().structType().get(column).getDataType(); + var colOp = new ColumnOp(column, dataType); + var litOp = new LiteralOp(stringValue.getValue(), dataType); + var children = List.of(colOp, litOp); + return NonLeafOp.createPartitionFilter(children, operator); + } else { + var dataType = metadata.tableSchema().structType().get(column).getDataType(); + var colOp = new ColumnOp(column, dataType); + var litOp = new LiteralOp(value.toString(), dataType); + var children = List.of(colOp, litOp); + return NonLeafOp.createPartitionFilter(children, operator); + } + } else throw new ExpressionNotSupportedException(predicate); + } catch (JSQLParserException e) { + throw new PredicateParsingException(e); + } + } + + public static ColumnRange createColumnRange(String name, EvalContext ctx, DataType valueType) + throws NonExistingColumnException { + var fileStats = ctx.getStatsValues(); + var values = Optional.ofNullable(fileStats.get(name)) + .orElseThrow(() -> new NonExistingColumnException(name)); + return new ColumnRange(values.getLeft(), values.getRight(), valueType); + } + + public static EvalContext createEvalContext(AddFile file) throws PredicateParsingException { + var statsString = file.getStats(); + var partitionValues = file.getPartitionValues(); + + try { + var fileStats = objectMapper.readValue(statsString, FileStats.class); + var maxValues = fileStats.maxValues; + var mappedMinMaxPairs = new java.util.HashMap>(); + fileStats.getMinValues().forEach((minK, minV) -> { + String maxV = maxValues.get(minK); + Pair minMaxPair = Pair.of(minV, maxV); + mappedMinMaxPairs.put(minK, minMaxPair); + }); + return new EvalContext(partitionValues, mappedMinMaxPairs); + } catch (JsonProcessingException e) { + // should never happen, depends on if the delta implementation changes + throw new PredicateParsingException(e); + } + } +} diff --git a/server/core/src/main/java/io/whitefox/core/ReadTableRequest.java b/server/core/src/main/java/io/whitefox/core/ReadTableRequest.java index aa6df33ab..55950d454 100644 --- a/server/core/src/main/java/io/whitefox/core/ReadTableRequest.java +++ b/server/core/src/main/java/io/whitefox/core/ReadTableRequest.java @@ -7,19 +7,30 @@ public interface ReadTableRequest { - public static class ReadTableVersion implements ReadTableRequest { - private final List predicateHints; + class ReadTableVersion implements ReadTableRequest { + private final Optional> predicateHints; + private final Optional jsonPredicateHints; private final Optional limitHint; private final Long version; - public ReadTableVersion(List predicateHints, Optional limitHint, Long version) { + public ReadTableVersion( + Optional> predicateHints, + Optional jsonPredicateHints, + Optional limitHint, + Long version) { + this.predicateHints = predicateHints; + this.jsonPredicateHints = jsonPredicateHints; this.limitHint = limitHint; this.version = version; } - public List predicateHints() { + public Optional jsonPredicateHints() { + return jsonPredicateHints; + } + + public Optional> predicateHints() { return predicateHints; } @@ -38,6 +49,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; ReadTableVersion that = (ReadTableVersion) o; return Objects.equals(predicateHints, that.predicateHints) + && Objects.equals(jsonPredicateHints, that.jsonPredicateHints) && Objects.equals(limitHint, that.limitHint) && Objects.equals(version, that.version); } @@ -45,31 +57,43 @@ public boolean equals(Object o) { @Override @SkipCoverageGenerated public int hashCode() { - return Objects.hash(predicateHints, limitHint, version); + return Objects.hash(predicateHints, jsonPredicateHints, limitHint, version); } @Override @SkipCoverageGenerated public String toString() { return "ReadTableVersion{" + "predicateHints=" - + predicateHints + ", limitHint=" + + predicateHints + "jsonPredicateHints=" + + jsonPredicateHints + ", limitHint=" + limitHint + ", version=" + version + '}'; } } - public static class ReadTableAsOfTimestamp implements ReadTableRequest { - private final List predicateHints; + class ReadTableAsOfTimestamp implements ReadTableRequest { + private final Optional> predicateHints; + private final Optional limitHint; + private final Optional jsonPredicateHints; private final Long timestamp; public ReadTableAsOfTimestamp( - List predicateHints, Optional limitHint, Long timestamp) { + Optional> predicateHints, + Optional jsonPredicateHints, + Optional limitHint, + Long timestamp) { + this.predicateHints = predicateHints; + this.jsonPredicateHints = jsonPredicateHints; this.limitHint = limitHint; this.timestamp = timestamp; } + public Optional jsonPredicateHints() { + return jsonPredicateHints; + } + @Override @SkipCoverageGenerated public boolean equals(Object o) { @@ -77,6 +101,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; ReadTableAsOfTimestamp that = (ReadTableAsOfTimestamp) o; return Objects.equals(predicateHints, that.predicateHints) + && Objects.equals(jsonPredicateHints, that.jsonPredicateHints) && Objects.equals(limitHint, that.limitHint) && Objects.equals(timestamp, that.timestamp); } @@ -84,19 +109,20 @@ public boolean equals(Object o) { @Override @SkipCoverageGenerated public int hashCode() { - return Objects.hash(predicateHints, limitHint, timestamp); + return Objects.hash(jsonPredicateHints, predicateHints, limitHint, timestamp); } @Override @SkipCoverageGenerated public String toString() { return "ReadTableAsOfTimestamp{" + "predicateHints=" - + predicateHints + ", limitHint=" + + predicateHints + "jsonPredicateHints=" + + jsonPredicateHints + ", limitHint=" + limitHint + ", timestamp=" + timestamp + '}'; } - public List predicateHints() { + public Optional> predicateHints() { return predicateHints; } @@ -109,19 +135,28 @@ public Long timestamp() { } } - public static class ReadTableCurrentVersion implements ReadTableRequest { - private final List predicateHints; + class ReadTableCurrentVersion implements ReadTableRequest { + private final Optional> predicateHints; + private final Optional jsonPredicateHints; private final Optional limitHint; - public ReadTableCurrentVersion(List predicateHints, Optional limitHint) { + public ReadTableCurrentVersion( + Optional> predicateHints, + Optional jsonPredicateHints, + Optional limitHint) { this.predicateHints = predicateHints; + this.jsonPredicateHints = jsonPredicateHints; this.limitHint = limitHint; } - public List predicateHints() { + public Optional> predicateHints() { return predicateHints; } + public Optional jsonPredicateHints() { + return jsonPredicateHints; + } + public Optional limitHint() { return limitHint; } @@ -130,7 +165,8 @@ public Optional limitHint() { @SkipCoverageGenerated public String toString() { return "ReadTableCurrentVersion{" + "predicateHints=" - + predicateHints + ", limitHint=" + + predicateHints + "jsonPredicateHints=" + + jsonPredicateHints + ", limitHint=" + limitHint + '}'; } @@ -141,13 +177,14 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; ReadTableCurrentVersion that = (ReadTableCurrentVersion) o; return Objects.equals(predicateHints, that.predicateHints) + && Objects.equals(jsonPredicateHints, that.jsonPredicateHints) && Objects.equals(limitHint, that.limitHint); } @Override @SkipCoverageGenerated public int hashCode() { - return Objects.hash(predicateHints, limitHint); + return Objects.hash(jsonPredicateHints, predicateHints, limitHint); } } } diff --git a/server/core/src/main/java/io/whitefox/core/services/DeltaSharedTable.java b/server/core/src/main/java/io/whitefox/core/services/DeltaSharedTable.java index 44f31e129..cbeb66819 100644 --- a/server/core/src/main/java/io/whitefox/core/services/DeltaSharedTable.java +++ b/server/core/src/main/java/io/whitefox/core/services/DeltaSharedTable.java @@ -1,14 +1,25 @@ package io.whitefox.core.services; +import static io.whitefox.core.PredicateUtils.evaluateJsonPredicate; +import static io.whitefox.core.PredicateUtils.evaluateSqlPredicate; + import io.delta.standalone.DeltaLog; import io.delta.standalone.Snapshot; +import io.delta.standalone.actions.AddFile; import io.whitefox.core.*; +import io.whitefox.core.Metadata; +import io.whitefox.core.TableSchema; +import io.whitefox.core.types.predicates.PredicateException; import java.sql.Timestamp; +import java.util.List; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.log4j.Logger; public class DeltaSharedTable implements InternalSharedTable { + private final Logger logger = Logger.getLogger(this.getClass()); + private final DeltaLog deltaLog; private final TableSchemaConverter tableSchemaConverter; private final SharedTable tableDetails; @@ -76,23 +87,70 @@ public Optional getTableVersion(Optional startingTimestamp) { return getSnapshot(startingTimestamp).map(Snapshot::getVersion); } + public boolean filterFilesBasedOnSqlPredicates( + Optional> predicates, AddFile f, Metadata metadata) { + // if there are no predicates return all possible files + if (predicates.isEmpty()) { + return true; + } + try { + var ctx = PredicateUtils.createEvalContext(f); + return predicates.get().stream().allMatch(p -> evaluateSqlPredicate(p, ctx, f, metadata)); + } catch (PredicateException e) { + logger.debug("Caught exception: " + e.getMessage()); + logger.info("File: " + f.getPath() + + " will be used in processing due to failure in parsing or processing the predicate"); + return true; + } + } + + public boolean filterFilesBasedOnJsonPredicates(Optional predicates, AddFile f) { + // if there are no predicates return all possible files + if (predicates.isEmpty()) { + return true; + } + try { + var ctx = PredicateUtils.createEvalContext(f); + return evaluateJsonPredicate(predicates, ctx, f); + } catch (PredicateException e) { + logger.debug("Caught exception: " + e.getMessage()); + logger.info("File: " + f.getPath() + + " will be used in processing due to failure in parsing or processing the predicate"); + return true; + } + } + public ReadTableResultToBeSigned queryTable(ReadTableRequest readTableRequest) { + Optional predicates; + Optional> sqlPredicates; Snapshot snapshot; if (readTableRequest instanceof ReadTableRequest.ReadTableCurrentVersion) { snapshot = deltaLog.snapshot(); + predicates = + ((ReadTableRequest.ReadTableCurrentVersion) readTableRequest).jsonPredicateHints(); + sqlPredicates = + ((ReadTableRequest.ReadTableCurrentVersion) readTableRequest).predicateHints(); } else if (readTableRequest instanceof ReadTableRequest.ReadTableAsOfTimestamp) { snapshot = deltaLog.getSnapshotForTimestampAsOf( ((ReadTableRequest.ReadTableAsOfTimestamp) readTableRequest).timestamp()); + predicates = + ((ReadTableRequest.ReadTableAsOfTimestamp) readTableRequest).jsonPredicateHints(); + sqlPredicates = ((ReadTableRequest.ReadTableAsOfTimestamp) readTableRequest).predicateHints(); } else if (readTableRequest instanceof ReadTableRequest.ReadTableVersion) { snapshot = deltaLog.getSnapshotForVersionAsOf( ((ReadTableRequest.ReadTableVersion) readTableRequest).version()); + predicates = ((ReadTableRequest.ReadTableVersion) readTableRequest).jsonPredicateHints(); + sqlPredicates = ((ReadTableRequest.ReadTableVersion) readTableRequest).predicateHints(); } else { throw new IllegalArgumentException("Unknown ReadTableRequest type: " + readTableRequest); } + var metadata = metadataFromSnapshot(snapshot); return new ReadTableResultToBeSigned( new Protocol(Optional.of(1)), - metadataFromSnapshot(snapshot), + metadata, snapshot.getAllFiles().stream() + .filter(f -> filterFilesBasedOnJsonPredicates(predicates, f)) + .filter(f -> filterFilesBasedOnSqlPredicates(sqlPredicates, f, metadata)) .map(f -> new TableFileToBeSigned( location() + "/" + f.getPath(), f.getSize(), diff --git a/server/core/src/main/java/io/whitefox/core/types/BasePrimitiveType.java b/server/core/src/main/java/io/whitefox/core/types/BasePrimitiveType.java index 1de9ca463..0f6bb5ee3 100644 --- a/server/core/src/main/java/io/whitefox/core/types/BasePrimitiveType.java +++ b/server/core/src/main/java/io/whitefox/core/types/BasePrimitiveType.java @@ -38,7 +38,7 @@ public static List getAllPrimitiveTypes() { put("boolean", BooleanType.BOOLEAN); put("byte", ByteType.BYTE); put("short", ShortType.SHORT); - put("integer", IntegerType.INTEGER); + put("int", IntegerType.INTEGER); put("long", LongType.LONG); put("float", FloatType.FLOAT); put("double", DoubleType.DOUBLE); diff --git a/server/core/src/main/java/io/whitefox/core/types/IntegerType.java b/server/core/src/main/java/io/whitefox/core/types/IntegerType.java index 546426a86..d8b1040f7 100644 --- a/server/core/src/main/java/io/whitefox/core/types/IntegerType.java +++ b/server/core/src/main/java/io/whitefox/core/types/IntegerType.java @@ -7,6 +7,6 @@ public class IntegerType extends BasePrimitiveType { public static final IntegerType INTEGER = new IntegerType(); private IntegerType() { - super("integer"); + super("int"); } } diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/BaseOp.java b/server/core/src/main/java/io/whitefox/core/types/predicates/BaseOp.java new file mode 100644 index 000000000..c95bb0fb9 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/BaseOp.java @@ -0,0 +1,118 @@ +package io.whitefox.core.types.predicates; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.whitefox.core.types.*; +import java.util.List; +import java.util.Objects; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "op") +@JsonSubTypes({ + @JsonSubTypes.Type( + value = LeafOp.class, + names = {"column", "literal"}), + @JsonSubTypes.Type(value = EqualOp.class, name = "equal"), + @JsonSubTypes.Type(value = NotOp.class, name = "not"), + @JsonSubTypes.Type(value = OrOp.class, name = "or"), + @JsonSubTypes.Type(value = AndOp.class, name = "and"), + @JsonSubTypes.Type(value = LessThanOp.class, name = "lessThan"), + @JsonSubTypes.Type(value = LessThanOrEqualOp.class, name = "lessThanOrEqual"), + @JsonSubTypes.Type(value = GreaterThanOp.class, name = "greaterThan"), + @JsonSubTypes.Type(value = GreaterThanOrEqualOp.class, name = "greaterThanOrEqual") +}) +public interface BaseOp { + + void validate() throws PredicateException; + + default Boolean isSupportedType(DataType valueType, EvaluatorVersion version) { + if (version == EvaluatorVersion.V2) { + return (valueType instanceof BooleanType + || valueType instanceof IntegerType + || valueType instanceof StringType + || valueType instanceof DateType + || valueType instanceof LongType + || valueType instanceof TimestampType + || valueType instanceof FloatType + || valueType instanceof DoubleType); + } else + return (valueType instanceof BooleanType + || valueType instanceof IntegerType + || valueType instanceof StringType + || valueType instanceof DateType + || valueType instanceof LongType); + } + + Object eval(EvalContext ctx) throws PredicateException; + + default Boolean evalExpectBoolean(EvalContext ctx) throws PredicateException { + var res = eval(ctx); + if (res instanceof Boolean) { + return (Boolean) res; + } else { + throw new WrongExpectedTypeException(res, Boolean.class); + } + } + + List getAllChildren(); + + default Boolean treeDepthExceeds(Integer depth) { + if (depth <= 0) { + return true; + } else { + return getAllChildren().stream().anyMatch(c -> c.treeDepthExceeds(depth - 1)); + } + } +} + +// marker interface for operator arity used for easier exception handling +interface Arity {} +; + +// Represents a unary operation. +interface UnaryOp extends Arity { + // Validates number of children to be 1. + default void validateChildren(List children) throws PredicateException { + if (children.size() != 1) throw new PredicateValidationException(children.size(), this, 1); + try { + children.get(0).validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + } +} + +interface BinaryOp extends Arity { + // Validates number of children to be 2. + default void validateChildren(List children) throws PredicateException { + if (children.size() != 2) throw new PredicateValidationException(children.size(), this, 2); + + // otherwise cannot throw exception in method call of lambda + for (BaseOp c : children) { + c.validate(); + } + + var child1 = children.get(0); + var child2 = children.get(1); + if (child1 instanceof LeafOp && child2 instanceof LeafOp) { + var leftType = ((LeafOp) child1).getOpValueType(); + var rightType = ((LeafOp) child2).getOpValueType(); + if (!Objects.equals(leftType, rightType)) { + throw new TypeMismatchException(leftType, rightType); + } + } + } +} + +// not used currently +interface NaryOp extends Arity { + // Validates number of children to be at least 2. + default void validateChildren(List children) throws PredicateException { + if (children.size() < 2) { + throw new PredicateValidationException(children.size(), this, 2); + } + + for (BaseOp c : children) { + c.validate(); + } + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/ColumnOp.java b/server/core/src/main/java/io/whitefox/core/types/predicates/ColumnOp.java new file mode 100644 index 000000000..6cfec32b9 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/ColumnOp.java @@ -0,0 +1,70 @@ +package io.whitefox.core.types.predicates; + +import static io.whitefox.core.PredicateUtils.createColumnRange; +import static io.whitefox.core.types.predicates.EvaluatorVersion.V1; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.whitefox.core.ColumnRange; +import io.whitefox.core.types.BooleanType; +import io.whitefox.core.types.DataType; +import java.util.Objects; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "column") +public class ColumnOp extends LeafOp { + + @JsonProperty("name") + String name; + + public ColumnOp() { + super(); + } + + public ColumnOp(String name, DataType valueType) { + this.name = name; + this.valueType = valueType; + } + + // Determine if the column value is null. + @Override + public Boolean isNull(EvalContext ctx) { + return ctx.partitionValues.get(name) == null && !ctx.getStatsValues().containsKey(name); + } + + @Override + public Boolean evalExpectBoolean(EvalContext ctx) { + if (!Objects.equals(valueType, BooleanType.BOOLEAN)) { + throw new IllegalArgumentException("Unsupported type for boolean evaluation: " + valueType); + } + return Boolean.valueOf(resolve(ctx)); + } + + public ColumnRange evalExpectColumnRange(EvalContext ctx) throws NonExistingColumnException { + return createColumnRange(name, ctx, valueType); + } + + @Override + public DataType getOpValueType() { + return valueType; + } + + @Override + public Object eval(EvalContext ctx) { + // TODO: handle case of null column + column ranges + return new ColumnRange(resolve(ctx), valueType); + } + + public void validate() throws PredicateException { + if (name == null) { + throw new NonExistingColumnException("Name must be specified: " + this); + } + if (!this.isSupportedType(valueType, V1)) { + throw new TypeNotSupportedException(valueType); + } + } + + private String resolve(EvalContext ctx) { + // TODO: handle case of null column + column ranges + return ctx.partitionValues.getOrDefault(name, null); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/DataTypeDeserializer.java b/server/core/src/main/java/io/whitefox/core/types/predicates/DataTypeDeserializer.java new file mode 100644 index 000000000..2b81a8d5a --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/DataTypeDeserializer.java @@ -0,0 +1,53 @@ +package io.whitefox.core.types.predicates; + +import static io.whitefox.core.types.DateType.DATE; +import static io.whitefox.core.types.DoubleType.DOUBLE; +import static io.whitefox.core.types.FloatType.FLOAT; +import static io.whitefox.core.types.IntegerType.INTEGER; +import static io.whitefox.core.types.LongType.LONG; +import static io.whitefox.core.types.StringType.STRING; +import static io.whitefox.core.types.TimestampType.TIMESTAMP; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import io.whitefox.core.types.*; +import java.io.IOException; + +public class DataTypeDeserializer extends StdDeserializer { + + // needed for jackson + public DataTypeDeserializer() { + this(null); + } + + public DataTypeDeserializer(Class vc) { + super(vc); + } + + @Override + public DataType deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + JsonNode node = jsonParser.getCodec().readTree(jsonParser); + String valueType = node.asText(); + DataType primitive = BasePrimitiveType.createPrimitive(valueType); + if (DATE.equals(primitive)) { + return DATE; + } else if (INTEGER.equals(primitive)) { + return INTEGER; + } else if (DOUBLE.equals(primitive)) { + return DOUBLE; + } else if (FLOAT.equals(primitive)) { + return FLOAT; + } else if (STRING.equals(primitive)) { + return STRING; + } else if (TIMESTAMP.equals(primitive)) { + return TIMESTAMP; + } else if (LONG.equals(primitive)) { + return LONG; + } + throw new JsonParseException("Unknown type passed inside a json predicate: " + valueType); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/EvalContext.java b/server/core/src/main/java/io/whitefox/core/types/predicates/EvalContext.java new file mode 100644 index 000000000..e9ad7cd12 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/EvalContext.java @@ -0,0 +1,24 @@ +package io.whitefox.core.types.predicates; + +import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; + +public class EvalContext { + + public EvalContext( + Map partitionValues, Map> statsValues) { + this.partitionValues = partitionValues; + this.statsValues = statsValues; + } + + final Map partitionValues; + final Map> statsValues; + + public Map getPartitionValues() { + return partitionValues; + } + + public Map> getStatsValues() { + return statsValues; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/EvalHelper.java b/server/core/src/main/java/io/whitefox/core/types/predicates/EvalHelper.java new file mode 100644 index 000000000..f015b957a --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/EvalHelper.java @@ -0,0 +1,119 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.ColumnRange; +import io.whitefox.core.types.*; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Objects; +import org.apache.commons.lang3.tuple.Pair; + +// Only for partition values +public class EvalHelper { + + private static LeafEvaluationResult validateAndGetRange( + ColumnOp columnChild, LiteralOp literalChild, EvalContext ctx) throws PredicateException { + var columnRange = columnChild.evalExpectColumnRange(ctx); + var rightVal = literalChild.evalExpectValueAndType(ctx).getSingleValue(); + + return LeafEvaluationResult.createFromRange(new RangeEvaluationResult(columnRange, rightVal)); + } + + private static LeafEvaluationResult validateAndGetTypeAndValue( + ColumnOp columnOp, LiteralOp literalOp, EvalContext ctx) throws PredicateException { + var columnType = columnOp.evalExpectValueAndType(ctx).getValueType(); + var columnValue = columnOp.evalExpectValueAndType(ctx).getSingleValue(); + + var literalType = literalOp.evalExpectValueAndType(ctx).getValueType(); + var literalValue = literalOp.evalExpectValueAndType(ctx).getSingleValue(); + // If the types don't match, it implies a malformed predicate tree. + // We simply throw an exception, which will cause filtering to be skipped. + if (!Objects.equals(columnType, literalType)) { + throw new TypeMismatchException(columnType, literalType); + } + + if (columnValue == null) { + return validateAndGetRange(columnOp, literalOp, ctx); + } + + // We throw an exception for nulls, which will skip filtering. + if (literalValue == null) { + throw new NullTypeException(columnOp, literalOp); + } + + return LeafEvaluationResult.createFromPartitionColumn( + new PartitionEvaluationResult(new ColumnRange(columnValue, columnType), literalValue)); + } + + private static Pair arrangeChildren(List children) { + if (children.get(0) instanceof ColumnOp) + return Pair.of((ColumnOp) children.get(0), (LiteralOp) children.get(1)); + else return Pair.of((ColumnOp) children.get(1), (LiteralOp) children.get(0)); + } + + // allows throwing an exception from a function passed as an argument + @FunctionalInterface + interface BiFunctionWithException { + R apply(T t, U u) throws E; + } + + static Boolean evaluate( + List children, + EvalContext ctx, + BiFunctionWithException condition) + throws PredicateException { + var columnOp = arrangeChildren(children).getLeft(); + var literalOp = arrangeChildren(children).getRight(); + + var leafEvaluationResult = validateAndGetTypeAndValue(columnOp, literalOp, ctx); + + if (leafEvaluationResult.rangeEvaluationResult.isPresent()) { + var evaluationResult = leafEvaluationResult.rangeEvaluationResult.get(); + var columnRange = evaluationResult.getColumnRange(); + var value = evaluationResult.getValue(); + return condition.apply(columnRange, value); + } else if (leafEvaluationResult.partitionEvaluationResult.isPresent()) { + var evaluationResult = leafEvaluationResult.partitionEvaluationResult.get(); + var literalValue = evaluationResult.getLiteralValue(); + + return condition.apply(evaluationResult.getPartitionValue(), literalValue); + } else throw new PredicateColumnEvaluationException(ctx); + } + + static Boolean equal(List children, EvalContext ctx) throws PredicateException { + return evaluate(children, ctx, ColumnRange::contains); + } + + static Boolean lessThan(List children, EvalContext ctx) throws PredicateException { + return evaluate(children, ctx, ColumnRange::canBeLess); + } + + // Validates that the specified value is in the correct format. + // Throws an exception otherwise. + public static void validateValue(String value, DataType valueType) + throws TypeValidationException { + try { + if (BooleanType.BOOLEAN.equals(valueType)) { + Boolean.parseBoolean(value); + } else if (IntegerType.INTEGER.equals(valueType)) { + Integer.parseInt(value); + } else if (LongType.LONG.equals(valueType)) { + Long.parseLong(value); + } else if (DateType.DATE.equals(valueType)) { + Date.valueOf(value); + } else if (FloatType.FLOAT.equals(valueType)) { + Float.parseFloat(value); + } else if (DoubleType.DOUBLE.equals(valueType)) { + Double.parseDouble(value); + } else if (TimestampType.TIMESTAMP.equals(valueType)) { + Timestamp.valueOf(value); + } else if (StringType.STRING.equals(valueType)) { + return; + } else { + throw new TypeNotSupportedException(valueType); + } + } catch (Exception e) { + throw new TypeValidationException(value, valueType); + } + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/EvaluatorVersion.java b/server/core/src/main/java/io/whitefox/core/types/predicates/EvaluatorVersion.java new file mode 100644 index 000000000..14828873f --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/EvaluatorVersion.java @@ -0,0 +1,12 @@ +package io.whitefox.core.types.predicates; + +public enum EvaluatorVersion { + V1("v1"), + V2("v2"); + + public final String value; + + EvaluatorVersion(String value) { + this.value = value; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/ExpressionNotSupportedException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/ExpressionNotSupportedException.java new file mode 100644 index 000000000..c25526922 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/ExpressionNotSupportedException.java @@ -0,0 +1,14 @@ +package io.whitefox.core.types.predicates; + +public class ExpressionNotSupportedException extends PredicateException { + private final String expression; + + public ExpressionNotSupportedException(String expression) { + this.expression = expression; + } + + @Override + public String getMessage() { + return "Unsupported expression: " + expression.toString(); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/LeafEvaluationResult.java b/server/core/src/main/java/io/whitefox/core/types/predicates/LeafEvaluationResult.java new file mode 100644 index 000000000..3e3ca790c --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/LeafEvaluationResult.java @@ -0,0 +1,25 @@ +package io.whitefox.core.types.predicates; + +import java.util.Optional; + +public class LeafEvaluationResult { + + Optional rangeEvaluationResult; + Optional partitionEvaluationResult; + + public LeafEvaluationResult( + Optional rangeEvaluationResult, + Optional partitionEvaluationResult) { + this.rangeEvaluationResult = rangeEvaluationResult; + this.partitionEvaluationResult = partitionEvaluationResult; + } + + public static LeafEvaluationResult createFromRange(RangeEvaluationResult rangeEvaluationResult) { + return new LeafEvaluationResult(Optional.of(rangeEvaluationResult), Optional.empty()); + } + + public static LeafEvaluationResult createFromPartitionColumn( + PartitionEvaluationResult partitionEvaluationResult) { + return new LeafEvaluationResult(Optional.empty(), Optional.of(partitionEvaluationResult)); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/LeafOp.java b/server/core/src/main/java/io/whitefox/core/types/predicates/LeafOp.java new file mode 100644 index 000000000..b7dc7e586 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/LeafOp.java @@ -0,0 +1,37 @@ +package io.whitefox.core.types.predicates; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.whitefox.core.ColumnRange; +import io.whitefox.core.types.DataType; +import java.util.List; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "op") +@JsonSubTypes({ + @JsonSubTypes.Type(value = ColumnOp.class, name = "column"), + @JsonSubTypes.Type(value = LiteralOp.class, name = "literal") +}) +public abstract class LeafOp implements BaseOp { + + abstract Boolean isNull(EvalContext ctx); + + @JsonProperty("valueType") + DataType valueType; + + ColumnRange evalExpectValueAndType(EvalContext ctx) throws PredicateException { + var res = eval(ctx); + if (res instanceof ColumnRange) { + return (ColumnRange) res; + } else { + throw new WrongExpectedTypeException(res, ColumnRange.class); + } + } + + @Override + public List getAllChildren() { + return List.of(); + } + + abstract DataType getOpValueType(); +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/LiteralOp.java b/server/core/src/main/java/io/whitefox/core/types/predicates/LiteralOp.java new file mode 100644 index 000000000..0fb87fd47 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/LiteralOp.java @@ -0,0 +1,49 @@ +package io.whitefox.core.types.predicates; + +import static io.whitefox.core.types.predicates.EvaluatorVersion.V1; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.whitefox.core.ColumnRange; +import io.whitefox.core.types.DataType; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "literal") +public class LiteralOp extends LeafOp { + @JsonProperty("value") + String value; + + @Override + public void validate() throws PredicateException { + if (value == null) { + throw new IllegalArgumentException("Value must be specified: " + this); + } + if (!isSupportedType(valueType, V1)) { + throw new IllegalArgumentException("Unsupported type: " + valueType); + } + EvalHelper.validateValue(value, valueType); + } + + @Override + public Object eval(EvalContext ctx) { + return new ColumnRange(value, valueType); + } + + public LiteralOp() { + super(); + } + + public LiteralOp(String value, DataType valueType) { + this.value = value; + this.valueType = valueType; + } + + @Override + public Boolean isNull(EvalContext ctx) { + return false; + } + + @Override + public DataType getOpValueType() { + return valueType; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/NonExistingColumnException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/NonExistingColumnException.java new file mode 100644 index 000000000..725a53e92 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/NonExistingColumnException.java @@ -0,0 +1,14 @@ +package io.whitefox.core.types.predicates; + +public class NonExistingColumnException extends PredicateException { + private final String name; + + public NonExistingColumnException(String name) { + this.name = name; + } + + @Override + public String getMessage() { + return "Column " + name + " does not exist in the file statistics"; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/NonLeafOp.java b/server/core/src/main/java/io/whitefox/core/types/predicates/NonLeafOp.java new file mode 100644 index 000000000..69b20957f --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/NonLeafOp.java @@ -0,0 +1,352 @@ +package io.whitefox.core.types.predicates; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.util.List; +import java.util.stream.Collectors; + +// Represents a non-leaf operation. +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "op") +@JsonSubTypes({ + @JsonSubTypes.Type(value = EqualOp.class, name = "equal"), + @JsonSubTypes.Type(value = NotOp.class, name = "not"), + @JsonSubTypes.Type(value = OrOp.class, name = "or"), + @JsonSubTypes.Type(value = IsNullOp.class, name = "null"), + @JsonSubTypes.Type(value = AndOp.class, name = "and"), + @JsonSubTypes.Type(value = LessThanOp.class, name = "lessThan"), + @JsonSubTypes.Type(value = LessThanOrEqualOp.class, name = "lessThanOrEqual"), + @JsonSubTypes.Type(value = GreaterThanOp.class, name = "greaterThan"), + @JsonSubTypes.Type(value = GreaterThanOrEqualOp.class, name = "greaterThanOrEqual"), + @JsonSubTypes.Type(value = DifferentThanOp.class, name = "differentThan") +}) +public abstract class NonLeafOp implements BaseOp { + + @JsonProperty("children") + List children; + + public static NonLeafOp createPartitionFilter(List children, String operator) + throws PredicateException { + switch (operator) { + case "=": + return new EqualOp(children); + case "<": + return new LessThanOp(children); + case "<=": + return new LessThanOrEqualOp(children); + case ">": + return new GreaterThanOp(children); + case ">=": + return new GreaterThanOrEqualOp(children); + case "<>": + return new DifferentThanOp(children); + case "isnull": + return new IsNullOp(children); + default: + // TODO: add not supported sql exception + throw new ExpressionNotSupportedException(operator); + } + } + + public List getAllChildren() { + // TODO flat map every child + return List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList()); + } +} + +class IsNullOp extends NonLeafOp implements UnaryOp { + + @JsonProperty("children") + List children; + + public IsNullOp(List children) { + this.children = children; + } + + public IsNullOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) { + return ((LeafOp) children.get(0)).isNull(ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "equal") +class EqualOp extends NonLeafOp implements BinaryOp { + @JsonProperty("children") + List children; + + public EqualOp() { + super(); + } + + public EqualOp(List children) { + this.children = children; + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + this.validate(); + return EvalHelper.equal(children, ctx); + } +} + +// not used in JsonPredicates, only for SQL +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "differentThan") +class DifferentThanOp extends NonLeafOp implements BinaryOp { + @JsonProperty("children") + List children; + + public DifferentThanOp() { + super(); + } + + public DifferentThanOp(List children) { + this.children = children; + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + this.validate(); + return !EvalHelper.equal(children, ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "lessThan") +class LessThanOp extends NonLeafOp implements BinaryOp { + @JsonProperty("children") + List children; + + public LessThanOp(List children) { + this.children = children; + } + + public LessThanOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + return EvalHelper.lessThan(children, ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "lessThanOrEqual") +class LessThanOrEqualOp extends NonLeafOp implements BinaryOp { + + @JsonProperty("children") + List children; + + public LessThanOrEqualOp(List children) { + this.children = children; + } + + public LessThanOrEqualOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + return EvalHelper.lessThan(children, ctx) || EvalHelper.equal(children, ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "greaterThan") +class GreaterThanOp extends NonLeafOp implements BinaryOp { + + @JsonProperty("children") + List children; + + public GreaterThanOp() { + super(); + } + + public GreaterThanOp(List children) { + this.children = children; + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + return !EvalHelper.lessThan(children, ctx) && !EvalHelper.equal(children, ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "greaterThanOrEqual") +class GreaterThanOrEqualOp extends NonLeafOp implements BinaryOp { + + @JsonProperty("children") + List children; + + public GreaterThanOrEqualOp() { + super(); + } + + public GreaterThanOrEqualOp(List children) { + this.children = children; + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + return !EvalHelper.lessThan(children, ctx); + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "and") +class AndOp extends NonLeafOp implements BinaryOp { + + public AndOp(List children) { + this.children = children; + } + + public AndOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren(children); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + // short-circuits, so not all exceptions will be thrown + for (BaseOp c : children) { + if (!c.evalExpectBoolean(ctx)) { + return false; + } + } + return true; + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "or") +class OrOp extends NonLeafOp implements BinaryOp { + + public OrOp(List children) { + this.children = children; + } + + public OrOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren(children); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + for (BaseOp c : children) { + if (c.evalExpectBoolean(ctx)) { + return true; + } + } + return false; + } +} + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "not") +class NotOp extends NonLeafOp implements UnaryOp { + + @JsonProperty("children") + List children; + + public NotOp(List children) { + this.children = children; + } + + public NotOp() { + super(); + } + + @Override + public void validate() throws PredicateException { + validateChildren( + List.copyOf(children).stream().map(c -> (BaseOp) c).collect(Collectors.toList())); + } + + @Override + public Object eval(EvalContext ctx) throws PredicateException { + try { + this.validate(); + } catch (PredicateException e) { + throw new RuntimeException(e); + } + return !children.get(0).evalExpectBoolean(ctx); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/NullTypeException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/NullTypeException.java new file mode 100644 index 000000000..5f53116bf --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/NullTypeException.java @@ -0,0 +1,20 @@ +package io.whitefox.core.types.predicates; + +public class NullTypeException extends PredicateException { + + private final BaseOp leftChild; + private final BaseOp rightChild; + + public NullTypeException(BaseOp leftChild, BaseOp rightChild) { + this.leftChild = leftChild; + this.rightChild = rightChild; + } + + @Override + public String getMessage() { + // TODO: Currently means that the column is not in the partition columns + // Expected is to mean that it is not present at all(use file stats) + return "Comparison with a null value is not supported: " + leftChild.getClass() + " and " + + rightChild.getClass(); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/PartitionEvaluationResult.java b/server/core/src/main/java/io/whitefox/core/types/predicates/PartitionEvaluationResult.java new file mode 100644 index 000000000..71031e935 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/PartitionEvaluationResult.java @@ -0,0 +1,22 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.ColumnRange; + +public class PartitionEvaluationResult { + + ColumnRange partitionValue; + String literalValue; + + public PartitionEvaluationResult(ColumnRange partitionValue, String literalValue) { + this.partitionValue = partitionValue; + this.literalValue = literalValue; + } + + public ColumnRange getPartitionValue() { + return partitionValue; + } + + public String getLiteralValue() { + return literalValue; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateColumnEvaluationException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateColumnEvaluationException.java new file mode 100644 index 000000000..ceaedd1f0 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateColumnEvaluationException.java @@ -0,0 +1,19 @@ +package io.whitefox.core.types.predicates; + +public class PredicateColumnEvaluationException extends PredicateException { + + private final EvalContext ctx; + + public PredicateColumnEvaluationException(EvalContext ctx) { + this.ctx = ctx; + } + + @Override + public String getMessage() { + return "Column from your query does not exist in either partition columns or regular columns of the table." + + "Partition columns: " + + ctx.getPartitionValues().keySet().stream().reduce((s1, s2) -> s1 + "|" + s2) + "\n" + + "Regular columns: " + + ctx.getStatsValues().keySet().stream().reduce((s1, s2) -> s1 + "|" + s2) + "\n"; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateException.java new file mode 100644 index 000000000..701105531 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateException.java @@ -0,0 +1,3 @@ +package io.whitefox.core.types.predicates; + +public class PredicateException extends Exception {} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateParsingException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateParsingException.java new file mode 100644 index 000000000..9fb890005 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateParsingException.java @@ -0,0 +1,18 @@ +package io.whitefox.core.types.predicates; + +import java.util.Arrays; + +public class PredicateParsingException extends PredicateException { + + private final Exception cause; + + public PredicateParsingException(Exception cause) { + this.cause = cause; + } + + @Override + public String getMessage() { + return "Parsing of predicate failed due to: " + cause.getMessage() + "\n Stack trace: " + + Arrays.toString(cause.getStackTrace()); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateValidationException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateValidationException.java new file mode 100644 index 000000000..b77ac6153 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/PredicateValidationException.java @@ -0,0 +1,25 @@ +package io.whitefox.core.types.predicates; + +public class PredicateValidationException extends PredicateException { + + private final int actualNumOfChildren; + private final Arity op; + private final int expectedNumOfChildren; + + public PredicateValidationException( + int actualNumOfChildren, Arity op, int expectedNumOfChildren) { + this.actualNumOfChildren = actualNumOfChildren; + this.op = op; + this.expectedNumOfChildren = expectedNumOfChildren; + } + + @Override + public String getMessage() { + if (op instanceof NaryOp) + return op + " : expected at least " + expectedNumOfChildren + " children, but found " + + actualNumOfChildren + " children"; + else + return op + " : expected " + expectedNumOfChildren + " children, but found " + + actualNumOfChildren + " children"; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/RangeEvaluationResult.java b/server/core/src/main/java/io/whitefox/core/types/predicates/RangeEvaluationResult.java new file mode 100644 index 000000000..3ae18b1e5 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/RangeEvaluationResult.java @@ -0,0 +1,22 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.ColumnRange; + +public class RangeEvaluationResult { + + ColumnRange columnRange; + String value; + + public RangeEvaluationResult(ColumnRange columnRange, String value) { + this.columnRange = columnRange; + this.value = value; + } + + public ColumnRange getColumnRange() { + return columnRange; + } + + public String getValue() { + return value; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/TypeMismatchException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeMismatchException.java new file mode 100644 index 000000000..45f585bd5 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeMismatchException.java @@ -0,0 +1,19 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.types.DataType; + +public class TypeMismatchException extends PredicateException { + + private final DataType lType; + private final DataType rType; + + public TypeMismatchException(DataType lType, DataType rType) { + this.lType = lType; + this.rType = rType; + } + + @Override + public String getMessage() { + return "Type are not matching between: " + lType.toString() + " and " + rType; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/TypeNotSupportedException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeNotSupportedException.java new file mode 100644 index 000000000..6415e93dd --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeNotSupportedException.java @@ -0,0 +1,16 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.types.DataType; + +public class TypeNotSupportedException extends PredicateException { + private final DataType type; + + public TypeNotSupportedException(DataType type) { + this.type = type; + } + + @Override + public String getMessage() { + return "Unsupported type: " + type.toString(); + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/TypeValidationException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeValidationException.java new file mode 100644 index 000000000..4c96a4e46 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/TypeValidationException.java @@ -0,0 +1,19 @@ +package io.whitefox.core.types.predicates; + +import io.whitefox.core.types.DataType; + +public class TypeValidationException extends PredicateException { + + private final String value; + private final DataType valueType; + + public TypeValidationException(String value, DataType valueType) { + this.value = value; + this.valueType = valueType; + } + + @Override + public String getMessage() { + return "Error validating value: " + value + " for type " + valueType; + } +} diff --git a/server/core/src/main/java/io/whitefox/core/types/predicates/WrongExpectedTypeException.java b/server/core/src/main/java/io/whitefox/core/types/predicates/WrongExpectedTypeException.java new file mode 100644 index 000000000..391f655d4 --- /dev/null +++ b/server/core/src/main/java/io/whitefox/core/types/predicates/WrongExpectedTypeException.java @@ -0,0 +1,18 @@ +package io.whitefox.core.types.predicates; + +public class WrongExpectedTypeException extends PredicateException { + + private final Object evaluationResult; + private final Class expectedType; + + public WrongExpectedTypeException(Object evaluationResult, Class expectedType) { + this.evaluationResult = evaluationResult; + this.expectedType = expectedType; + } + + @Override + public String getMessage() { + return "Evaluation of a Root or Non-Leaf predicate is expected to be of " + expectedType + + " type, instead got: " + evaluationResult.getClass(); + } +} diff --git a/server/core/src/test/java/io/whitefox/core/services/DeltaShareServiceTest.java b/server/core/src/test/java/io/whitefox/core/services/DeltaShareServiceTest.java index 676d45a9b..90e2f8f17 100644 --- a/server/core/src/test/java/io/whitefox/core/services/DeltaShareServiceTest.java +++ b/server/core/src/test/java/io/whitefox/core/services/DeltaShareServiceTest.java @@ -1,10 +1,12 @@ package io.whitefox.core.services; import io.whitefox.DeltaTestUtils; +import io.whitefox.core.*; import io.whitefox.core.Principal; import io.whitefox.core.Schema; import io.whitefox.core.Share; import io.whitefox.core.SharedTable; +import io.whitefox.core.services.exceptions.TableNotFound; import io.whitefox.persistence.StorageManager; import io.whitefox.persistence.memory.InMemoryStorageManager; import java.util.Collections; @@ -228,4 +230,52 @@ public void tableMetadataNotFound() { deltaSharesService.getTableMetadata("name", "default", "tableNotFound", Optional.empty()); Assertions.assertTrue(resultTable.isEmpty()); } + + @Test + @DisabledOnOs(OS.WINDOWS) + public void queryExistingTable() { + var shares = List.of(createShare( + "name", + "key", + Map.of( + "default", + new Schema( + "default", + List.of(new SharedTable( + "partitioned-delta-table", + "default", + "name", + DeltaTestUtils.deltaTable("partitioned-delta-table"))), + "name")))); + StorageManager storageManager = new InMemoryStorageManager(shares); + DeltaSharesService deltaSharesService = + new DeltaSharesServiceImpl(storageManager, 100, tableLoaderFactory, fileSignerFactory); + var resultTable = deltaSharesService.queryTable( + "name", + "default", + "partitioned-delta-table", + new ReadTableRequest.ReadTableCurrentVersion( + Optional.empty(), Optional.empty(), Optional.empty())); + Assertions.assertEquals(9, resultTable.files().size()); + } + + @Test + public void queryNonExistingTable() { + var shares = List.of(createShare( + "name", + "key", + Map.of( + "default", + new Schema( + "default", + List.of(new SharedTable( + "table1", "default", "name", DeltaTestUtils.deltaTable("location1"))), + "name")))); + StorageManager storageManager = new InMemoryStorageManager(shares); + DeltaSharesService deltaSharesService = + new DeltaSharesServiceImpl(storageManager, 100, tableLoaderFactory, fileSignerFactory); + Assertions.assertThrows( + TableNotFound.class, + () -> deltaSharesService.queryTable("name", "default", "tableNotFound", null)); + } } diff --git a/server/core/src/test/java/io/whitefox/core/services/DeltaSharedTableTest.java b/server/core/src/test/java/io/whitefox/core/services/DeltaSharedTableTest.java index 322b4f80c..ae9bf1024 100644 --- a/server/core/src/test/java/io/whitefox/core/services/DeltaSharedTableTest.java +++ b/server/core/src/test/java/io/whitefox/core/services/DeltaSharedTableTest.java @@ -5,9 +5,13 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.wildfly.common.Assert.assertTrue; +import io.whitefox.core.Protocol; +import io.whitefox.core.ReadTableRequest; import io.whitefox.core.SharedTable; +import java.time.format.DateTimeParseException; +import java.util.List; import java.util.Optional; -import java.util.concurrent.ExecutionException; +import org.apache.commons.lang3.tuple.Pair; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.condition.DisabledOnOs; import org.junit.jupiter.api.condition.OS; @@ -16,7 +20,7 @@ public class DeltaSharedTableTest { @Test - void getTableVersion() throws ExecutionException, InterruptedException { + void getTableVersion() { var PTable = new SharedTable("delta-table", "default", "share1", deltaTable("delta-table")); var DTable = DeltaSharedTable.of(PTable); var version = DTable.getTableVersion(Optional.empty()); @@ -39,7 +43,7 @@ void getUnknownTableMetadata() { } @Test - void getTableVersionNonExistingTable() throws ExecutionException, InterruptedException { + void getTableVersionNonExistingTable() { var PTable = new SharedTable("delta-table", "default", "share1", deltaTable("delta-table-not-exists")); var exception = assertThrows(IllegalArgumentException.class, () -> DeltaSharedTable.of(PTable)); @@ -47,7 +51,7 @@ void getTableVersionNonExistingTable() throws ExecutionException, InterruptedExc } @Test - void getTableVersionWithTimestamp() throws ExecutionException, InterruptedException { + void getTableVersionWithTimestamp() { var PTable = new SharedTable("delta-table", "default", "share1", deltaTable("delta-table")); var DTable = DeltaSharedTable.of(PTable); var version = DTable.getTableVersion(TestDateUtils.parseTimestamp("2023-09-30T10:15:30+01:00")); @@ -55,10 +59,160 @@ void getTableVersionWithTimestamp() throws ExecutionException, InterruptedExcept } @Test - void getTableVersionWithFutureTimestamp() throws ExecutionException, InterruptedException { + void getTableVersionWithFutureTimestamp() { var PTable = new SharedTable("delta-table", "default", "share1", deltaTable("delta-table")); var DTable = DeltaSharedTable.of(PTable); var version = DTable.getTableVersion(TestDateUtils.parseTimestamp("2024-10-20T10:15:30+01:00")); assertEquals(Optional.empty(), version); } + + @Test + void getTableVersionWithMalformedTimestamp() { + var PTable = new SharedTable("delta-table", "default", "share1", deltaTable("delta-table")); + var DTable = DeltaSharedTable.of(PTable); + assertThrows( + DateTimeParseException.class, + () -> DTable.getTableVersion(TestDateUtils.parseTimestamp("221rfewdsad10:15:30+01:00"))); + } + + @Test + void queryTableWithoutPredicate() { + var PTable = new SharedTable( + "partitioned-delta-table", "default", "share1", deltaTable("partitioned-delta-table")); + var DTable = DeltaSharedTable.of(PTable); + var request = new ReadTableRequest.ReadTableCurrentVersion( + Optional.empty(), Optional.empty(), Optional.empty()); + var response = DTable.queryTable(request); + assertEquals(response.protocol(), new Protocol(Optional.of(1))); + assertEquals(response.other().size(), 9); + } + + @Test + void queryTableWithSqlPredicates() { + + var predicatesAndExpectedResult = List.of( + Pair.of(List.of("date = '2021-08-15'"), 4), + Pair.of(List.of("date < '2021-08-14'"), 5), + Pair.of(List.of("date > '2021-08-04'"), 9), + Pair.of(List.of("date is NULL"), 0), + Pair.of(List.of("date >= '2021-08-15'"), 4), + Pair.of(List.of("date <> '2021-08-15'"), 5), + Pair.of(List.of("date <= '2021-08-15'"), 9)); + + var PTable = new SharedTable( + "partitioned-delta-table", "default", "share1", deltaTable("partitioned-delta-table")); + var DTable = DeltaSharedTable.of(PTable); + + predicatesAndExpectedResult.forEach(p -> { + var request = new ReadTableRequest.ReadTableCurrentVersion( + Optional.of(p.getLeft()), Optional.empty(), Optional.empty()); + var response = DTable.queryTable(request); + assertEquals(p.getRight(), response.other().size()); + }); + } + + @Test + void queryTableWithNonPartitionSqlPredicate() { + var predicates = List.of("id < 30"); + var tableName = "partitioned-delta-table-with-multiple-columns"; + + var PTable = new SharedTable(tableName, "default", "share1", deltaTable(tableName)); + var DTable = DeltaSharedTable.of(PTable); + var request = new ReadTableRequest.ReadTableCurrentVersion( + Optional.of(predicates), Optional.empty(), Optional.empty()); + var response = DTable.queryTable(request); + assertEquals(1, response.other().size()); + } + + @Test + void queryTableWithJsonPredicates() { + + var predicatesAndExpectedResult = List.of( + Pair.of( + "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"dating\",\"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-09-12\",\"valueType\":\"date\"}\n" + + " ]\n" + + "}", + 2), + Pair.of( + "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"date\",\"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-09-12\",\"valueType\":\"date\"}\n" + + " ]\n" + + "}", + 1), + Pair.of( + "{" + + " \"op\":\"lessThan\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"31\",\"valueType\":\"int\"}\n" + + " ]\n" + + "}", + 1), + Pair.of( + "{" + + " \"op\":\"lessThanOrEqual\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"31\",\"valueType\":\"int\"}\n" + + " ]\n" + + "}", + 2), + Pair.of( + "{" + + " \"op\":\"lessThan\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"45\",\"valueType\":\"int\"}\n" + + " ]\n" + + "}", + 2), + Pair.of( + "{" + + " \"op\":\"isNull\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"}" + + " ]\n" + + "}", + 2), + Pair.of( + "{\n" + " \"op\":\"and\",\n" + + " \"children\":[\n" + + " {\n" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"date\",\"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2022-02-06\",\"valueType\":\"date\"}\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"op\":\"greaterThanOrEqual\",\"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"31\",\"valueType\":\"int\"}\n" + + " ]\n" + + " }\n" + + " ]\n" + + "}", + 0)); + + var PTable = new SharedTable( + "partitioned-delta-table-with-multiple-columns", + "default", + "share1", + deltaTable("partitioned-delta-table-with-multiple-columns")); + var DTable = DeltaSharedTable.of(PTable); + + predicatesAndExpectedResult.forEach(p -> { + var request = new ReadTableRequest.ReadTableCurrentVersion( + Optional.empty(), Optional.of(p.getLeft()), Optional.empty()); + var response = DTable.queryTable(request); + assertEquals(p.getRight(), response.other().size()); + }); + } } diff --git a/server/core/src/test/java/io/whitefox/core/types/MapTypeTest.java b/server/core/src/test/java/io/whitefox/core/types/MapTypeTest.java index 7863da510..223273c81 100644 --- a/server/core/src/test/java/io/whitefox/core/types/MapTypeTest.java +++ b/server/core/src/test/java/io/whitefox/core/types/MapTypeTest.java @@ -83,7 +83,7 @@ public void testToJson() { MapType mapType = new MapType(keyType, valueType, valueContainsNull); assertEquals( - "{\"type\": \"map\",\"keyType\": \"string\",\"valueType\": \"integer\",\"valueContainsNull\": true}", + "{\"type\": \"map\",\"keyType\": \"string\",\"valueType\": \"int\",\"valueContainsNull\": true}", mapType.toJson()); } } diff --git a/server/core/src/test/java/io/whitefox/core/types/PredicateUtilsTest.java b/server/core/src/test/java/io/whitefox/core/types/PredicateUtilsTest.java new file mode 100644 index 000000000..9e4b63c70 --- /dev/null +++ b/server/core/src/test/java/io/whitefox/core/types/PredicateUtilsTest.java @@ -0,0 +1,41 @@ +package io.whitefox.core.types; + +import static io.whitefox.DeltaTestUtils.deltaTable; +import static io.whitefox.DeltaTestUtils.deltaTableUri; + +import io.delta.standalone.DeltaLog; +import io.delta.standalone.actions.AddFile; +import io.whitefox.core.PredicateUtils; +import io.whitefox.core.SharedTable; +import io.whitefox.core.types.predicates.*; +import java.util.ArrayList; +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; + +public class PredicateUtilsTest { + + @Test + @DisabledOnOs(OS.WINDOWS) + void testCreateEvalContext() throws PredicateParsingException { + var PTable = new SharedTable( + "partitioned-delta-table-with-multiple-columns", + "default", + "share1", + deltaTable("partitioned-delta-table-with-multiple-columns")); + + var log = DeltaLog.forTable( + new Configuration(), deltaTableUri("partitioned-delta-table-with-multiple-columns")); + var contexts = new ArrayList(); + for (AddFile file : log.snapshot().getAllFiles()) { + EvalContext evalContext = PredicateUtils.createEvalContext(file); + contexts.add(evalContext); + } + assert (contexts.size() == 2); + var c1 = contexts.get(0); + var c2 = contexts.get(1); + assert (c1.getPartitionValues().get("date").equals("2022-02-06")); + assert (c2.getPartitionValues().get("date").equals("2021-09-12")); + } +} diff --git a/server/core/src/test/java/io/whitefox/core/types/predicates/EvalHelperTest.java b/server/core/src/test/java/io/whitefox/core/types/predicates/EvalHelperTest.java new file mode 100644 index 000000000..1299305ac --- /dev/null +++ b/server/core/src/test/java/io/whitefox/core/types/predicates/EvalHelperTest.java @@ -0,0 +1,164 @@ +package io.whitefox.core.types.predicates; + +import static org.junit.jupiter.api.Assertions.*; + +import io.whitefox.core.types.*; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.Test; + +public class EvalHelperTest { + + @Test + void testLessThanOnPartitionColumns() throws PredicateException { + var evalContext1 = new EvalContext(Map.of("date", "2020-10-10"), Map.of()); + var children1 = + List.of(new ColumnOp("date", DateType.DATE), new LiteralOp("2020-10-11", DateType.DATE)); + assertTrue(EvalHelper.lessThan(children1, evalContext1)); + var evalContext2 = new EvalContext(Map.of("integerCol", "19"), Map.of()); + var children2 = List.of( + new ColumnOp("integerCol", IntegerType.INTEGER), new LiteralOp("20", IntegerType.INTEGER)); + assertTrue(EvalHelper.lessThan(children2, evalContext2)); + var evalContext3 = new EvalContext(Map.of("long", "20"), Map.of()); + var children3 = + List.of(new ColumnOp("long", LongType.LONG), new LiteralOp("21", LongType.LONG)); + assertTrue(EvalHelper.lessThan(children3, evalContext3)); + var evalContext4 = new EvalContext(Map.of("float", "2.97"), Map.of()); + var children4 = + List.of(new ColumnOp("float", FloatType.FLOAT), new LiteralOp("2.98", FloatType.FLOAT)); + assertTrue(EvalHelper.lessThan(children4, evalContext4)); + var children5 = + List.of(new LiteralOp("21", LongType.LONG), new ColumnOp("long", LongType.LONG)); + assertTrue(EvalHelper.lessThan(children5, evalContext3)); + var evalContext6 = new EvalContext(Map.of("float", "2.99"), Map.of()); + var children6 = + List.of(new ColumnOp("float", FloatType.FLOAT), new LiteralOp("2.98", FloatType.FLOAT)); + assertFalse(EvalHelper.lessThan(children6, evalContext6)); + } + + @Test + void testEqualOnPartitionColumns() throws PredicateException { + var evalContext1 = new EvalContext(Map.of("date", "2020-10-11"), Map.of()); + var children1 = + List.of(new ColumnOp("date", DateType.DATE), new LiteralOp("2020-10-11", DateType.DATE)); + assertTrue(EvalHelper.equal(children1, evalContext1)); + var evalContext2 = new EvalContext(Map.of("integerCol", "20"), Map.of()); + var children2 = List.of( + new ColumnOp("integerCol", IntegerType.INTEGER), new LiteralOp("20", IntegerType.INTEGER)); + assertTrue(EvalHelper.equal(children2, evalContext2)); + var evalContext3 = new EvalContext(Map.of("long", "20"), Map.of()); + var children3 = + List.of(new ColumnOp("long", LongType.LONG), new LiteralOp("20", LongType.LONG)); + assertTrue(EvalHelper.equal(children3, evalContext3)); + var evalContext4 = new EvalContext(Map.of("boolean", "true"), Map.of()); + var children4 = List.of( + new ColumnOp("boolean", BooleanType.BOOLEAN), new LiteralOp("true", BooleanType.BOOLEAN)); + assertTrue(EvalHelper.equal(children4, evalContext4)); + var evalContext5 = new EvalContext(Map.of("float", "2.99"), Map.of()); + var children5 = + List.of(new ColumnOp("float", FloatType.FLOAT), new LiteralOp("2.99", FloatType.FLOAT)); + assertTrue(EvalHelper.equal(children5, evalContext5)); + } + + @Test + void testEqualOnRegularColumns() throws PredicateException { + var evalContext1 = new EvalContext( + Map.of("date", "2020-10-11"), Map.of("id", Pair.of("2020-11-10", "2020-11-12"))); + var children1 = + List.of(new ColumnOp("id", DateType.DATE), new LiteralOp("2020-11-11", DateType.DATE)); + assertTrue(EvalHelper.equal(children1, evalContext1)); + var evalContext2 = new EvalContext(Map.of("integer", "20"), Map.of("id", Pair.of("20", "29"))); + var children2 = + List.of(new ColumnOp("id", IntegerType.INTEGER), new LiteralOp("20", IntegerType.INTEGER)); + assertTrue(EvalHelper.equal(children2, evalContext2)); + var evalContext3 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("20", "29"))); + var children3 = List.of(new ColumnOp("id", LongType.LONG), new LiteralOp("21", LongType.LONG)); + assertTrue(EvalHelper.equal(children3, evalContext3)); + var evalContext4 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("2.99", "3.01"))); + var children4 = + List.of(new ColumnOp("id", FloatType.FLOAT), new LiteralOp("3.0", FloatType.FLOAT)); + assertTrue(EvalHelper.equal(children4, evalContext4)); + var evalContext5 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("aaaa", "cccc"))); + var children5 = + List.of(new ColumnOp("id", StringType.STRING), new LiteralOp("aabb", StringType.STRING)); + assertTrue(EvalHelper.equal(children5, evalContext5)); + var evalContext6 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("2.99", "3.01"))); + var children6 = + List.of(new ColumnOp("id", DoubleType.DOUBLE), new LiteralOp("3.0", DoubleType.DOUBLE)); + assertTrue(EvalHelper.equal(children6, evalContext6)); + var evalContext7 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("true", "true"))); + var children7 = List.of( + new ColumnOp("id", BooleanType.BOOLEAN), new LiteralOp("true", BooleanType.BOOLEAN)); + assertTrue(EvalHelper.equal(children7, evalContext7)); + var evalContext8 = new EvalContext( + Map.of("long", "20"), + Map.of("id", Pair.of("2022-08-10 06:02:03.000000", "2022-12-10 06:02:03.000000"))); + var children8 = List.of( + new ColumnOp("id", TimestampType.TIMESTAMP), + new LiteralOp("2022-10-10 06:02:03.000000", TimestampType.TIMESTAMP)); + assertTrue(EvalHelper.equal(children8, evalContext8)); + } + + @Test + void testLessThanOnRegularColumns() throws PredicateException { + var evalContext1 = new EvalContext( + Map.of("date", "2020-10-11"), Map.of("id", Pair.of("2020-11-10", "2020-11-12"))); + var children1 = + List.of(new ColumnOp("id", DateType.DATE), new LiteralOp("2020-12-12", DateType.DATE)); + assertTrue(EvalHelper.lessThan(children1, evalContext1)); + var evalContext2 = new EvalContext(Map.of("integer", "20"), Map.of("id", Pair.of("20", "29"))); + var children2 = + List.of(new ColumnOp("id", IntegerType.INTEGER), new LiteralOp("30", IntegerType.INTEGER)); + assertTrue(EvalHelper.lessThan(children2, evalContext2)); + var evalContext3 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("20", "29"))); + var children3 = List.of(new ColumnOp("id", LongType.LONG), new LiteralOp("30", LongType.LONG)); + assertTrue(EvalHelper.lessThan(children3, evalContext3)); + var evalContext4 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("2.99", "3.01"))); + var children4 = + List.of(new ColumnOp("id", FloatType.FLOAT), new LiteralOp("3.02", FloatType.FLOAT)); + assertTrue(EvalHelper.lessThan(children4, evalContext4)); + var evalContext5 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("2.99", "3.01"))); + var children5 = + List.of(new ColumnOp("id", DoubleType.DOUBLE), new LiteralOp("3.02", DoubleType.DOUBLE)); + assertTrue(EvalHelper.lessThan(children5, evalContext5)); + var evalContext6 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("2.99", "3.01"))); + var children6 = + List.of(new ColumnOp("id", DoubleType.DOUBLE), new LiteralOp("3.02", DoubleType.DOUBLE)); + assertTrue(EvalHelper.lessThan(children6, evalContext6)); + var evalContext7 = new EvalContext(Map.of("long", "20"), Map.of("id", Pair.of("aaaa", "cccc"))); + var children7 = + List.of(new ColumnOp("id", StringType.STRING), new LiteralOp("dddd", StringType.STRING)); + assertTrue(EvalHelper.lessThan(children7, evalContext7)); + var evalContext8 = new EvalContext( + Map.of("long", "20"), + Map.of("id", Pair.of("2022-08-10 06:02:03.000000", "2022-12-10 06:02:03.000000"))); + var children8 = List.of( + new ColumnOp("id", TimestampType.TIMESTAMP), + new LiteralOp("2022-12-11 06:02:03.000000", TimestampType.TIMESTAMP)); + assertTrue(EvalHelper.lessThan(children8, evalContext8)); + } + + @Test + void testValidateAndGetRange() throws PredicateException { + var evalContext1 = + new EvalContext(Map.of("date", "2020-10-11"), Map.of("id", Pair.of("20", "29"))); + var children1 = List.of( + new ColumnOp("notId", IntegerType.INTEGER), new LiteralOp("30", IntegerType.INTEGER)); + assertThrows( + NonExistingColumnException.class, () -> EvalHelper.lessThan(children1, evalContext1)); + var children2 = + List.of(new ColumnOp("id", IntegerType.INTEGER), new LiteralOp("30", IntegerType.INTEGER)); + assertTrue(EvalHelper.lessThan(children2, evalContext1)); + assertFalse(EvalHelper.equal(children2, evalContext1)); + var reversedChildren = + List.of(new LiteralOp("30", IntegerType.INTEGER), new ColumnOp("id", IntegerType.INTEGER)); + assertFalse(EvalHelper.equal(reversedChildren, evalContext1)); + var evalContext2 = + new EvalContext(Map.of("date", "2020-10-11"), Map.of("id", Pair.of("20", "29"))); + var children3 = + List.of(new ColumnOp("notId", IntegerType.INTEGER), new LiteralOp("30", DateType.DATE)); + assertThrows(TypeMismatchException.class, () -> EvalHelper.lessThan(children3, evalContext2)); + assertThrows(TypeMismatchException.class, () -> EvalHelper.equal(children3, evalContext2)); + } +} diff --git a/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateExceptionsTest.java b/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateExceptionsTest.java new file mode 100644 index 000000000..e78a77850 --- /dev/null +++ b/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateExceptionsTest.java @@ -0,0 +1,84 @@ +package io.whitefox.core.types.predicates; + +import static io.whitefox.DeltaTestUtils.deltaTableUri; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import io.delta.standalone.DeltaLog; +import io.delta.standalone.actions.AddFile; +import io.whitefox.core.PredicateUtils; +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; + +@DisabledOnOs(OS.WINDOWS) +public class PredicateExceptionsTest { + + DeltaLog log = DeltaLog.forTable( + new Configuration(), deltaTableUri("partitioned-delta-table-with-multiple-columns")); + AddFile file = log.snapshot().getAllFiles().get(0); + + @Test + void testTypeNotSupportedExceptionGettingThrown() throws PredicateParsingException { + + EvalContext context = PredicateUtils.createEvalContext(file); + var predicate = "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"dating\",\"valueType\":\"float\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-09-12\",\"valueType\":\"float\"}\n" + + " ]\n" + + "}"; + + var parsed = PredicateUtils.parseJsonPredicate(predicate); + assertThrows(TypeNotSupportedException.class, () -> parsed.evalExpectBoolean(context)); + } + + @Test + void testNonExistingColumnExceptionGettingThrown() throws PredicateParsingException { + + EvalContext context = PredicateUtils.createEvalContext(file); + var predicate = "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"notPresent\",\"valueType\": \"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-09-12\",\"valueType\":\"date\"}\n" + + " ]\n" + + "}"; + + var parsed = PredicateUtils.parseJsonPredicate(predicate); + assertThrows(NonExistingColumnException.class, () -> parsed.evalExpectBoolean(context)); + } + + @Test + void testTypeMismatchExceptionGettingThrown() throws PredicateParsingException { + + EvalContext context = PredicateUtils.createEvalContext(file); + var predicate = "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"abcd\",\"valueType\":\"string\"}\n" + + " ]\n" + + "}"; + + var parsed = PredicateUtils.parseJsonPredicate(predicate); + assertThrows(TypeMismatchException.class, () -> parsed.evalExpectBoolean(context)); + } + + @Test + void testTypeValidationExceptionGettingThrown() throws PredicateParsingException { + + EvalContext context = PredicateUtils.createEvalContext(file); + var predicate = "{" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"abcd\",\"valueType\":\"int\"}\n" + + " ]\n" + + "}"; + + var parsed = PredicateUtils.parseJsonPredicate(predicate); + assertThrows(TypeValidationException.class, () -> parsed.evalExpectBoolean(context)); + } +} diff --git a/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateParsingTest.java b/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateParsingTest.java new file mode 100644 index 000000000..f70ea8730 --- /dev/null +++ b/server/core/src/test/java/io/whitefox/core/types/predicates/PredicateParsingTest.java @@ -0,0 +1,126 @@ +package io.whitefox.core.types.predicates; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +import io.whitefox.core.Metadata; +import io.whitefox.core.PredicateUtils; +import io.whitefox.core.TableSchema; +import io.whitefox.core.types.DateType; +import io.whitefox.core.types.StructField; +import io.whitefox.core.types.StructType; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.junit.jupiter.api.Test; + +public class PredicateParsingTest { + + @Test + void testParsingOfJsonEqual() throws PredicateException { + + String predicate = "{\n" + " \"op\": \"equal\",\n" + + " \"children\": [\n" + + " {\"op\": \"column\", \"name\":\"hireDate\", \"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-04-29\",\"valueType\":\"date\"}\n" + + " ]\n" + + "}"; + var op = PredicateUtils.parseJsonPredicate(predicate); + op.validate(); + assert (op instanceof EqualOp); + assert (((EqualOp) op).children.size() == 2); + } + + @Test + void testParsingOfInvalidSql() { + var meta = new Metadata( + "id", + Optional.empty(), + Optional.empty(), + Metadata.Format.PARQUET, + new TableSchema( + new StructType(List.of(new StructField("date", DateType.DATE, true, Map.of())))), + List.of("date", "age"), + Map.of(), + Optional.empty(), + Optional.empty(), + Optional.empty()); + + var ctx = new EvalContext(Map.of("date", "date"), Map.of()); + var predicate = "date LIKE '2021-09-09'"; + + assertThrows( + ExpressionNotSupportedException.class, + () -> PredicateUtils.parseSqlPredicate(predicate, ctx, meta)); + } + + @Test + void testParsingOfSqlEqual() throws PredicateException { + var ctx = new EvalContext(Map.of("date", "date"), Map.of()); + var meta = new Metadata( + "id", + Optional.empty(), + Optional.empty(), + Metadata.Format.PARQUET, + new TableSchema( + new StructType(List.of(new StructField("date", DateType.DATE, true, Map.of())))), + List.of("date", "age"), + Map.of(), + Optional.empty(), + Optional.empty(), + Optional.empty()); + var predicate = "date = '2021-09-09'"; + var op = PredicateUtils.parseSqlPredicate(predicate, ctx, meta); + op.validate(); + assert (op instanceof EqualOp); + assert (((EqualOp) op).children.size() == 2); + } + + @Test + void testParsingOfNested() throws PredicateException { + String predicate = "{\n" + " \"op\":\"and\",\n" + + " \"children\":[\n" + + " {\n" + + " \"op\":\"equal\",\n" + + " \"children\":[\n" + + " {\"op\":\"column\",\"name\":\"hireDate\",\"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-04-29\",\"valueType\":\"date\"}\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"op\":\"lessThan\",\"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"25\",\"valueType\":\"int\"}\n" + + " ]\n" + + " }\n" + + " ]\n" + + "}"; + var op = PredicateUtils.parseJsonPredicate(predicate); + op.validate(); + assert (op instanceof AndOp); + assert (((AndOp) op).children.size() == 2); + assert (((AndOp) op).children.get(0) instanceof EqualOp); + } + + @Test + void testCustomExceptionOnBadJson() { + String predicate = "{\n" + " \"op\":\"and\",\n" + + " \"children\":[\n" + + " {\n" + + " \"op\":\"equals\",\n" + + " \"children\":[\n" + + " {\"op\":\"columna\",\"name\":\"hireDate\",\"valueType\":\"date\"},\n" + + " {\"op\":\"literal\",\"value\":\"2021-04-29\",\"valueType\":\"date\"}\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"op\":\"lessThans\",\"children\":[\n" + + " {\"op\":\"column\",\"name\":\"id\",\"valueType\":\"int\"},\n" + + " {\"op\":\"literal\",\"value\":\"25\",\"valueType\":\"int\"}\n" + + " ]\n" + + " }\n" + + " ]\n" + + "}"; + assertThrows( + PredicateParsingException.class, () -> PredicateUtils.parseJsonPredicate(predicate)); + } +} diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000000.json.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 000000000..f996377c0 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000000.json.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000001.json.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 000000000..269e811ac Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000001.json.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000002.json.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 000000000..582d4ecf2 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/.00000000000000000002.json.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000000.json b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000000.json new file mode 100644 index 000000000..5fff7515e --- /dev/null +++ b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000000.json @@ -0,0 +1,12 @@ +{"commitInfo":{"timestamp":1700421575700,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"date\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"9","numOutputRows":"9","numOutputBytes":"8432"},"engineInfo":"Apache-Spark/3.3.0 Delta-Lake/2.3.0","txnId":"ae63370b-3ee6-486b-9338-628e85824801"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"d344945b-8128-46c7-9d0e-5076f109549d","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"uid\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["date"],"configuration":{},"createdTime":1700421566245}} +{"add":{"path":"date=2021-08-09/part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0,\"uid\":\"82a911ca-1753-432a-8cd3-974f7ea1\"},\"maxValues\":{\"id\":0,\"uid\":\"82a911ca-1753-432a-8cd3-974f7ea1�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1,\"uid\":\"755a31b6-92d2-4531-ab58-e7cb1237\"},\"maxValues\":{\"id\":1,\"uid\":\"755a31b6-92d2-4531-ab58-e7cb1237�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2,\"uid\":\"581dfbf3-7fac-4b6d-959a-c53dbb48\"},\"maxValues\":{\"id\":2,\"uid\":\"581dfbf3-7fac-4b6d-959a-c53dbb48�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3,\"uid\":\"555a1916-27fa-44cb-ba70-aab79278\"},\"maxValues\":{\"id\":3,\"uid\":\"555a1916-27fa-44cb-ba70-aab79278�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4,\"uid\":\"37f7d866-8ade-430b-86af-10c3f1e3\"},\"maxValues\":{\"id\":4,\"uid\":\"37f7d866-8ade-430b-86af-10c3f1e3�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6,\"uid\":\"1081a468-d905-4e20-b598-649a665b\"},\"maxValues\":{\"id\":6,\"uid\":\"1081a468-d905-4e20-b598-649a665b�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":936,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7,\"uid\":\"2a22110c-ea3d-42cf-9e20-3cfc88bd\"},\"maxValues\":{\"id\":7,\"uid\":\"2a22110c-ea3d-42cf-9e20-3cfc88bd�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":937,"modificationTime":1700421574000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8,\"uid\":\"1d6017e1-a5c0-46a4-baa2-8f81d9f0\"},\"maxValues\":{\"id\":8,\"uid\":\"1d6017e1-a5c0-46a4-baa2-8f81d9f0�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":937,"modificationTime":1700421575000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9,\"uid\":\"09a39ec3-14c3-4547-9868-b270857e\"},\"maxValues\":{\"id\":9,\"uid\":\"09a39ec3-14c3-4547-9868-b270857e�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000001.json b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000001.json new file mode 100644 index 000000000..20c90c75b --- /dev/null +++ b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000001.json @@ -0,0 +1,12 @@ +{"commitInfo":{"timestamp":1700421782717,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"date\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"2","numOutputRows":"9","numOutputBytes":"2047"},"engineInfo":"Apache-Spark/3.3.0 Delta-Lake/2.3.0","txnId":"78717ed7-f5d8-4798-8401-6eab5d73f535"}} +{"add":{"path":"date=2021-08-09/part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":1044,"modificationTime":1700421781000,"dataChange":true,"stats":"{\"numRecords\":5,\"minValues\":{\"id\":0,\"uid\":\"5cbc37a4-8817-448f-8dfb-16de9e57\"},\"maxValues\":{\"id\":4,\"uid\":\"ef77f98c-639d-45b7-994d-419f7477�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":1003,"modificationTime":1700421781000,"dataChange":true,"stats":"{\"numRecords\":4,\"minValues\":{\"id\":6,\"uid\":\"1bf1a81c-9e24-43f0-aede-bf1d8782\"},\"maxValues\":{\"id\":9,\"uid\":\"c1a9d043-3d8d-4bfa-868f-eb8cb447�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"remove":{"path":"date=2021-08-15/part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet","deletionTimestamp":1700421782710,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-15"},"size":936}} +{"remove":{"path":"date=2021-08-15/part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-15"},"size":937}} +{"remove":{"path":"date=2021-08-09/part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":937}} +{"remove":{"path":"date=2021-08-09/part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":937}} +{"remove":{"path":"date=2021-08-15/part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-15"},"size":937}} +{"remove":{"path":"date=2021-08-15/part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-15"},"size":937}} +{"remove":{"path":"date=2021-08-09/part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":937}} +{"remove":{"path":"date=2021-08-09/part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":937}} +{"remove":{"path":"date=2021-08-09/part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet","deletionTimestamp":1700421782711,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":937}} diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000002.json b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000002.json new file mode 100644 index 000000000..d8e84daa2 --- /dev/null +++ b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/_delta_log/00000000000000000002.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1704459098200,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"date\"]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"2","numOutputRows":"19","numOutputBytes":"2504"},"engineInfo":"Apache-Spark/3.3.0 Delta-Lake/2.3.0","txnId":"0775569b-f4aa-445e-bd97-31ad2f713bac"}} +{"add":{"path":"date=2021-09-12/part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet","partitionValues":{"date":"2021-09-12"},"size":1229,"modificationTime":1704459094688,"dataChange":true,"stats":"{\"numRecords\":9,\"minValues\":{\"id\":31,\"uid\":\"042b58d9-f526-4b1e-a712-1d2b62c8\"},\"maxValues\":{\"id\":39,\"uid\":\"f7af65ac-289c-4142-8fc8-5ce6e3c4�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"add":{"path":"date=2022-02-06/part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet","partitionValues":{"date":"2022-02-06"},"size":1275,"modificationTime":1704459094688,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":20,\"uid\":\"06215b25-a3c6-4d69-b000-3487dc0b\"},\"maxValues\":{\"id\":29,\"uid\":\"bd2322fe-fc6c-4417-a3f9-e55b1a53�\"},\"nullCount\":{\"id\":0,\"uid\":0}}"}} +{"remove":{"path":"date=2021-08-09/part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet","deletionTimestamp":1704459098199,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-09"},"size":1044}} +{"remove":{"path":"date=2021-08-15/part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet","deletionTimestamp":1704459098199,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"date":"2021-08-15"},"size":1003}} diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet.crc new file mode 100644 index 000000000..15a06d517 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet.crc new file mode 100644 index 000000000..db3e7da8f Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet.crc new file mode 100644 index 000000000..76632d542 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet.crc new file mode 100644 index 000000000..46bdd2ef2 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet.crc new file mode 100644 index 000000000..067a6a903 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet.crc new file mode 100644 index 000000000..4e609f8a4 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/.part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet new file mode 100644 index 000000000..ecf13bf37 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00000-1dcbbcb3-4245-4f20-86d8-01231ba570e7.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet new file mode 100644 index 000000000..8d9f1be5a Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00001-6e82ece6-3597-4ad0-8843-1f6b1e3bc4f6.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet new file mode 100644 index 000000000..e897b0644 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00003-86eb45e1-a91f-4751-a091-4896b9f69ec8.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet new file mode 100644 index 000000000..c206488da Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00004-80d754aa-1567-453c-b491-1a44b562fa3c.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet new file mode 100644 index 000000000..bfbdaa503 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00006-f290581b-367f-435c-8466-2ce9f04cb3ef.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet new file mode 100644 index 000000000..dad5f92bc Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-09/part-00007-1714c8b5-a279-45f8-bc6c-937559d2c39a.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet.crc new file mode 100644 index 000000000..a38485a96 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet.crc new file mode 100644 index 000000000..fcaf10e37 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet.crc new file mode 100644 index 000000000..af18c5385 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet.crc new file mode 100644 index 000000000..0e1d595b7 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet.crc new file mode 100644 index 000000000..bec4280c0 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/.part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet new file mode 100644 index 000000000..68aea46ef Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00001-f38b954d-41f3-41c3-a7a7-3115e28814b7.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet new file mode 100644 index 000000000..3a789a4c6 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00009-dc49acd6-1a95-4e37-bf4b-cfbd789ff11c.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet new file mode 100644 index 000000000..a46769ede Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00011-f2482f68-663e-4380-bded-b62c066e32c9.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet new file mode 100644 index 000000000..9dce7655b Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00013-fd83bc7c-cb1e-4bf5-8e25-44a270e58690.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet new file mode 100644 index 000000000..8be7d7807 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-08-15/part-00015-78628903-d770-4d3b-984b-365fde6e183a.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/.part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/.part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet.crc new file mode 100644 index 000000000..dc3b35723 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/.part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet new file mode 100644 index 000000000..cb167afdb Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2021-09-12/part-00000-78e140ba-2b39-4742-b3e0-9d4ea6c178d9.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/.part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/.part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet.crc new file mode 100644 index 000000000..4aa73fde6 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/.part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet new file mode 100644 index 000000000..b82d35273 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table-with-multiple-columns/date=2022-02-06/part-00001-09dfde6d-8885-4ae4-9b7d-01a49d5e5e9a.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/.00000000000000000000.json.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 000000000..614fc5eb7 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/.00000000000000000000.json.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/00000000000000000000.json b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/00000000000000000000.json new file mode 100644 index 000000000..18bd945b6 --- /dev/null +++ b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/_delta_log/00000000000000000000.json @@ -0,0 +1,12 @@ +{"commitInfo":{"timestamp":1698863537194,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"date\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"9","numOutputRows":"9","numOutputBytes":"4302"},"engineInfo":"Apache-Spark/3.3.0 Delta-Lake/2.3.0","txnId":"bfc58406-b971-4499-8ae3-e61ff56c5c1f"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"d07c494b-e59b-4848-9bcd-2a08b307bce1","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["date"],"configuration":{},"createdTime":1698863528592}} +{"add":{"path":"date=2021-08-09/part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-09/part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet","partitionValues":{"date":"2021-08-09"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":478,"modificationTime":1698863536000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"date=2021-08-15/part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet","partitionValues":{"date":"2021-08-15"},"size":478,"modificationTime":1698863537000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet.crc new file mode 100644 index 000000000..fe3a6258e Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet.crc new file mode 100644 index 000000000..d5e19e43a Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet.crc new file mode 100644 index 000000000..40544516f Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet.crc new file mode 100644 index 000000000..caa0eea6e Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet.crc new file mode 100644 index 000000000..1f3c5de55 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/.part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet new file mode 100644 index 000000000..d6cec93b8 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00001-ef9d73d7-88f2-41af-80ae-e8fe85adadf6.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet new file mode 100644 index 000000000..114c2e107 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00003-a29698f2-7d2c-402d-9c0b-5d8b63a32bd2.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet new file mode 100644 index 000000000..c655515d2 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00004-1602573a-de57-4d2c-aeef-38ece808b77f.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet new file mode 100644 index 000000000..32faa5bbc Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00006-8669602d-9175-4640-ac90-bf9424a681bc.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet new file mode 100644 index 000000000..80b30c4b8 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-09/part-00007-2a965cbf-a36c-442e-a923-016823f92121.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet.crc new file mode 100644 index 000000000..dcca5ede7 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet.crc new file mode 100644 index 000000000..ed77d8c8b Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet.crc new file mode 100644 index 000000000..bc8291cd2 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet.crc b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet.crc new file mode 100644 index 000000000..15b7ddafc Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/.part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet.crc differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet new file mode 100644 index 000000000..a880f782c Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00009-28b727ce-c7d7-4863-a0c5-9497e34cde7d.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet new file mode 100644 index 000000000..4ca9f6bb3 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00011-ab578c13-0186-4f09-82e4-b521351e5a8c.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet new file mode 100644 index 000000000..7a0cc3f40 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00013-def8488a-2bf2-4623-bfba-f238deee6371.c000.snappy.parquet differ diff --git a/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet new file mode 100644 index 000000000..c2cae0218 Binary files /dev/null and b/server/core/src/testFixtures/resources/delta/samples/partitioned-delta-table/date=2021-08-15/part-00015-7ca44de2-8b15-465d-86f5-5de21ed67eee.c000.snappy.parquet differ