diff --git a/dagger-common/build.gradle b/dagger-common/build.gradle index 842714d5f..0b6d26593 100644 --- a/dagger-common/build.gradle +++ b/dagger-common/build.gradle @@ -64,6 +64,7 @@ dependencies { dependenciesCommonJar 'com.jayway.jsonpath:json-path:2.4.0' dependenciesCommonJar 'io.odpf:stencil:0.1.6' dependenciesCommonJar 'com.google.code.gson:gson:2.8.2' + dependenciesCommonJar 'org.apache.parquet:parquet-column:1.12.2' testImplementation 'junit:junit:4.13' testImplementation 'org.jmockit:jmockit:1.25' diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/exceptions/serde/SimpleGroupParsingException.java b/dagger-common/src/main/java/io/odpf/dagger/common/exceptions/serde/SimpleGroupParsingException.java new file mode 100644 index 000000000..c651d17d4 --- /dev/null +++ b/dagger-common/src/main/java/io/odpf/dagger/common/exceptions/serde/SimpleGroupParsingException.java @@ -0,0 +1,11 @@ +package io.odpf.dagger.common.exceptions.serde; + +/** + * This runtime exception is thrown when a field cannot be parsed from a Parquet SimpleGroup. + **/ +public class SimpleGroupParsingException extends RuntimeException { + + public SimpleGroupParsingException(String message) { + super(message); + } +} diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/DaggerDeserializer.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/DaggerDeserializer.java new file mode 100644 index 000000000..928148ae5 --- /dev/null +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/DaggerDeserializer.java @@ -0,0 +1,9 @@ +package io.odpf.dagger.common.serde; + +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; + +import java.io.Serializable; + +public interface DaggerDeserializer extends Serializable, ResultTypeQueryable { + +} diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/json/deserialization/JsonDeserializer.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/json/deserialization/JsonDeserializer.java index af8ba4c47..3f234424e 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/json/deserialization/JsonDeserializer.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/json/deserialization/JsonDeserializer.java @@ -1,5 +1,6 @@ package io.odpf.dagger.common.serde.json.deserialization; +import io.odpf.dagger.common.serde.DaggerDeserializer; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.formats.json.JsonRowDeserializationSchema; @@ -16,7 +17,7 @@ import static io.odpf.dagger.common.core.Constants.ROWTIME; -public class JsonDeserializer implements KafkaDeserializationSchema { +public class JsonDeserializer implements KafkaDeserializationSchema, DaggerDeserializer { private final JsonRowDeserializationSchema jsonRowDeserializationSchema; private final int rowtimeIdx; private final TypeInformation typeInformation; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidation.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidation.java new file mode 100644 index 000000000..1542252f5 --- /dev/null +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidation.java @@ -0,0 +1,9 @@ +package io.odpf.dagger.common.serde.parquet; + +import org.apache.parquet.example.data.simple.SimpleGroup; + +public class SimpleGroupValidation { + public static boolean checkFieldExistsAndIsInitialized(SimpleGroup simpleGroup, String fieldName) { + return simpleGroup.getType().containsField(fieldName) && simpleGroup.getFieldRepetitionCount(fieldName) != 0; + } +} diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializer.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializer.java new file mode 100644 index 000000000..749274f3e --- /dev/null +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializer.java @@ -0,0 +1,71 @@ +package io.odpf.dagger.common.serde.parquet.deserialization; + +import com.google.protobuf.Descriptors; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.exceptions.DescriptorNotFoundException; +import io.odpf.dagger.common.exceptions.serde.DaggerDeserializationException; +import io.odpf.dagger.common.exceptions.serde.SimpleGroupParsingException; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoType; +import io.odpf.dagger.common.serde.proto.protohandler.RowFactory; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.types.Row; +import org.apache.parquet.example.data.simple.SimpleGroup; + +import java.sql.Timestamp; +import java.time.Instant; + +public class SimpleGroupDeserializer implements DaggerDeserializer { + private final String protoClassName; + private final int timestampFieldIndex; + private final StencilClientOrchestrator stencilClientOrchestrator; + private final TypeInformation typeInformation; + + public SimpleGroupDeserializer(String protoClassName, int timestampFieldIndex, String rowtimeAttributeName, StencilClientOrchestrator stencilClientOrchestrator) { + this.protoClassName = protoClassName; + this.timestampFieldIndex = timestampFieldIndex; + this.stencilClientOrchestrator = stencilClientOrchestrator; + this.typeInformation = new ProtoType(protoClassName, rowtimeAttributeName, stencilClientOrchestrator).getRowType(); + } + + private Descriptors.Descriptor getProtoParser() { + Descriptors.Descriptor dsc = stencilClientOrchestrator.getStencilClient().get(protoClassName); + if (dsc == null) { + throw new DescriptorNotFoundException(); + } + return dsc; + } + + public Row deserialize(SimpleGroup simpleGroup) { + Descriptors.Descriptor descriptor = getProtoParser(); + try { + Row row = RowFactory.createRow(descriptor, simpleGroup, 2); + return addTimestampFieldToRow(row, simpleGroup, descriptor); + } catch (RuntimeException e) { + throw new DaggerDeserializationException(e); + } + } + + private Row addTimestampFieldToRow(Row row, SimpleGroup simpleGroup, Descriptors.Descriptor descriptor) { + Descriptors.FieldDescriptor fieldDescriptor = descriptor.findFieldByNumber(timestampFieldIndex); + String timestampFieldName = fieldDescriptor.getName(); + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, timestampFieldName)) { + long timeInMillis = simpleGroup.getLong(timestampFieldName, 0); + Instant instant = Instant.ofEpochMilli(timeInMillis); + + row.setField(row.getArity() - 2, true); + row.setField(row.getArity() - 1, Timestamp.from(instant)); + return row; + } else { + String errMessage = String.format("Could not extract timestamp with field name %s from simple group of type %s", + fieldDescriptor.getName(), simpleGroup.getType().toString()); + throw new SimpleGroupParsingException(errMessage); + } + } + + @Override + public TypeInformation getProducedType() { + return this.typeInformation; + } +} diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/deserialization/ProtoDeserializer.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/deserialization/ProtoDeserializer.java index 90a6fe32b..a6590936a 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/deserialization/ProtoDeserializer.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/deserialization/ProtoDeserializer.java @@ -1,5 +1,6 @@ package io.odpf.dagger.common.serde.proto.deserialization; +import io.odpf.dagger.common.serde.DaggerDeserializer; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; import org.apache.flink.types.Row; @@ -22,7 +23,7 @@ /** * Deserializer for protobuf messages. */ -public class ProtoDeserializer implements KafkaDeserializationSchema { +public class ProtoDeserializer implements KafkaDeserializationSchema, DaggerDeserializer { private final String protoClassName; private final int timestampFieldIndex; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandler.java index 63bb33175..8ef643ca0 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandler.java @@ -1,11 +1,13 @@ package io.odpf.dagger.common.serde.proto.protohandler; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import io.odpf.dagger.common.exceptions.serde.EnumFieldNotFoundException; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The type Enum proto handler. @@ -28,7 +30,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -54,10 +56,22 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { return String.valueOf(field).trim(); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + String defaultEnumValue = fieldDescriptor.getEnumType().findValueByNumber(0).getName(); + String fieldName = fieldDescriptor.getName(); + if (simpleGroup != null && SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + String parquetEnumValue = simpleGroup.getString(fieldName, 0); + Descriptors.EnumValueDescriptor enumValueDescriptor = fieldDescriptor.getEnumType().findValueByName(parquetEnumValue); + return enumValueDescriptor == null ? defaultEnumValue : enumValueDescriptor.getName(); + } + return defaultEnumValue; + } + @Override public Object transformToJson(Object field) { return field; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandler.java index 88f28003d..a86cc4bb7 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandler.java @@ -7,6 +7,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.types.Row; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.List; @@ -35,7 +36,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -64,7 +65,7 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { ArrayList rows = new ArrayList<>(); if (field != null) { List protos = (List) field; @@ -73,6 +74,11 @@ public Object transformFromKafka(Object field) { return rows.toArray(); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + return null; + } + @Override public Object transformToJson(Object field) { return null; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandler.java index 54e9bab8b..d6e84e076 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandler.java @@ -7,6 +7,7 @@ import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.DynamicMessage; import com.google.protobuf.DynamicMessage.Builder; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.List; import java.util.Map; @@ -35,7 +36,7 @@ public boolean canHandle() { } @Override - public Builder transformForKafka(Builder builder, Object field) { + public Builder transformToProtoBuilder(Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -49,7 +50,7 @@ public Builder transformForKafka(Builder builder, Object field) { if (index < rowElement.getArity()) { ProtoHandler protoHandler = ProtoHandlerFactory.getProtoHandler(nestedFieldDescriptor); if (rowElement.getField(index) != null) { - protoHandler.transformForKafka(elementBuilder, rowElement.getField(index)); + protoHandler.transformToProtoBuilder(elementBuilder, rowElement.getField(index)); } } } @@ -63,10 +64,15 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { return RowFactory.createRow((DynamicMessage) field); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + return null; + } + @Override public Object transformToJson(Object field) { if (jsonRowSerializationSchema == null) { diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandler.java index 43d419b0e..8efea007d 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandler.java @@ -7,6 +7,7 @@ import io.odpf.dagger.common.serde.proto.protohandler.typehandler.PrimitiveTypeHandler; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The type Primitive proto handler. @@ -29,7 +30,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { return field != null ? builder.setField(fieldDescriptor, transform(field)) : builder; } @@ -41,7 +42,7 @@ public Object transformFromPostProcessor(Object field) { private Object transform(Object field) { PrimitiveTypeHandler primitiveTypeHandler = PrimitiveTypeHandlerFactory.getTypeHandler(fieldDescriptor); try { - return primitiveTypeHandler.getValue(field); + return primitiveTypeHandler.parseObject(field); } catch (NumberFormatException e) { String errMessage = String.format("type mismatch of field: %s, expecting %s type, actual type %s", fieldDescriptor.getName(), fieldDescriptor.getType(), field.getClass()); throw new InvalidDataTypeException(errMessage); @@ -49,10 +50,16 @@ private Object transform(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { return field; } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + PrimitiveTypeHandler primitiveTypeHandler = PrimitiveTypeHandlerFactory.getTypeHandler(fieldDescriptor); + return primitiveTypeHandler.parseSimpleGroup(simpleGroup); + } + @Override public Object transformToJson(Object field) { return field; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/ProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/ProtoHandler.java index a25a839fc..89c529775 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/ProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/ProtoHandler.java @@ -3,6 +3,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The interface Proto handler. @@ -16,13 +17,13 @@ public interface ProtoHandler { boolean canHandle(); /** - * Transform for kafka dynamic message . builder. + * Transform to protobuf message builder. * * @param builder the builder * @param field the field * @return the dynamic message . builder */ - DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field); + DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field); /** * Transform from post processor object. @@ -33,12 +34,20 @@ public interface ProtoHandler { Object transformFromPostProcessor(Object field); /** - * Transform from kafka object. + * Transform from protobuf message. * * @param field the field * @return the object */ - Object transformFromKafka(Object field); + Object transformFromProto(Object field); + + /** + * Transform from parquet SimpleGroup. + * + * @param simpleGroup the SimpleGroup object + * @return the transformed object + */ + Object transformFromParquet(SimpleGroup simpleGroup); /** * Transform to json object. diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandler.java index bab6cc748..e14a948ef 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandler.java @@ -7,6 +7,7 @@ import com.google.gson.Gson; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.List; @@ -36,7 +37,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { return builder; } @@ -46,10 +47,15 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { return getValue(field); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + return null; + } + @Override public Object transformToJson(Object field) { return GSON.toJson(getValue(field)); diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandler.java index ee20e7970..c10938b2b 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandler.java @@ -9,6 +9,7 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.DynamicMessage.Builder; import net.minidev.json.JSONArray; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.Arrays; @@ -39,7 +40,7 @@ public boolean canHandle() { } @Override - public Builder transformForKafka(Builder builder, Object field) { + public Builder transformToProtoBuilder(Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -74,7 +75,7 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { ArrayList rows = new ArrayList<>(); if (field != null) { List protos = (List) field; @@ -83,6 +84,11 @@ public Object transformFromKafka(Object field) { return rows.toArray(); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + return null; + } + @Override public Object transformToJson(Object field) { if (jsonRowSerializationSchema == null) { @@ -110,7 +116,7 @@ private void handleNestedField(Builder elementBuilder, List nes if (index < row.getArity()) { ProtoHandler protoHandler = ProtoHandlerFactory.getProtoHandler(nestedFieldDescriptor); - protoHandler.transformForKafka(elementBuilder, row.getField(index)); + protoHandler.transformToProtoBuilder(elementBuilder, row.getField(index)); } } } diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandler.java index 5f4c9f0d2..e0472a1dd 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandler.java @@ -7,6 +7,7 @@ import com.google.gson.Gson; import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.Arrays; @@ -37,7 +38,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -61,11 +62,16 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { PrimitiveTypeHandler primitiveTypeHandler = PrimitiveTypeHandlerFactory.getTypeHandler(fieldDescriptor); return primitiveTypeHandler.getArray(field); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + return null; + } + @Override public Object transformToJson(Object field) { return GSON.toJson(field); diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandler.java index 131904caa..58e3ef425 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandler.java @@ -5,6 +5,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The type Repeated struct message proto handler. @@ -28,7 +29,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { return builder; } @@ -38,7 +39,12 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { + return null; + } + + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { return null; } diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactory.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactory.java index 6fdb76baf..e567b920d 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactory.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactory.java @@ -5,6 +5,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.List; import java.util.Map; @@ -47,11 +48,25 @@ public static Row createRow(DynamicMessage proto, int extraColumns) { Row row = new Row(descriptorFields.size() + extraColumns); for (FieldDescriptor fieldDescriptor : descriptorFields) { ProtoHandler protoHandler = ProtoHandlerFactory.getProtoHandler(fieldDescriptor); - row.setField(fieldDescriptor.getIndex(), protoHandler.transformFromKafka(proto.getField(fieldDescriptor))); + row.setField(fieldDescriptor.getIndex(), protoHandler.transformFromProto(proto.getField(fieldDescriptor))); } return row; } + public static Row createRow(Descriptors.Descriptor descriptor, SimpleGroup simpleGroup, int extraColumns) { + List descriptorFields = descriptor.getFields(); + Row row = new Row(descriptorFields.size() + extraColumns); + for (FieldDescriptor fieldDescriptor : descriptorFields) { + ProtoHandler protoHandler = ProtoHandlerFactory.getProtoHandler(fieldDescriptor); + row.setField(fieldDescriptor.getIndex(), protoHandler.transformFromParquet(simpleGroup)); + } + return row; + } + + public static Row createRow(Descriptors.Descriptor descriptor, SimpleGroup simpleGroup) { + return createRow(descriptor, simpleGroup, 0); + } + /** * Create row from specfied proto and extra columns equals to zero. * diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandler.java index d80c4f745..bb22b3717 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandler.java @@ -5,6 +5,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The type Struct message proto handler. @@ -28,7 +29,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { return builder; } @@ -38,7 +39,12 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { + return null; + } + + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { return null; } diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandler.java index 353b641b1..321790c70 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandler.java @@ -1,11 +1,13 @@ package io.odpf.dagger.common.serde.proto.protohandler; +import com.google.protobuf.Timestamp; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.types.Row; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; -import com.google.protobuf.Timestamp; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.text.SimpleDateFormat; import java.time.Instant; @@ -19,6 +21,9 @@ */ public class TimestampProtoHandler implements ProtoHandler { private static final int SECOND_TO_MS_FACTOR = 1000; + private static final long DEFAULT_SECONDS_VALUE = 0L; + private static final int DEFAULT_NANOS_VALUE = 0; + private static final int MS_TO_NANOS_FACTOR = 1000_000; private static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); private Descriptors.FieldDescriptor fieldDescriptor; @@ -38,7 +43,7 @@ public boolean canHandle() { } @Override - public DynamicMessage.Builder transformForKafka(DynamicMessage.Builder builder, Object field) { + public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) { if (!canHandle() || field == null) { return builder; } @@ -93,10 +98,26 @@ public Object transformFromPostProcessor(Object field) { } @Override - public Object transformFromKafka(Object field) { + public Object transformFromProto(Object field) { return RowFactory.createRow((DynamicMessage) field); } + @Override + public Object transformFromParquet(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + if (simpleGroup != null && SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + + /* conversion from ms to nanos borrowed from Instant.java class and inlined here for performance reasons */ + long timeInMillis = simpleGroup.getLong(fieldName, 0); + long seconds = Math.floorDiv(timeInMillis, SECOND_TO_MS_FACTOR); + int mos = (int) Math.floorMod(timeInMillis, SECOND_TO_MS_FACTOR); + int nanos = mos * MS_TO_NANOS_FACTOR; + return Row.of(seconds, nanos); + } else { + return Row.of(DEFAULT_SECONDS_VALUE, DEFAULT_NANOS_VALUE); + } + } + @Override public Object transformToJson(Object field) { Row timeField = (Row) field; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandler.java index 2011dd259..4d79c2f53 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandler.java @@ -5,6 +5,8 @@ import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import java.util.List; @@ -29,10 +31,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return Boolean.parseBoolean(getValueOrDefault(field, "false")); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getBoolean(fieldName, 0); + } else { + /* return default value */ + return false; + } + } + @Override public Object getArray(Object field) { boolean[] inputValues = new boolean[0]; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandler.java index 568b53d43..958f075fe 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandler.java @@ -1,11 +1,13 @@ package io.odpf.dagger.common.serde.proto.protohandler.typehandler; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import com.google.protobuf.ByteString; import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.List; @@ -31,10 +33,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return field; } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + byte[] byteArray = simpleGroup.getBinary(fieldName, 0).getBytes(); + return ByteString.copyFrom(byteArray); + } else { + return null; + } + } + @Override public Object getArray(Object field) { List inputValues = new ArrayList<>(); diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandler.java index e72eb0b8b..4cc0c2d07 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandler.java @@ -3,8 +3,10 @@ import com.google.common.primitives.Doubles; import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.List; @@ -29,10 +31,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return Double.parseDouble(getValueOrDefault(field, "0")); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getDouble(fieldName, 0); + } else { + /* return default value */ + return 0.0D; + } + } + @Override public Object getArray(Object field) { double[] inputValues = new double[0]; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandler.java index cababed55..9f42fb7d2 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandler.java @@ -3,8 +3,10 @@ import com.google.common.primitives.Floats; import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.List; @@ -29,10 +31,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return Float.parseFloat(getValueOrDefault(field, "0")); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getFloat(fieldName, 0); + } else { + /* return default value */ + return 0.0F; + } + } + @Override public Object getArray(Object field) { diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandler.java index 94c39e1ad..1fbc1931c 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandler.java @@ -3,8 +3,10 @@ import com.google.common.primitives.Ints; import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.List; @@ -29,10 +31,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return Integer.parseInt(getValueOrDefault(field, "0")); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getInteger(fieldName, 0); + } else { + /* return default value */ + return 0; + } + } + @Override public Object getArray(Object field) { int[] inputValues = new int[0]; diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandler.java index ee82ed310..4ee15d02a 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandler.java @@ -2,8 +2,10 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.List; @@ -29,10 +31,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return Long.parseLong(getValueOrDefault(field, "0")); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getLong(fieldName, 0); + } else { + /* return default value */ + return 0L; + } + } + @Override public Object getArray(Object field) { List inputValues = new ArrayList<>(); diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/PrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/PrimitiveTypeHandler.java index a6d3de1c6..071905389 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/PrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/PrimitiveTypeHandler.java @@ -1,6 +1,7 @@ package io.odpf.dagger.common.serde.proto.protohandler.typehandler; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.parquet.example.data.simple.SimpleGroup; /** * The interface Primitive type handler. @@ -14,12 +15,20 @@ public interface PrimitiveTypeHandler { boolean canHandle(); /** - * Gets value. + * Parses a primitive value from an object. * - * @param field the field - * @return the value + * @param field the object to be parsed + * @return the parsed value + */ + Object parseObject(Object field); + + /** + * Parses a primitive value from a SimpleGroup object. + * + * @param simpleGroup SimpleGroup object to be parsed + * @return the parsed value */ - Object getValue(Object field); + Object parseSimpleGroup(SimpleGroup simpleGroup); /** * Gets array. diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandler.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandler.java index 13114182d..91021f419 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandler.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandler.java @@ -2,9 +2,11 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import io.odpf.dagger.common.serde.parquet.SimpleGroupValidation; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.parquet.example.data.simple.SimpleGroup; import java.util.ArrayList; import java.util.List; @@ -30,10 +32,23 @@ public boolean canHandle() { } @Override - public Object getValue(Object field) { + public Object parseObject(Object field) { return getValueOrDefault(field, ""); } + @Override + public Object parseSimpleGroup(SimpleGroup simpleGroup) { + String fieldName = fieldDescriptor.getName(); + + /* this if branch checks that the field name exists in the simple group schema and is initialized */ + if (SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, fieldName)) { + return simpleGroup.getString(fieldName, 0); + } else { + /* return default value */ + return ""; + } + } + @Override public Object getArray(Object field) { List inputValues = new ArrayList<>(); diff --git a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/serialization/ProtoSerializer.java b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/serialization/ProtoSerializer.java index e8cd0af8c..15b863b8a 100644 --- a/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/serialization/ProtoSerializer.java +++ b/dagger-common/src/main/java/io/odpf/dagger/common/serde/proto/serialization/ProtoSerializer.java @@ -140,7 +140,7 @@ private DynamicMessage.Builder populateBuilder(DynamicMessage.Builder builder, D ProtoHandler protoHandler = ProtoHandlerFactory.getProtoHandler(fieldDescriptor); if (data != null) { try { - builder = protoHandler.transformForKafka(builder, data); + builder = protoHandler.transformToProtoBuilder(builder, data); } catch (IllegalArgumentException e) { String protoType = fieldDescriptor.getType().toString(); if (fieldDescriptor.isRepeated()) { diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidationTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidationTest.java new file mode 100644 index 000000000..c98b0bd74 --- /dev/null +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/SimpleGroupValidationTest.java @@ -0,0 +1,45 @@ +package io.odpf.dagger.common.serde.parquet; + +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; +import org.junit.Test; + +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.Types.buildMessage; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class SimpleGroupValidationTest { + + @Test + public void checkFieldExistsAndIsInitializedShouldReturnFalseWhenFieldIsNotInitializedInSimpleGroup() { + GroupType parquetSchema = buildMessage() + .required(INT32).named("primitive-type-column") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertFalse(SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, "primitive-type-column")); + } + + @Test + public void checkFieldExistsAndIsInitializedShouldReturnFalseWhenFieldIsNotPresentInSimpleGroup() { + GroupType parquetSchema = buildMessage() + .required(INT32).named("primitive-type-column") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("primitive-type-column", 34); + + assertFalse(SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, "unknown-column")); + } + + @Test + public void checkFieldExistsAndIsInitializedShouldReturnTrueWhenFieldIsBothPresentAndInitializedInSimpleGroup() { + GroupType parquetSchema = buildMessage() + .required(INT32).named("primitive-type-column") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("primitive-type-column", 34); + + assertTrue(SimpleGroupValidation.checkFieldExistsAndIsInitialized(simpleGroup, "primitive-type-column")); + } +} diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializerTest.java new file mode 100644 index 000000000..9e443d188 --- /dev/null +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/parquet/deserialization/SimpleGroupDeserializerTest.java @@ -0,0 +1,200 @@ +package io.odpf.dagger.common.serde.parquet.deserialization; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.exceptions.DescriptorNotFoundException; +import io.odpf.dagger.common.exceptions.serde.DaggerDeserializationException; +import io.odpf.dagger.consumer.TestBookingLogKey; +import io.odpf.dagger.consumer.TestPrimitiveMessage; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.PrimitiveType; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.sql.Timestamp; +import java.time.Instant; + +import static io.odpf.dagger.common.core.Constants.*; +import static io.odpf.dagger.common.core.Constants.SCHEMA_REGISTRY_STENCIL_URLS_DEFAULT; +import static org.apache.flink.api.common.typeinfo.Types.*; +import static org.apache.flink.api.common.typeinfo.Types.SQL_TIMESTAMP; +import static org.junit.Assert.*; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class SimpleGroupDeserializerTest { + + private StencilClientOrchestrator stencilClientOrchestrator; + + @Mock + private Configuration configuration; + + @Before + public void setUp() { + initMocks(this); + when(configuration.getBoolean(SCHEMA_REGISTRY_STENCIL_ENABLE_KEY, SCHEMA_REGISTRY_STENCIL_ENABLE_DEFAULT)).thenReturn(SCHEMA_REGISTRY_STENCIL_ENABLE_DEFAULT); + when(configuration.getString(SCHEMA_REGISTRY_STENCIL_URLS_KEY, SCHEMA_REGISTRY_STENCIL_URLS_DEFAULT)).thenReturn(SCHEMA_REGISTRY_STENCIL_URLS_DEFAULT); + stencilClientOrchestrator = new StencilClientOrchestrator(configuration); + } + + @Test + public void shouldReturnProducedType() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestBookingLogKey.class.getTypeName(), 5, "rowtime", stencilClientOrchestrator); + TypeInformation producedType = simpleGroupDeserializer.getProducedType(); + assertArrayEquals( + new String[]{"service_type", "order_number", "order_url", "status", "event_timestamp", INTERNAL_VALIDATION_FIELD_KEY, "rowtime"}, + ((RowTypeInfo) producedType).getFieldNames()); + assertArrayEquals( + new TypeInformation[]{STRING, STRING, STRING, STRING, ROW_NAMED(new String[]{"seconds", "nanos"}, LONG, INT), BOOLEAN, SQL_TIMESTAMP}, + ((RowTypeInfo) producedType).getFieldTypes()); + } + + @Test + public void shouldDeserializeSimpleGroupOfPrimitiveTypesIntoRow() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestPrimitiveMessage.class.getTypeName(), 9, "rowtime", stencilClientOrchestrator); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(PrimitiveType.PrimitiveTypeName.BOOLEAN).named("is_valid") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_number") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_hash") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("latitude") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("longitude") + .required(PrimitiveType.PrimitiveTypeName.FLOAT).named("price") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("packet_count") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("phone") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("event_timestamp") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("service_type") + .named("TestGroupType"); + + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("is_valid", true); + simpleGroup.add("order_number", "ORDER_1322432"); + String byteString = "g362vxv3ydg73g2ss"; + simpleGroup.add("order_hash", Binary.fromConstantByteArray(byteString.getBytes())); + simpleGroup.add("latitude", Double.MAX_VALUE); + simpleGroup.add("longitude", Double.MIN_VALUE); + simpleGroup.add("price", Float.MAX_VALUE); + simpleGroup.add("packet_count", Integer.MAX_VALUE); + simpleGroup.add("phone", Long.MAX_VALUE); + long currentTimeInMillis = Instant.now().toEpochMilli(); + long seconds = Instant.ofEpochMilli(currentTimeInMillis).getEpochSecond(); + int nanos = Instant.ofEpochMilli(currentTimeInMillis).getNano(); + simpleGroup.add("event_timestamp", currentTimeInMillis); + simpleGroup.add("service_type", "GO_RIDE"); + + Row row = simpleGroupDeserializer.deserialize(simpleGroup); + + assertEquals("ORDER_1322432", row.getField(getProtoIndex("order_number"))); + assertEquals(Float.MAX_VALUE, row.getField(getProtoIndex("price"))); + Row actualTimestampRow = (Row) row.getField(getProtoIndex("event_timestamp")); + assertNotNull(actualTimestampRow); + assertEquals(seconds, actualTimestampRow.getField(0)); + assertEquals(nanos, actualTimestampRow.getField(1)); + } + + @Test + public void shouldAddExtraFieldsToRow() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestPrimitiveMessage.class.getTypeName(), 9, "rowtime", stencilClientOrchestrator); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(PrimitiveType.PrimitiveTypeName.BOOLEAN).named("is_valid") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_number") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_hash") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("latitude") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("longitude") + .required(PrimitiveType.PrimitiveTypeName.FLOAT).named("price") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("packet_count") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("phone") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("event_timestamp") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("service_type") + .named("TestGroupType"); + + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + long currentTimeInMillis = Instant.now().toEpochMilli(); + simpleGroup.add("event_timestamp", currentTimeInMillis); + int expectedRowCount = TestPrimitiveMessage.getDescriptor().getFields().size() + 2; + + Row row = simpleGroupDeserializer.deserialize(simpleGroup); + + assertEquals(expectedRowCount, row.getArity()); + assertEquals(true, row.getField(row.getArity() - 2)); + assertEquals(Timestamp.from(Instant.ofEpochMilli(currentTimeInMillis)), row.getField(row.getArity() - 1)); + } + + @Test + public void shouldSetDefaultValueForAllPrimitiveTypeFieldsExceptTimestampIfMissingInSimpleGroup() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestPrimitiveMessage.class.getTypeName(), 9, "rowtime", stencilClientOrchestrator); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(PrimitiveType.PrimitiveTypeName.BOOLEAN).named("is_valid") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_number") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_hash") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("latitude") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("longitude") + .required(PrimitiveType.PrimitiveTypeName.FLOAT).named("price") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("packet_count") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("phone") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("event_timestamp") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("service_type") + .named("TestGroupType"); + + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + long currentTimeInMillis = Instant.now().toEpochMilli(); + simpleGroup.add("event_timestamp", currentTimeInMillis); + + Row row = simpleGroupDeserializer.deserialize(simpleGroup); + + assertEquals(false, row.getField(getProtoIndex("is_valid"))); + assertEquals("", row.getField(getProtoIndex("order_number"))); + assertNull(row.getField(getProtoIndex("order_hash"))); + assertEquals(0.0D, row.getField(getProtoIndex("latitude"))); + assertEquals(0.0F, row.getField(getProtoIndex("price"))); + assertEquals(0, row.getField(getProtoIndex("packet_count"))); + assertEquals(0L, row.getField(getProtoIndex("phone"))); + assertEquals("UNKNOWN", row.getField(getProtoIndex("service_type"))); + } + + @Test + public void shouldThrowExceptionIfTimestampIsMissingInSimpleGroup() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestPrimitiveMessage.class.getTypeName(), 9, "rowtime", stencilClientOrchestrator); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(PrimitiveType.PrimitiveTypeName.BOOLEAN).named("is_valid") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_number") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("order_hash") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("latitude") + .required(PrimitiveType.PrimitiveTypeName.DOUBLE).named("longitude") + .required(PrimitiveType.PrimitiveTypeName.FLOAT).named("price") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("packet_count") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("phone") + .required(PrimitiveType.PrimitiveTypeName.INT64).named("event_timestamp") + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("service_type") + .named("TestGroupType"); + + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + Assert.assertThrows(DaggerDeserializationException.class, + () -> simpleGroupDeserializer.deserialize(simpleGroup)); + } + + @Test + public void shouldThrowExceptionIfSimpleGroupIsNull() { + SimpleGroupDeserializer simpleGroupDeserializer = new SimpleGroupDeserializer(TestPrimitiveMessage.class.getTypeName(), 9, "rowtime", stencilClientOrchestrator); + + Assert.assertThrows(DaggerDeserializationException.class, + () -> simpleGroupDeserializer.deserialize(null)); + } + + @Test + public void shouldThrowExceptionIfDescriptorIsNotFound() { + assertThrows(DescriptorNotFoundException.class, + () -> new SimpleGroupDeserializer(String.class.getTypeName(), 6, "rowtime", stencilClientOrchestrator)); + } + + private int getProtoIndex(String propertyName) { + return TestPrimitiveMessage.getDescriptor().findFieldByName(propertyName).getIndex(); + } +} diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandlerTest.java index b2ff1216f..957033270 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/EnumProtoHandlerTest.java @@ -2,15 +2,16 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; -import io.odpf.dagger.consumer.TestBookingLogMessage; -import io.odpf.dagger.consumer.TestRepeatedEnumMessage; -import io.odpf.dagger.consumer.TestServiceType; +import io.odpf.dagger.consumer.*; import io.odpf.dagger.common.exceptions.serde.EnumFieldNotFoundException; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Assert; import org.junit.Test; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.junit.Assert.*; public class EnumProtoHandlerTest { @@ -45,7 +46,7 @@ public void shouldReturnTheSameBuilderWithoutSettingIfCanNotHandle() { EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - assertEquals("", enumProtoHandler.transformForKafka(builder, 123).getField(fieldDescriptor)); + assertEquals("", enumProtoHandler.transformToProtoBuilder(builder, 123).getField(fieldDescriptor)); } @Test @@ -54,7 +55,7 @@ public void shouldReturnTheSameBuilderWithoutSettingIfNullPassed() { EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - assertEquals("", enumProtoHandler.transformForKafka(builder, null).getField(fieldDescriptor)); + assertEquals("", enumProtoHandler.transformToProtoBuilder(builder, null).getField(fieldDescriptor)); } @Test @@ -63,7 +64,7 @@ public void shouldSetTheFieldPassedInTheBuilderForEnumFieldTypeDescriptor() { EnumProtoHandler enumProtoHandler = new EnumProtoHandler(enumFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(enumFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = enumProtoHandler.transformForKafka(builder, "GO_RIDE"); + DynamicMessage.Builder returnedBuilder = enumProtoHandler.transformToProtoBuilder(builder, "GO_RIDE"); assertEquals(TestServiceType.Enum.GO_RIDE.getValueDescriptor(), returnedBuilder.getField(enumFieldDescriptor)); } @@ -73,7 +74,7 @@ public void shouldThrowExceptionIfFieldNotFoundInGivenEnumFieldTypeDescriptor() EnumProtoHandler enumProtoHandler = new EnumProtoHandler(enumFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(enumFieldDescriptor.getContainingType()); - EnumFieldNotFoundException exception = Assert.assertThrows(EnumFieldNotFoundException.class, () -> enumProtoHandler.transformForKafka(builder, "test")); + EnumFieldNotFoundException exception = Assert.assertThrows(EnumFieldNotFoundException.class, () -> enumProtoHandler.transformToProtoBuilder(builder, "test")); assertEquals("field: test not found in io.odpf.dagger.consumer.TestBookingLogMessage.service_type", exception.getMessage()); } @@ -152,7 +153,7 @@ public void shouldTransformValueForKafka() { Descriptors.Descriptor descriptor = TestBookingLogMessage.getDescriptor(); Descriptors.FieldDescriptor fieldDescriptor = descriptor.findFieldByName("status"); EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); - assertEquals("DRIVER_FOUND", enumProtoHandler.transformFromKafka("DRIVER_FOUND")); + assertEquals("DRIVER_FOUND", enumProtoHandler.transformFromProto("DRIVER_FOUND")); } @Test @@ -162,4 +163,45 @@ public void shouldConvertEnumToJsonString() { assertEquals("DRIVER_FOUND", value); } + + @Test + public void shouldReturnStringEnumValueWhenSimpleGroupIsPassed() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("status"); + String expectedEnum = fieldDescriptor.getEnumType().getValues().get(1).getName(); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("status") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("status", expectedEnum); + EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); + + Object actualEnum = enumProtoHandler.transformFromParquet(simpleGroup); + + assertEquals(expectedEnum, actualEnum); + } + + @Test + public void shouldReturnDefaultEnumStringWhenNullIsPassedToTransformFromParquet() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("status"); + EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); + + Object actualEnumValue = enumProtoHandler.transformFromParquet(null); + + assertEquals("UNKNOWN", actualEnumValue); + } + + @Test + public void shouldReturnDefaultEnumStringWhenEnumValueInsideSimpleGroupIsNotPresentInProtoDefinition() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("status"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("status") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("status", "NON_EXISTENT_ENUM"); + EnumProtoHandler enumProtoHandler = new EnumProtoHandler(fieldDescriptor); + + Object actualEnum = enumProtoHandler.transformFromParquet(simpleGroup); + + assertEquals("UNKNOWN", actualEnum); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandlerTest.java index 1fa10ca38..8769a1daa 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MapProtoHandlerTest.java @@ -11,6 +11,8 @@ import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestComplexMap; import io.odpf.dagger.consumer.TestMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Assert; import org.junit.Test; @@ -20,9 +22,7 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; public class MapProtoHandlerTest { @@ -48,7 +48,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfCannotHandle() { MapProtoHandler mapProtoHandler = new MapProtoHandler(otherFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(otherFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformForKafka(builder, "123"); + DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformToProtoBuilder(builder, "123"); assertEquals("", returnedBuilder.getField(otherFieldDescriptor)); } @@ -58,7 +58,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfNullPassed() { MapProtoHandler mapProtoHandler = new MapProtoHandler(mapFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(mapFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformForKafka(builder, null); + DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformToProtoBuilder(builder, null); List entries = (List) returnedBuilder.getField(mapFieldDescriptor); assertEquals(0, entries.size()); } @@ -73,7 +73,7 @@ public void shouldSetMapFieldIfStringMapPassed() { inputMap.put("a", "123"); inputMap.put("b", "456"); - DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformForKafka(builder, inputMap); + DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformToProtoBuilder(builder, inputMap); List entries = (List) returnedBuilder.getField(mapFieldDescriptor); assertEquals(2, entries.size()); @@ -103,7 +103,7 @@ public void shouldSetMapFieldIfArrayofObjectsHavingRowsWithStringFieldsPassed() inputRows.add(inputRow1); inputRows.add(inputRow2); - DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformForKafka(builder, inputRows.toArray()); + DynamicMessage.Builder returnedBuilder = mapProtoHandler.transformToProtoBuilder(builder, inputRows.toArray()); List entries = (List) returnedBuilder.getField(mapFieldDescriptor); assertEquals(2, entries.size()); @@ -124,7 +124,7 @@ public void shouldThrowExceptionIfRowsPassedAreNotOfArityTwo() { Row inputRow = new Row(3); inputRows.add(inputRow); IllegalArgumentException exception = Assert.assertThrows(IllegalArgumentException.class, - () -> mapProtoHandler.transformForKafka(builder, inputRows.toArray())); + () -> mapProtoHandler.transformToProtoBuilder(builder, inputRows.toArray())); assertEquals("Row: +I[null, null, null] of size: 3 cannot be converted to map", exception.getMessage()); } @@ -182,7 +182,7 @@ public void shouldReturnArrayOfRowHavingSameSizeAsInputMapForTransformForKafka() DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestBookingLogMessage.getDescriptor(), driverProfileFlattenLogMessage.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(2, outputValues.size()); } @@ -201,7 +201,7 @@ public void shouldReturnArrayOfRowHavingFieldsSetAsInputMapAndOfSizeTwoForTransf DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestBookingLogMessage.getDescriptor(), driverProfileFlattenLogMessage.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals("a", ((Row) outputValues.get(0)).getField(0)); assertEquals("123", ((Row) outputValues.get(0)).getField(1)); @@ -222,7 +222,7 @@ public void shouldReturnArrayOfRowHavingSameSizeAsInputMapHavingComplexDataField DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(2, outputValues.size()); } @@ -238,7 +238,7 @@ public void shouldReturnArrayOfRowsHavingFieldsSetAsInputMapHavingComplexDataFie DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(1, ((Row) outputValues.get(0)).getField(0)); assertEquals("123", ((Row) ((Row) outputValues.get(0)).getField(1)).getField(0)); @@ -262,7 +262,7 @@ public void shouldReturnArrayOfRowsHavingFieldsSetAsInputMapHavingComplexDataFie DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(0, ((Row) outputValues.get(0)).getField(0)); assertEquals("123", ((Row) ((Row) outputValues.get(0)).getField(1)).getField(0)); @@ -281,7 +281,7 @@ public void shouldReturnArrayOfRowsHavingFieldsSetAsInputMapHavingComplexDataFie DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(1, ((Row) outputValues.get(0)).getField(0)); assertEquals("", ((Row) ((Row) outputValues.get(0)).getField(1)).getField(0)); @@ -300,7 +300,7 @@ public void shouldReturnArrayOfRowsHavingFieldsSetAsInputMapHavingComplexDataFie DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(0, ((Row) outputValues.get(0)).getField(0)); assertEquals("", ((Row) ((Row) outputValues.get(0)).getField(1)).getField(0)); @@ -319,7 +319,7 @@ public void shouldReturnArrayOfRowsHavingFieldsSetAsInputMapHavingComplexDataFie DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestComplexMap.getDescriptor(), testComplexMap.toByteArray()); - List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromKafka(dynamicMessage.getField(mapFieldDescriptor))); + List outputValues = Arrays.asList((Object[]) mapProtoHandler.transformFromProto(dynamicMessage.getField(mapFieldDescriptor))); assertEquals(0, ((Row) outputValues.get(0)).getField(0)); assertEquals("", ((Row) ((Row) outputValues.get(0)).getField(1)).getField(0)); @@ -344,4 +344,14 @@ public void shouldReturnTypeInformation() { assertEquals(Types.OBJECT_ARRAY(Types.ROW_NAMED(new String[]{"key", "value"}, Types.STRING, Types.STRING)), mapProtoHandler.getTypeInformation()); } + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("metadata"); + MapProtoHandler protoHandler = new MapProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandlerTest.java index 5ffc9ea75..d6d2c6eeb 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/MessageProtoHandlerTest.java @@ -9,13 +9,13 @@ import com.google.protobuf.InvalidProtocolBufferException; import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestPaymentOptionMetadata; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.HashMap; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; public class MessageProtoHandlerTest { @@ -41,8 +41,8 @@ public void shouldReturnTheSameBuilderWithoutSettingFieldIfCanNotHandle() { MessageProtoHandler messsageProtoHandler = new MessageProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - assertEquals(builder, messsageProtoHandler.transformForKafka(builder, 123)); - assertEquals("", messsageProtoHandler.transformForKafka(builder, 123).getField(fieldDescriptor)); + assertEquals(builder, messsageProtoHandler.transformToProtoBuilder(builder, 123)); + assertEquals("", messsageProtoHandler.transformToProtoBuilder(builder, 123).getField(fieldDescriptor)); } @Test @@ -51,7 +51,7 @@ public void shouldReturnTheSameBuilderWithoutSettingFieldIfNullPassed() { MessageProtoHandler messsageProtoHandler = new MessageProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - DynamicMessage.Builder outputBuilder = messsageProtoHandler.transformForKafka(builder, null); + DynamicMessage.Builder outputBuilder = messsageProtoHandler.transformToProtoBuilder(builder, null); assertEquals(builder, outputBuilder); assertEquals("", outputBuilder.getField(fieldDescriptor)); } @@ -65,7 +65,7 @@ public void shouldSetTheFieldsPassedInTheBuilderForMessageFieldTypeDescriptorIfA Row inputRow = new Row(2); inputRow.setField(0, "test1"); inputRow.setField(1, "test2"); - DynamicMessage.Builder returnedBuilder = messageProtoHandler.transformForKafka(builder, inputRow); + DynamicMessage.Builder returnedBuilder = messageProtoHandler.transformToProtoBuilder(builder, inputRow); TestPaymentOptionMetadata returnedValue = TestPaymentOptionMetadata.parseFrom(((DynamicMessage) returnedBuilder.getField(messageFieldDescriptor)).toByteArray()); @@ -82,7 +82,7 @@ public void shouldSetTheFieldsPassedInTheBuilderForMessageFieldTypeDescriptorIfA Row inputRow = new Row(1); inputRow.setField(0, "test1"); - DynamicMessage.Builder returnedBuilder = messageProtoHandler.transformForKafka(builder, inputRow); + DynamicMessage.Builder returnedBuilder = messageProtoHandler.transformToProtoBuilder(builder, inputRow); TestPaymentOptionMetadata returnedValue = TestPaymentOptionMetadata.parseFrom(((DynamicMessage) returnedBuilder.getField(messageFieldDescriptor)).toByteArray()); @@ -147,7 +147,7 @@ public void shouldReturnRowGivenAMapForFieldDescriptorOfTypeMessageIfAllValueAre Descriptors.Descriptor descriptor = TestBookingLogMessage.getDescriptor(); Descriptors.FieldDescriptor fieldDescriptor = descriptor.findFieldByName("payment_option_metadata"); - Row value = (Row) new MessageProtoHandler(fieldDescriptor).transformFromKafka(dynamicMessage.getField(fieldDescriptor)); + Row value = (Row) new MessageProtoHandler(fieldDescriptor).transformFromProto(dynamicMessage.getField(fieldDescriptor)); assertEquals("test1", value.getField(0)); assertEquals("test2", value.getField(1)); @@ -165,7 +165,7 @@ public void shouldReturnRowGivenAMapForFieldDescriptorOfTypeMessageIfAllValueAre Descriptors.Descriptor descriptor = TestBookingLogMessage.getDescriptor(); Descriptors.FieldDescriptor fieldDescriptor = descriptor.findFieldByName("payment_option_metadata"); - Row value = (Row) new MessageProtoHandler(fieldDescriptor).transformFromKafka(dynamicMessage.getField(fieldDescriptor)); + Row value = (Row) new MessageProtoHandler(fieldDescriptor).transformFromProto(dynamicMessage.getField(fieldDescriptor)); assertEquals("test1", value.getField(0)); assertEquals("", value.getField(1)); @@ -193,4 +193,15 @@ public void shouldConvertComplexRowDataToJsonString() { assertEquals("{\"masked_card\":\"test1\",\"network\":\"test2\"}", String.valueOf(value)); } + + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("payment_option_metadata"); + MessageProtoHandler protoHandler = new MessageProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandlerTest.java index 61d9239d9..017c119f5 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/PrimitiveProtoHandlerTest.java @@ -1,16 +1,21 @@ package io.odpf.dagger.common.serde.proto.protohandler; +import com.google.protobuf.ByteString; +import io.odpf.dagger.consumer.TestMessageEnvelope; import org.apache.flink.api.common.typeinfo.Types; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import io.odpf.dagger.common.exceptions.serde.InvalidDataTypeException; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.junit.Assert.*; public class PrimitiveProtoHandlerTest { @@ -28,7 +33,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfNullFieldIsPassed() { PrimitiveProtoHandler primitiveProtoHandler = new PrimitiveProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = primitiveProtoHandler.transformForKafka(builder, null); + DynamicMessage.Builder returnedBuilder = primitiveProtoHandler.transformToProtoBuilder(builder, null); assertEquals("", returnedBuilder.getField(fieldDescriptor)); } @@ -38,7 +43,7 @@ public void shouldSetFieldPassedInTheBuilder() { PrimitiveProtoHandler primitiveProtoHandler = new PrimitiveProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = primitiveProtoHandler.transformForKafka(builder, "123"); + DynamicMessage.Builder returnedBuilder = primitiveProtoHandler.transformToProtoBuilder(builder, "123"); assertEquals("123", returnedBuilder.getField(fieldDescriptor)); } @@ -89,13 +94,13 @@ public void shouldThrowInvalidDataTypeExceptionInCaseOfTypeMismatchForPostProces } @Test - public void shouldReturnSameValueForTransformForKafka() { + public void shouldReturnSameValueForTransformFromKafka() { Descriptors.Descriptor descriptor = TestBookingLogMessage.getDescriptor(); Descriptors.FieldDescriptor stringFieldDescriptor = descriptor.findFieldByName("order_number"); PrimitiveProtoHandler primitiveProtoHandler = new PrimitiveProtoHandler(stringFieldDescriptor); - assertEquals(123, primitiveProtoHandler.transformFromKafka(123)); - assertEquals("123", primitiveProtoHandler.transformFromKafka("123")); + assertEquals(123, primitiveProtoHandler.transformFromProto(123)); + assertEquals("123", primitiveProtoHandler.transformFromProto("123")); } @Test @@ -113,4 +118,21 @@ public void shouldConvertPrimitiveStringToJsonString() { assertEquals("123", value); } + + @Test + public void shouldReturnParsedValueForTransformFromParquet() { + Descriptors.FieldDescriptor fieldDescriptor = TestMessageEnvelope.getDescriptor().findFieldByName("log_key"); + String testString = "test-string"; + ByteString expectedByteString = ByteString.copyFrom(testString.getBytes()); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("log_key") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("log_key", Binary.fromConstantByteArray(expectedByteString.toByteArray())); + PrimitiveProtoHandler primitiveProtoHandler = new PrimitiveProtoHandler(fieldDescriptor); + + ByteString actualByteString = (ByteString) primitiveProtoHandler.transformFromParquet(simpleGroup); + + assertEquals(expectedByteString, actualByteString); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandlerTest.java index e5c673b02..d159f2a45 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedEnumProtoHandlerTest.java @@ -10,14 +10,14 @@ import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestEnumMessage; import io.odpf.dagger.consumer.TestRepeatedEnumMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Collections; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; public class RepeatedEnumProtoHandlerTest { @Test @@ -50,7 +50,7 @@ public void shouldReturnTheSameBuilderWithoutSettingAnyValue() { RepeatedEnumProtoHandler repeatedEnumProtoHandler = new RepeatedEnumProtoHandler(repeatedEnumFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(repeatedEnumFieldDescriptor.getContainingType()); - assertEquals(Collections.EMPTY_LIST, repeatedEnumProtoHandler.transformForKafka(builder, 123).getField(repeatedEnumFieldDescriptor)); + assertEquals(Collections.EMPTY_LIST, repeatedEnumProtoHandler.transformToProtoBuilder(builder, 123).getField(repeatedEnumFieldDescriptor)); } @Test @@ -95,7 +95,7 @@ public void shouldTransformValueForKafkaAsStringArray() throws InvalidProtocolBu Descriptors.FieldDescriptor repeatedEnumFieldDescriptor = TestRepeatedEnumMessage.getDescriptor().findFieldByName("test_enums"); RepeatedEnumProtoHandler repeatedEnumProtoHandler = new RepeatedEnumProtoHandler(repeatedEnumFieldDescriptor); - String[] outputValues = (String[]) repeatedEnumProtoHandler.transformFromKafka(dynamicMessage.getField(repeatedEnumFieldDescriptor)); + String[] outputValues = (String[]) repeatedEnumProtoHandler.transformFromProto(dynamicMessage.getField(repeatedEnumFieldDescriptor)); assertEquals("UNKNOWN", outputValues[0]); } @@ -105,8 +105,19 @@ public void shouldTransformValueForKafkaAsEmptyStringArrayForNull() { Descriptors.FieldDescriptor repeatedEnumFieldDescriptor = TestRepeatedEnumMessage.getDescriptor().findFieldByName("test_enums"); RepeatedEnumProtoHandler repeatedEnumProtoHandler = new RepeatedEnumProtoHandler(repeatedEnumFieldDescriptor); - String[] outputValues = (String[]) repeatedEnumProtoHandler.transformFromKafka(null); + String[] outputValues = (String[]) repeatedEnumProtoHandler.transformFromProto(null); assertEquals(0, outputValues.length); } + + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("test_enums"); + RepeatedEnumProtoHandler protoHandler = new RepeatedEnumProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandlerTest.java index 765534ccf..9aab6d6bf 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedMessageProtoHandlerTest.java @@ -10,6 +10,8 @@ import io.odpf.dagger.consumer.TestFeedbackLogMessage; import io.odpf.dagger.consumer.TestReason; import net.minidev.json.JSONArray; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; @@ -19,9 +21,7 @@ import static org.apache.flink.api.common.typeinfo.Types.OBJECT_ARRAY; import static org.apache.flink.api.common.typeinfo.Types.ROW_NAMED; import static org.apache.flink.api.common.typeinfo.Types.STRING; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; public class RepeatedMessageProtoHandlerTest { @@ -47,8 +47,8 @@ public void shouldReturnTheSameBuilderWithoutSettingFieldIfCanNotHandle() { RepeatedMessageProtoHandler repeatedMesssageProtoHandler = new RepeatedMessageProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - assertEquals(builder, repeatedMesssageProtoHandler.transformForKafka(builder, 123)); - assertEquals("", repeatedMesssageProtoHandler.transformForKafka(builder, 123).getField(fieldDescriptor)); + assertEquals(builder, repeatedMesssageProtoHandler.transformToProtoBuilder(builder, 123)); + assertEquals("", repeatedMesssageProtoHandler.transformToProtoBuilder(builder, 123).getField(fieldDescriptor)); } @Test @@ -57,7 +57,7 @@ public void shouldReturnTheSameBuilderWithoutSettingFieldIfNullPassed() { RepeatedMessageProtoHandler repeatedMesssageProtoHandler = new RepeatedMessageProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); - DynamicMessage.Builder outputBuilder = repeatedMesssageProtoHandler.transformForKafka(builder, null); + DynamicMessage.Builder outputBuilder = repeatedMesssageProtoHandler.transformToProtoBuilder(builder, null); assertEquals(builder, outputBuilder); assertEquals("", outputBuilder.getField(fieldDescriptor)); } @@ -80,7 +80,7 @@ public void shouldSetTheFieldsPassedInTheBuilderForRepeatedMessageFieldTypeDescr inputRows.add(inputRow1); inputRows.add(inputRow2); - DynamicMessage.Builder returnedBuilder = repeatedMessageProtoHandler.transformForKafka(builder, inputRows.toArray()); + DynamicMessage.Builder returnedBuilder = repeatedMessageProtoHandler.transformToProtoBuilder(builder, inputRows.toArray()); List reasons = (List) returnedBuilder.getField(repeatedMessageFieldDescriptor); @@ -112,7 +112,7 @@ public void shouldSetTheFieldsPassedInTheBuilderForRepeatedMessageFieldTypeDescr inputRows.add(inputRow1); inputRows.add(inputRow2); - DynamicMessage.Builder returnedBuilder = repeatedMesssageProtoHandler.transformForKafka(builder, inputRows); + DynamicMessage.Builder returnedBuilder = repeatedMesssageProtoHandler.transformToProtoBuilder(builder, inputRows); List reasons = (List) returnedBuilder.getField(repeatedMessageFieldDescriptor); @@ -144,7 +144,7 @@ public void shouldSetTheFieldsNotPassedInTheBuilderForRepeatedMessageFieldTypeDe inputRows.add(inputRow1); inputRows.add(inputRow2); - DynamicMessage.Builder returnedBuilder = repeatedMesssageProtoHandler.transformForKafka(builder, inputRows); + DynamicMessage.Builder returnedBuilder = repeatedMesssageProtoHandler.transformToProtoBuilder(builder, inputRows); List reasons = (List) returnedBuilder.getField(repeatedMessageFieldDescriptor); @@ -240,7 +240,7 @@ public void shouldReturnArrayOfRowsGivenAListForFieldDescriptorOfTypeRepeatedMes public void shouldReturnEmptyArrayOfRowsIfNullPassedForKafkaTransform() { Descriptors.FieldDescriptor repeatedMessageFieldDescriptor = TestFeedbackLogMessage.getDescriptor().findFieldByName("reason"); - Object[] values = (Object[]) new RepeatedMessageProtoHandler(repeatedMessageFieldDescriptor).transformFromKafka(null); + Object[] values = (Object[]) new RepeatedMessageProtoHandler(repeatedMessageFieldDescriptor).transformFromProto(null); assertEquals(0, values.length); } @@ -256,7 +256,7 @@ public void shouldReturnArrayOfRowsGivenAListForFieldDescriptorOfTypeRepeatedMes Descriptors.FieldDescriptor repeatedMessageFieldDescriptor = TestFeedbackLogMessage.getDescriptor().findFieldByName("reason"); - Object[] values = (Object[]) new RepeatedMessageProtoHandler(repeatedMessageFieldDescriptor).transformFromKafka(dynamicMessage.getField(repeatedMessageFieldDescriptor)); + Object[] values = (Object[]) new RepeatedMessageProtoHandler(repeatedMessageFieldDescriptor).transformFromProto(dynamicMessage.getField(repeatedMessageFieldDescriptor)); assertEquals(repeatedMessageFieldDescriptor.getMessageType().getFields().size(), ((Row) values[0]).getArity()); assertEquals(repeatedMessageFieldDescriptor.getMessageType().getFields().size(), ((Row) values[1]).getArity()); @@ -296,4 +296,14 @@ public void shouldConvertRepeatedComplexRowDataToJsonString() { assertEquals("[{\"reason_id\":\"reason1\",\"group_id\":\"group1\"}, {\"reason_id\":\"reason2\",\"group_id\":\"group2\"}]", String.valueOf(value)); } + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("reason"); + RepeatedMessageProtoHandler protoHandler = new RepeatedMessageProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandlerTest.java index 31f1e5c72..e69550994 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedPrimitiveProtoHandlerTest.java @@ -11,6 +11,8 @@ import io.odpf.dagger.common.exceptions.serde.InvalidDataTypeException; import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestRepeatedEnumMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Assert; import org.junit.Test; @@ -18,10 +20,7 @@ import java.util.List; import static java.util.Arrays.asList; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; public class RepeatedPrimitiveProtoHandlerTest { @@ -63,7 +62,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfCannotHandle() { RepeatedPrimitiveProtoHandler repeatedPrimitiveProtoHandler = new RepeatedPrimitiveProtoHandler(otherFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(otherFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformForKafka(builder, "123"); + DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformToProtoBuilder(builder, "123"); assertEquals("", returnedBuilder.getField(otherFieldDescriptor)); } @@ -73,7 +72,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfNullFieldIsPassed() { RepeatedPrimitiveProtoHandler repeatedPrimitiveProtoHandler = new RepeatedPrimitiveProtoHandler(repeatedFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(repeatedFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformForKafka(builder, null); + DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformToProtoBuilder(builder, null); List outputValues = (List) returnedBuilder.getField(repeatedFieldDescriptor); assertEquals(0, outputValues.size()); } @@ -86,7 +85,7 @@ public void shouldSetEmptyListInBuilderIfEmptyListIfPassed() { ArrayList inputValues = new ArrayList<>(); - DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformForKafka(builder, inputValues); + DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformToProtoBuilder(builder, inputValues); List outputValues = (List) returnedBuilder.getField(repeatedFieldDescriptor); assertEquals(0, outputValues.size()); } @@ -101,7 +100,7 @@ public void shouldSetFieldPassedInTheBuilderAsAList() { inputValues.add("test1"); inputValues.add("test2"); - DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformForKafka(builder, inputValues); + DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformToProtoBuilder(builder, inputValues); List outputValues = (List) returnedBuilder.getField(repeatedFieldDescriptor); assertEquals(asList("test1", "test2"), outputValues); } @@ -116,7 +115,7 @@ public void shouldSetFieldPassedInTheBuilderAsArray() { inputValues.add("test1"); inputValues.add("test2"); - DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformForKafka(builder, inputValues.toArray()); + DynamicMessage.Builder returnedBuilder = repeatedPrimitiveProtoHandler.transformToProtoBuilder(builder, inputValues.toArray()); List outputValues = (List) returnedBuilder.getField(repeatedFieldDescriptor); assertEquals(asList("test1", "test2"), outputValues); } @@ -206,7 +205,7 @@ public void shouldReturnAllFieldsInAListOfObjectsIfMultipleFieldsPassedWithSameT .build(); DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestBookingLogMessage.getDescriptor(), goLifeBookingLogMessage.toByteArray()); - String[] outputValues = (String[]) repeatedPrimitiveProtoHandler.transformFromKafka(dynamicMessage.getField(repeatedFieldDescriptor)); + String[] outputValues = (String[]) repeatedPrimitiveProtoHandler.transformFromProto(dynamicMessage.getField(repeatedFieldDescriptor)); assertArrayEquals(new String[]{"1", "2", "3"}, outputValues); } @@ -216,7 +215,7 @@ public void shouldThrowUnsupportedDataTypeExceptionInCaseOfInCaseOfEnumForKafkaT Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("status"); RepeatedPrimitiveProtoHandler repeatedPrimitiveProtoHandler = new RepeatedPrimitiveProtoHandler(fieldDescriptor); DataTypeNotSupportedException exception = Assert.assertThrows(DataTypeNotSupportedException.class, - () -> repeatedPrimitiveProtoHandler.transformFromKafka("CREATED")); + () -> repeatedPrimitiveProtoHandler.transformFromProto("CREATED")); assertEquals("Data type ENUM not supported in primitive type handlers", exception.getMessage()); } @@ -239,4 +238,15 @@ public void shouldConvertRepeatedRowDataToJsonString() { Object value = new RepeatedPrimitiveProtoHandler(repeatedFieldDescriptor).transformToJson(inputValues); assertEquals("[\"test1\",\"test2\"]", String.valueOf(value)); } + + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("meta_array"); + RepeatedPrimitiveProtoHandler protoHandler = new RepeatedPrimitiveProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandlerTest.java index 7e1ada3f6..a45982726 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RepeatedStructMessageProtoHandlerTest.java @@ -8,6 +8,8 @@ import com.google.protobuf.DynamicMessage; import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestNestedRepeatedMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.Collections; @@ -45,7 +47,7 @@ public void shouldReturnTheSameBuilderWithoutSettingAnyValue() { Descriptors.FieldDescriptor repeatedStructFieldDescriptor = TestNestedRepeatedMessage.getDescriptor().findFieldByName("metadata"); RepeatedStructMessageProtoHandler repeatedStructMessageProtoHandler = new RepeatedStructMessageProtoHandler(repeatedStructFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(repeatedStructFieldDescriptor.getContainingType()); - assertEquals(Collections.EMPTY_LIST, repeatedStructMessageProtoHandler.transformForKafka(builder, 123).getField(repeatedStructFieldDescriptor)); + assertEquals(Collections.EMPTY_LIST, repeatedStructMessageProtoHandler.transformToProtoBuilder(builder, 123).getField(repeatedStructFieldDescriptor)); } @Test @@ -59,7 +61,7 @@ public void shouldReturnNullForTransformForPostProcessor() { public void shouldReturnNullForTransformForKafka() { Descriptors.FieldDescriptor repeatedStructFieldDescriptor = TestNestedRepeatedMessage.getDescriptor().findFieldByName("metadata"); RepeatedStructMessageProtoHandler repeatedStructMessageProtoHandler = new RepeatedStructMessageProtoHandler(repeatedStructFieldDescriptor); - assertNull(repeatedStructMessageProtoHandler.transformFromKafka("test")); + assertNull(repeatedStructMessageProtoHandler.transformFromProto("test")); } @Test @@ -71,4 +73,15 @@ public void shouldReturnTypeInformation() { assertEquals(expectedTypeInformation, actualTypeInformation); } + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("metadata"); + RepeatedStructMessageProtoHandler protoHandler = new RepeatedStructMessageProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } + } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactoryTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactoryTest.java index 4eaa9f5cd..1a15d11bb 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactoryTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/RowFactoryTest.java @@ -1,5 +1,7 @@ package io.odpf.dagger.common.serde.proto.protohandler; +import io.odpf.dagger.consumer.TestPrimitiveMessage; +import io.odpf.dagger.consumer.TestReason; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -10,14 +12,19 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; import org.junit.Test; +import java.time.Instant; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.junit.Assert.*; public class RowFactoryTest { @@ -119,4 +126,92 @@ public void shouldBeAbleToCreateAValidCopyOfTheRowCreated() throws InvalidProtoc assertEquals(copy.toString(), row.toString()); } + @Test + public void shouldCreateRowWithPositionIndexingFromSimpleGroup() { + Descriptors.Descriptor descriptor = TestPrimitiveMessage.getDescriptor(); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BOOLEAN).named("is_valid") + .required(BINARY).named("order_number") + .required(BINARY).named("order_hash") + .required(DOUBLE).named("latitude") + .required(DOUBLE).named("longitude") + .required(FLOAT).named("price") + .required(INT32).named("packet_count") + .required(INT64).named("phone") + .required(INT64).named("event_timestamp") + .required(BINARY).named("service_type") + .named("TestGroupType"); + + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("is_valid", true); + simpleGroup.add("order_number", "ORDER_1322432"); + String byteString = "g362vxv3ydg73g2ss"; + simpleGroup.add("order_hash", Binary.fromConstantByteArray(byteString.getBytes())); + simpleGroup.add("latitude", Double.MAX_VALUE); + simpleGroup.add("longitude", Double.MIN_VALUE); + simpleGroup.add("price", Float.MAX_VALUE); + simpleGroup.add("packet_count", Integer.MAX_VALUE); + simpleGroup.add("phone", Long.MAX_VALUE); + long currentTimeInMillis = Instant.now().toEpochMilli(); + long seconds = Instant.ofEpochMilli(currentTimeInMillis).getEpochSecond(); + int nanos = Instant.ofEpochMilli(currentTimeInMillis).getNano(); + simpleGroup.add("event_timestamp", currentTimeInMillis); + simpleGroup.add("service_type", "GO_RIDE"); + + Row row = RowFactory.createRow(descriptor, simpleGroup); + + assertNotNull(row); + assertEquals(10, row.getArity()); + assertEquals(true, row.getField(0)); + assertEquals(Double.MIN_VALUE, row.getField(4)); + Row actualTimestampRow = (Row) row.getField(8); + assertNotNull(actualTimestampRow); + assertEquals(seconds, actualTimestampRow.getField(0)); + assertEquals(nanos, actualTimestampRow.getField(1)); + } + + @Test + public void shouldCreateRowFromSimpleGroupWithExtraFieldsSetToNull() { + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("reason_id") + .required(BINARY).named("group_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("reason_id", "some reason id"); + simpleGroup.add("group_id", "some group id"); + + Row actualRow = RowFactory.createRow(TestReason.getDescriptor(), simpleGroup, 2); + + assertEquals(4, actualRow.getArity()); + assertNull(actualRow.getField(2)); + assertNull(actualRow.getField(3)); + } + + @Test + public void shouldBeAbleToCreateAValidCopyOfTheRowCreatedFromSimpleGroup() { + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("reason_id") + .required(BINARY).named("group_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("reason_id", "some reason id"); + simpleGroup.add("group_id", "some group id"); + + Row actualRow = RowFactory.createRow(TestReason.getDescriptor(), simpleGroup); + + ArrayList> typeInformations = new ArrayList<>(); + ExecutionConfig config = new ExecutionConfig(); + TestReason.getDescriptor().getFields().forEach(fieldDescriptor -> { + typeInformations.add(ProtoHandlerFactory.getProtoHandler(fieldDescriptor).getTypeInformation()); + }); + ArrayList> typeSerializers = new ArrayList<>(); + typeInformations.forEach(rowTypeInformation -> { + typeSerializers.add(rowTypeInformation.createSerializer(config)); + }); + RowSerializer rowSerializer = new RowSerializer(typeSerializers.toArray(new TypeSerializer[0])); + + Row copy = rowSerializer.copy(actualRow); + + assertEquals(copy.toString(), actualRow.toString()); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandlerTest.java index 4c78d247a..267462cd0 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/StructMessageProtoHandlerTest.java @@ -8,6 +8,8 @@ import com.google.protobuf.DynamicMessage; import io.odpf.dagger.consumer.TestBookingLogMessage; import io.odpf.dagger.consumer.TestRepeatedEnumMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -44,7 +46,7 @@ public void shouldReturnTheSameBuilderWithoutSettingAnyValue() { StructMessageProtoHandler structMessageProtoHandler = new StructMessageProtoHandler(fieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(fieldDescriptor.getContainingType()); assertEquals(DynamicMessage.getDefaultInstance(fieldDescriptor.getContainingType()).getAllFields().size(), - ((DynamicMessage) structMessageProtoHandler.transformForKafka(builder, 123).getField(fieldDescriptor)).getAllFields().size()); + ((DynamicMessage) structMessageProtoHandler.transformToProtoBuilder(builder, 123).getField(fieldDescriptor)).getAllFields().size()); } @Test @@ -58,7 +60,7 @@ public void shouldReturnNullForTransformForPostProcessor() { public void shouldReturnNullForTransformForKafka() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("profile_data"); StructMessageProtoHandler structMessageProtoHandler = new StructMessageProtoHandler(fieldDescriptor); - assertNull(structMessageProtoHandler.transformFromKafka("test")); + assertNull(structMessageProtoHandler.transformFromProto("test")); } @Test @@ -70,4 +72,14 @@ public void shouldReturnTypeInformation() { assertEquals(expectedTypeInformation, actualTypeInformation); } + @Test + public void shouldReturnNullWhenTransformFromParquetIsCalledWithAnyArgument() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("profile_data"); + StructMessageProtoHandler protoHandler = new StructMessageProtoHandler(fieldDescriptor); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + assertNull(protoHandler.transformFromParquet(simpleGroup)); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandlerTest.java index 06a8bb4e1..2dab74b0b 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/TimestampProtoHandlerTest.java @@ -8,6 +8,8 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.sql.Timestamp; @@ -15,10 +17,8 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.junit.Assert.*; public class TimestampProtoHandlerTest { @Test @@ -43,7 +43,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfCannotHandle() { TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(otherFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(otherFieldDescriptor.getContainingType()); - DynamicMessage.Builder returnedBuilder = timestampProtoHandler.transformForKafka(builder, "123"); + DynamicMessage.Builder returnedBuilder = timestampProtoHandler.transformToProtoBuilder(builder, "123"); assertEquals("", returnedBuilder.getField(otherFieldDescriptor)); } @@ -53,7 +53,7 @@ public void shouldReturnSameBuilderWithoutSettingFieldIfNullFieldIsPassed() thro TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(timestampFieldDescriptor); DynamicMessage.Builder builder = DynamicMessage.newBuilder(timestampFieldDescriptor.getContainingType()); - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, null).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, null).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(0L, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -69,7 +69,7 @@ public void shouldSetTimestampIfInstanceOfJavaSqlTimestampPassed() throws Invali long milliSeconds = System.currentTimeMillis(); Timestamp inputTimestamp = new Timestamp(milliSeconds); - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, inputTimestamp).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, inputTimestamp).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(milliSeconds / 1000, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -87,7 +87,7 @@ public void shouldSetTimestampIfInstanceOfLocalDateTimePassed() throws InvalidPr Timestamp inputTimestamp = new Timestamp(milliSeconds); LocalDateTime localDateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(milliSeconds), ZoneOffset.UTC); - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, localDateTime).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, localDateTime).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(milliSeconds / 1000, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -103,7 +103,7 @@ public void shouldSetTimestampIfRowHavingTimestampIsPassed() throws InvalidProto int nanos = (int) (System.currentTimeMillis() * 1000000); Row inputRow = Row.of(seconds, nanos); - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, inputRow).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, inputRow).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(seconds, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -119,7 +119,7 @@ public void shouldThrowExceptionIfRowOfArityOtherThanTwoIsPassed() { Row inputRow = new Row(3); try { - timestampProtoHandler.transformForKafka(builder, inputRow).build(); + timestampProtoHandler.transformToProtoBuilder(builder, inputRow).build(); } catch (Exception e) { assertEquals(IllegalArgumentException.class, e.getClass()); assertEquals("Row: +I[null, null, null] of size: 3 cannot be converted to timestamp", e.getMessage()); @@ -134,7 +134,7 @@ public void shouldSetTimestampIfInstanceOfNumberPassed() throws InvalidProtocolB long seconds = System.currentTimeMillis() / 1000; - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, seconds).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, seconds).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(seconds, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -149,7 +149,7 @@ public void shouldSetTimestampIfInstanceOfInstantPassed() throws InvalidProtocol Instant instant = Instant.now(); - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, instant).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, instant).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(instant.getEpochSecond(), bookingLogMessage.getEventTimestamp().getSeconds()); @@ -164,7 +164,7 @@ public void shouldSetTimestampIfInstanceOfStringPassed() throws InvalidProtocolB String inputTimestamp = "2019-03-28T05:50:13Z"; - DynamicMessage dynamicMessage = timestampProtoHandler.transformForKafka(builder, inputTimestamp).build(); + DynamicMessage dynamicMessage = timestampProtoHandler.transformToProtoBuilder(builder, inputTimestamp).build(); TestBookingLogMessage bookingLogMessage = TestBookingLogMessage.parseFrom(dynamicMessage.toByteArray()); assertEquals(1553752213, bookingLogMessage.getEventTimestamp().getSeconds()); @@ -227,7 +227,7 @@ public void shouldTransformTimestampForDynamicMessageForKafka() throws InvalidPr .build(); DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestBookingLogMessage.getDescriptor(), bookingLogMessage.toByteArray()); TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); - Row row = (Row) timestampProtoHandler.transformFromKafka(dynamicMessage.getField(fieldDescriptor)); + Row row = (Row) timestampProtoHandler.transformFromProto(dynamicMessage.getField(fieldDescriptor)); assertEquals(Row.of(10L, 10), row); } @@ -240,7 +240,7 @@ public void shouldSetDefaultValueForDynamicMessageForKafkaIfValuesNotSet() throw .build(); DynamicMessage dynamicMessage = DynamicMessage.parseFrom(TestBookingLogMessage.getDescriptor(), bookingLogMessage.toByteArray()); TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); - Row row = (Row) timestampProtoHandler.transformFromKafka(dynamicMessage.getField(fieldDescriptor)); + Row row = (Row) timestampProtoHandler.transformFromProto(dynamicMessage.getField(fieldDescriptor)); assertEquals(Row.of(0L, 0), row); } @@ -256,4 +256,64 @@ public void shouldConvertTimestampToJsonString() { assertEquals("2020-09-14 11:43:48", String.valueOf(value)); } + + @Test + public void shouldTransformEpochInMillisFromSimpleGroup() { + long sampleTimeInMillis = Instant.now().toEpochMilli(); + Instant instant = Instant.ofEpochMilli(sampleTimeInMillis); + Row expectedRow = Row.of(instant.getEpochSecond(), instant.getNano()); + + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("event_timestamp"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("event_timestamp") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("event_timestamp", sampleTimeInMillis); + + TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); + Row actualRow = (Row) timestampProtoHandler.transformFromParquet(simpleGroup); + + assertEquals(expectedRow, actualRow); + } + + @Test + public void shouldReturnDefaultTimestampRowDuringTransformIfNullIsPassedToTransformFromParquet() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("event_timestamp"); + TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); + + Row actualRow = (Row) timestampProtoHandler.transformFromParquet(null); + + Row expectedRow = Row.of(0L, 0); + assertEquals(expectedRow, actualRow); + } + + @Test + public void shouldReturnDefaultTimestampRowDuringTransformIfSimpleGroupDoesNotContainField() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("event_timestamp"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); + Row actualRow = (Row) timestampProtoHandler.transformFromParquet(simpleGroup); + + Row expectedRow = Row.of(0L, 0); + assertEquals(expectedRow, actualRow); + } + + @Test + public void shouldReturnDefaultTimestampRowDuringTransformIfSimpleGroupDoesNotContainValueForField() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("event_timestamp"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("event_timestamp") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + + TimestampProtoHandler timestampProtoHandler = new TimestampProtoHandler(fieldDescriptor); + Row actualRow = (Row) timestampProtoHandler.transformFromParquet(simpleGroup); + + Row expectedRow = Row.of(0L, 0); + assertEquals(expectedRow, actualRow); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandlerTest.java index b4b064abb..2786eac82 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/BooleanPrimitiveTypeHandlerTest.java @@ -4,18 +4,20 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class BooleanPrimitiveTypeHandlerTest { - @Test public void shouldHandleBooleanTypes() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); @@ -36,7 +38,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeBool() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); BooleanPrimitiveTypeHandler booleanPrimitiveTypeHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); - Object value = booleanPrimitiveTypeHandler.getValue(actualValue); + Object value = booleanPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -48,7 +50,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeBool() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); BooleanPrimitiveTypeHandler booleanPrimitiveTypeHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); - Object value = booleanPrimitiveTypeHandler.getValue(String.valueOf(actualValue)); + Object value = booleanPrimitiveTypeHandler.parseObject(String.valueOf(actualValue)); assertEquals(actualValue, value); } @@ -58,7 +60,7 @@ public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeBool Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); BooleanPrimitiveTypeHandler booleanPrimitiveTypeHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); - Object value = booleanPrimitiveTypeHandler.getValue(null); + Object value = booleanPrimitiveTypeHandler.parseObject(null); assertEquals(false, value); } @@ -98,4 +100,48 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((boolean[]) actualValues).length); } + + @Test + public void shouldFetchParsedValueForFieldOfTypeBoolInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BOOLEAN).named("customer_dynamic_surge_enabled") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("customer_dynamic_surge_enabled", true); + + BooleanPrimitiveTypeHandler booleanHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); + Object actualValue = booleanHandler.parseSimpleGroup(simpleGroup); + + assertEquals(true, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BOOLEAN).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + BooleanPrimitiveTypeHandler booleanHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = booleanHandler.parseSimpleGroup(simpleGroup); + + assertEquals(false, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("customer_dynamic_surge_enabled"); + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BOOLEAN).named("customer_dynamic_surge_enabled") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + BooleanPrimitiveTypeHandler booleanHandler = new BooleanPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = booleanHandler.parseSimpleGroup(simpleGroup); + + assertEquals(false, actualValue); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandlerTest.java index 9676d7f4c..0696f5225 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/ByteStringPrimitiveTypeHandlerTest.java @@ -6,15 +6,16 @@ import com.google.protobuf.ByteString; import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestMessageEnvelope; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.junit.Assert.*; public class ByteStringPrimitiveTypeHandlerTest { @Test @@ -37,7 +38,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeByteString() { Descriptors.FieldDescriptor fieldDescriptor = TestMessageEnvelope.getDescriptor().findFieldByName("log_key"); ByteStringPrimitiveTypeHandler byteStringPrimitiveTypeHandler = new ByteStringPrimitiveTypeHandler(fieldDescriptor); - Object value = byteStringPrimitiveTypeHandler.getValue(actualValue); + Object value = byteStringPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -73,4 +74,49 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((ByteString[]) actualValues).length); } + @Test + public void shouldFetchUTF8EncodedByteStringForFieldOfTypeBinaryInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestMessageEnvelope.getDescriptor().findFieldByName("log_key"); + String testString = "test-string"; + ByteString expectedByteString = ByteString.copyFrom(testString.getBytes()); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("log_key") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("log_key", Binary.fromConstantByteArray(expectedByteString.toByteArray())); + ByteStringPrimitiveTypeHandler byteStringHandler = new ByteStringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = byteStringHandler.parseSimpleGroup(simpleGroup); + + assertEquals(expectedByteString, actualValue); + } + + @Test + public void shouldReturnNullIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestMessageEnvelope.getDescriptor().findFieldByName("log_key"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + ByteStringPrimitiveTypeHandler byteStringHandler = new ByteStringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = byteStringHandler.parseSimpleGroup(simpleGroup); + + assertNull(actualValue); + } + + @Test + public void shouldReturnNullIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestMessageEnvelope.getDescriptor().findFieldByName("log_key"); + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("log_key") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + ByteStringPrimitiveTypeHandler byteStringHandler = new ByteStringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = byteStringHandler.parseSimpleGroup(simpleGroup); + + assertNull(actualValue); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandlerTest.java index 760b8e95a..151074002 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/DoublePrimitiveTypeHandlerTest.java @@ -3,11 +3,14 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestBookingLogMessage; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; import static org.junit.Assert.*; public class DoublePrimitiveTypeHandlerTest { @@ -32,7 +35,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeDouble() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); DoublePrimitiveTypeHandler doublePrimitiveTypeHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); - Object value = doublePrimitiveTypeHandler.getValue(actualValue); + Object value = doublePrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -43,7 +46,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeDouble() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); DoublePrimitiveTypeHandler doublePrimitiveTypeHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); - Object value = doublePrimitiveTypeHandler.getValue(String.valueOf(actualValue)); + Object value = doublePrimitiveTypeHandler.parseObject(String.valueOf(actualValue)); assertEquals(actualValue, value); } @@ -52,7 +55,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeDouble() { public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeDouble() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); DoublePrimitiveTypeHandler doublePrimitiveTypeHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); - Object value = doublePrimitiveTypeHandler.getValue(null); + Object value = doublePrimitiveTypeHandler.parseObject(null); assertEquals(0.0D, value); } @@ -90,4 +93,50 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((double[]) actualValues).length); } + @Test + public void shouldFetchParsedValueForFieldOfTypeDoubleInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(DOUBLE).named("cash_amount") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("cash_amount", 34.23D); + + DoublePrimitiveTypeHandler doubleHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); + Object actualValue = doubleHandler.parseSimpleGroup(simpleGroup); + + assertEquals(34.23D, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); + + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(DOUBLE).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + DoublePrimitiveTypeHandler doubleHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = doubleHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0.0D, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cash_amount"); + + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(DOUBLE).named("cash_amount") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + DoublePrimitiveTypeHandler doubleHandler = new DoublePrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = doubleHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0.0D, actualValue); + } + } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandlerTest.java index 183e085af..073e2d1d0 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/FloatPrimitiveTypeHandlerTest.java @@ -4,11 +4,15 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.Types.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -34,7 +38,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeFloat() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); FloatPrimitiveTypeHandler floatPrimitiveTypeHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); - Object value = floatPrimitiveTypeHandler.getValue(actualValue); + Object value = floatPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -45,7 +49,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeFloat() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); FloatPrimitiveTypeHandler floatPrimitiveTypeHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); - Object value = floatPrimitiveTypeHandler.getValue(String.valueOf(actualValue)); + Object value = floatPrimitiveTypeHandler.parseObject(String.valueOf(actualValue)); assertEquals(actualValue, value); } @@ -54,7 +58,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeFloat() { public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeFloat() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); FloatPrimitiveTypeHandler floatPrimitiveTypeHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); - Object value = floatPrimitiveTypeHandler.getValue(null); + Object value = floatPrimitiveTypeHandler.parseObject(null); assertEquals(0.0f, value); } @@ -92,4 +96,49 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((float[]) actualValues).length); } + @Test + public void shouldFetchParsedValueForFieldOfTypeFloatInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); + GroupType parquetSchema = requiredGroup() + .required(FLOAT).named("amount_paid_by_cash") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("amount_paid_by_cash", 32.56F); + FloatPrimitiveTypeHandler floatHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = floatHandler.parseSimpleGroup(simpleGroup); + + assertEquals(32.56F, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); + GroupType parquetSchema = requiredGroup() + .required(FLOAT).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + FloatPrimitiveTypeHandler floatHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = floatHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0.0F, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("amount_paid_by_cash"); + + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = requiredGroup() + .required(FLOAT).named("amount_paid_by_cash") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + FloatPrimitiveTypeHandler floatHandler = new FloatPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = floatHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0.0F, actualValue); + } + } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandlerTest.java index b8444a30d..e32d6613a 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/IntegerPrimitiveTypeHandlerTest.java @@ -4,11 +4,14 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -36,7 +39,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeInteger() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); IntegerPrimitiveTypeHandler integerPrimitiveTypeHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); - Object value = integerPrimitiveTypeHandler.getValue(actualValue); + Object value = integerPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -47,7 +50,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeInteger() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); IntegerPrimitiveTypeHandler integerPrimitiveTypeHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); - Object value = integerPrimitiveTypeHandler.getValue(String.valueOf(actualValue)); + Object value = integerPrimitiveTypeHandler.parseObject(String.valueOf(actualValue)); assertEquals(actualValue, value); } @@ -56,7 +59,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeInteger() { public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeInteger() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); IntegerPrimitiveTypeHandler integerPrimitiveTypeHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); - Object value = integerPrimitiveTypeHandler.getValue(null); + Object value = integerPrimitiveTypeHandler.parseObject(null); assertEquals(0, value); } @@ -94,4 +97,50 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((int[]) actualValues).length); } + @Test + public void shouldFetchParsedValueForFieldOfTypeIntegerInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); + + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT32).named("cancel_reason_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("cancel_reason_id", 34); + + IntegerPrimitiveTypeHandler integerHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); + Object actualValue = integerHandler.parseSimpleGroup(simpleGroup); + + assertEquals(34, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); + + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT32).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + IntegerPrimitiveTypeHandler integerHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = integerHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("cancel_reason_id"); + + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT32).named("cancel_reason_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + IntegerPrimitiveTypeHandler integerHandler = new IntegerPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = integerHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0, actualValue); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandlerTest.java index 8f3fbe275..5439255d8 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/LongPrimitiveTypeHandlerTest.java @@ -4,11 +4,14 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestAggregatedSupplyMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -36,7 +39,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeLong() { Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); LongPrimitiveTypeHandler longPrimitiveTypeHandler = new LongPrimitiveTypeHandler(fieldDescriptor); - Object value = longPrimitiveTypeHandler.getValue(actualValue); + Object value = longPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -47,7 +50,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeLong() { Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); LongPrimitiveTypeHandler longPrimitiveTypeHandler = new LongPrimitiveTypeHandler(fieldDescriptor); - Object value = longPrimitiveTypeHandler.getValue(String.valueOf(actualValue)); + Object value = longPrimitiveTypeHandler.parseObject(String.valueOf(actualValue)); assertEquals(actualValue, value); } @@ -56,7 +59,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeLong() { public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeLong() { Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); LongPrimitiveTypeHandler longPrimitiveTypeHandler = new LongPrimitiveTypeHandler(fieldDescriptor); - Object value = longPrimitiveTypeHandler.getValue(null); + Object value = longPrimitiveTypeHandler.parseObject(null); assertEquals(0L, value); } @@ -92,4 +95,49 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((Long[]) actualValues).length); } + @Test + public void shouldFetchParsedValueForFieldOfTypeLongInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("s2_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("s2_id", 101828L); + LongPrimitiveTypeHandler longHandler = new LongPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = longHandler.parseSimpleGroup(simpleGroup); + + assertEquals(101828L, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); + + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + LongPrimitiveTypeHandler longHandler = new LongPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = longHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0L, actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestAggregatedSupplyMessage.getDescriptor().findFieldByName("s2_id"); + + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(INT64).named("s2_id") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + LongPrimitiveTypeHandler longHandler = new LongPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = longHandler.parseSimpleGroup(simpleGroup); + + assertEquals(0L, actualValue); + } } diff --git a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandlerTest.java b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandlerTest.java index b555f3c81..bf2190881 100644 --- a/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandlerTest.java +++ b/dagger-common/src/test/java/io/odpf/dagger/common/serde/proto/protohandler/typehandler/StringPrimitiveTypeHandlerTest.java @@ -5,11 +5,14 @@ import com.google.protobuf.Descriptors; import io.odpf.dagger.consumer.TestBookingLogMessage; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -37,7 +40,7 @@ public void shouldFetchValueForFieldForFieldDescriptorOfTypeString() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); StringPrimitiveTypeHandler stringPrimitiveTypeHandler = new StringPrimitiveTypeHandler(fieldDescriptor); - Object value = stringPrimitiveTypeHandler.getValue(actualValue); + Object value = stringPrimitiveTypeHandler.parseObject(actualValue); assertEquals(actualValue, value); } @@ -48,7 +51,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeString() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); StringPrimitiveTypeHandler stringPrimitiveTypeHandler = new StringPrimitiveTypeHandler(fieldDescriptor); - Object value = stringPrimitiveTypeHandler.getValue(actualValue); + Object value = stringPrimitiveTypeHandler.parseObject(actualValue); assertEquals("23", value); } @@ -57,7 +60,7 @@ public void shouldFetchParsedValueForFieldForFieldDescriptorOfTypeString() { public void shouldFetchDefaultValueIfValueNotPresentForFieldDescriptorOfTypeString() { Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); StringPrimitiveTypeHandler stringPrimitiveTypeHandler = new StringPrimitiveTypeHandler(fieldDescriptor); - Object value = stringPrimitiveTypeHandler.getValue(null); + Object value = stringPrimitiveTypeHandler.parseObject(null); assertEquals("", value); } @@ -93,4 +96,47 @@ public void shouldReturnEmptyArrayOnNull() { assertEquals(0, ((String[]) actualValues).length); } + @Test + public void shouldFetchParsedValueForFieldOfTypeStringInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("order_number") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + simpleGroup.add("order_number", "some-value"); + StringPrimitiveTypeHandler stringHandler = new StringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = stringHandler.parseSimpleGroup(simpleGroup); + + assertEquals("some-value", actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotPresentInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("some-other-field") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + StringPrimitiveTypeHandler stringHandler = new StringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = stringHandler.parseSimpleGroup(simpleGroup); + + assertEquals("", actualValue); + } + + @Test + public void shouldFetchDefaultValueIfFieldNotInitializedWithAValueInSimpleGroup() { + Descriptors.FieldDescriptor fieldDescriptor = TestBookingLogMessage.getDescriptor().findFieldByName("order_number"); + /* The field is added to the schema but not assigned a value */ + GroupType parquetSchema = org.apache.parquet.schema.Types.requiredGroup() + .required(BINARY).named("order_number") + .named("TestGroupType"); + SimpleGroup simpleGroup = new SimpleGroup(parquetSchema); + StringPrimitiveTypeHandler stringHandler = new StringPrimitiveTypeHandler(fieldDescriptor); + + Object actualValue = stringHandler.parseSimpleGroup(simpleGroup); + + assertEquals("", actualValue); + } } diff --git a/dagger-common/src/test/proto/TestLogMessage.proto b/dagger-common/src/test/proto/TestLogMessage.proto index 86ee8d232..362bcd908 100644 --- a/dagger-common/src/test/proto/TestLogMessage.proto +++ b/dagger-common/src/test/proto/TestLogMessage.proto @@ -292,3 +292,16 @@ message TestApiLogMessage { google.protobuf.Timestamp event_timestamp = 1; repeated string request_headers_extra = 2; } + +message TestPrimitiveMessage { + bool is_valid = 1; + string order_number = 2; + bytes order_hash = 3; + double latitude = 4; + double longitude = 5; + float price = 6; + int32 packet_count = 7; + int64 phone = 8; + google.protobuf.Timestamp event_timestamp = 9; + TestServiceType.Enum service_type = 10; +} \ No newline at end of file diff --git a/dagger-core/build.gradle b/dagger-core/build.gradle index 2fa8cadd7..bb9827a2f 100644 --- a/dagger-core/build.gradle +++ b/dagger-core/build.gradle @@ -84,6 +84,7 @@ dependencies { dependenciesJar 'org.asynchttpclient:async-http-client:2.10.1' dependenciesJar 'io.vertx:vertx-pg-client:3.9.0' dependenciesJar 'org.apache.commons:commons-pool2:2.4.3' + dependenciesJar 'org.apache.parquet:parquet-protobuf:1.12.2' testImplementation project(':dagger-common').sourceSets.test.output testImplementation 'junit:junit:4.13' diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java b/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java index a2ed9b426..e502c975c 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java @@ -1,5 +1,15 @@ package io.odpf.dagger.core; +import io.odpf.dagger.core.source.Stream; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.ApiExpression; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + import io.odpf.dagger.common.configuration.Configuration; import io.odpf.dagger.common.core.StencilClientOrchestrator; import io.odpf.dagger.common.core.StreamInfo; @@ -16,17 +26,8 @@ import io.odpf.dagger.core.processors.types.PostProcessor; import io.odpf.dagger.core.processors.types.Preprocessor; import io.odpf.dagger.core.sink.SinkOrchestrator; -import io.odpf.dagger.core.source.Stream; import io.odpf.dagger.core.source.StreamsFactory; import io.odpf.dagger.core.utils.Constants; -import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.ApiExpression; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -84,7 +85,6 @@ public StreamManager registerConfigs() { return this; } - /** * Register source with pre processors stream manager. * @@ -97,13 +97,13 @@ public StreamManager registerSourceWithPreProcessors() { getStreams().forEach(stream -> { String tableName = stream.getStreamName(); WatermarkStrategyDefinition watermarkStrategyDefinition = getSourceWatermarkDefinition(enablePerPartitionWatermark); - DataStream kafkaStream = stream.registerSource(executionEnvironment, watermarkStrategyDefinition.getWatermarkStrategy(watermarkDelay), stream.getStreamName()); + DataStream dataStream = stream.registerSource(executionEnvironment, watermarkStrategyDefinition.getWatermarkStrategy(watermarkDelay)); StreamWatermarkAssigner streamWatermarkAssigner = new StreamWatermarkAssigner(new LastColumnWatermark()); DataStream rowSingleOutputStreamOperator = streamWatermarkAssigner - .assignTimeStampAndWatermark(kafkaStream, watermarkDelay, enablePerPartitionWatermark); + .assignTimeStampAndWatermark(dataStream, watermarkDelay, enablePerPartitionWatermark); - TableSchema tableSchema = TableSchema.fromTypeInfo(kafkaStream.getType()); + TableSchema tableSchema = TableSchema.fromTypeInfo(dataStream.getType()); StreamInfo streamInfo = new StreamInfo(rowSingleOutputStreamOperator, tableSchema.getFieldNames()); streamInfo = addPreProcessor(streamInfo, tableName, preProcessorConfig); @@ -197,10 +197,6 @@ protected StreamInfo createStreamInfo(Table table) { return new StreamInfo(stream, table.getSchema().getFieldNames()); } - List getStreams() { - return StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); - } - private StreamInfo addPostProcessor(StreamInfo streamInfo) { List postProcessors = PostProcessorFactory.getPostProcessors(configuration, stencilClientOrchestrator, streamInfo.getColumnNames(), telemetryExporter); for (PostProcessor postProcessor : postProcessors) { @@ -222,4 +218,8 @@ private void addSink(StreamInfo streamInfo) { sinkOrchestrator.addSubscriber(telemetryExporter); streamInfo.getDataStream().sinkTo(sinkOrchestrator.getSink(configuration, streamInfo.getColumnNames(), stencilClientOrchestrator)); } + + List getStreams() { + return StreamsFactory.getStreams(configuration, stencilClientOrchestrator); + } } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactory.java b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactory.java new file mode 100644 index 000000000..2af435449 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactory.java @@ -0,0 +1,31 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.core.exception.DaggerConfigurationException; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class DaggerDeserializerFactory { + public static DaggerDeserializer create(StreamConfig streamConfig, Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { + return getDaggerDeserializerProviders(streamConfig, configuration, stencilClientOrchestrator) + .stream() + .filter(DaggerDeserializerProvider::canProvide) + .findFirst() + .orElseThrow(() -> new DaggerConfigurationException("No suitable deserializer could be constructed for the given stream configuration.")) + .getDaggerDeserializer(); + } + + private static List> getDaggerDeserializerProviders(StreamConfig streamConfig, Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { + return Stream.of( + new JsonDeserializerProvider(streamConfig), + new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator), + new SimpleGroupDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator)) + .collect(Collectors.toList()); + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerProvider.java b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerProvider.java new file mode 100644 index 000000000..405973429 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/DaggerDeserializerProvider.java @@ -0,0 +1,8 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.serde.DaggerDeserializer; + +public interface DaggerDeserializerProvider { + DaggerDeserializer getDaggerDeserializer(); + boolean canProvide(); +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/JsonDeserializerProvider.java b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/JsonDeserializerProvider.java new file mode 100644 index 000000000..fb0df4a2a --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/JsonDeserializerProvider.java @@ -0,0 +1,44 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.DataTypes; +import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.types.Row; + +import java.util.Arrays; +import java.util.HashSet; + +import static io.odpf.dagger.common.serde.DataTypes.JSON; +import static io.odpf.dagger.core.source.SourceName.KAFKA_SOURCE; +import static io.odpf.dagger.core.source.SourceName.KAFKA_CONSUMER; + +public class JsonDeserializerProvider implements DaggerDeserializerProvider { + private final StreamConfig streamConfig; + private static final HashSet COMPATIBLE_SOURCES = new HashSet<>(Arrays.asList(KAFKA_SOURCE, KAFKA_CONSUMER)); + private static final DataTypes COMPATIBLE_INPUT_SCHEMA_TYPE = JSON; + + public JsonDeserializerProvider(StreamConfig streamConfig) { + this.streamConfig = streamConfig; + } + + @Override + public DaggerDeserializer getDaggerDeserializer() { + return new JsonDeserializer(streamConfig.getJsonSchema(), streamConfig.getJsonEventTimestampFieldName()); + } + + @Override + public boolean canProvide() { + SourceDetails[] sourceDetailsList = streamConfig.getSourceDetails(); + for (SourceDetails sourceDetails : sourceDetailsList) { + SourceName sourceName = sourceDetails.getSourceName(); + DataTypes inputSchemaType = DataTypes.valueOf(streamConfig.getDataType()); + if (!COMPATIBLE_SOURCES.contains(sourceName) || !inputSchemaType.equals(COMPATIBLE_INPUT_SCHEMA_TYPE)) { + return false; + } + } + return true; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProvider.java b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProvider.java new file mode 100644 index 000000000..10803a808 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProvider.java @@ -0,0 +1,55 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.DataTypes; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.types.Row; + +import java.util.Arrays; +import java.util.HashSet; + +import static io.odpf.dagger.common.serde.DataTypes.PROTO; +import static io.odpf.dagger.core.source.SourceName.KAFKA_SOURCE; +import static io.odpf.dagger.core.source.SourceName.KAFKA_CONSUMER; +import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT; +import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_KEY; + +public class ProtoDeserializerProvider implements DaggerDeserializerProvider { + protected final StreamConfig streamConfig; + protected final Configuration configuration; + protected final StencilClientOrchestrator stencilClientOrchestrator; + private static final HashSet COMPATIBLE_SOURCES = new HashSet<>(Arrays.asList(KAFKA_SOURCE, KAFKA_CONSUMER)); + private static final DataTypes COMPATIBLE_INPUT_SCHEMA_TYPE = PROTO; + + public ProtoDeserializerProvider(StreamConfig streamConfig, Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.stencilClientOrchestrator = stencilClientOrchestrator; + } + + @Override + public DaggerDeserializer getDaggerDeserializer() { + int timestampFieldIndex = Integer.parseInt(streamConfig.getEventTimestampFieldIndex()); + String protoClassName = streamConfig.getProtoClass(); + String rowTimeAttributeName = configuration.getString(FLINK_ROWTIME_ATTRIBUTE_NAME_KEY, FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT); + return new ProtoDeserializer(protoClassName, timestampFieldIndex, rowTimeAttributeName, stencilClientOrchestrator); + } + + @Override + public boolean canProvide() { + SourceDetails[] sourceDetailsList = streamConfig.getSourceDetails(); + for (SourceDetails sourceDetails : sourceDetailsList) { + SourceName sourceName = sourceDetails.getSourceName(); + DataTypes inputSchemaType = DataTypes.valueOf(streamConfig.getDataType()); + if (!COMPATIBLE_SOURCES.contains(sourceName) || !inputSchemaType.equals(COMPATIBLE_INPUT_SCHEMA_TYPE)) { + return false; + } + } + return true; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProvider.java b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProvider.java new file mode 100644 index 000000000..a341c465c --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProvider.java @@ -0,0 +1,51 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.DataTypes; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.types.Row; + +import static io.odpf.dagger.common.serde.DataTypes.PROTO; +import static io.odpf.dagger.core.source.SourceName.PARQUET_SOURCE; +import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT; +import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_KEY; + +public class SimpleGroupDeserializerProvider implements DaggerDeserializerProvider { + protected final StreamConfig streamConfig; + protected final Configuration configuration; + protected final StencilClientOrchestrator stencilClientOrchestrator; + private static final SourceName COMPATIBLE_SOURCE = PARQUET_SOURCE; + private static final DataTypes COMPATIBLE_INPUT_SCHEMA_TYPE = PROTO; + + public SimpleGroupDeserializerProvider(StreamConfig streamConfig, Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.stencilClientOrchestrator = stencilClientOrchestrator; + } + + @Override + public DaggerDeserializer getDaggerDeserializer() { + int timestampFieldIndex = Integer.parseInt(streamConfig.getEventTimestampFieldIndex()); + String protoClassName = streamConfig.getProtoClass(); + String rowTimeAttributeName = configuration.getString(FLINK_ROWTIME_ATTRIBUTE_NAME_KEY, FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT); + return new SimpleGroupDeserializer(protoClassName, timestampFieldIndex, rowTimeAttributeName, stencilClientOrchestrator); + } + + @Override + public boolean canProvide() { + SourceDetails[] sourceDetailsList = streamConfig.getSourceDetails(); + for (SourceDetails sourceDetails : sourceDetailsList) { + SourceName sourceName = sourceDetails.getSourceName(); + DataTypes inputSchemaType = DataTypes.valueOf(streamConfig.getDataType()); + if (!sourceName.equals(COMPATIBLE_SOURCE) || !inputSchemaType.equals(COMPATIBLE_INPUT_SCHEMA_TYPE)) { + return false; + } + } + return true; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/exception/ParquetFileSourceReaderInitializationException.java b/dagger-core/src/main/java/io/odpf/dagger/core/exception/ParquetFileSourceReaderInitializationException.java new file mode 100644 index 000000000..5023f5cba --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/exception/ParquetFileSourceReaderInitializationException.java @@ -0,0 +1,10 @@ +package io.odpf.dagger.core.exception; + +/*** + * This exception is thrown when the reader for Parquet FileSource could not be initialized. + */ +public class ParquetFileSourceReaderInitializationException extends RuntimeException { + public ParquetFileSourceReaderInitializationException(Throwable cause) { + super(cause); + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSource.java index dc41ed2f4..885385330 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSource.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSource.java @@ -1,27 +1,11 @@ package io.odpf.dagger.core.source; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.types.Row; -/** - * Interface for all types of Sources in Dagger. - */ -public interface DaggerSource { +public interface DaggerSource { + DataStream register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy); - /** - * Can handle boolean. - * - * @return boolean according to configured source - */ - boolean canHandle(); - - /** - * @param executionEnvironment the flink execution environment - * @param watermarkStrategy configured watermark strategy - * @param streamName name of the stream - * @return DataStream after registration - */ - DataStreamSource register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy, String streamName); + boolean canBuild(); } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSourceFactory.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSourceFactory.java index 1a0907215..98f72d38c 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSourceFactory.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/DaggerSourceFactory.java @@ -1,19 +1,37 @@ package io.odpf.dagger.core.source; import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; import io.odpf.dagger.core.exception.InvalidDaggerSourceException; -import io.odpf.dagger.core.source.kafka.DaggerKafkaSource; -import io.odpf.dagger.core.source.kafka.DaggerOldKafkaSource; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import io.odpf.dagger.core.source.flinkkafkaconsumer.FlinkKafkaConsumerDaggerSource; +import io.odpf.dagger.core.source.kafka.KafkaDaggerSource; +import io.odpf.dagger.core.source.parquet.ParquetDaggerSource; +import org.apache.flink.types.Row; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; import java.util.stream.Stream; public class DaggerSourceFactory { - public static DaggerSource createDaggerSource(StreamConfig streamConfig, KafkaDeserializationSchema deserializationSchema, Configuration configuration) { - return Stream.of(new DaggerOldKafkaSource(streamConfig, deserializationSchema, configuration), new DaggerKafkaSource(streamConfig, deserializationSchema, configuration)) - .filter(DaggerSource::canHandle) + public static DaggerSource create(StreamConfig streamConfig, Configuration configuration, DaggerDeserializer deserializer) { + List> daggerSources = getDaggerSources(streamConfig, configuration, deserializer); + return daggerSources.stream() + .filter(DaggerSource::canBuild) .findFirst() - .orElseThrow(() -> new InvalidDaggerSourceException(String.format("Dagger source: %s not supported", streamConfig.getSourceType()))); + .orElseThrow(() -> { + String sourceDetails = Arrays.toString(streamConfig.getSourceDetails()); + String message = String.format("No suitable DaggerSource can be created as per SOURCE_DETAILS config %s", sourceDetails); + return new InvalidDaggerSourceException(message); + }); + } + + private static List> getDaggerSources(StreamConfig streamConfig, Configuration configuration, DaggerDeserializer deserializer) { + KafkaDaggerSource kafkaDaggerSource = new KafkaDaggerSource(streamConfig, configuration, deserializer); + FlinkKafkaConsumerDaggerSource flinkKafkaConsumerDaggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, deserializer); + ParquetDaggerSource parquetDaggerSource = new ParquetDaggerSource(streamConfig, configuration, deserializer); + return Stream.of(kafkaDaggerSource, flinkKafkaConsumerDaggerSource, parquetDaggerSource) + .collect(Collectors.toList()); } } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceDetails.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceDetails.java new file mode 100644 index 000000000..db33dd0b9 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceDetails.java @@ -0,0 +1,24 @@ +package io.odpf.dagger.core.source; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; + +import java.io.Serializable; + +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_NAME_KEY; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_TYPE_KEY; + +public class SourceDetails implements Serializable { + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_NAME_KEY) + @Getter + private SourceName sourceName; + + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_TYPE_KEY) + @Getter + private SourceType sourceType; + + public SourceDetails(SourceName sourceName, SourceType sourceType) { + this.sourceName = sourceName; + this.sourceType = sourceType; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceName.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceName.java index 12048d162..aa697b548 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceName.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceName.java @@ -1,6 +1,16 @@ package io.odpf.dagger.core.source; +import com.google.gson.annotations.SerializedName; + +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA_CONSUMER; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_NAME_PARQUET; + public enum SourceName { + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA) KAFKA_SOURCE, - OLD_KAFKA_SOURCE + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_NAME_PARQUET) + PARQUET_SOURCE, + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA_CONSUMER) + KAFKA_CONSUMER } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceType.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceType.java new file mode 100644 index 000000000..8e8ede485 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/SourceType.java @@ -0,0 +1,13 @@ +package io.odpf.dagger.core.source; + +import com.google.gson.annotations.SerializedName; + +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_TYPE_BOUNDED; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_DETAILS_SOURCE_TYPE_UNBOUNDED; + +public enum SourceType { + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_TYPE_BOUNDED) + BOUNDED, + @SerializedName(STREAM_SOURCE_DETAILS_SOURCE_TYPE_UNBOUNDED) + UNBOUNDED +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/Stream.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/Stream.java index 96e0cd49c..82412cb3e 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/Stream.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/Stream.java @@ -1,32 +1,47 @@ package io.odpf.dagger.core.source; -import io.odpf.dagger.common.serde.DataTypes; +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.core.deserializer.DaggerDeserializerFactory; import lombok.Getter; -import lombok.NonNull; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -public class Stream { +import java.io.Serializable; + +public class Stream implements Serializable { @Getter - @NonNull - private final DaggerSource daggerSource; + private final DaggerSource daggerSource; @Getter - @NonNull private final String streamName; - @Getter - @NonNull - private DataTypes inputDataType; - public Stream(DaggerSource daggerSource, String streamName, DataTypes inputDataType) { + Stream(DaggerSource daggerSource, String streamName) { this.daggerSource = daggerSource; this.streamName = streamName; - this.inputDataType = inputDataType; } - public DataStream registerSource(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy, String stream) { - return daggerSource.register(executionEnvironment, watermarkStrategy, stream); + public DataStream registerSource(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy) { + return daggerSource.register(executionEnvironment, watermarkStrategy); } -} + public static class Builder { + private final StreamConfig streamConfig; + private final Configuration configuration; + private final StencilClientOrchestrator stencilClientOrchestrator; + + public Builder(StreamConfig streamConfig, Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.stencilClientOrchestrator = stencilClientOrchestrator; + } + + public Stream build() { + DaggerDeserializer daggerDeserializer = DaggerDeserializerFactory.create(streamConfig, configuration, stencilClientOrchestrator); + DaggerSource daggerSource = DaggerSourceFactory.create(streamConfig, configuration, daggerDeserializer); + return new Stream(daggerSource, streamConfig.getSchemaTable()); + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamConfig.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamConfig.java index e2046e0b2..358374991 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamConfig.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamConfig.java @@ -1,12 +1,15 @@ package io.odpf.dagger.core.source; +import io.odpf.dagger.core.source.parquet.SourceParquetReadOrderStrategy; +import io.odpf.dagger.core.source.parquet.SourceParquetSchemaMatchStrategy; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; + import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.annotations.SerializedName; import com.google.gson.stream.JsonReader; import io.odpf.dagger.common.configuration.Configuration; import lombok.Getter; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import java.io.StringReader; @@ -14,7 +17,9 @@ import java.util.Properties; import java.util.regex.Pattern; -import static io.odpf.dagger.common.core.Constants.*; +import static io.odpf.dagger.common.core.Constants.INPUT_STREAMS; +import static io.odpf.dagger.common.core.Constants.STREAM_INPUT_SCHEMA_PROTO_CLASS; +import static io.odpf.dagger.common.core.Constants.STREAM_INPUT_SCHEMA_TABLE; import static io.odpf.dagger.core.utils.Constants.*; public class StreamConfig { @@ -71,15 +76,24 @@ public class StreamConfig { @SerializedName(STREAM_INPUT_DATATYPE) private String dataType; - @SerializedName(STREAM_INPUT_SOURCE_TYPE) - private String sourceType; + @SerializedName(STREAM_SOURCE_DETAILS_KEY) + private SourceDetails[] sourceDetails; - public String getSourceType() { - if (sourceType == null) { - sourceType = SourceName.OLD_KAFKA_SOURCE.toString(); - } - return sourceType; - } + @SerializedName(STREAM_SOURCE_PARQUET_BILLING_PROJECT_KEY) + @Getter + private String parquetBillingProject; + + @SerializedName(STREAM_SOURCE_PARQUET_FILE_PATHS_KEY) + @Getter + private String[] parquetFilePaths; + + @SerializedName(STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_KEY) + @Getter + private SourceParquetReadOrderStrategy parquetFilesReadOrderStrategy; + + @SerializedName(STREAM_SOURCE_PARQUET_SCHEMA_MATCH_STRATEGY_KEY) + @Getter + private SourceParquetSchemaMatchStrategy parquetSchemaMatchStrategy; public String getDataType() { if (dataType == null) { @@ -88,6 +102,14 @@ public String getDataType() { return dataType; } + public SourceDetails[] getSourceDetails() { + if (sourceDetails == null) { + return new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}; + } else { + return sourceDetails; + } + } + public String getAutoOffsetReset() { if (autoOffsetReset == null) { autoOffsetReset = "latest"; diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamsFactory.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamsFactory.java index fa3919bfd..c8118cc8e 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamsFactory.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/StreamsFactory.java @@ -2,39 +2,19 @@ import io.odpf.dagger.common.configuration.Configuration; import io.odpf.dagger.common.core.StencilClientOrchestrator; -import io.odpf.dagger.core.processors.telemetry.processor.MetricsTelemetryExporter; -import io.odpf.dagger.core.source.kafka.builder.JsonDataStreamBuilder; -import io.odpf.dagger.core.source.kafka.builder.ProtoDataStreamBuilder; -import io.odpf.dagger.core.source.kafka.builder.StreamBuilder; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; public class StreamsFactory { - - public static List getStreams(Configuration configuration, - StencilClientOrchestrator stencilClientOrchestrator, - MetricsTelemetryExporter telemetryExporter) { + public static List getStreams(Configuration configuration, StencilClientOrchestrator stencilClientOrchestrator) { StreamConfig[] streamConfigs = StreamConfig.parse(configuration); ArrayList streams = new ArrayList<>(); for (StreamConfig streamConfig : streamConfigs) { - List dataStreams = Arrays - .asList(new JsonDataStreamBuilder(streamConfig, configuration), - new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration)); - StreamBuilder streamBuilder = dataStreams.stream() - .filter(dataStream -> dataStream.canBuild()) - .findFirst() - .orElse(new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration)); - - streamBuilder.notifySubscriber(telemetryExporter); - - Stream stream = streamBuilder - .build(); - streams.add(stream); + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + streams.add(builder.build()); } return streams; } } - diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustom.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustom.java similarity index 97% rename from dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustom.java rename to dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustom.java index 5dee68575..06b30eea2 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustom.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustom.java @@ -1,4 +1,4 @@ -package io.odpf.dagger.core.source.kafka; +package io.odpf.dagger.core.source.flinkkafkaconsumer; import io.odpf.dagger.common.configuration.Configuration; import io.odpf.dagger.core.metrics.reporters.ErrorReporter; diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSource.java new file mode 100644 index 000000000..c15eeb3ea --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSource.java @@ -0,0 +1,57 @@ +package io.odpf.dagger.core.source.flinkkafkaconsumer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.dagger.core.source.DaggerSource; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.types.Row; + +import static io.odpf.dagger.core.source.SourceName.KAFKA_CONSUMER; +import static io.odpf.dagger.core.source.SourceType.UNBOUNDED; + +public class FlinkKafkaConsumerDaggerSource implements DaggerSource { + + private final DaggerDeserializer deserializer; + private final StreamConfig streamConfig; + private final Configuration configuration; + private static final SourceName SUPPORTED_SOURCE_NAME = KAFKA_CONSUMER; + private static final SourceType SUPPORTED_SOURCE_TYPE = UNBOUNDED; + + public FlinkKafkaConsumerDaggerSource(StreamConfig streamConfig, Configuration configuration, DaggerDeserializer deserializer) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.deserializer = deserializer; + } + + FlinkKafkaConsumerCustom buildSource() { + KafkaDeserializationSchema kafkaDeserializationSchema = (KafkaDeserializationSchema) deserializer; + return new FlinkKafkaConsumerCustom(streamConfig.getTopicPattern(), + kafkaDeserializationSchema, streamConfig.getKafkaProps(configuration), configuration); + } + + @Override + public DataStream register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy) { + FlinkKafkaConsumerCustom source = buildSource(); + return executionEnvironment.addSource(source.assignTimestampsAndWatermarks(watermarkStrategy)); + } + + @Override + public boolean canBuild() { + SourceDetails[] sourceDetailsArray = streamConfig.getSourceDetails(); + if (sourceDetailsArray.length != 1) { + return false; + } else { + SourceName sourceName = sourceDetailsArray[0].getSourceName(); + SourceType sourceType = sourceDetailsArray[0].getSourceType(); + return sourceName.equals(SUPPORTED_SOURCE_NAME) && sourceType.equals(SUPPORTED_SOURCE_TYPE) + && deserializer instanceof KafkaDeserializationSchema; + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSource.java deleted file mode 100644 index 03a694748..000000000 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSource.java +++ /dev/null @@ -1,45 +0,0 @@ -package io.odpf.dagger.core.source.kafka; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.core.source.DaggerSource; -import io.odpf.dagger.core.source.SourceName; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.connector.kafka.source.KafkaSource; -import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.types.Row; - -public class DaggerKafkaSource implements DaggerSource { - private StreamConfig streamConfig; - private final KafkaDeserializationSchema deserializationSchema; - private final Configuration configuration; - - public DaggerKafkaSource(StreamConfig streamConfig, KafkaDeserializationSchema deserializationSchema, Configuration configuration) { - this.streamConfig = streamConfig; - this.deserializationSchema = deserializationSchema; - this.configuration = configuration; - } - - @Override - public boolean canHandle() { - return SourceName.valueOf(streamConfig.getSourceType()).equals(SourceName.KAFKA_SOURCE); - } - - @Override - public DataStreamSource register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy, String streamName) { - KafkaSource kafkaSource = getKafkaSource(); - return executionEnvironment.fromSource(kafkaSource, watermarkStrategy, streamName); - } - - KafkaSource getKafkaSource() { - return KafkaSource.builder() - .setTopicPattern(streamConfig.getTopicPattern()) - .setStartingOffsets(streamConfig.getStartingOffset()) - .setProperties(streamConfig.getKafkaProps(configuration)) - .setDeserializer(KafkaRecordDeserializationSchema.of(deserializationSchema)) - .build(); - } -} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSource.java deleted file mode 100644 index 94be2239d..000000000 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSource.java +++ /dev/null @@ -1,38 +0,0 @@ -package io.odpf.dagger.core.source.kafka; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.core.source.DaggerSource; -import io.odpf.dagger.core.source.SourceName; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.types.Row; - -public class DaggerOldKafkaSource implements DaggerSource { - private StreamConfig streamConfig; - private final KafkaDeserializationSchema deserializationSchema; - private final Configuration configuration; - - public DaggerOldKafkaSource(StreamConfig streamConfig, KafkaDeserializationSchema deserializationSchema, Configuration configuration) { - this.streamConfig = streamConfig; - this.deserializationSchema = deserializationSchema; - this.configuration = configuration; - } - - @Override - public boolean canHandle() { - return SourceName.valueOf(streamConfig.getSourceType()).equals(SourceName.OLD_KAFKA_SOURCE); - } - - @Override - public DataStreamSource register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy, String streamName) { - FlinkKafkaConsumerCustom flinkKafkaConsumerCustom = getFlinkKafkaConsumerCustom(); - return executionEnvironment.addSource(flinkKafkaConsumerCustom.assignTimestampsAndWatermarks(watermarkStrategy)); - } - - FlinkKafkaConsumerCustom getFlinkKafkaConsumerCustom() { - return new FlinkKafkaConsumerCustom(streamConfig.getTopicPattern(), deserializationSchema, streamConfig.getKafkaProps(configuration), configuration); - } -} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSource.java new file mode 100644 index 000000000..e2459200a --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSource.java @@ -0,0 +1,62 @@ +package io.odpf.dagger.core.source.kafka; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.dagger.core.source.DaggerSource; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.types.Row; + +import static io.odpf.dagger.core.source.SourceName.KAFKA_SOURCE; +import static io.odpf.dagger.core.source.SourceType.UNBOUNDED; + +public class KafkaDaggerSource implements DaggerSource { + private final DaggerDeserializer deserializer; + private final StreamConfig streamConfig; + private final Configuration configuration; + private static final SourceName SUPPORTED_SOURCE_NAME = KAFKA_SOURCE; + private static final SourceType SUPPORTED_SOURCE_TYPE = UNBOUNDED; + + public KafkaDaggerSource(StreamConfig streamConfig, Configuration configuration, DaggerDeserializer deserializer) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.deserializer = deserializer; + } + + KafkaSource buildSource() { + KafkaRecordDeserializationSchema kafkaRecordDeserializationSchema = KafkaRecordDeserializationSchema + .of((KafkaDeserializationSchema) deserializer); + return KafkaSource.builder() + .setTopicPattern(streamConfig.getTopicPattern()) + .setStartingOffsets(streamConfig.getStartingOffset()) + .setProperties(streamConfig.getKafkaProps(configuration)) + .setDeserializer(kafkaRecordDeserializationSchema) + .build(); + } + + @Override + public DataStream register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy) { + return executionEnvironment.fromSource(buildSource(), watermarkStrategy, streamConfig.getSchemaTable()); + } + + @Override + public boolean canBuild() { + SourceDetails[] sourceDetailsArray = streamConfig.getSourceDetails(); + if (sourceDetailsArray.length != 1) { + return false; + } else { + SourceName sourceName = sourceDetailsArray[0].getSourceName(); + SourceType sourceType = sourceDetailsArray[0].getSourceType(); + return sourceName.equals(SUPPORTED_SOURCE_NAME) && sourceType.equals(SUPPORTED_SOURCE_TYPE) + && deserializer instanceof KafkaDeserializationSchema; + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilder.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilder.java deleted file mode 100644 index 876a68428..000000000 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilder.java +++ /dev/null @@ -1,44 +0,0 @@ -package io.odpf.dagger.core.source.kafka.builder; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.common.serde.DataTypes; -import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.types.Row; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class JsonDataStreamBuilder extends StreamBuilder { - private final StreamConfig streamConfig; - - private Map> metrics = new HashMap<>(); - - public JsonDataStreamBuilder(StreamConfig streamConfig, Configuration configuration) { - super(streamConfig, configuration); - this.streamConfig = streamConfig; - } - - @Override - void addTelemetry() { - addDefaultMetrics(metrics); - } - - @Override - Map> getMetrics() { - return metrics; - } - - @Override - public boolean canBuild() { - return getInputDataType() == DataTypes.JSON; - } - - - @Override - KafkaDeserializationSchema getDeserializationSchema() { - return new JsonDeserializer(streamConfig.getJsonSchema(), streamConfig.getJsonEventTimestampFieldName()); - } -} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilder.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilder.java deleted file mode 100644 index aa8d175b2..000000000 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilder.java +++ /dev/null @@ -1,55 +0,0 @@ -package io.odpf.dagger.core.source.kafka.builder; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.common.core.StencilClientOrchestrator; -import io.odpf.dagger.common.serde.DataTypes; -import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static io.odpf.dagger.core.metrics.telemetry.TelemetryTypes.INPUT_PROTO; -import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT; -import static io.odpf.dagger.core.utils.Constants.FLINK_ROWTIME_ATTRIBUTE_NAME_KEY; - -public class ProtoDataStreamBuilder extends StreamBuilder { - private final String protoClassName; - private final StreamConfig streamConfig; - private StencilClientOrchestrator stencilClientOrchestrator; - private Configuration configuration; - private Map> metrics = new HashMap<>(); - - public ProtoDataStreamBuilder(StreamConfig streamConfig, StencilClientOrchestrator stencilClientOrchestrator, Configuration configuration) { - super(streamConfig, configuration); - this.streamConfig = streamConfig; - this.protoClassName = streamConfig.getProtoClass(); - this.stencilClientOrchestrator = stencilClientOrchestrator; - this.configuration = configuration; - } - - @Override - void addTelemetry() { - addDefaultMetrics(metrics); - addMetric(metrics, INPUT_PROTO.getValue(), protoClassName); - } - - @Override - Map> getMetrics() { - return metrics; - } - - @Override - public boolean canBuild() { - return getInputDataType() == DataTypes.PROTO; - } - - @Override - KafkaDeserializationSchema getDeserializationSchema() { - int timestampFieldIndex = Integer.parseInt(streamConfig.getEventTimestampFieldIndex()); - String rowTimeAttributeName = configuration.getString(FLINK_ROWTIME_ATTRIBUTE_NAME_KEY, FLINK_ROWTIME_ATTRIBUTE_NAME_DEFAULT); - return new ProtoDeserializer(protoClassName, timestampFieldIndex, rowTimeAttributeName, stencilClientOrchestrator); - } -} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/StreamBuilder.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/StreamBuilder.java deleted file mode 100644 index e997ad0ce..000000000 --- a/dagger-core/src/main/java/io/odpf/dagger/core/source/kafka/builder/StreamBuilder.java +++ /dev/null @@ -1,74 +0,0 @@ -package io.odpf.dagger.core.source.kafka.builder; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.common.serde.DataTypes; -import io.odpf.dagger.core.metrics.telemetry.TelemetryPublisher; -import io.odpf.dagger.core.source.DaggerSource; -import io.odpf.dagger.core.source.DaggerSourceFactory; -import io.odpf.dagger.core.source.Stream; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import static io.odpf.dagger.core.metrics.telemetry.TelemetryTypes.INPUT_STREAM; -import static io.odpf.dagger.core.metrics.telemetry.TelemetryTypes.INPUT_TOPIC; - -public abstract class StreamBuilder implements TelemetryPublisher { - - private StreamConfig streamConfig; - private Configuration configuration; - - public StreamBuilder(StreamConfig streamConfig, Configuration configuration) { - this.streamConfig = streamConfig; - this.configuration = configuration; - } - - @Override - public void preProcessBeforeNotifyingSubscriber() { - addTelemetry(); - } - - @Override - public Map> getTelemetry() { - return getMetrics(); - } - - abstract void addTelemetry(); - - abstract Map> getMetrics(); - - public abstract boolean canBuild(); - - public Stream build() { - return new Stream(createSource(), streamConfig.getSchemaTable(), getInputDataType()); - } - - private DaggerSource createSource() { - return DaggerSourceFactory.createDaggerSource(streamConfig, getDeserializationSchema(), configuration); - } - - abstract KafkaDeserializationSchema getDeserializationSchema(); - - DataTypes getInputDataType() { - return DataTypes.valueOf(streamConfig.getDataType()); - } - - void addMetric(Map> metrics, String key, String value) { - metrics.computeIfAbsent(key, k -> new ArrayList<>()).add(value); - } - - protected Map> addDefaultMetrics(Map> metrics) { - String topicPattern = streamConfig.getKafkaTopicNames(); - List topicsToReport = new ArrayList<>(); - topicsToReport.addAll(Arrays.asList(topicPattern.split("\\|"))); - topicsToReport.forEach(topic -> addMetric(metrics, INPUT_TOPIC.getValue(), topic)); - - String streamName = streamConfig.getKafkaName(); - addMetric(metrics, INPUT_STREAM.getValue(), streamName); - return metrics; - } -} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSource.java new file mode 100644 index 000000000..5ecdd34c3 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSource.java @@ -0,0 +1,105 @@ +package io.odpf.dagger.core.source.parquet; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.core.exception.DaggerConfigurationException; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.dagger.core.source.DaggerSource; +import io.odpf.dagger.core.source.parquet.reader.ParquetReader; +import io.odpf.dagger.core.source.parquet.reader.ReaderProvider; +import io.odpf.dagger.core.source.parquet.splitassigner.ChronologyOrderedSplitAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.function.Supplier; + +import static io.odpf.dagger.core.source.SourceName.PARQUET_SOURCE; +import static io.odpf.dagger.core.source.SourceType.BOUNDED; + +public class ParquetDaggerSource implements DaggerSource { + private final DaggerDeserializer deserializer; + private final StreamConfig streamConfig; + private final Configuration configuration; + private static final SourceType SUPPORTED_SOURCE_TYPE = BOUNDED; + private static final SourceName SUPPORTED_SOURCE_NAME = PARQUET_SOURCE; + + public ParquetDaggerSource(StreamConfig streamConfig, Configuration configuration, DaggerDeserializer deserializer) { + this.streamConfig = streamConfig; + this.configuration = configuration; + this.deserializer = deserializer; + } + + @Override + public DataStream register(StreamExecutionEnvironment executionEnvironment, WatermarkStrategy watermarkStrategy) { + return executionEnvironment.fromSource(buildFileSource(), watermarkStrategy, streamConfig.getSchemaTable()); + } + + @Override + public boolean canBuild() { + SourceDetails[] sourceDetailsArray = streamConfig.getSourceDetails(); + if (sourceDetailsArray.length != 1) { + return false; + } else { + SourceName sourceName = sourceDetailsArray[0].getSourceName(); + SourceType sourceType = sourceDetailsArray[0].getSourceType(); + return sourceName.equals(SUPPORTED_SOURCE_NAME) && sourceType.equals(SUPPORTED_SOURCE_TYPE) + && deserializer instanceof SimpleGroupDeserializer; + } + } + + FileSource buildFileSource() { + ParquetFileSource.Builder parquetFileSourceBuilder = ParquetFileSource.Builder.getInstance(); + ParquetFileRecordFormat parquetFileRecordFormat = buildParquetFileRecordFormat(); + FileSplitAssigner.Provider splitAssignerProvider = buildParquetFileSplitAssignerProvider(); + Path[] filePaths = buildFlinkFilePaths(); + + ParquetFileSource parquetFileSource = parquetFileSourceBuilder.setFilePaths(filePaths) + .setConfiguration(configuration) + .setFileRecordFormat(parquetFileRecordFormat) + .setSourceType(SUPPORTED_SOURCE_TYPE) + .setFileSplitAssigner(splitAssignerProvider) + .build(); + return parquetFileSource.buildFileSource(); + } + + private Path[] buildFlinkFilePaths() { + String[] parquetFilePaths = streamConfig.getParquetFilePaths(); + return Arrays.stream(parquetFilePaths) + .map(Path::new) + .toArray(Path[]::new); + } + + private FileSplitAssigner.Provider buildParquetFileSplitAssignerProvider() { + SourceParquetReadOrderStrategy readOrderStrategy = streamConfig.getParquetFilesReadOrderStrategy(); + switch (readOrderStrategy) { + case EARLIEST_TIME_URL_FIRST: + return ChronologyOrderedSplitAssigner::new; + case EARLIEST_INDEX_FIRST: + default: + throw new DaggerConfigurationException("Error: file split assignment strategy not configured or not supported yet."); + } + } + + private ParquetFileRecordFormat buildParquetFileRecordFormat() { + SimpleGroupDeserializer simpleGroupDeserializer = (SimpleGroupDeserializer) deserializer; + ReaderProvider parquetFileReaderProvider = new ParquetReader.ParquetReaderProvider(simpleGroupDeserializer); + ParquetFileRecordFormat.Builder parquetFileRecordFormatBuilder = ParquetFileRecordFormat.Builder.getInstance(); + Supplier> typeInformationProvider = (Supplier> & Serializable) simpleGroupDeserializer::getProducedType; + return parquetFileRecordFormatBuilder + .setParquetFileReaderProvider(parquetFileReaderProvider) + .setTypeInformationProvider(typeInformationProvider) + .build(); + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormat.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormat.java new file mode 100644 index 000000000..6bb5747fc --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormat.java @@ -0,0 +1,73 @@ +package io.odpf.dagger.core.source.parquet; + +import static com.google.api.client.util.Preconditions.checkArgument; + +import io.odpf.dagger.core.source.parquet.reader.ReaderProvider; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import java.util.function.Supplier; + +public class ParquetFileRecordFormat implements FileRecordFormat { + private final ReaderProvider parquetFileReaderProvider; + private final Supplier> typeInformationProvider; + + private ParquetFileRecordFormat(ReaderProvider parquetFileReaderProvider, Supplier> typeInformationProvider) { + this.parquetFileReaderProvider = parquetFileReaderProvider; + this.typeInformationProvider = typeInformationProvider; + } + + @Override + public Reader createReader(Configuration config, Path filePath, long splitOffset, long splitLength) { + return parquetFileReaderProvider.getReader(filePath.toString()); + } + + /* TO DO: Need to implement a way on how to re-create the reader from saved state or from checkpoint */ + @Override + public Reader restoreReader(Configuration config, Path filePath, long restoredOffset, long splitOffset, long splitLength) { + throw new UnsupportedOperationException("Error: Restoring a reader from saved state is not implemented yet"); + } + + @Override + public boolean isSplittable() { + return false; + } + + @Override + public TypeInformation getProducedType() { + return typeInformationProvider.get(); + } + + public static class Builder { + private ReaderProvider parquetFileReaderProvider; + private Supplier> typeInformationProvider; + + public static Builder getInstance() { + return new Builder(); + } + + private Builder() { + this.parquetFileReaderProvider = null; + this.typeInformationProvider = null; + } + + public Builder setParquetFileReaderProvider(ReaderProvider parquetFileReaderProvider) { + this.parquetFileReaderProvider = parquetFileReaderProvider; + return this; + } + + public Builder setTypeInformationProvider(Supplier> typeInformationProvider) { + this.typeInformationProvider = typeInformationProvider; + return this; + } + + public ParquetFileRecordFormat build() { + checkArgument(parquetFileReaderProvider != null, "ReaderProvider is required but is set as null"); + checkArgument(typeInformationProvider != null, "TypeInformationProvider is required but is set as null"); + return new ParquetFileRecordFormat(parquetFileReaderProvider, typeInformationProvider); + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileSource.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileSource.java new file mode 100644 index 000000000..8eae3a484 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/ParquetFileSource.java @@ -0,0 +1,109 @@ +package io.odpf.dagger.core.source.parquet; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.core.source.SourceType; +import lombok.Getter; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; +import org.apache.flink.connector.file.src.assigners.LocalityAwareSplitAssigner; +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +import static com.google.api.client.util.Preconditions.checkArgument; +import static io.odpf.dagger.core.source.SourceType.BOUNDED; + +public class ParquetFileSource implements Serializable { + @Getter + private final SourceType sourceType; + @Getter + private final Path[] filePaths; + @Getter + private final Configuration configuration; + @Getter + private final FileRecordFormat fileRecordFormat; + @Getter + private final FileSplitAssigner.Provider fileSplitAssigner; + + private ParquetFileSource(SourceType sourceType, + Configuration configuration, + FileRecordFormat fileRecordFormat, + Path[] filePaths, + FileSplitAssigner.Provider fileSplitAssigner) { + this.sourceType = sourceType; + this.configuration = configuration; + this.filePaths = filePaths; + this.fileRecordFormat = fileRecordFormat; + this.fileSplitAssigner = fileSplitAssigner; + } + + public FileSource buildFileSource() { + return FileSource.forRecordFileFormat(fileRecordFormat, filePaths) + .setSplitAssigner(fileSplitAssigner) + .build(); + } + + public static class Builder { + private SourceType sourceType; + private Path[] filePaths; + private FileRecordFormat fileRecordFormat; + private Configuration configuration; + private FileSplitAssigner.Provider fileSplitAssigner; + + public static Builder getInstance() { + return new Builder(); + } + + private Builder() { + this.sourceType = SourceType.BOUNDED; + this.configuration = null; + this.fileRecordFormat = null; + this.filePaths = new Path[0]; + this.fileSplitAssigner = LocalityAwareSplitAssigner::new; + } + + public Builder setSourceType(SourceType sourceType) { + this.sourceType = sourceType; + return this; + } + + public Builder setFileRecordFormat(FileRecordFormat fileRecordFormat) { + this.fileRecordFormat = fileRecordFormat; + return this; + } + + public Builder setFileSplitAssigner(FileSplitAssigner.Provider fileSplitAssigner) { + this.fileSplitAssigner = fileSplitAssigner; + return this; + } + + public Builder setFilePaths(Path[] filePaths) { + this.filePaths = filePaths; + return this; + } + + public Builder setConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + /* other validations if required before creating the file source can be put here */ + /* for example, checking that all the file paths conform to just one partitioning strategy */ + private void sanityCheck() { + checkArgument(fileRecordFormat != null, "FileRecordFormat is required but is set as null"); + checkArgument(filePaths.length != 0, "At least one file path is required but none are provided"); + checkArgument(sourceType == BOUNDED, "Running Parquet FileSource in UNBOUNDED mode is not supported yet"); + } + + public ParquetFileSource build() { + sanityCheck(); + return new ParquetFileSource(sourceType, + configuration, + fileRecordFormat, + filePaths, + fileSplitAssigner); + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetReadOrderStrategy.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetReadOrderStrategy.java new file mode 100644 index 000000000..2f6849ff4 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetReadOrderStrategy.java @@ -0,0 +1,13 @@ +package io.odpf.dagger.core.source.parquet; + +import com.google.gson.annotations.SerializedName; + +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_INDEX_FIRST; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_TIME_URL_FIRST; + +public enum SourceParquetReadOrderStrategy { + @SerializedName(STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_TIME_URL_FIRST) + EARLIEST_TIME_URL_FIRST, + @SerializedName(STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_INDEX_FIRST) + EARLIEST_INDEX_FIRST +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetSchemaMatchStrategy.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetSchemaMatchStrategy.java new file mode 100644 index 000000000..8300f4dc7 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/SourceParquetSchemaMatchStrategy.java @@ -0,0 +1,13 @@ +package io.odpf.dagger.core.source.parquet; + +import com.google.gson.annotations.SerializedName; + +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_PARQUET_BACKWARD_COMPATIBLE_SCHEMA_MATCH_STRATEGY; +import static io.odpf.dagger.core.utils.Constants.STREAM_SOURCE_PARQUET_SAME_SCHEMA_MATCH_STRATEGY; + +public enum SourceParquetSchemaMatchStrategy { + @SerializedName(STREAM_SOURCE_PARQUET_SAME_SCHEMA_MATCH_STRATEGY) + SAME_SCHEMA_WITH_FAIL_ON_MISMATCH, + @SerializedName(STREAM_SOURCE_PARQUET_BACKWARD_COMPATIBLE_SCHEMA_MATCH_STRATEGY) + BACKWARD_COMPATIBLE_SCHEMA_WITH_FAIL_ON_TYPE_MISMATCH +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ParquetReader.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ParquetReader.java new file mode 100644 index 000000000..ceae39203 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ParquetReader.java @@ -0,0 +1,128 @@ +package io.odpf.dagger.core.source.parquet.reader; + +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.core.exception.ParquetFileSourceReaderInitializationException; +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.ColumnIOFactory; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.io.RecordReader; +import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; + +public class ParquetReader implements FileRecordFormat.Reader { + private final Path hadoopFilePath; + private final SimpleGroupDeserializer simpleGroupDeserializer; + private long currentRecordIndex; + private final ParquetFileReader parquetFileReader; + private long rowCount; + private boolean isRecordReaderInitialized; + private RecordReader recordReader; + private final MessageType schema; + private static final Logger LOGGER = LoggerFactory.getLogger(ParquetReader.class.getName()); + + private ParquetReader(Path hadoopFilePath, SimpleGroupDeserializer simpleGroupDeserializer, ParquetFileReader parquetFileReader) throws IOException { + this.hadoopFilePath = hadoopFilePath; + this.simpleGroupDeserializer = simpleGroupDeserializer; + this.parquetFileReader = parquetFileReader; + this.schema = this.parquetFileReader.getFileMetaData().getSchema(); + this.isRecordReaderInitialized = false; + } + + private boolean checkIfNullPage(PageReadStore page) { + if (page == null) { + String logMessage = String.format("No more data found in Parquet file %s", hadoopFilePath.getName()); + LOGGER.info(logMessage); + return true; + } + return false; + } + + private void changeReaderPosition(PageReadStore pages) { + rowCount = pages.getRowCount(); + currentRecordIndex = 0; + MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema); + recordReader = columnIO.getRecordReader(pages, new GroupRecordConverter(schema)); + } + + private void initializeRecordReader() throws IOException { + PageReadStore nextPage = parquetFileReader.readNextRowGroup(); + changeReaderPosition(nextPage); + this.isRecordReaderInitialized = true; + String logMessage = String.format("Successfully created the ParquetFileReader and RecordReader for file %s", hadoopFilePath.getName()); + LOGGER.info(logMessage); + } + + private Row readRecords() throws IOException { + if (currentRecordIndex >= rowCount) { + PageReadStore nextPage = parquetFileReader.readNextRowGroup(); + if (checkIfNullPage(nextPage)) { + return null; + } + changeReaderPosition(nextPage); + } + return readAndDeserialize(); + } + + @Nullable + @Override + public Row read() throws IOException { + if (!isRecordReaderInitialized) { + initializeRecordReader(); + } + return readRecords(); + } + + private Row readAndDeserialize() { + SimpleGroup simpleGroup = (SimpleGroup) recordReader.read(); + currentRecordIndex++; + return simpleGroupDeserializer.deserialize(simpleGroup); + } + + @Override + public void close() throws IOException { + parquetFileReader.close(); + closeRecordReader(); + String logMessage = String.format("Closed the ParquetFileReader and de-referenced the RecordReader for file %s", hadoopFilePath.getName()); + LOGGER.info(logMessage); + } + + private void closeRecordReader() { + if (isRecordReaderInitialized) { + this.isRecordReaderInitialized = false; + } + recordReader = null; + } + + public static class ParquetReaderProvider implements ReaderProvider { + private final SimpleGroupDeserializer simpleGroupDeserializer; + + public ParquetReaderProvider(SimpleGroupDeserializer simpleGroupDeserializer) { + this.simpleGroupDeserializer = simpleGroupDeserializer; + } + + @Override + public ParquetReader getReader(String filePath) { + try { + Configuration conf = new Configuration(); + Path hadoopFilePath = new Path(filePath); + ParquetFileReader parquetFileReader = ParquetFileReader.open(HadoopInputFile.fromPath(hadoopFilePath, conf)); + return new ParquetReader(hadoopFilePath, simpleGroupDeserializer, parquetFileReader); + } catch (IOException | RuntimeException ex) { + throw new ParquetFileSourceReaderInitializationException(ex); + } + } + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ReaderProvider.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ReaderProvider.java new file mode 100644 index 000000000..3fb592970 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/reader/ReaderProvider.java @@ -0,0 +1,11 @@ +package io.odpf.dagger.core.source.parquet.reader; + +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +@FunctionalInterface +public interface ReaderProvider extends Serializable { + FileRecordFormat.Reader getReader(String filePath); +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssigner.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssigner.java new file mode 100644 index 000000000..b1922adc7 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssigner.java @@ -0,0 +1,104 @@ +package io.odpf.dagger.core.source.parquet.splitassigner; + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nullable; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.time.Instant; +import java.util.Collection; +import java.util.Comparator; +import java.util.Optional; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class ChronologyOrderedSplitAssigner implements FileSplitAssigner { + private final PriorityBlockingQueue unassignedSplits; + private final Pattern filePathPattern = Pattern.compile("^.*/dt=([0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9])/(hr=([0-9][0-9]))?.*$"); + private static final int INITIAL_DEFAULT_CAPACITY = 11; + + public ChronologyOrderedSplitAssigner(Collection fileSourceSplits) { + this.unassignedSplits = new PriorityBlockingQueue<>(INITIAL_DEFAULT_CAPACITY, getFileSourceSplitComparator()); + for (FileSourceSplit split : fileSourceSplits) { + validateAndAddSplits(split); + } + } + + private void validateAndAddSplits(FileSourceSplit split) { + try { + Instant instant = parseInstantFromFilePath(split.path()); + this.unassignedSplits.add(new InstantEnrichedSplit(split, instant)); + } catch (ParseException ex) { + throw new IllegalArgumentException(ex); + } + } + + @Override + public Optional getNext(@Nullable String hostname) { + InstantEnrichedSplit instantEnrichedSplit = unassignedSplits.poll(); + if (instantEnrichedSplit == null) { + return Optional.empty(); + } + return Optional.of(instantEnrichedSplit.getFileSourceSplit()); + } + + @Override + public void addSplits(Collection splits) { + for (FileSourceSplit split : splits) { + validateAndAddSplits(split); + } + } + + @Override + public Collection remainingSplits() { + return unassignedSplits + .stream() + .map(InstantEnrichedSplit::getFileSourceSplit) + .collect(Collectors.toList()); + } + + private Comparator getFileSourceSplitComparator() { + return (instantEnrichedSplit1, instantEnrichedSplit2) -> { + Instant instant1 = instantEnrichedSplit1.getInstant(); + Instant instant2 = instantEnrichedSplit2.getInstant(); + if (instant1.isBefore(instant2)) { + return -1; + } else if (instant1.isAfter(instant2)) { + return 1; + } else { + return 0; + } + }; + } + + private Instant parseInstantFromFilePath(Path path) throws ParseException { + Matcher matcher = filePathPattern.matcher(path.toString()); + final int hourMatcherGroupNumber = 3; + final int dateMatcherGroupNumber = 1; + boolean matchFound = matcher.find(); + if (matchFound && matcher.group(hourMatcherGroupNumber) != null && matcher.group(dateMatcherGroupNumber) != null) { + return convertToInstant(matcher.group(dateMatcherGroupNumber), matcher.group(hourMatcherGroupNumber)); + } else if (matchFound && matcher.group(hourMatcherGroupNumber) == null && matcher.group(dateMatcherGroupNumber) != null) { + return convertToInstant(matcher.group(dateMatcherGroupNumber)); + } else { + String message = String.format("Cannot extract timestamp from filepath for deciding order of processing.\n" + + "File path doesn't abide with any partitioning strategy: %s", path); + throw new ParseException(message, 0); + } + } + + private Instant convertToInstant(String dateSegment) throws ParseException { + SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd"); + return formatter.parse(dateSegment).toInstant(); + } + + private Instant convertToInstant(String dateSegment, String hourSegment) throws ParseException { + SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH"); + String dateHourString = String.join(" ", dateSegment, hourSegment); + return formatter.parse(dateHourString).toInstant(); + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/IndexOrderedSplitAssigner.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/IndexOrderedSplitAssigner.java new file mode 100644 index 000000000..e66194ca0 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/IndexOrderedSplitAssigner.java @@ -0,0 +1,30 @@ +package io.odpf.dagger.core.source.parquet.splitassigner; + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.Optional; + +/* TODO */ +public class IndexOrderedSplitAssigner implements FileSplitAssigner { + + public IndexOrderedSplitAssigner(Collection fileSourceSplits) { + } + + @Override + public Optional getNext(@Nullable String hostname) { + return Optional.empty(); + } + + @Override + public void addSplits(Collection splits) { + + } + + @Override + public Collection remainingSplits() { + return null; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/InstantEnrichedSplit.java b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/InstantEnrichedSplit.java new file mode 100644 index 000000000..73b683a77 --- /dev/null +++ b/dagger-core/src/main/java/io/odpf/dagger/core/source/parquet/splitassigner/InstantEnrichedSplit.java @@ -0,0 +1,19 @@ +package io.odpf.dagger.core.source.parquet.splitassigner; + +import lombok.Getter; +import org.apache.flink.connector.file.src.FileSourceSplit; + +import java.io.Serializable; +import java.time.Instant; + +public class InstantEnrichedSplit implements Serializable { + @Getter + private final FileSourceSplit fileSourceSplit; + @Getter + private final Instant instant; + + public InstantEnrichedSplit(FileSourceSplit fileSourceSplit, Instant instant) { + this.fileSourceSplit = fileSourceSplit; + this.instant = instant; + } +} diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java b/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java index a6a683779..4db163156 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java @@ -95,8 +95,25 @@ public class Constants { public static final String STREAM_SOURCE_KAFKA_TOPIC_NAMES_KEY = "SOURCE_KAFKA_TOPIC_NAMES"; public static final String STREAM_INPUT_STREAM_NAME_KEY = "SOURCE_KAFKA_NAME"; + public static final String STREAM_SOURCE_DETAILS_KEY = "SOURCE_DETAILS"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_TYPE_KEY = "SOURCE_TYPE"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_TYPE_BOUNDED = "BOUNDED"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_TYPE_UNBOUNDED = "UNBOUNDED"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_NAME_KEY = "SOURCE_NAME"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA = "KAFKA_SOURCE"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_NAME_PARQUET = "PARQUET_SOURCE"; + public static final String STREAM_SOURCE_DETAILS_SOURCE_NAME_KAFKA_CONSUMER = "KAFKA_CONSUMER"; + public static final String STREAM_SOURCE_PARQUET_FILE_PATHS_KEY = "SOURCE_PARQUET_FILE_PATHS"; + public static final String STREAM_SOURCE_PARQUET_BILLING_PROJECT_KEY = "SOURCE_PARQUET_BILLING_PROJECT"; + public static final String STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_KEY = "SOURCE_PARQUET_READ_ORDER_STRATEGY"; + public static final String STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_TIME_URL_FIRST = "EARLIEST_TIME_URL_FIRST"; + public static final String STREAM_SOURCE_PARQUET_READ_ORDER_STRATEGY_EARLIEST_INDEX_FIRST = "EARLIEST_INDEX_FIRST"; + public static final String STREAM_SOURCE_PARQUET_SCHEMA_MATCH_STRATEGY_KEY = "SOURCE_PARQUET_SCHEMA_MATCH_STRATEGY"; + public static final String STREAM_SOURCE_PARQUET_SAME_SCHEMA_MATCH_STRATEGY = "SAME_SCHEMA_WITH_FAIL_ON_MISMATCH"; + public static final String STREAM_SOURCE_PARQUET_BACKWARD_COMPATIBLE_SCHEMA_MATCH_STRATEGY = "BACKWARD_COMPATIBLE_SCHEMA_WITH_FAIL_ON_TYPE_MISMATCH"; + + public static final String STREAM_INPUT_DATATYPE = "INPUT_DATATYPE"; - public static final String STREAM_INPUT_SOURCE_TYPE = "INPUT_SOURCE_TYPE"; public static final String STREAM_INPUT_SCHEMA_JSON_EVENT_TIMESTAMP_FIELD_NAME_KEY = "INPUT_SCHEMA_JSON_EVENT_TIMESTAMP_FIELD_NAME"; public static final String STREAM_INPUT_SCHEMA_JSON_SCHEMA_KEY = "INPUT_SCHEMA_JSON_SCHEMA"; diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/StreamManagerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/StreamManagerTest.java index 0600d7cd5..ff8c1af83 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/StreamManagerTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/StreamManagerTest.java @@ -53,7 +53,6 @@ public class StreamManagerTest { + " \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"flink-sql-flud-gp0330\",\n" + " \"INPUT_SCHEMA_PROTO_CLASS\": \"io.odpf.dagger.consumer.TestBookingLogMessage\",\n" + " \"INPUT_SCHEMA_TABLE\": \"data_stream\",\n" - + " \"INPUT_SOURCE_TYPE\": \"KAFKA_SOURCE\",\n" + " \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\"\n" + " }\n" + "]"; @@ -94,9 +93,6 @@ public class StreamManagerTest { @Mock private Stream stream; - @Mock - private WatermarkStrategy watermarkStrategy; - @Before public void setup() { @@ -147,7 +143,7 @@ public void shouldRegisterRequiredConfigsOnExecutionEnvironment() { @Test public void shouldRegisterSourceWithPreprocessorsWithWaterMarks() { when(singleOutputStream.assignTimestampsAndWatermarks(any(WatermarkStrategy.class))).thenReturn(singleOutputStream); - when(stream.registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class), any(String.class))).thenReturn(singleOutputStream); + when(stream.registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class))).thenReturn(singleOutputStream); when(singleOutputStream.getType()).thenReturn(typeInformation); when(stream.getStreamName()).thenReturn("data_stream"); @@ -161,7 +157,7 @@ public void shouldRegisterSourceWithPreprocessorsWithWaterMarks() { @Test public void shouldCreateValidSourceWithWatermarks() { when(singleOutputStream.assignTimestampsAndWatermarks(any(WatermarkStrategy.class))).thenReturn(singleOutputStream); - when(stream.registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class), any(String.class))).thenReturn(singleOutputStream); + when(stream.registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class))).thenReturn(singleOutputStream); when(singleOutputStream.getType()).thenReturn(typeInformation); when(stream.getStreamName()).thenReturn("data_stream"); @@ -169,7 +165,7 @@ public void shouldCreateValidSourceWithWatermarks() { streamManagerStub.registerConfigs(); streamManagerStub.registerSourceWithPreProcessors(); - verify(stream, Mockito.times(1)).registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class), any(String.class)); + verify(stream, Mockito.times(1)).registerSource(any(StreamExecutionEnvironment.class), any(WatermarkStrategy.class)); } @Test diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactoryTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactoryTest.java new file mode 100644 index 000000000..36f7fa332 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/DaggerDeserializerFactoryTest.java @@ -0,0 +1,90 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.consumer.TestBookingLogMessage; +import io.odpf.dagger.core.exception.DaggerConfigurationException; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.stencil.client.StencilClient; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class DaggerDeserializerFactoryTest { + + @Mock + private StreamConfig streamConfig; + + @Mock + private Configuration configuration; + + @Mock + private StencilClientOrchestrator stencilClientOrchestrator; + + @Mock + private StencilClient stencilClient; + + @Before + public void setUp() throws Exception { + initMocks(this); + } + + @Test + public void shouldReturnJsonDeserializerWhenConfigured() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + when(streamConfig.getJsonSchema()).thenReturn("{ \"$schema\": \"https://json-schema.org/draft/2020-12/schema\", \"$id\": \"https://example.com/product.schema.json\", \"title\": \"Product\", \"description\": \"A product from Acme's catalog\", \"type\": \"object\", \"properties\": { \"id\": { \"description\": \"The unique identifier for a product\", \"type\": \"string\" }, \"time\": { \"description\": \"event timestamp of the event\", \"type\": \"string\", \"format\" : \"date-time\" } }, \"required\": [ \"id\", \"time\" ] }"); + + DaggerDeserializer daggerDeserializer = DaggerDeserializerFactory.create(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(daggerDeserializer instanceof JsonDeserializer); + } + + @Test + public void shouldReturnProtoDeserializerWhenConfigured() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + DaggerDeserializer daggerDeserializer = DaggerDeserializerFactory.create(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(daggerDeserializer instanceof ProtoDeserializer); + } + + @Test + public void shouldReturnSimpleGroupDeserializerWhenConfigured() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + DaggerDeserializer daggerDeserializer = DaggerDeserializerFactory.create(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(daggerDeserializer instanceof SimpleGroupDeserializer); + } + + @Test + public void shouldThrowRuntimeExceptionIfNoDeserializerCouldBeCreatedFromConfigs() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + assertThrows(DaggerConfigurationException.class, () -> DaggerDeserializerFactory.create(streamConfig, configuration, stencilClientOrchestrator)); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/JsonDeserializerProviderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/JsonDeserializerProviderTest.java new file mode 100644 index 000000000..f889ad8aa --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/JsonDeserializerProviderTest.java @@ -0,0 +1,79 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class JsonDeserializerProviderTest { + @Mock + private StreamConfig streamConfig; + + @Before + public void setUp() throws Exception { + initMocks(this); + } + + @Test + public void shouldBeAbleToProvideJsonDeserializerWhenSourceNameIsKafkaAndSchemaTypeIsJSON() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + JsonDeserializerProvider provider = new JsonDeserializerProvider(streamConfig); + + assertTrue(provider.canProvide()); + } + + @Test + public void shouldBeAbleToProvideJsonDeserializerWhenSourceNameIsKafkaConsumerAndSchemaTypeIsJSON() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + JsonDeserializerProvider provider = new JsonDeserializerProvider(streamConfig); + + assertTrue(provider.canProvide()); + } + + @Test + public void shouldNotProvideJsonDeserializerWhenSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + JsonDeserializerProvider provider = new JsonDeserializerProvider(streamConfig); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldNotProvideJsonDeserializerWhenSchemaTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + JsonDeserializerProvider provider = new JsonDeserializerProvider(streamConfig); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldReturnJsonDeserializerForSupportedSourceNameAndSchemaType() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + when(streamConfig.getJsonSchema()).thenReturn("{ \"$schema\": \"https://json-schema.org/draft/2020-12/schema\", \"$id\": \"https://example.com/product.schema.json\", \"title\": \"Product\", \"description\": \"A product from Acme's catalog\", \"type\": \"object\", \"properties\": { \"id\": { \"description\": \"The unique identifier for a product\", \"type\": \"string\" }, \"time\": { \"description\": \"event timestamp of the event\", \"type\": \"string\", \"format\" : \"date-time\" } }, \"required\": [ \"id\", \"time\" ] }"); + + JsonDeserializerProvider provider = new JsonDeserializerProvider(streamConfig); + DaggerDeserializer daggerDeserializer = provider.getDaggerDeserializer(); + + assertTrue(daggerDeserializer instanceof JsonDeserializer); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProviderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProviderTest.java new file mode 100644 index 000000000..5c1188476 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/ProtoDeserializerProviderTest.java @@ -0,0 +1,94 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.consumer.TestBookingLogMessage; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.stencil.client.StencilClient; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class ProtoDeserializerProviderTest { + @Mock + private StreamConfig streamConfig; + + @Mock + private Configuration configuration; + + @Mock + private StencilClientOrchestrator stencilClientOrchestrator; + + @Mock + private StencilClient stencilClient; + + @Before + public void setUp() throws Exception { + initMocks(this); + } + + @Test + public void shouldBeAbleToProvideProtoDeserializerWhenSourceNameIsKafkaAndSchemaTypeIsPROTO() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + ProtoDeserializerProvider provider = new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(provider.canProvide()); + } + + @Test + public void shouldBeAbleToProvideProtoDeserializerWhenSourceNameIsKafkaConsumerAndSchemaTypeIsPROTO() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + ProtoDeserializerProvider provider = new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(provider.canProvide()); + } + + @Test + public void shouldNotProvideProtoDeserializerWhenSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + ProtoDeserializerProvider provider = new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldNotProvideProtoDeserializerWhenSchemaTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + ProtoDeserializerProvider provider = new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldReturnProtoDeserializerForSupportedSourceNameAndSchemaType() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + ProtoDeserializerProvider provider = new ProtoDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + DaggerDeserializer daggerDeserializer = provider.getDaggerDeserializer(); + + assertTrue(daggerDeserializer instanceof ProtoDeserializer); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProviderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProviderTest.java new file mode 100644 index 000000000..3838b9b6b --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/deserializer/SimpleGroupDeserializerProviderTest.java @@ -0,0 +1,85 @@ +package io.odpf.dagger.core.deserializer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.consumer.TestBookingLogMessage; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import io.odpf.stencil.client.StencilClient; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class SimpleGroupDeserializerProviderTest { + + @Mock + private StreamConfig streamConfig; + + @Mock + private Configuration configuration; + + @Mock + private StencilClientOrchestrator stencilClientOrchestrator; + + @Mock + private StencilClient stencilClient; + + @Before + public void setUp() throws Exception { + initMocks(this); + } + + @Test + public void shouldBeAbleToProvideSimpleGroupDeserializerWhenSourceNameIsParquetAndSchemaTypeIsPROTO() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + SimpleGroupDeserializerProvider provider = new SimpleGroupDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertTrue(provider.canProvide()); + } + + @Test + public void shouldNotProvideSimpleGroupDeserializerWhenSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + + SimpleGroupDeserializerProvider provider = new SimpleGroupDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldNotProvideSimpleGroupDeserializerWhenSchemaTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + + SimpleGroupDeserializerProvider provider = new SimpleGroupDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + + assertFalse(provider.canProvide()); + } + + @Test + public void shouldReturnSimpleGroupDeserializerForSupportedSourceNameAndSchemaType() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + SimpleGroupDeserializerProvider provider = new SimpleGroupDeserializerProvider(streamConfig, configuration, stencilClientOrchestrator); + DaggerDeserializer daggerDeserializer = provider.getDaggerDeserializer(); + + assertTrue(daggerDeserializer instanceof SimpleGroupDeserializer); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/DaggerSourceFactoryTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/DaggerSourceFactoryTest.java index cdcea31af..50845d98f 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/DaggerSourceFactoryTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/DaggerSourceFactoryTest.java @@ -1,73 +1,67 @@ package io.odpf.dagger.core.source; import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.core.source.kafka.DaggerKafkaSource; -import io.odpf.dagger.core.source.kafka.DaggerOldKafkaSource; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.junit.Assert; +import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.core.exception.InvalidDaggerSourceException; +import io.odpf.dagger.core.source.flinkkafkaconsumer.FlinkKafkaConsumerDaggerSource; +import io.odpf.dagger.core.source.kafka.KafkaDaggerSource; +import io.odpf.dagger.core.source.parquet.ParquetDaggerSource; +import org.apache.flink.types.Row; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; +import org.mockito.Mockito; -import java.util.HashMap; -import java.util.Properties; -import java.util.regex.Pattern; - -import static org.junit.Assert.assertThrows; -import static org.mockito.ArgumentMatchers.any; +import static org.junit.Assert.*; import static org.mockito.Mockito.when; import static org.mockito.MockitoAnnotations.initMocks; public class DaggerSourceFactoryTest { - @Mock private StreamConfig streamConfig; - @Mock - private KafkaDeserializationSchema deserializationSchema; - @Mock private Configuration configuration; @Before - public void setup() { + public void setUp() throws Exception { initMocks(this); - HashMap kafkaPropMap = new HashMap<>(); - kafkaPropMap.put("group.id", "dummy-consumer-group"); - kafkaPropMap.put("bootstrap.servers", "localhost:9092"); + } - Properties properties = new Properties(); - properties.putAll(kafkaPropMap); + @Test + public void shouldReturnKafkaDaggerSourceWhenConfigured() { + ProtoDeserializer deserializer = Mockito.mock(ProtoDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + DaggerSource daggerSource = DaggerSourceFactory.create(streamConfig, configuration, deserializer); - when(streamConfig.getDataType()).thenReturn("PROTO"); - when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); - when(streamConfig.getEventTimestampFieldIndex()).thenReturn("1"); - when(streamConfig.getKafkaProps(any())).thenReturn(properties); - when(streamConfig.getStartingOffset()).thenReturn(OffsetsInitializer.committedOffsets(OffsetResetStrategy.valueOf("LATEST"))); - when(streamConfig.getSchemaTable()).thenReturn("test-table"); - when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test")); + assertTrue(daggerSource instanceof KafkaDaggerSource); } @Test - public void shouldCreateOldKafkaSourceIfGivenInConfig() { - when(streamConfig.getSourceType()).thenReturn("OLD_KAFKA_SOURCE"); - DaggerSource daggerSource = DaggerSourceFactory.createDaggerSource(streamConfig, deserializationSchema, configuration); - Assert.assertTrue(daggerSource instanceof DaggerOldKafkaSource); + public void shouldReturnFlinkKafkaConsumerDaggerSourceWhenConfigured() { + JsonDeserializer deserializer = Mockito.mock(JsonDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + DaggerSource daggerSource = DaggerSourceFactory.create(streamConfig, configuration, deserializer); + + assertTrue(daggerSource instanceof FlinkKafkaConsumerDaggerSource); } @Test - public void shouldCreateKafkaSourceIfGivenInConfig() { - when(streamConfig.getSourceType()).thenReturn("KAFKA_SOURCE"); - DaggerSource daggerSource = DaggerSourceFactory.createDaggerSource(streamConfig, deserializationSchema, configuration); - Assert.assertTrue(daggerSource instanceof DaggerKafkaSource); + public void shouldReturnParquetDaggerSourceWhenConfigured() { + SimpleGroupDeserializer deserializer = Mockito.mock(SimpleGroupDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + DaggerSource daggerSource = DaggerSourceFactory.create(streamConfig, configuration, deserializer); + + assertTrue(daggerSource instanceof ParquetDaggerSource); } @Test - public void shouldThrowExceptionIfNotSupportedSourceGiven() { - when(streamConfig.getSourceType()).thenReturn("TEST"); - assertThrows(IllegalArgumentException.class, - () -> DaggerSourceFactory.createDaggerSource(streamConfig, deserializationSchema, configuration)); + public void shouldThrowRuntimeExceptionIfNoDaggerSourceCouldBeCreatedAsPerConfigs() { + SimpleGroupDeserializer deserializer = Mockito.mock(SimpleGroupDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.UNBOUNDED)}); + + assertThrows(InvalidDaggerSourceException.class, () -> DaggerSourceFactory.create(streamConfig, configuration, deserializer)); } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamConfigTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamConfigTest.java index 75ad736d5..fada741d9 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamConfigTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamConfigTest.java @@ -2,6 +2,8 @@ import com.google.gson.JsonSyntaxException; import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.core.source.parquet.SourceParquetReadOrderStrategy; +import io.odpf.dagger.core.source.parquet.SourceParquetSchemaMatchStrategy; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.junit.Assert; import org.junit.Before; @@ -15,8 +17,11 @@ import java.util.regex.Pattern; import static io.odpf.dagger.common.core.Constants.INPUT_STREAMS; +import static io.odpf.dagger.core.source.SourceName.KAFKA_CONSUMER; +import static io.odpf.dagger.core.source.SourceType.UNBOUNDED; import static io.odpf.dagger.core.utils.Constants.SOURCE_KAFKA_CONSUME_LARGE_MESSAGE_ENABLE_DEFAULT; import static io.odpf.dagger.core.utils.Constants.SOURCE_KAFKA_CONSUME_LARGE_MESSAGE_ENABLE_KEY; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; import static org.mockito.MockitoAnnotations.initMocks; @@ -38,7 +43,7 @@ public void shouldParseStreamConfigs() { when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - Assert.assertEquals(1, streamConfigs.length); + assertEquals(1, streamConfigs.length); } @Test @@ -47,14 +52,14 @@ public void shouldSetConfigurationsFromJsonStreamConfig() { StreamConfig[] streamConfigs = StreamConfig.parse(configuration); StreamConfig currConfig = streamConfigs[0]; - Assert.assertEquals("false", currConfig.getAutoCommitEnable()); - Assert.assertEquals("latest", currConfig.getAutoOffsetReset()); - Assert.assertEquals("PROTO", currConfig.getDataType()); - Assert.assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); - Assert.assertEquals("41", currConfig.getEventTimestampFieldIndex()); - Assert.assertEquals("test-topic", currConfig.getKafkaTopicNames()); - Assert.assertEquals("data_stream", currConfig.getSchemaTable()); - Assert.assertEquals("local-kafka-stream", currConfig.getKafkaName()); + assertEquals("false", currConfig.getAutoCommitEnable()); + assertEquals("latest", currConfig.getAutoOffsetReset()); + assertEquals("PROTO", currConfig.getDataType()); + assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); + assertEquals("41", currConfig.getEventTimestampFieldIndex()); + assertEquals("test-topic", currConfig.getKafkaTopicNames()); + assertEquals("data_stream", currConfig.getSchemaTable()); + assertEquals("local-kafka-stream", currConfig.getKafkaName()); } @Test @@ -62,27 +67,27 @@ public void shouldParseMultipleStreamsFromStreamConfigJson() { when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"false\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" }, {\"INPUT_SCHEMA_TABLE\": \"data_stream_1\", \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_DATATYPE\": \"JSON\", \"INPUT_SCHEMA_JSON_SCHEMA\": \"{ \\\"$schema\\\": \\\"https://json-schema.org/draft/2020-12/schema\\\", \\\"$id\\\": \\\"https://example.com/product.schema.json\\\", \\\"title\\\": \\\"Product\\\", \\\"description\\\": \\\"A product from Acme's catalog\\\", \\\"type\\\": \\\"object\\\", \\\"properties\\\": { \\\"id\\\": { \\\"description\\\": \\\"The unique identifier for a product\\\", \\\"type\\\": \\\"string\\\" }, \\\"time\\\": { \\\"description\\\": \\\"event timestamp of the event\\\", \\\"type\\\": \\\"string\\\", \\\"format\\\" : \\\"date-time\\\" } }, \\\"required\\\": [ \\\"id\\\", \\\"time\\\" ] }\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"true\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - Assert.assertEquals(2, streamConfigs.length); + assertEquals(2, streamConfigs.length); StreamConfig currConfig = streamConfigs[0]; - Assert.assertEquals("false", currConfig.getAutoCommitEnable()); - Assert.assertEquals("latest", currConfig.getAutoOffsetReset()); - Assert.assertEquals("PROTO", currConfig.getDataType()); - Assert.assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); - Assert.assertEquals("41", currConfig.getEventTimestampFieldIndex()); - Assert.assertEquals("test-topic", currConfig.getKafkaTopicNames()); - Assert.assertEquals("data_stream", currConfig.getSchemaTable()); - Assert.assertEquals("local-kafka-stream", currConfig.getKafkaName()); + assertEquals("false", currConfig.getAutoCommitEnable()); + assertEquals("latest", currConfig.getAutoOffsetReset()); + assertEquals("PROTO", currConfig.getDataType()); + assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); + assertEquals("41", currConfig.getEventTimestampFieldIndex()); + assertEquals("test-topic", currConfig.getKafkaTopicNames()); + assertEquals("data_stream", currConfig.getSchemaTable()); + assertEquals("local-kafka-stream", currConfig.getKafkaName()); StreamConfig currConfigNext = streamConfigs[1]; - Assert.assertEquals("true", currConfigNext.getAutoCommitEnable()); - Assert.assertEquals("latest", currConfigNext.getAutoOffsetReset()); - Assert.assertEquals("JSON", currConfigNext.getDataType()); - Assert.assertEquals("dummy-consumer-group", currConfigNext.getConsumerGroupId()); - Assert.assertEquals("41", currConfigNext.getEventTimestampFieldIndex()); - Assert.assertEquals("test-topic", currConfigNext.getKafkaTopicNames()); - Assert.assertEquals("data_stream_1", currConfigNext.getSchemaTable()); - Assert.assertEquals("local-kafka-stream", currConfigNext.getKafkaName()); + assertEquals("true", currConfigNext.getAutoCommitEnable()); + assertEquals("latest", currConfigNext.getAutoOffsetReset()); + assertEquals("JSON", currConfigNext.getDataType()); + assertEquals("dummy-consumer-group", currConfigNext.getConsumerGroupId()); + assertEquals("41", currConfigNext.getEventTimestampFieldIndex()); + assertEquals("test-topic", currConfigNext.getKafkaTopicNames()); + assertEquals("data_stream_1", currConfigNext.getSchemaTable()); + assertEquals("local-kafka-stream", currConfigNext.getKafkaName()); } @@ -102,7 +107,7 @@ public void shouldParseKafkaProperties() { Properties properties = new Properties(); properties.putAll(kafkaPropMap); - Assert.assertEquals(properties, streamConfigs[0].getKafkaProps(configuration)); + assertEquals(properties, streamConfigs[0].getKafkaProps(configuration)); } @Test @@ -121,7 +126,7 @@ public void shouldAddAdditionalKafkaConfigToKafkaProperties() { Properties properties = new Properties(); properties.putAll(kafkaPropMap); - Assert.assertEquals(properties, streamConfigs[0].getKafkaProps(configuration)); + assertEquals(properties, streamConfigs[0].getKafkaProps(configuration)); } @Test @@ -129,7 +134,7 @@ public void shouldSetValidDataTypeIfNotGiven() { when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - Assert.assertEquals("PROTO", streamConfigs[0].getDataType()); + assertEquals("PROTO", streamConfigs[0].getDataType()); } @Test @@ -137,7 +142,7 @@ public void shouldSetValidDataTypeIfGiven() { when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ {\"INPUT_DATATYPE\": \"JSON\", \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET_Random\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - Assert.assertEquals("JSON", streamConfigs[0].getDataType()); + assertEquals("JSON", streamConfigs[0].getDataType()); } @Test @@ -145,7 +150,7 @@ public void shouldGetTopicPattern() { when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - Assert.assertEquals(Pattern.compile("test-topic").pattern(), streamConfigs[0].getTopicPattern().pattern()); + assertEquals(Pattern.compile("test-topic").pattern(), streamConfigs[0].getTopicPattern().pattern()); } @Test @@ -154,7 +159,7 @@ public void shouldGetOffsetResetStrategy() { StreamConfig[] streamConfigs = StreamConfig.parse(configuration); OffsetResetStrategy autoOffsetResetStrategy = streamConfigs[0].getStartingOffset().getAutoOffsetResetStrategy(); - Assert.assertEquals(OffsetResetStrategy.valueOf("LATEST"), autoOffsetResetStrategy); + assertEquals(OffsetResetStrategy.valueOf("LATEST"), autoOffsetResetStrategy); } @Test @@ -165,36 +170,56 @@ public void shouldThrowInCaseOfInvalidStream() { } @Test - public void shouldReturnConfiguredSourceType() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"INPUT_SOURCE_TYPE\": \"KAFKA_SOURCE\",\"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"false\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); + public void shouldGetSourceDetails() { + when(configuration.getString(INPUT_STREAMS, "")) + .thenReturn("[{" + + "\"SOURCE_DETAILS\": " + + "[{\"SOURCE_TYPE\": \"BOUNDED\", \"SOURCE_NAME\": \"PARQUET_SOURCE\"}," + + "{\"SOURCE_TYPE\": \"UNBOUNDED\", \"SOURCE_NAME\": \"KAFKA_SOURCE\"}]" + + "}]"); StreamConfig[] streamConfigs = StreamConfig.parse(configuration); - StreamConfig currConfig = streamConfigs[0]; - Assert.assertEquals("false", currConfig.getAutoCommitEnable()); - Assert.assertEquals("latest", currConfig.getAutoOffsetReset()); - Assert.assertEquals("PROTO", currConfig.getDataType()); - Assert.assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); - Assert.assertEquals("41", currConfig.getEventTimestampFieldIndex()); - Assert.assertEquals("test-topic", currConfig.getKafkaTopicNames()); - Assert.assertEquals("data_stream", currConfig.getSchemaTable()); - Assert.assertEquals("local-kafka-stream", currConfig.getKafkaName()); - Assert.assertEquals("KAFKA_SOURCE", currConfig.getSourceType()); + SourceDetails[] sourceDetails = streamConfigs[0].getSourceDetails(); + assertEquals(SourceType.valueOf("BOUNDED"), sourceDetails[0].getSourceType()); + assertEquals(SourceName.valueOf("PARQUET_SOURCE"), sourceDetails[0].getSourceName()); + assertEquals(SourceType.valueOf("UNBOUNDED"), sourceDetails[1].getSourceType()); + assertEquals(SourceName.valueOf("KAFKA_SOURCE"), sourceDetails[1].getSourceName()); } @Test - public void shouldReturnDefaultSourceTypeAsOldKafkaSource() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"false\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); + public void shouldGetUnboundedKafkaConsumerAsSourceDetailsWhenNotGiven() { + when(configuration.getString(INPUT_STREAMS, "")) + .thenReturn("[{\"INPUT_SCHEMA_TABLE\": \"data_stream\"," + + "\"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"true\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"test-group-13\"," + + "\"SOURCE_KAFKA_NAME\": \"local-kafka-stream\"," + + "\"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\"," + + "\"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\"}]"); + StreamConfig[] streamConfigs = StreamConfig.parse(configuration); + SourceDetails[] sourceDetails = streamConfigs[0].getSourceDetails(); - StreamConfig currConfig = streamConfigs[0]; - Assert.assertEquals("false", currConfig.getAutoCommitEnable()); - Assert.assertEquals("latest", currConfig.getAutoOffsetReset()); - Assert.assertEquals("PROTO", currConfig.getDataType()); - Assert.assertEquals("dummy-consumer-group", currConfig.getConsumerGroupId()); - Assert.assertEquals("41", currConfig.getEventTimestampFieldIndex()); - Assert.assertEquals("test-topic", currConfig.getKafkaTopicNames()); - Assert.assertEquals("data_stream", currConfig.getSchemaTable()); - Assert.assertEquals("local-kafka-stream", currConfig.getKafkaName()); - Assert.assertEquals("OLD_KAFKA_SOURCE", currConfig.getSourceType()); + assertEquals(1, sourceDetails.length); + assertEquals(UNBOUNDED, sourceDetails[0].getSourceType()); + assertEquals(KAFKA_CONSUMER, sourceDetails[0].getSourceName()); + } + + @Test + public void shouldGetParquetSourceProperties() { + when(configuration.getString(INPUT_STREAMS, "")) + .thenReturn("[{\"SOURCE_PARQUET_FILE_PATHS\": [\"gs://some-parquet-path\", \"gs://another-parquet-path\"]," + + "\"SOURCE_PARQUET_BILLING_PROJECT\": \"data-project\"," + + "\"SOURCE_PARQUET_READ_ORDER_STRATEGY\": \"EARLIEST_TIME_URL_FIRST\"," + + "\"SOURCE_PARQUET_SCHEMA_MATCH_STRATEGY\": \"BACKWARD_COMPATIBLE_SCHEMA_WITH_FAIL_ON_TYPE_MISMATCH\"" + + "}]"); + StreamConfig[] streamConfigs = StreamConfig.parse(configuration); + + Assert.assertArrayEquals(new String[]{"gs://some-parquet-path", "gs://another-parquet-path"}, streamConfigs[0].getParquetFilePaths()); + assertEquals("data-project", streamConfigs[0].getParquetBillingProject()); + assertEquals(SourceParquetReadOrderStrategy.valueOf("EARLIEST_TIME_URL_FIRST"), streamConfigs[0].getParquetFilesReadOrderStrategy()); + assertEquals(SourceParquetSchemaMatchStrategy.valueOf("BACKWARD_COMPATIBLE_SCHEMA_WITH_FAIL_ON_TYPE_MISMATCH"), streamConfigs[0].getParquetSchemaMatchStrategy()); } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamTest.java new file mode 100644 index 000000000..fa16c5da0 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamTest.java @@ -0,0 +1,148 @@ +package io.odpf.dagger.core.source; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.core.StencilClientOrchestrator; +import io.odpf.dagger.consumer.TestBookingLogMessage; +import io.odpf.dagger.core.source.flinkkafkaconsumer.FlinkKafkaConsumerDaggerSource; +import io.odpf.dagger.core.source.kafka.KafkaDaggerSource; +import io.odpf.dagger.core.source.parquet.ParquetDaggerSource; +import io.odpf.stencil.client.StencilClient; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + + +import java.util.HashMap; +import java.util.Properties; +import java.util.regex.Pattern; + +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class StreamTest { + + @Mock + private StencilClientOrchestrator stencilClientOrchestrator; + + @Mock + private StencilClient stencilClient; + + @Mock + private Configuration configuration; + + @Mock + private StreamConfig streamConfig; + + @Mock + private StreamExecutionEnvironment streamExecutionEnvironment; + + @Mock + private WatermarkStrategy watermarkStrategy; + + @Mock + private DaggerSource mockDaggerSource; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldBeAbleToBuildAStreamWithKafkaDaggerSourceAndProtoSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(streamConfig.getSchemaTable()).thenReturn("data_stream"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + Stream stream = builder.build(); + + assertTrue(stream.getDaggerSource() instanceof KafkaDaggerSource); + } + + @Test + public void shouldBeAbleToBuildAStreamWithFlinkKafkaConsumerDaggerSourceAndProtoSchema() { + HashMap kafkaPropMap = new HashMap<>(); + kafkaPropMap.put("group.id", "dummy-consumer-group"); + kafkaPropMap.put("bootstrap.servers", "localhost:9092"); + + Properties properties = new Properties(); + properties.putAll(kafkaPropMap); + + when(streamConfig.getKafkaProps(any())).thenReturn(properties); + when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test-topic")); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(streamConfig.getSchemaTable()).thenReturn("data_stream"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + Stream stream = builder.build(); + + assertTrue(stream.getDaggerSource() instanceof FlinkKafkaConsumerDaggerSource); + } + + @Test + public void shouldBeAbleToBuildAStreamWithKafkaDaggerSourceAndJsonSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + when(streamConfig.getSchemaTable()).thenReturn("data_stream"); + when(streamConfig.getJsonSchema()).thenReturn("{ \"$schema\": \"https://json-schema.org/draft/2020-12/schema\", \"$id\": \"https://example.com/product.schema.json\", \"title\": \"Product\", \"description\": \"A product from Acme's catalog\", \"type\": \"object\", \"properties\": { \"id\": { \"description\": \"The unique identifier for a product\", \"type\": \"string\" }, \"time\": { \"description\": \"event timestamp of the event\", \"type\": \"string\", \"format\" : \"date-time\" } }, \"required\": [ \"id\", \"time\" ] }"); + + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + Stream stream = builder.build(); + + assertTrue(stream.getDaggerSource() instanceof KafkaDaggerSource); + } + + @Test + public void shouldBeAbleToBuildAStreamWithFlinkKafkaConsumerDaggerSourceAndJsonSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + when(streamConfig.getDataType()).thenReturn("JSON"); + when(streamConfig.getSchemaTable()).thenReturn("data_stream"); + when(streamConfig.getJsonSchema()).thenReturn("{ \"$schema\": \"https://json-schema.org/draft/2020-12/schema\", \"$id\": \"https://example.com/product.schema.json\", \"title\": \"Product\", \"description\": \"A product from Acme's catalog\", \"type\": \"object\", \"properties\": { \"id\": { \"description\": \"The unique identifier for a product\", \"type\": \"string\" }, \"time\": { \"description\": \"event timestamp of the event\", \"type\": \"string\", \"format\" : \"date-time\" } }, \"required\": [ \"id\", \"time\" ] }"); + + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + Stream stream = builder.build(); + + assertTrue(stream.getDaggerSource() instanceof FlinkKafkaConsumerDaggerSource); + } + + @Test + public void shouldBeAbleToBuildAStreamWithParquetDaggerSourceAndProtoSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + when(streamConfig.getEventTimestampFieldIndex()).thenReturn("5"); + when(streamConfig.getDataType()).thenReturn("PROTO"); + when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); + when(streamConfig.getSchemaTable()).thenReturn("data_stream"); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); + Stream.Builder builder = new Stream.Builder(streamConfig, configuration, stencilClientOrchestrator); + Stream stream = builder.build(); + + assertTrue(stream.getDaggerSource() instanceof ParquetDaggerSource); + } + + @Test + public void shouldInvokeTheDaggerSourceRegistrationMethodWhenRegisterSourceIsCalled() { + Stream stream = new Stream(mockDaggerSource, "some-stream"); + + stream.registerSource(streamExecutionEnvironment, watermarkStrategy); + + verify(mockDaggerSource, times(1)).register(streamExecutionEnvironment, watermarkStrategy); + + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamsFactoryTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamsFactoryTest.java index b37ef32fb..c5f2de52f 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamsFactoryTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/StreamsFactoryTest.java @@ -1,29 +1,27 @@ package io.odpf.dagger.core.source; +import io.odpf.dagger.core.source.flinkkafkaconsumer.FlinkKafkaConsumerDaggerSource; +import io.odpf.dagger.core.source.kafka.KafkaDaggerSource; import io.odpf.stencil.client.StencilClient; import com.google.gson.JsonSyntaxException; import io.odpf.dagger.common.configuration.Configuration; import io.odpf.dagger.common.core.StencilClientOrchestrator; -import io.odpf.dagger.common.serde.DataTypes; import io.odpf.dagger.consumer.TestBookingLogMessage; -import io.odpf.dagger.core.processors.telemetry.processor.MetricsTelemetryExporter; -import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.mockito.Mock; + import java.util.List; import static io.odpf.dagger.common.core.Constants.INPUT_STREAMS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; import static org.mockito.MockitoAnnotations.initMocks; public class StreamsFactoryTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - @Mock private StencilClientOrchestrator stencilClientOrchestrator; @@ -33,89 +31,71 @@ public class StreamsFactoryTest { @Mock private Configuration configuration; - @Mock - private MetricsTelemetryExporter telemetryExporter; - @Before public void setup() { initMocks(this); } @Test - public void shouldCreateSingleStreamFromSingleStreamConfig() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); - when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); - when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); - - Assert.assertEquals(1, streams.size()); - Assert.assertTrue(streams.get(0) instanceof Stream); - } - - @Test - public void shouldCreateStreamWithProtoDatatype() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); - when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); - when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); - - Assert.assertEquals(DataTypes.PROTO, streams.get(0).getInputDataType()); - - } - - @Test - public void shouldCreateStreamWithProtoDatatypeIfNotSpecified() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); - when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); - when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); + public void shouldReturnListOfStreamsCreatedFromConfiguration() { + when(configuration.getString(INPUT_STREAMS, "")) + .thenReturn("[{\"INPUT_SCHEMA_TABLE\": \"data_stream_1\"," + + "\"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic-1\"," + + "\"INPUT_DATATYPE\": \"PROTO\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"true\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"test-group-13\"," + + "\"SOURCE_KAFKA_NAME\": \"local-kafka-stream-1\"," + + "\"SOURCE_DETAILS\": [{\"SOURCE_TYPE\": \"UNBOUNDED\", \"SOURCE_NAME\": \"KAFKA_CONSUMER\"}]," + + "\"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\"," + + "\"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\"}," + + "{\"INPUT_SCHEMA_TABLE\": \"data_stream_2\"," + + "\"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic-2\"," + + "\"SOURCE_DETAILS\": [{\"SOURCE_TYPE\": \"UNBOUNDED\", \"SOURCE_NAME\": \"KAFKA_SOURCE\"}]," + + "\"INPUT_DATATYPE\": \"JSON\"," + + "\"INPUT_SCHEMA_JSON_SCHEMA\" : \"{ \\\"$schema\\\": \\\"https://json-schema.org/draft/2020-12/schema\\\", \\\"$id\\\": \\\"https://example.com/product.schema.json\\\", \\\"title\\\": \\\"Product\\\", \\\"description\\\": \\\"A product from Acme's catalog\\\", \\\"type\\\": \\\"object\\\", \\\"properties\\\": { \\\"id\\\": { \\\"description\\\": \\\"The unique identifier for a product\\\", \\\"type\\\": \\\"string\\\" }, \\\"time\\\": { \\\"description\\\": \\\"event timestamp of the event\\\", \\\"type\\\": \\\"string\\\", \\\"format\\\" : \\\"date-time\\\" } }, \\\"required\\\": [ \\\"id\\\", \\\"time\\\" ] }\"," + + "\"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\"," + + "\"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\"," + + "\"SOURCE_KAFKA_NAME\": \"local-kafka-stream-2\" }]"); - Assert.assertEquals(DataTypes.PROTO, streams.get(0).getInputDataType()); - } - - @Test - public void shouldCreateStreamWithJSONDatatype() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_DATATYPE\": \"JSON\", \"INPUT_SCHEMA_JSON_SCHEMA\" : \"{ \\\"$schema\\\": \\\"https://json-schema.org/draft/2020-12/schema\\\", \\\"$id\\\": \\\"https://example.com/product.schema.json\\\", \\\"title\\\": \\\"Product\\\", \\\"description\\\": \\\"A product from Acme's catalog\\\", \\\"type\\\": \\\"object\\\", \\\"properties\\\": { \\\"id\\\": { \\\"description\\\": \\\"The unique identifier for a product\\\", \\\"type\\\": \\\"string\\\" }, \\\"time\\\": { \\\"description\\\": \\\"event timestamp of the event\\\", \\\"type\\\": \\\"string\\\", \\\"format\\\" : \\\"date-time\\\" } }, \\\"required\\\": [ \\\"id\\\", \\\"time\\\" ] }\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); - - Assert.assertEquals(DataTypes.JSON, streams.get(0).getInputDataType()); - } - - @Test - public void shouldCreateMultipleStreams() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" }, { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_DATATYPE\": \"JSON\", \"INPUT_SCHEMA_JSON_SCHEMA\": \"{ \\\"$schema\\\": \\\"https://json-schema.org/draft/2020-12/schema\\\", \\\"$id\\\": \\\"https://example.com/product.schema.json\\\", \\\"title\\\": \\\"Product\\\", \\\"description\\\": \\\"A product from Acme's catalog\\\", \\\"type\\\": \\\"object\\\", \\\"properties\\\": { \\\"id\\\": { \\\"description\\\": \\\"The unique identifier for a product\\\", \\\"type\\\": \\\"string\\\" }, \\\"time\\\": { \\\"description\\\": \\\"event timestamp of the event\\\", \\\"type\\\": \\\"string\\\", \\\"format\\\" : \\\"date-time\\\" } }, \\\"required\\\": [ \\\"id\\\", \\\"time\\\" ] }\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); - Assert.assertEquals(2, streams.size()); - } - - @Test - public void shouldCreateStreamsWithDifferentTypes() { - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" }, { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_DATATYPE\": \"JSON\", \"INPUT_SCHEMA_JSON_SCHEMA\": \"{ \\\"$schema\\\": \\\"https://json-schema.org/draft/2020-12/schema\\\", \\\"$id\\\": \\\"https://example.com/product.schema.json\\\", \\\"title\\\": \\\"Product\\\", \\\"description\\\": \\\"A product from Acme's catalog\\\", \\\"type\\\": \\\"object\\\", \\\"properties\\\": { \\\"id\\\": { \\\"description\\\": \\\"The unique identifier for a product\\\", \\\"type\\\": \\\"string\\\" }, \\\"time\\\": { \\\"description\\\": \\\"event timestamp of the event\\\", \\\"type\\\": \\\"string\\\", \\\"format\\\" : \\\"date-time\\\" } }, \\\"required\\\": [ \\\"id\\\", \\\"time\\\" ] }\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \"local-kafka-stream\" } ]"); - when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); - when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - - List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); + List streams = StreamsFactory.getStreams(configuration, stencilClientOrchestrator); - Assert.assertEquals(DataTypes.PROTO, streams.get(0).getInputDataType()); - Assert.assertEquals(DataTypes.JSON, streams.get(1).getInputDataType()); + assertEquals(2, streams.size()); + assertTrue(streams.get(0).getDaggerSource() instanceof FlinkKafkaConsumerDaggerSource); + assertEquals("data_stream_1", streams.get(0).getStreamName()); + assertTrue(streams.get(1).getDaggerSource() instanceof KafkaDaggerSource); + assertEquals("data_stream_2", streams.get(1).getStreamName()); } @Test public void shouldThrowErrorForInvalidStreamConfig() { - thrown.expect(JsonSyntaxException.class); - when(configuration.getString(INPUT_STREAMS, "")).thenReturn("[ { \"SOURCE_KAFKA_TOPIC_NAMES\": \"test-topic\", \"INPUT_SCHEMA_TABLE\": \"data_stream\", \"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\", \"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\", \"SOURCE_KAFKA_CONSUMER_CONFIG_BOOTSTRAP_SERVERS\": \"localhost:9092\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_COMMIT_ENABLE\": \"\", \"SOURCE_KAFKA_CONSUMER_CONFIG_AUTO_OFFSET_RESET\": \"latest\", \"SOURCE_KAFKA_CONSUMER_CONFIG_GROUP_ID\": \"dummy-consumer-group\", \"SOURCE_KAFKA_NAME\": \\\"local-kafka-stream\" } ]"); + when(configuration.getString(INPUT_STREAMS, "")) + .thenReturn("[{\"INPUT_SCHEMA_TABLE\": \"data_stream\"," + + "\"INPUT_DATATYPE\": \"PROTO\"," + + "\"SOURCE_PARQUET_READ_ORDER_STRATEGY\": \\\"EARLIEST_TIME_URL_FIRST," + + "\"SOURCE_PARQUET_FILE_PATHS\": [\"gs://p-godata-id-mainstream-bedrock/carbon-offset-transaction-log/dt=2022-02-05/hr=09/\", \"gs://p-godata-id-mainstream-bedrock/carbon-offset-transaction-log/dt=2022-02-03/hr=14/\"]," + + "\"SOURCE_DETAILS\": [{\"SOURCE_TYPE\": \"BOUNDED\", \"SOURCE_NAME\": \"PARQUET_SOURCE\"}]," + + "\"INPUT_SCHEMA_PROTO_CLASS\": \"com.tests.TestMessage\"," + + "\"INPUT_SCHEMA_EVENT_TIMESTAMP_FIELD_INDEX\": \"41\"}]"); when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); + + assertThrows(JsonSyntaxException.class, + () -> StreamsFactory.getStreams(configuration, stencilClientOrchestrator)); } @Test public void shouldThrowNullPointerIfStreamConfigIsNotGiven() { - thrown.expect(NullPointerException.class); when(configuration.getString(INPUT_STREAMS, "")).thenReturn(""); - StreamsFactory.getStreams(configuration, stencilClientOrchestrator, telemetryExporter); + + assertThrows(NullPointerException.class, + () -> StreamsFactory.getStreams(configuration, stencilClientOrchestrator)); } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustomTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustomTest.java similarity index 99% rename from dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustomTest.java rename to dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustomTest.java index 9f0d575a7..c95218380 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/FlinkKafkaConsumerCustomTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerCustomTest.java @@ -1,4 +1,4 @@ -package io.odpf.dagger.core.source.kafka; +package io.odpf.dagger.core.source.flinkkafkaconsumer; import io.odpf.dagger.common.configuration.Configuration; import io.odpf.dagger.core.metrics.reporters.ErrorReporter; diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSourceTest.java new file mode 100644 index 000000000..96958907d --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/flinkkafkaconsumer/FlinkKafkaConsumerDaggerSourceTest.java @@ -0,0 +1,106 @@ +package io.odpf.dagger.core.source.flinkkafkaconsumer; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.json.deserialization.JsonDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class FlinkKafkaConsumerDaggerSourceTest { + + @Mock + private Configuration configuration; + + @Mock + private StreamConfig streamConfig; + + @Mock + private DaggerDeserializer daggerDeserializer; + + @Mock + private FlinkKafkaConsumerCustom flinkKafkaConsumerCustom; + + @Mock + private WatermarkStrategy watermarkStrategy; + + @Mock + private StreamExecutionEnvironment streamExecutionEnvironment; + + @Before + public void setup() { + initMocks(this); + daggerDeserializer = Mockito.mock(JsonDeserializer.class); + } + + @Test + public void shouldBeAbleToBuildSourceIfSourceDetailsIsUnboundedKafkaConsumerAndDaggerDeserializerIsKafkaDeserializationSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertTrue(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceDetailsContainsMultipleBackToBackSources() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED), + new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.UNBOUNDED)}); + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.BOUNDED)}); + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfDeserializerTypeIsUnsupported() { + DaggerDeserializer unsupportedDeserializer = Mockito.mock(SimpleGroupDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.UNBOUNDED)}); + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, unsupportedDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldBeAbleToRegisterSourceWithExecutionEnvironmentForCorrectConfiguration() { + FlinkKafkaConsumerDaggerSource daggerSource = new FlinkKafkaConsumerDaggerSource(streamConfig, configuration, daggerDeserializer); + FlinkKafkaConsumerDaggerSource daggerSourceSpy = Mockito.spy(daggerSource); + doReturn(flinkKafkaConsumerCustom).when(daggerSourceSpy).buildSource(); + when(flinkKafkaConsumerCustom.assignTimestampsAndWatermarks(watermarkStrategy)).thenReturn(flinkKafkaConsumerCustom); + + daggerSourceSpy.register(streamExecutionEnvironment, watermarkStrategy); + + verify(flinkKafkaConsumerCustom, times(1)).assignTimestampsAndWatermarks(watermarkStrategy); + verify(streamExecutionEnvironment, times(1)).addSource(flinkKafkaConsumerCustom); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSourceTest.java deleted file mode 100644 index 0199e707c..000000000 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerKafkaSourceTest.java +++ /dev/null @@ -1,92 +0,0 @@ -package io.odpf.dagger.core.source.kafka; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.connector.kafka.source.KafkaSource; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.types.Row; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; - -import java.util.HashMap; -import java.util.Properties; -import java.util.regex.Pattern; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.*; -import static org.mockito.MockitoAnnotations.initMocks; - -public class DaggerKafkaSourceTest { - @Mock - private StreamConfig streamConfig; - - @Mock - private KafkaDeserializationSchema deserializationSchema; - - @Mock - private Configuration configuration; - - @Mock - private StreamExecutionEnvironment environment; - - @Mock - private WatermarkStrategy watermarkStrategy; - - @Mock - private KafkaSource kafkaSource; - - @Before - public void setUp() { - initMocks(this); - HashMap kafkaPropMap = new HashMap<>(); - kafkaPropMap.put("group.id", "dummy-consumer-group"); - kafkaPropMap.put("bootstrap.servers", "localhost:9092"); - - Properties properties = new Properties(); - properties.putAll(kafkaPropMap); - when(streamConfig.getKafkaProps(any())).thenReturn(properties); - when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test")); - when(streamConfig.getStartingOffset()).thenReturn(OffsetsInitializer.committedOffsets(OffsetResetStrategy.valueOf("LATEST"))); - } - - @Test - public void shouldHandleKafkaSource() { - when(streamConfig.getSourceType()).thenReturn("KAFKA_SOURCE"); - DaggerKafkaSource daggerKafkaSource = new DaggerKafkaSource(streamConfig, deserializationSchema, configuration); - Assert.assertTrue(daggerKafkaSource.canHandle()); - } - - @Test - public void shouldNotHandleOtherSourceType() { - when(streamConfig.getSourceType()).thenReturn("OLD_KAFKA_SOURCE"); - DaggerKafkaSource daggerKafkaSource = new DaggerKafkaSource(streamConfig, deserializationSchema, configuration); - Assert.assertFalse(daggerKafkaSource.canHandle()); - } - - @Test - public void shouldRegisterOnExecutionEnvironment() { - DaggerKafkaSource daggerKafkaSource = new DaggerKafkaSourceStub(streamConfig, deserializationSchema, configuration); - daggerKafkaSource.register(environment, watermarkStrategy, "data_stream_0"); - - verify(environment, times(1)).fromSource(kafkaSource, watermarkStrategy, "data_stream_0"); - } - - class DaggerKafkaSourceStub extends DaggerKafkaSource { - - DaggerKafkaSourceStub(StreamConfig streamConfig, KafkaDeserializationSchema deserializationSchema, Configuration configuration) { - super(streamConfig, deserializationSchema, configuration); - } - - @Override - KafkaSource getKafkaSource() { - return kafkaSource; - } - } - -} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSourceTest.java deleted file mode 100644 index 27add4cb2..000000000 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/DaggerOldKafkaSourceTest.java +++ /dev/null @@ -1,94 +0,0 @@ -package io.odpf.dagger.core.source.kafka; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.core.source.StreamConfig; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.types.Row; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; - -import java.util.HashMap; -import java.util.Properties; -import java.util.regex.Pattern; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.*; -import static org.mockito.MockitoAnnotations.initMocks; - -public class DaggerOldKafkaSourceTest { - - @Mock - private StreamConfig streamConfig; - - @Mock - private KafkaDeserializationSchema deserializationSchema; - - @Mock - private Configuration configuration; - - @Mock - private StreamExecutionEnvironment environment; - - @Mock - private WatermarkStrategy watermarkStrategy; - - @Mock - private FlinkKafkaConsumerCustom flinkKafkaConsumerCustom; - - @Mock - private FlinkKafkaConsumerBase kafkaConsumerBaseWithWatermarks; - - @Before - public void setUp() { - initMocks(this); - HashMap kafkaPropMap = new HashMap<>(); - kafkaPropMap.put("group.id", "dummy-consumer-group"); - kafkaPropMap.put("bootstrap.servers", "localhost:9092"); - - Properties properties = new Properties(); - properties.putAll(kafkaPropMap); - when(streamConfig.getKafkaProps(any())).thenReturn(properties); - when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test")); - } - - @Test - public void shouldHandleOldKafkaSource() { - when(streamConfig.getSourceType()).thenReturn("OLD_KAFKA_SOURCE"); - DaggerOldKafkaSource daggerOldKafkaSource = new DaggerOldKafkaSource(streamConfig, deserializationSchema, configuration); - Assert.assertTrue(daggerOldKafkaSource.canHandle()); - } - - @Test - public void shouldNotHandleOtherSourceType() { - when(streamConfig.getSourceType()).thenReturn("KAFKA_SOURCE"); - DaggerOldKafkaSource daggerOldKafkaSource = new DaggerOldKafkaSource(streamConfig, deserializationSchema, configuration); - Assert.assertFalse(daggerOldKafkaSource.canHandle()); - } - - @Test - public void shouldRegisterOnExecutionEnvironment() { - when(flinkKafkaConsumerCustom.assignTimestampsAndWatermarks(watermarkStrategy)).thenReturn(kafkaConsumerBaseWithWatermarks); - DaggerOldKafkaSource daggerOldKafkaSource = new DaggerOldKafkaSourceStub(streamConfig, deserializationSchema, configuration); - daggerOldKafkaSource.register(environment, watermarkStrategy, "data_stream_0"); - - verify(flinkKafkaConsumerCustom, times(1)).assignTimestampsAndWatermarks(watermarkStrategy); - verify(environment, times(1)).addSource(kafkaConsumerBaseWithWatermarks); - } - - class DaggerOldKafkaSourceStub extends DaggerOldKafkaSource { - - DaggerOldKafkaSourceStub(StreamConfig streamConfig, KafkaDeserializationSchema deserializationSchema, Configuration configuration) { - super(streamConfig, deserializationSchema, configuration); - } - - @Override - FlinkKafkaConsumerCustom getFlinkKafkaConsumerCustom() { - return flinkKafkaConsumerCustom; - } - } -} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSourceTest.java new file mode 100644 index 000000000..56f1f9eb3 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/KafkaDaggerSourceTest.java @@ -0,0 +1,108 @@ +package io.odpf.dagger.core.source.kafka; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; + + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class KafkaDaggerSourceTest { + @Mock + private Configuration configuration; + + @Mock + private StreamConfig streamConfig; + + @Mock + private DaggerDeserializer daggerDeserializer; + + @Mock + private KafkaSource kafkaSource; + + @Mock + private WatermarkStrategy strategy; + + @Mock + private StreamExecutionEnvironment streamExecutionEnvironment; + + @Before + public void setup() { + initMocks(this); + daggerDeserializer = Mockito.mock(ProtoDeserializer.class); + } + + @Test + public void shouldBeAbleToBuildSourceIfSourceDetailsIsUnboundedKafkaAndDaggerDeserializerIsKafkaDeserializationSchema() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + KafkaDaggerSource daggerSource = new KafkaDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertTrue(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceDetailsContainsMultipleBackToBackSources() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED), + new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + KafkaDaggerSource daggerSource = new KafkaDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.UNBOUNDED)}); + KafkaDaggerSource daggerSource = new KafkaDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.BOUNDED)}); + KafkaDaggerSource daggerSource = new KafkaDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfDeserializerTypeIsUnsupported() { + DaggerDeserializer unsupportedDeserializer = Mockito.mock(SimpleGroupDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_SOURCE, SourceType.UNBOUNDED)}); + KafkaDaggerSource daggerSource = new KafkaDaggerSource(streamConfig, configuration, unsupportedDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldBeAbleToRegisterSourceWithExecutionEnvironmentForCorrectConfiguration() { + when(streamConfig.getSchemaTable()).thenReturn("data_stream_0"); + + KafkaDaggerSource kafkaDaggerSource = new KafkaDaggerSource(streamConfig, configuration, daggerDeserializer); + KafkaDaggerSource kafkaDaggerSourceSpy = Mockito.spy(kafkaDaggerSource); + doReturn(kafkaSource).when(kafkaDaggerSourceSpy).buildSource(); + + kafkaDaggerSourceSpy.register(streamExecutionEnvironment, strategy); + + verify(streamExecutionEnvironment, times(1)).fromSource(kafkaSource, strategy, "data_stream_0"); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilderTest.java deleted file mode 100644 index 7973022cd..000000000 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/JsonDataStreamBuilderTest.java +++ /dev/null @@ -1,111 +0,0 @@ -package io.odpf.dagger.core.source.kafka.builder; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.common.serde.DataTypes; -import io.odpf.dagger.core.source.Stream; -import io.odpf.dagger.core.source.StreamConfig; -import io.odpf.dagger.core.source.kafka.DaggerOldKafkaSource; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.mockito.Mock; - -import java.util.*; -import java.util.regex.Pattern; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.when; -import static org.mockito.MockitoAnnotations.initMocks; - -public class JsonDataStreamBuilderTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Mock - private Configuration configuration; - - @Mock - private StreamConfig streamConfig; - - @Before - public void setup() { - initMocks(this); - } - - @Test - public void shouldProcessJSONStream() { - when(streamConfig.getDataType()).thenReturn("JSON"); - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - - Assert.assertTrue(jsonDataStreamBuilder.canBuild()); - } - - @Test - public void shouldParseDataTypeFromStreamConfig() { - when(streamConfig.getDataType()).thenReturn("JSON"); - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - - Assert.assertEquals(DataTypes.JSON, jsonDataStreamBuilder.getInputDataType()); - } - - @Test - public void shouldIgnoreProtoStream() { - when(streamConfig.getDataType()).thenReturn("PROTO"); - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - - Assert.assertFalse(jsonDataStreamBuilder.canBuild()); - } - - @Test - public void shouldBuildJSONStreamIfConfigured() { - HashMap kafkaPropMap = new HashMap<>(); - kafkaPropMap.put("group.id", "dummy-consumer-group"); - kafkaPropMap.put("bootstrap.servers", "localhost:9092"); - - Properties properties = new Properties(); - properties.putAll(kafkaPropMap); - - when(streamConfig.getDataType()).thenReturn("JSON"); - when(streamConfig.getJsonSchema()).thenReturn("{ \"$schema\": \"https://json-schema.org/draft/2020-12/schema\", \"$id\": \"https://example.com/product.schema.json\", \"title\": \"Product\", \"description\": \"A product from Acme's catalog\", \"type\": \"object\", \"properties\": { \"id\": { \"description\": \"The unique identifier for a product\", \"type\": \"string\" }, \"time\": { \"description\": \"event timestamp of the event\", \"type\": \"string\", \"format\" : \"date-time\" } }, \"required\": [ \"id\", \"time\" ] }"); - when(streamConfig.getEventTimestampFieldIndex()).thenReturn("1"); - - when(streamConfig.getKafkaProps(any())).thenReturn(properties); - when(streamConfig.getStartingOffset()).thenReturn(OffsetsInitializer.committedOffsets(OffsetResetStrategy.valueOf("LATEST"))); - when(streamConfig.getSchemaTable()).thenReturn("test-table"); - when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test")); - when(streamConfig.getSourceType()).thenReturn("OLD_KAFKA_SOURCE"); - - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - - Stream build = jsonDataStreamBuilder.build(); - - Assert.assertEquals(DataTypes.JSON, build.getInputDataType()); - Assert.assertTrue(build.getDaggerSource() instanceof DaggerOldKafkaSource); - Assert.assertEquals("test-table", build.getStreamName()); - } - - @Test - public void shouldAddMetricsSpecificToKafkaSource() { - when(streamConfig.getKafkaTopicNames()).thenReturn("test-topic"); - when(streamConfig.getKafkaName()).thenReturn("test-kafka"); - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - jsonDataStreamBuilder.addTelemetry(); - - Map> metrics = jsonDataStreamBuilder.getMetrics(); - - Assert.assertEquals(Arrays.asList(new String[]{"test-topic"}), metrics.get("input_topic")); - Assert.assertEquals(Arrays.asList(new String[]{"test-kafka"}), metrics.get("input_stream")); - } - - @Test - public void shouldFailToCreateStreamIfSomeConfigsAreMissing() { - thrown.expect(NullPointerException.class); - JsonDataStreamBuilder jsonDataStreamBuilder = new JsonDataStreamBuilder(streamConfig, configuration); - - jsonDataStreamBuilder.build(); - } -} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilderTest.java deleted file mode 100644 index 1e765bda9..000000000 --- a/dagger-core/src/test/java/io/odpf/dagger/core/source/kafka/builder/ProtoDataStreamBuilderTest.java +++ /dev/null @@ -1,124 +0,0 @@ -package io.odpf.dagger.core.source.kafka.builder; - -import io.odpf.dagger.common.configuration.Configuration; -import io.odpf.dagger.common.core.StencilClientOrchestrator; -import io.odpf.dagger.common.serde.DataTypes; -import io.odpf.dagger.consumer.TestBookingLogMessage; -import io.odpf.dagger.core.source.Stream; -import io.odpf.dagger.core.source.StreamConfig; -import io.odpf.dagger.core.source.kafka.DaggerOldKafkaSource; -import io.odpf.stencil.client.StencilClient; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.mockito.Mock; - -import java.util.*; -import java.util.regex.Pattern; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.when; -import static org.mockito.MockitoAnnotations.initMocks; - -public class ProtoDataStreamBuilderTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Mock - private Configuration configuration; - - @Mock - private StencilClientOrchestrator stencilClientOrchestrator; - - @Mock - private StreamConfig streamConfig; - - @Mock - private StencilClient stencilClient; - - @Before - public void setup() { - initMocks(this); - } - - @Test - public void shouldProcessProtoStream() { - when(streamConfig.getDataType()).thenReturn("PROTO"); - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - - Assert.assertTrue(protoDataStreamBuilder.canBuild()); - } - - @Test - public void shouldParseDataTypeFromStreamConfig() { - when(streamConfig.getDataType()).thenReturn("PROTO"); - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - - Assert.assertEquals(DataTypes.PROTO, protoDataStreamBuilder.getInputDataType()); - } - - @Test - public void shouldIgnoreJsonStream() { - when(streamConfig.getDataType()).thenReturn("JSON"); - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - - Assert.assertFalse(protoDataStreamBuilder.canBuild()); - } - - @Test - public void shouldBuildProtoStreamIfConfigured() { - HashMap kafkaPropMap = new HashMap<>(); - kafkaPropMap.put("group.id", "dummy-consumer-group"); - kafkaPropMap.put("bootstrap.servers", "localhost:9092"); - - Properties properties = new Properties(); - properties.putAll(kafkaPropMap); - - when(streamConfig.getDataType()).thenReturn("PROTO"); - when(streamConfig.getProtoClass()).thenReturn("com.tests.TestMessage"); - when(streamConfig.getEventTimestampFieldIndex()).thenReturn("1"); - when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); - when(stencilClient.get("com.tests.TestMessage")).thenReturn(TestBookingLogMessage.getDescriptor()); - when(streamConfig.getKafkaProps(any())).thenReturn(properties); - when(streamConfig.getStartingOffset()).thenReturn(OffsetsInitializer.committedOffsets(OffsetResetStrategy.valueOf("LATEST"))); - when(streamConfig.getSchemaTable()).thenReturn("test-table"); - when(streamConfig.getTopicPattern()).thenReturn(Pattern.compile("test")); - when(streamConfig.getSourceType()).thenReturn("OLD_KAFKA_SOURCE"); - - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - - Stream build = protoDataStreamBuilder.build(); - - Assert.assertEquals(DataTypes.PROTO, build.getInputDataType()); - Assert.assertTrue(build.getDaggerSource() instanceof DaggerOldKafkaSource); - Assert.assertEquals("test-table", build.getStreamName()); - } - - @Test - public void shouldAddMetricsSpecificToKafkaSource() { - when(streamConfig.getKafkaTopicNames()).thenReturn("test-topic"); - when(streamConfig.getProtoClass()).thenReturn("test-class"); - when(streamConfig.getKafkaName()).thenReturn("test-kafka"); - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - protoDataStreamBuilder.addTelemetry(); - - Map> metrics = protoDataStreamBuilder.getMetrics(); - - Assert.assertEquals(Arrays.asList(new String[]{"test-topic"}), metrics.get("input_topic")); - Assert.assertEquals(Arrays.asList(new String[]{"test-class"}), metrics.get("input_proto")); - Assert.assertEquals(Arrays.asList(new String[]{"test-kafka"}), metrics.get("input_stream")); - } - - @Test - public void shouldFailToCreateStreamIfSomeConfigsAreMissing() { - thrown.expect(NullPointerException.class); - when(streamConfig.getEventTimestampFieldIndex()).thenReturn("1"); - ProtoDataStreamBuilder protoDataStreamBuilder = new ProtoDataStreamBuilder(streamConfig, stencilClientOrchestrator, configuration); - - protoDataStreamBuilder.build(); - } -} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSourceTest.java new file mode 100644 index 000000000..0b359e68a --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetDaggerSourceTest.java @@ -0,0 +1,138 @@ +package io.odpf.dagger.core.source.parquet; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.common.serde.DaggerDeserializer; +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.common.serde.proto.deserialization.ProtoDeserializer; +import io.odpf.dagger.core.exception.DaggerConfigurationException; +import io.odpf.dagger.core.source.SourceDetails; +import io.odpf.dagger.core.source.SourceName; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.StreamConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + + +import static io.odpf.dagger.core.source.parquet.SourceParquetReadOrderStrategy.EARLIEST_INDEX_FIRST; +import static io.odpf.dagger.core.source.parquet.SourceParquetReadOrderStrategy.EARLIEST_TIME_URL_FIRST; +import static org.junit.Assert.*; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +@PrepareForTest(FileSource.class) +@RunWith(PowerMockRunner.class) +public class ParquetDaggerSourceTest { + @Mock + private Configuration configuration; + + @Mock + private StreamConfig streamConfig; + + @Mock + private DaggerDeserializer daggerDeserializer; + + @Mock + private WatermarkStrategy strategy; + + @Mock + private StreamExecutionEnvironment streamExecutionEnvironment; + + private FileSource fileSource; + + @Before + public void setup() { + initMocks(this); + daggerDeserializer = Mockito.mock(SimpleGroupDeserializer.class); + /* FileSource is a final class and hence cannot be mocked using vanilla Mockito */ + fileSource = PowerMockito.mock(FileSource.class); + } + + @Test + public void shouldBeAbleToBuildSourceIfSourceDetailsIsBoundedParquetAndDaggerDeserializerIsSimpleGroupDeserializer() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertTrue(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceDetailsContainsMultipleBackToBackSources() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED), + new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceNameIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.KAFKA_CONSUMER, SourceType.BOUNDED)}); + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfSourceTypeIsUnsupported() { + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.UNBOUNDED)}); + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldNotBeAbleToBuildSourceIfDeserializerTypeIsUnsupported() { + DaggerDeserializer unsupportedDeserializer = Mockito.mock(ProtoDeserializer.class); + when(streamConfig.getSourceDetails()).thenReturn(new SourceDetails[]{new SourceDetails(SourceName.PARQUET_SOURCE, SourceType.BOUNDED)}); + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, unsupportedDeserializer); + + assertFalse(daggerSource.canBuild()); + } + + @Test + public void shouldBeAbleToRegisterSourceWithExecutionEnvironmentForCorrectConfiguration() { + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + ParquetDaggerSource daggerSourceSpy = Mockito.spy(daggerSource); + doReturn(fileSource).when(daggerSourceSpy).buildFileSource(); + when(streamConfig.getSchemaTable()).thenReturn("data_stream_0"); + + daggerSourceSpy.register(streamExecutionEnvironment, strategy); + + verify(streamExecutionEnvironment, times(1)).fromSource(fileSource, strategy, "data_stream_0"); + } + + @Test + public void shouldUseStreamConfigurationToBuildTheFileSource() { + /* the below call mocks ensure that the function calls are indeed made to build the source and the code compiles */ + when(streamConfig.getSchemaTable()).thenReturn("data_stream_0"); + when(streamConfig.getParquetFilesReadOrderStrategy()).thenReturn(EARLIEST_TIME_URL_FIRST); + when(streamConfig.getParquetFilePaths()).thenReturn(new String[]{"gs://sshsh", "gs://shadd"}); + + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + daggerSource.register(streamExecutionEnvironment, strategy); + } + + @Test + public void shouldThrowRuntimeExceptionIfReadOrderStrategyIsNotSupported() { + when(streamConfig.getParquetFilesReadOrderStrategy()).thenReturn(EARLIEST_INDEX_FIRST); + when(streamConfig.getParquetFilePaths()).thenReturn(new String[]{"gs://sshsh", "gs://shadd"}); + + ParquetDaggerSource daggerSource = new ParquetDaggerSource(streamConfig, configuration, daggerDeserializer); + + assertThrows(DaggerConfigurationException.class, () -> daggerSource.register(streamExecutionEnvironment, strategy)); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormatTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormatTest.java new file mode 100644 index 000000000..144003862 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileRecordFormatTest.java @@ -0,0 +1,98 @@ +package io.odpf.dagger.core.source.parquet; + + +import io.odpf.dagger.core.source.parquet.reader.ParquetReader; +import io.odpf.dagger.core.source.parquet.reader.ReaderProvider; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.util.function.Supplier; + +import static org.junit.Assert.*; +import static org.mockito.MockitoAnnotations.initMocks; + +public class ParquetFileRecordFormatTest { + + @Mock + private ParquetReader parquetReader; + + @Mock + private TypeInformation typeInformation; + + @Mock + private Configuration configuration; + + private final ReaderProvider readerProviderMock = (filePath) -> parquetReader; + private final Supplier> typeInformationProviderMock = () -> typeInformation; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldBuildAFileRecordFormatAsPerConfiguredParameters() { + ParquetFileRecordFormat.Builder builder = ParquetFileRecordFormat.Builder.getInstance(); + ParquetFileRecordFormat parquetFileRecordFormat = builder.setParquetFileReaderProvider(readerProviderMock) + .setTypeInformationProvider(typeInformationProviderMock) + .build(); + + FileRecordFormat.Reader expectedReader = parquetFileRecordFormat.createReader(configuration, new Path("gs://file-path"), 0, 1024); + TypeInformation expectedTypeInformation = parquetFileRecordFormat.getProducedType(); + + assertEquals(expectedReader, parquetReader); + assertEquals(expectedTypeInformation, typeInformation); + } + + @Test + public void shouldThrowIllegalArgumentExceptionWhenReaderProviderIsNotConfigured() { + ParquetFileRecordFormat.Builder builder = ParquetFileRecordFormat.Builder.getInstance(); + + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, + () -> builder + .setTypeInformationProvider(typeInformationProviderMock) + .build()); + + assertEquals("ReaderProvider is required but is set as null", ex.getMessage()); + } + + @Test + public void shouldThrowIllegalArgumentExceptionWhenTypeInformationProviderIsNotConfigured() { + ParquetFileRecordFormat.Builder builder = ParquetFileRecordFormat.Builder.getInstance(); + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, + () -> builder + .setParquetFileReaderProvider(readerProviderMock) + .build()); + + assertEquals("TypeInformationProvider is required but is set as null", ex.getMessage()); + } + + @Test + public void shouldReturnFalseWhenIsSplittableIsCalled() { + ParquetFileRecordFormat.Builder builder = ParquetFileRecordFormat.Builder.getInstance(); + ParquetFileRecordFormat parquetFileRecordFormat = builder.setParquetFileReaderProvider(readerProviderMock) + .setTypeInformationProvider(typeInformationProviderMock) + .build(); + + assertFalse(parquetFileRecordFormat.isSplittable()); + } + + @Test + public void shouldThrowUnsupportedOperationExceptionWhenRestoreReaderIsCalled() { + ParquetFileRecordFormat.Builder builder = ParquetFileRecordFormat.Builder.getInstance(); + ParquetFileRecordFormat parquetFileRecordFormat = builder.setTypeInformationProvider(typeInformationProviderMock) + .setParquetFileReaderProvider(readerProviderMock) + .build(); + + UnsupportedOperationException ex = assertThrows(UnsupportedOperationException.class, + () -> parquetFileRecordFormat.restoreReader(configuration, new Path("gs://some-path"), 12, 0, 1024)); + + assertEquals("Error: Restoring a reader from saved state is not implemented yet", ex.getMessage()); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileSourceTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileSourceTest.java new file mode 100644 index 000000000..9933a77ae --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/ParquetFileSourceTest.java @@ -0,0 +1,122 @@ +package io.odpf.dagger.core.source.parquet; + +import io.odpf.dagger.common.configuration.Configuration; +import io.odpf.dagger.core.source.SourceType; +import io.odpf.dagger.core.source.parquet.ParquetFileSource.Builder; +import io.odpf.dagger.core.source.parquet.splitassigner.ChronologyOrderedSplitAssigner; +import org.apache.flink.connector.file.src.assigners.LocalityAwareSplitAssigner; +import org.apache.flink.connector.file.src.reader.FileRecordFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import static java.util.Collections.*; +import static org.junit.Assert.*; +import static org.mockito.MockitoAnnotations.initMocks; + +public class ParquetFileSourceTest { + @Mock + private Configuration configuration; + + @Mock + private FileRecordFormat fileRecordFormat; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldBuildParquetFileSourceAsPerArguments() { + Builder builder = Builder.getInstance(); + Path[] filePaths = new Path[]{new Path("gs://aadadc"), new Path("gs://sjsjhd")}; + ParquetFileSource parquetFileSource = builder.setConfiguration(configuration) + .setFileRecordFormat(fileRecordFormat) + .setSourceType(SourceType.BOUNDED) + .setFileSplitAssigner(ChronologyOrderedSplitAssigner::new) + .setFilePaths(filePaths) + .build(); + + assertTrue(parquetFileSource.getFileSplitAssigner().create(emptyList()) instanceof ChronologyOrderedSplitAssigner); + assertArrayEquals(filePaths, parquetFileSource.getFilePaths()); + assertEquals(fileRecordFormat, parquetFileSource.getFileRecordFormat()); + assertEquals(configuration, parquetFileSource.getConfiguration()); + assertEquals(SourceType.BOUNDED, parquetFileSource.getSourceType()); + } + + @Test + public void shouldThrowExceptionIfSourceTypeConfiguredAsUnbounded() { + Builder builder = Builder.getInstance(); + Path[] filePaths = new Path[]{new Path("gs://aadadc"), new Path("gs://sjsjhd")}; + + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, + () -> builder.setConfiguration(configuration) + .setFileRecordFormat(fileRecordFormat) + .setSourceType(SourceType.UNBOUNDED) + .setFileSplitAssigner(ChronologyOrderedSplitAssigner::new) + .setFilePaths(filePaths) + .build()); + + assertEquals("Running Parquet FileSource in UNBOUNDED mode is not supported yet", ex.getMessage()); + } + + @Test + public void shouldThrowExceptionIfFileRecordFormatIsNotSet() { + Builder builder = Builder.getInstance(); + Path[] filePaths = new Path[]{new Path("gs://aadadc"), new Path("gs://sjsjhd")}; + + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, + () -> builder.setConfiguration(configuration) + .setSourceType(SourceType.UNBOUNDED) + .setFileSplitAssigner(ChronologyOrderedSplitAssigner::new) + .setFilePaths(filePaths) + .build()); + + assertEquals("FileRecordFormat is required but is set as null", ex.getMessage()); + } + + @Test + public void shouldThrowExceptionIfNoFilePathsSet() { + Builder builder = Builder.getInstance(); + + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, + () -> builder.setConfiguration(configuration) + .setFileRecordFormat(fileRecordFormat) + .setSourceType(SourceType.BOUNDED) + .setFileSplitAssigner(ChronologyOrderedSplitAssigner::new) + .build()); + + assertEquals("At least one file path is required but none are provided", ex.getMessage()); + } + + @Test + public void shouldUseDefaultValueForSomeFieldsWhichAreNotConfiguredExplicitly() { + Builder builder = Builder.getInstance(); + Path[] filePaths = new Path[]{new Path("gs://aadadc"), new Path("gs://sjsjhd")}; + ParquetFileSource parquetFileSource = builder.setConfiguration(configuration) + .setFileRecordFormat(fileRecordFormat) + .setFilePaths(filePaths) + .build(); + + assertTrue(parquetFileSource.getFileSplitAssigner().create(emptyList()) instanceof LocalityAwareSplitAssigner); + assertEquals(SourceType.BOUNDED, parquetFileSource.getSourceType()); + } + + /* this test just verifies that the code for generating the FileSource compiles successfully and runs. */ + /* Since static methods of FileSource have been used and its member properties are not exposed, it's difficult to test the */ + /* returned blackbox object */ + @Test + public void shouldReturnAFileSourceMadeFromParquetFileSource() { + Builder builder = Builder.getInstance(); + Path[] filePaths = new Path[]{new Path("gs://aadadc"), new Path("gs://sjsjhd")}; + ParquetFileSource parquetFileSource = builder.setConfiguration(configuration) + .setFileRecordFormat(fileRecordFormat) + .setSourceType(SourceType.BOUNDED) + .setFileSplitAssigner(ChronologyOrderedSplitAssigner::new) + .setFilePaths(filePaths) + .build(); + + parquetFileSource.buildFileSource(); + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/reader/ParquetReaderTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/reader/ParquetReaderTest.java new file mode 100644 index 000000000..174619f62 --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/reader/ParquetReaderTest.java @@ -0,0 +1,168 @@ +package io.odpf.dagger.core.source.parquet.reader; + + +import io.odpf.dagger.common.serde.parquet.deserialization.SimpleGroupDeserializer; +import io.odpf.dagger.core.exception.ParquetFileSourceReaderInitializationException; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.flink.types.Row; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.Types.*; +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; +import static org.mockito.internal.verification.VerificationModeFactory.times; + +public class ParquetReaderTest { + @Mock + private SimpleGroupDeserializer deserializer; + + @Rule + public TemporaryFolder tempFolder = TemporaryFolder.builder().assureDeletion().build(); + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldCreateReadersConfiguredWithTheSameDeserializerButForDifferentFilePaths() throws IOException { + when(deserializer.deserialize(any(SimpleGroup.class))).thenReturn(Row.of("same", "deserializer")); + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + ClassLoader classLoader = getClass().getClassLoader(); + + String filePath1 = classLoader.getResource("test_file.parquet").getPath(); + ParquetReader reader1 = provider.getReader(filePath1); + + String filePath2 = classLoader.getResource("multiple_row_groups_test_file.parquet").getPath(); + ParquetReader reader2 = provider.getReader(filePath2); + + assertEquals(reader1.read(), reader2.read()); + } + + @Test + public void shouldReadFileAndCallDeserializerWithSimpleGroupWhenReadIsCalled() throws IOException { + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + ClassLoader classLoader = getClass().getClassLoader(); + ParquetReader reader = provider.getReader(classLoader.getResource("test_file.parquet").getPath()); + + reader.read(); + reader.read(); + reader.read(); + + SimpleGroup[] allExpectedSimpleGroups = getSimpleGroups(); + ArgumentCaptor simpleGroupCaptor = ArgumentCaptor.forClass(SimpleGroup.class); + verify(deserializer, times(3)).deserialize(simpleGroupCaptor.capture()); + List allActualSimpleGroups = simpleGroupCaptor.getAllValues(); + + for (int i = 0; i < 3; i++) { + SimpleGroup expectedSimpleGroup = allExpectedSimpleGroups[i]; + SimpleGroup actualSimpleGroup = allActualSimpleGroups.get(i); + assertEquals(expectedSimpleGroup.getType(), actualSimpleGroup.getType()); + assertEquals(expectedSimpleGroup.getString("name", 0), actualSimpleGroup.getString("name", 0)); + assertEquals(expectedSimpleGroup.getLong("age", 0), actualSimpleGroup.getLong("age", 0)); + assertEquals(expectedSimpleGroup.getString("residence", 0), actualSimpleGroup.getString("residence", 0)); + } + } + + @Test + public void shouldBeAbleToReadParquetFileContainingMultipleRowGroups() throws IOException { + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + ClassLoader classLoader = getClass().getClassLoader(); + ParquetReader reader = provider.getReader(classLoader.getResource("multiple_row_groups_test_file.parquet").getPath()); + + reader.read(); + reader.read(); + reader.read(); + reader.read(); + reader.read(); + reader.read(); + + SimpleGroup[] allExpectedSimpleGroups = ArrayUtils.addAll(getSimpleGroups(), getSimpleGroups()); + ArgumentCaptor simpleGroupCaptor = ArgumentCaptor.forClass(SimpleGroup.class); + verify(deserializer, times(6)).deserialize(simpleGroupCaptor.capture()); + List allActualSimpleGroups = simpleGroupCaptor.getAllValues(); + + for (int i = 0; i < 6; i++) { + SimpleGroup expectedSimpleGroup = allExpectedSimpleGroups[i]; + SimpleGroup actualSimpleGroup = allActualSimpleGroups.get(i); + assertEquals(expectedSimpleGroup.getType(), actualSimpleGroup.getType()); + assertEquals(expectedSimpleGroup.getString("name", 0), actualSimpleGroup.getString("name", 0)); + assertEquals(expectedSimpleGroup.getLong("age", 0), actualSimpleGroup.getLong("age", 0)); + assertEquals(expectedSimpleGroup.getString("residence", 0), actualSimpleGroup.getString("residence", 0)); + } + } + + @Test + public void shouldReturnDeserializedValueWhenRecordsPresentAndNullWhenNoMoreDataLeftToRead() throws IOException { + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + ClassLoader classLoader = getClass().getClassLoader(); + ParquetReader reader = provider.getReader(classLoader.getResource("test_file.parquet").getPath()); + when(deserializer.deserialize(any(SimpleGroup.class))).thenReturn(Row.of("some value")); + + assertEquals(Row.of("some value"), reader.read()); + assertEquals(Row.of("some value"), reader.read()); + assertEquals(Row.of("some value"), reader.read()); + assertNull(reader.read()); + } + + @Test + public void shouldThrowIOExceptionIfReadIsCalledAfterCallingClose() throws IOException { + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + ClassLoader classLoader = getClass().getClassLoader(); + ParquetReader reader = provider.getReader(classLoader.getResource("test_file.parquet").getPath()); + + reader.close(); + + assertThrows(IOException.class, reader::read); + } + + @Test + public void shouldThrowParquetFileSourceReaderInitializationExceptionIfCannotConstructReaderForTheFile() throws IOException { + final File tempFile = tempFolder.newFile("test_file.parquet"); + ParquetReader.ParquetReaderProvider provider = new ParquetReader.ParquetReaderProvider(deserializer); + + assertThrows(ParquetFileSourceReaderInitializationException.class, () -> provider.getReader(tempFile.getPath())); + } + + private SimpleGroup[] getSimpleGroups() { + GroupType expectedSchema = buildMessage() + .optional(BINARY).as(LogicalTypeAnnotation.stringType()).named("name") + .optional(INT64).named("age") + .optional(BINARY).as(LogicalTypeAnnotation.stringType()).named("residence") + .named("schema"); + + SimpleGroup group1 = new SimpleGroup(expectedSchema); + group1.add("name", "Ajay"); + group1.add("age", 24L); + group1.add("residence", "Mumbai"); + + SimpleGroup group2 = new SimpleGroup(expectedSchema); + group2.add("name", "Utkarsh"); + group2.add("age", 25L); + group2.add("residence", "Delhi"); + + SimpleGroup group3 = new SimpleGroup(expectedSchema); + group3.add("name", "Samay"); + group3.add("age", 29L); + group3.add("residence", "Pune"); + + return new SimpleGroup[]{group1, group2, group3}; + } +} diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssignerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssignerTest.java new file mode 100644 index 000000000..7064cc28a --- /dev/null +++ b/dagger-core/src/test/java/io/odpf/dagger/core/source/parquet/splitassigner/ChronologyOrderedSplitAssignerTest.java @@ -0,0 +1,127 @@ +package io.odpf.dagger.core.source.parquet.splitassigner; + + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.core.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import java.util.*; + +import static org.junit.Assert.*; + +public class ChronologyOrderedSplitAssignerTest { + + @Test + public void shouldReturnFileSplitsHavingOldestDateFilePathsFirstWhenFilePathURLHasOnlyDate() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/asdghsdhasd"), 0, 1024); + FileSourceSplit secondSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-02-29/ga6agad6ad"), 0, 1024); + FileSourceSplit thirdSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-11-31/hd7ahadh7agd"), 0, 1024); + FileSourceSplit fourthSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-12-31/hagga6a36dg"), 0, 1024); + List inputSplits = Arrays.asList(secondSplit, fourthSplit, firstSplit, thirdSplit); + FileSourceSplit[] expectedOrdering = new FileSourceSplit[]{firstSplit, secondSplit, thirdSplit, fourthSplit}; + + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(inputSplits); + + for (int i = 0; i < 4; i++) { + Optional split = splitAssigner.getNext(null); + assertTrue(split.isPresent()); + assertEquals("AssertionError when testing for file split number " + (i + 1), expectedOrdering[i], split.get()); + } + } + + @Test + public void shouldReturnFileSplitsHavingOldestTimeFilePathsFirstWhenFilePathURLHasBothDateAndHour() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/hr=00/hd6a7gad"), 0, 1024); + FileSourceSplit secondSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/hr=08/sa6advgad7"), 0, 1024); + FileSourceSplit thirdSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-11-31/hr=09/aga6adgad"), 0, 1024); + FileSourceSplit fourthSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-12-31/hr=23/ahaha4a5dg"), 0, 1024); + List inputSplits = Arrays.asList(secondSplit, fourthSplit, firstSplit, thirdSplit); + FileSourceSplit[] expectedOrdering = new FileSourceSplit[]{firstSplit, secondSplit, thirdSplit, fourthSplit}; + + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(inputSplits); + + for (int i = 0; i < 4; i++) { + Optional split = splitAssigner.getNext(null); + assertTrue(split.isPresent()); + assertEquals("AssertionError when testing for file split number " + (i + 1), expectedOrdering[i], split.get()); + } + } + + @Test + public void shouldReturnEmptyOptionalWhenNoMoreSplitsToReturn() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/shs6s5sdg"), 0, 1024); + List inputSplits = Arrays.asList(firstSplit); + + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(inputSplits); + + splitAssigner.getNext(null); + Optional nextSplit = splitAssigner.getNext(null); + + assertFalse(nextSplit.isPresent()); + } + + @Test + public void shouldThrowIllegalArgumentExceptionDuringConstructionItselfWhenFilePathsDoNotFollowPattern() { + FileSourceSplit split = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-130-12/shs6s5sdg"), 0, 1024); + + IllegalArgumentException ex = Assert.assertThrows(IllegalArgumentException.class, () -> new ChronologyOrderedSplitAssigner(Collections.singleton(split))); + + assertEquals("java.text.ParseException: Cannot extract timestamp from filepath for deciding order of processing.\n" + + "File path doesn't abide with any partitioning strategy: gs://my-bucket/bid-log/dt=2019-130-12/shs6s5sdg", ex.getMessage()); + } + + @Test + public void shouldAddNewFileSourceSplitsWithOldestDateFilePathsReturnedFirstWhenFilePathURLHasOnlyDate() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/asdghsdhasd"), 0, 1024); + FileSourceSplit secondSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-02-29/ga6agad6ad"), 0, 1024); + FileSourceSplit thirdSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-11-31/hd7ahadh7agd"), 0, 1024); + FileSourceSplit fourthSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-12-31/hagga6a36dg"), 0, 1024); + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(Collections.singleton(secondSplit)); + List remainingSplitsToAdd = Arrays.asList(fourthSplit, firstSplit, thirdSplit); + splitAssigner.addSplits(remainingSplitsToAdd); + + FileSourceSplit[] expectedOrdering = new FileSourceSplit[]{firstSplit, secondSplit, thirdSplit, fourthSplit}; + for (int i = 0; i < 4; i++) { + Optional split = splitAssigner.getNext(null); + assertTrue(split.isPresent()); + assertEquals("AssertionError when testing for file split number " + (i + 1), expectedOrdering[i], split.get()); + } + } + + @Test + public void shouldAddNewFileSourceSplitsWithOldestTimeFilePathsReturnedFirstWhenFilePathURLHasBothDateAndHour() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/hr=00/hd6a7gad"), 0, 1024); + FileSourceSplit secondSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/hr=08/sa6advgad7"), 0, 1024); + FileSourceSplit thirdSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-11-31/hr=09/aga6adgad"), 0, 1024); + FileSourceSplit fourthSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-12-31/hr=23/ahaha4a5dg"), 0, 1024); + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(Collections.singleton(secondSplit)); + List remainingSplitsToAdd = Arrays.asList(fourthSplit, firstSplit, thirdSplit); + splitAssigner.addSplits(remainingSplitsToAdd); + + FileSourceSplit[] expectedOrdering = new FileSourceSplit[]{firstSplit, secondSplit, thirdSplit, fourthSplit}; + for (int i = 0; i < 4; i++) { + Optional split = splitAssigner.getNext(null); + assertTrue(split.isPresent()); + assertEquals("AssertionError when testing for file split number " + (i + 1), expectedOrdering[i], split.get()); + } + } + + @Test + public void shouldReturnRemainingSplitsWhichAreNotAssignedYetInAscendingOrderOfFilePathTimestampURL() { + FileSourceSplit firstSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2019-10-12/asdghsdhasd"), 0, 1024); + FileSourceSplit secondSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-02-29/ga6agad6ad"), 0, 1024); + FileSourceSplit thirdSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-11-31/hd7ahadh7agd"), 0, 1024); + FileSourceSplit fourthSplit = new FileSourceSplit("1", new Path("gs://my-bucket/bid-log/dt=2020-12-31/hagga6a36dg"), 0, 1024); + List inputSplits = Arrays.asList(secondSplit, fourthSplit, firstSplit, thirdSplit); + + ChronologyOrderedSplitAssigner splitAssigner = new ChronologyOrderedSplitAssigner(inputSplits); + splitAssigner.getNext(null); + List remainingSplits = (List) splitAssigner.remainingSplits(); + + FileSourceSplit[] expectedSplits = new FileSourceSplit[]{secondSplit, thirdSplit, fourthSplit}; + for (int i = 0; i < 3; i++) { + assertEquals("AssertionError when testing for file split number " + (i + 1), expectedSplits[i], remainingSplits.get(i)); + } + } +} diff --git a/dagger-core/src/test/resources/multiple_row_groups_test_file.parquet b/dagger-core/src/test/resources/multiple_row_groups_test_file.parquet new file mode 100644 index 000000000..7f38e21f4 Binary files /dev/null and b/dagger-core/src/test/resources/multiple_row_groups_test_file.parquet differ diff --git a/dagger-core/src/test/resources/test_file.parquet b/dagger-core/src/test/resources/test_file.parquet new file mode 100644 index 000000000..567acd2c4 Binary files /dev/null and b/dagger-core/src/test/resources/test_file.parquet differ