From 8e3ec734a6f9109650d2d0a3ea782e6cdfeec305 Mon Sep 17 00:00:00 2001 From: esmioley Date: Mon, 8 Apr 2019 10:10:44 -0700 Subject: [PATCH] Update from latest Uber internal (#18) --- marmaray-tools/checkstyles | 1 + .../marmaray/tools/HDFSMetadataPrinter.java | 3 +- .../marmaray/tools/HDFSMetadataPruner.java | 5 +- marmaray/checkstyles | 1 + .../com/uber/marmaray/common/AvroPayload.java | 129 ++++-- .../marmaray/common/DispersalLengthType.java | 27 ++ .../marmaray/common/MetadataManagerType.java | 28 ++ .../common/actions/JobDagActions.java | 21 +- .../common/actions/ReporterAction.java | 4 +- ...CassandraMetadataManagerConfiguration.java | 112 +++++ .../CassandraSinkConfiguration.java | 82 +++- .../common/configuration/Configuration.java | 39 +- .../configuration/FileSinkConfiguration.java | 27 ++ .../FileSourceConfiguration.java | 11 +- .../HDFSMetadataManagerConfiguration.java | 38 ++ .../HDFSSchemaServiceConfiguration.java | 2 +- .../configuration/HiveConfiguration.java | 13 +- .../configuration/HoodieConfiguration.java | 44 +- .../HoodieIndexConfiguration.java | 13 +- .../configuration/KafkaConfiguration.java | 21 +- .../KafkaSourceConfiguration.java | 40 +- .../MetadataManagerConfiguration.java | 78 ++++ .../configuration/SparkConfiguration.java | 53 +++ .../data/AbstractDataConverter.java | 28 +- .../data/CassandraSinkCQLDataConverter.java | 122 ++++-- .../data/CassandraSinkDataConverter.java | 266 +++++++++--- .../data/FileSinkDataCSVConverter.java | 135 ++++++ .../data/FileSinkDataConverter.java | 88 ++-- .../data/FileSinkDataConverterFactory.java | 42 ++ .../data/FileSinkDataJSONConverter.java | 135 ++++++ .../data/HoodieSinkDataConverter.java | 12 + .../data/KafkaSourceDataConverter.java | 30 +- .../data/SparkSourceDataConverter.java | 62 ++- .../schema/CassandraSchemaConverter.java | 5 +- .../schema/DataFrameSchemaConverter.java | 60 +-- .../common/forkoperator/ForkOperator.java | 11 + .../com/uber/marmaray/common/job/Dag.java | 29 ++ .../ExecutionTimeJobExecutionStrategy.java | 23 +- .../common/job/IJobExecutionStrategy.java | 2 +- .../com/uber/marmaray/common/job/JobDag.java | 76 +++- .../uber/marmaray/common/job/JobManager.java | 214 +++++----- .../common/job/JsonHoodieIngestionJob.java | 147 ------- .../common/job/ThreadPoolService.java | 9 +- .../CassandraBasedMetadataManager.java | 366 ++++++++++++++++ .../metadata/HDFSDatePartitionManager.java | 3 +- .../common/metadata/HDFSMetadataManager.java | 59 ++- .../common/metadata/HDFSPartitionManager.java | 1 - .../metadata/HoodieBasedMetadataManager.java | 65 +-- .../common/metadata/IMetadataManager.java | 11 +- .../metadata/JobManagerMetadataTracker.java | 3 +- .../common/metadata/MultiMetadataManager.java | 190 +++++++++ .../common/metadata/NoOpMetadataManager.java | 11 + .../common/metrics/CassandraMetric.java | 17 + .../common/metrics/DataFeedMetricNames.java | 36 ++ .../common/metrics/DataFeedMetrics.java | 49 ++- .../marmaray/common/metrics/DoubleMetric.java | 4 +- .../common/metrics/ErrorCauseTagNames.java | 85 ++++ .../marmaray/common/metrics/LongMetric.java | 2 + .../uber/marmaray/common/metrics/Metric.java | 3 +- .../common/metrics/ModuleTagNames.java | 39 ++ .../marmaray/common/metrics/TimerMetric.java | 2 + .../reporters/IKafkaDataLossReporter.java | 11 + .../marmaray/common/reporters/Reporters.java | 7 + .../CassandraMetadataSchemaManager.java | 98 +++++ .../cassandra/CassandraSchemaField.java | 10 +- .../cassandra/CassandraSchemaManager.java | 394 ++++++++++++++++++ .../cassandra/CassandraSinkSchemaManager.java | 230 +--------- .../common/schema/cassandra/ClusterKey.java | 6 +- .../common/sinks/SinkStatManager.java | 12 +- .../sinks/cassandra/CassandraClientSink.java | 50 ++- .../sinks/cassandra/CassandraSSTableSink.java | 229 ++++++++-- .../common/sinks/cassandra/CassandraSink.java | 97 +++-- .../common/sinks/file/AwsFileSink.java | 126 +++++- .../marmaray/common/sinks/file/FileSink.java | 120 +++++- .../common/sinks/file/HdfsFileSink.java | 13 + .../common/sinks/hoodie/HoodieErrorSink.java | 9 +- .../common/sinks/hoodie/HoodieSink.java | 52 ++- .../DefaultHoodieDataPartitioner.java | 39 ++ .../uber/marmaray/common/sources/ISource.java | 6 + .../common/sources/IWorkUnitCalculator.java | 10 + .../common/sources/file/FileSource.java | 5 +- .../file/JSONFileSourceDataConverter.java | 14 +- .../common/sources/hive/HiveSource.java | 52 ++- .../hive/ParquetWorkUnitCalculator.java | 47 ++- .../sources/kafka/IKafkaOffsetSelector.java | 32 ++ .../kafka/KafkaBootstrapOffsetSelector.java | 8 +- .../sources/kafka/KafkaOffsetResetter.java | 30 ++ .../common/sources/kafka/KafkaSource.java | 236 +++++++++-- .../kafka/KafkaWorkUnitCalculator.java | 142 +++++-- .../kafka/LogBasedKafkaDataLossReporter.java | 14 + .../common/spark/MarmarayKryoSerializer.java | 70 ++++ .../uber/marmaray/common/spark/SparkArgs.java | 24 +- .../marmaray/common/spark/SparkFactory.java | 89 ++-- .../marmaray/common/status/BaseStatus.java | 55 +++ .../uber/marmaray/common/status/IStatus.java | 42 ++ .../common/status/JobManagerStatus.java | 52 +++ .../marmaray/utilities/CommandLineUtil.java | 9 +- .../uber/marmaray/utilities/ConfigUtil.java | 2 +- .../marmaray/utilities/ErrorTableUtil.java | 42 +- .../com/uber/marmaray/utilities/FSUtils.java | 18 +- .../uber/marmaray/utilities/HoodieUtil.java | 4 +- .../com/uber/marmaray/utilities/JobUtil.java | 13 + .../KafkaSourceConverterErrorExtractor.java | 5 - .../uber/marmaray/utilities/KafkaUtil.java | 14 +- .../marmaray/utilities/LongAccumulator.java | 84 ++++ .../uber/marmaray/utilities/SchemaUtil.java | 111 ++++- .../com/uber/marmaray/utilities/SizeUnit.java | 51 +++ .../uber/marmaray/utilities/SparkUtil.java | 143 +------ .../uber/marmaray/utilities/TimeUnitUtil.java | 31 ++ .../marmaray/utilities/TimestampInfo.java | 9 +- .../cluster/CassandraClusterInfo.java | 9 +- .../listener/SparkEventListener.java | 10 +- .../utilities/listener/TimeoutManager.java | 77 +++- .../java/com/uber/marmaray/TestSparkUtil.java | 18 +- .../TestCassandraSinkConfiguration.java | 14 + .../configuration/TestConfiguration.java | 16 + .../TestHoodieConfiguration.java | 32 ++ .../TestHoodieIndexConfiguration.java | 1 + .../TestKafkaSourceConfiguration.java | 69 +++ .../converters/TestAbstractDataConverter.java | 8 + .../TestCassandraDataFrameConverter.java | 7 +- .../TestCassandraSchemaConverter.java | 21 +- .../TestDataFrameDataConverter.java | 5 + .../TestDataFrameSchemaConverter.java | 6 +- .../TestCassandraSinkCQLDataConverter.java | 123 ++++++ .../data/TestCassandraSinkDataConverter.java | 270 ++++++++++++ .../data/TestFileSinkDataCSVConverter.java | 184 ++++++++ .../data/TestFileSinkDataConverter.java | 109 ----- .../data/TestFileSinkDataJSONConverter.java | 70 ++++ .../data/TestSparkSourceDataConverter.java | 106 +++++ ...TestExecutionTimeJobExecutionStrategy.java | 8 +- .../uber/marmaray/common/job/TestJobDag.java | 151 +++++++ .../marmaray/common/job/TestJobManager.java | 213 ++++++++++ .../common/job/TestThreadPoolService.java | 20 +- .../metadata/MemoryMetadataManager.java | 11 + .../TestCassandraBasedMetadataManager.java | 112 +++++ .../TestHDFSDatePartitionManager.java | 10 +- .../TestHDFSJobLevelMetadataTracker.java | 2 +- .../metadata/TestHDFSMetadataManager.java | 2 +- .../metadata/TestHDFSPartitionManager.java | 12 +- .../TestCassandraSinkSchemaManager.java | 131 ++++-- .../cassandra/TestCassandraClientSink.java | 37 +- .../cassandra/TestCassandraSSTableSink.java | 178 ++++++-- .../cassandra/TestCassandraSinkUtil.java | 8 +- .../common/sinks/file/FileSinkTestUtil.java | 4 +- .../common/sinks/file/TestAwsFileSink.java | 141 +++++-- .../common/sinks/file/TestFileSink.java | 22 +- .../common/sinks/file/TestHdfsFileSink.java | 6 +- .../common/sinks/hoodie/TestHoodieSink.java | 73 +++- .../file/TestFileWorkUnitCalculator.java | 6 +- .../common/sources/hive/TestHiveSource.java | 3 +- .../hive/TestHiveSourceConfiguration.java | 3 - .../hive/TestParquetWorkUnitCalculator.java | 8 +- .../spark/TestMarmarayKryoSerializer.java | 22 + .../common/spark/TestSparkFactory.java | 43 +- .../common/status/TestBaseStatus.java | 81 ++++ .../common/status/TestJobManagerStatus.java | 49 +++ .../common/util/AbstractSparkTest.java | 2 +- .../common/util/CassandraTestConstants.java | 10 + .../marmaray/common/util/FileHelperUtil.java | 20 + .../marmaray/common/util/HiveTestUtil.java | 4 +- .../marmaray/common/util/KafkaTestHelper.java | 61 ++- .../marmaray/common/util/SparkTestUtil.java | 4 +- .../marmaray/common/util/TestFsUtils.java | 3 +- .../common/util/TestParquetWriterUtil.java | 4 +- .../marmaray/common/util/TestSchemaUtil.java | 21 +- .../marmaray/utilities/ResourcesUtils.java | 14 + .../marmaray/utilities/TestKafkaUtil.java | 28 +- .../uber/marmaray/utilities/TestSizeUnit.java | 54 +++ .../marmaray/utilities/TestTimeUnitUtil.java | 28 ++ .../listener/TestTimeoutManager.java | 79 ++++ marmaray/src/test/resources/config.yaml | 2 + .../testPartition2/data.parquet | Bin 0 -> 454 bytes pom.xml | 6 +- 174 files changed, 7401 insertions(+), 1651 deletions(-) create mode 120000 marmaray-tools/checkstyles create mode 120000 marmaray/checkstyles create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/DispersalLengthType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/MetadataManagerType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraMetadataManagerConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSMetadataManagerConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/MetadataManagerConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/SparkConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataCSVConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverterFactory.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataJSONConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/Dag.java delete mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/JsonHoodieIngestionJob.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/CassandraBasedMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/MultiMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraMetric.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/ErrorCauseTagNames.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/ModuleTagNames.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/reporters/IKafkaDataLossReporter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraMetadataSchemaManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/partitioner/DefaultHoodieDataPartitioner.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/IKafkaOffsetSelector.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaOffsetResetter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/LogBasedKafkaDataLossReporter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/spark/MarmarayKryoSerializer.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/status/BaseStatus.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/status/IStatus.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/status/JobManagerStatus.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/LongAccumulator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/SizeUnit.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/TimeUnitUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaSourceConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkCQLDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataCSVConverter.java delete mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataJSONConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestSparkSourceDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/job/TestJobDag.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/job/TestJobManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestCassandraBasedMetadataManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/spark/TestMarmarayKryoSerializer.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/status/TestBaseStatus.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/status/TestJobManagerStatus.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/utilities/ResourcesUtils.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/utilities/TestSizeUnit.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/utilities/TestTimeUnitUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/utilities/listener/TestTimeoutManager.java create mode 100644 marmaray/src/test/resources/testData/testPartition1/testPartition2/data.parquet diff --git a/marmaray-tools/checkstyles b/marmaray-tools/checkstyles new file mode 120000 index 0000000..bf484d3 --- /dev/null +++ b/marmaray-tools/checkstyles @@ -0,0 +1 @@ +../checkstyles \ No newline at end of file diff --git a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java index 8b13c25..2ca8f12 100644 --- a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java +++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.tools; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.uber.marmaray.common.configuration.Configuration; @@ -63,7 +64,7 @@ public static void main(final String[] args) throws ParseException, IOException log.info("Printing contents of metadata file: " + metadataFilePath); final Configuration conf = new Configuration(); - final FileSystem fs = FSUtils.getFs(conf); + final FileSystem fs = FSUtils.getFs(conf, Optional.absent()); try (final InputStream is = new BufferedInputStream(fs.open(new Path(metadataFilePath)))) { try (final ObjectInputStream input = new ObjectInputStream(is)) { final Map metadataMap = HDFSMetadataManager.deserialize(input); diff --git a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java index b2bbe90..80d5641 100644 --- a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java +++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java @@ -20,6 +20,9 @@ import com.uber.marmaray.common.metadata.HDFSMetadataManager; import com.uber.marmaray.utilities.CommandLineUtil; import com.uber.marmaray.utilities.FSUtils; + +import com.google.common.base.Optional; + import java.io.IOException; import java.util.Comparator; import lombok.extern.slf4j.Slf4j; @@ -78,7 +81,7 @@ public static void main(final String[] args) throws ParseException, IOException : false; final Configuration conf = new Configuration(); - final FileSystem fs = FSUtils.getFs(conf); + final FileSystem fs = FSUtils.getFs(conf, Optional.absent()); if (fs.isDirectory(metadataPath)) { final FileStatus[] fileStatuses = fs.listStatus(metadataPath); diff --git a/marmaray/checkstyles b/marmaray/checkstyles new file mode 120000 index 0000000..bf484d3 --- /dev/null +++ b/marmaray/checkstyles @@ -0,0 +1 @@ +../checkstyles \ No newline at end of file diff --git a/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java index d9214de..a41902d 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java @@ -19,17 +19,18 @@ import com.google.common.base.Preconditions; import com.uber.marmaray.common.data.IData; import com.uber.marmaray.utilities.SparkUtil; +import lombok.AllArgsConstructor; import lombok.NonNull; import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.hibernate.validator.constraints.NotEmpty; -import scala.reflect.ClassManifestFactory; -import scala.reflect.ClassTag; import java.io.Serializable; +import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -43,45 +44,117 @@ @Slf4j public class AvroPayload implements IPayload, IData, Serializable { - private static final ClassTag recordClassTag = ClassManifestFactory.fromClass(GenericRecord.class); @NonNull - private final Map rootFields = new HashMap<>(); - @NonNull - private final byte[] byteRecord; + private IAvroPayloadInternal payloadInterval; public AvroPayload(@NonNull final GenericRecord record) { - this.byteRecord = SparkUtil.serialize(record, recordClassTag); - for (final Schema.Field f : record.getSchema().getFields()) { - if (!RECORD.equals(f.schema().getType())) { - this.rootFields.put(f.name(), record.get(f.name())); - } - } + this(record, true); } public AvroPayload(@NonNull final GenericRecord record, @NonNull final List fieldsToCache) { - this.byteRecord = SparkUtil.serialize(record, recordClassTag); - for (final String f : fieldsToCache) { - this.rootFields.put(f, record.get(f)); - } + this.payloadInterval = new SerializedAvroPayloadInternal(record, fieldsToCache); } - /** - * Avoid calling it to fetch top level record fields. - */ + public AvroPayload(@NonNull final GenericRecord record, final boolean serializeRecord) { + this.payloadInterval = + serializeRecord ? new SerializedAvroPayloadInternal(record) : new AvroPayloadInternal(record); + } + + @Override public GenericRecord getData() { - return SparkUtil.deserialize(this.byteRecord, recordClassTag); + return this.payloadInterval.getData(); + } + + public Object getField(@NotEmpty final String fieldName) { + return this.payloadInterval.getField(fieldName); + } + + public static List getSerializationClasses() { + return Arrays.asList(AvroPayload.class, + IAvroPayloadInternal.class, + AvroPayloadInternal.class, + SerializedAvroPayloadInternal.class); + } + + private interface IAvroPayloadInternal { + + /** + * returns cached {@link GenericRecord} data. + */ + GenericRecord getData(); + + /** + * Returns field stored at root level. + */ + Object getField(@NotEmpty final String fieldName); + } + + @AllArgsConstructor + private static class AvroPayloadInternal implements IAvroPayloadInternal { + + @NonNull + private final GenericRecord record; + + @Override + public GenericRecord getData() { + return this.record; + } + + @Override + public Object getField(@NotEmpty final String fieldName) { + return this.record.get(fieldName); + } } /** - * It only supports fetching fields at the root level of the record which are of type other than - * {@link org.apache.avro.generic.GenericData.Record}. - * - * @param fieldName name of the field at the root level of the record. + * It internally stores AvroPayload as byte[] to reduce memory footprint. */ - public Object getField(@NotEmpty final String fieldName) { - Preconditions.checkState(this.rootFields.containsKey(fieldName), - "field is not cached at root level :" + fieldName); - return this.rootFields.get(fieldName); + private static class SerializedAvroPayloadInternal implements IAvroPayloadInternal { + + private final Map rootFields; + private final byte[] byteRecord; + + public SerializedAvroPayloadInternal(@NonNull final GenericRecord record) { + this(record, getFieldsToCache(record)); + } + + public SerializedAvroPayloadInternal(@NonNull final GenericRecord record, + @NonNull final List fieldsToCache) { + this.byteRecord = SparkUtil.serialize(record, SparkUtil.GENERIC_RECORD_CLASS_TAG); + this.rootFields = new HashMap<>(); + for (final String f : fieldsToCache) { + this.rootFields.put(f, record.get(f)); + } + } + + /** + * Avoid calling it to fetch top level record fields. + */ + public GenericRecord getData() { + return SparkUtil.deserialize(this.byteRecord, SparkUtil.GENERIC_RECORD_CLASS_TAG); + } + + /** + * It only supports fetching fields at the root level of the record which are of type other than + * {@link org.apache.avro.generic.GenericData.Record}. + * + * @param fieldName name of the field at the root level of the record. + */ + public Object getField(@NotEmpty final String fieldName) { + Preconditions.checkState(this.rootFields.containsKey(fieldName), + "field is not cached at root level :" + fieldName); + return this.rootFields.get(fieldName); + } + + private static List getFieldsToCache(@NonNull final GenericRecord record) { + final List fieldsToCache = new LinkedList<>(); + for (final Schema.Field f : record.getSchema().getFields()) { + if (!RECORD.equals(f.schema().getType())) { + fieldsToCache.add(f.name()); + } + } + return fieldsToCache; + } } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/DispersalLengthType.java b/marmaray/src/main/java/com/uber/marmaray/common/DispersalLengthType.java new file mode 100644 index 0000000..afc0904 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/DispersalLengthType.java @@ -0,0 +1,27 @@ +/* Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common; + +/** + * {@link DispersalLengthType} defines if dispersed data is from one single day or not + * 1. SINGLE_DAY + * 2. MULTIPLE_DAY + */ +public enum DispersalLengthType { + SINGLE_DAY, + MULTIPLE_DAY +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/MetadataManagerType.java b/marmaray/src/main/java/com/uber/marmaray/common/MetadataManagerType.java new file mode 100644 index 0000000..186eac3 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/MetadataManagerType.java @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common; + +/** + * {@link MetadataManagerType} defines metadata manager type + * normal: on disk + * Cassandra: cassandra based + */ +public enum MetadataManagerType { + HDFS, + CASSANDRA, + MULTI +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java index b5d9db8..f54b7bd 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java @@ -52,22 +52,29 @@ public final class JobDagActions { * Actions are executed in parallel; execution status will not affect the others. */ - public static final String DEFAULT_NAME = "anonymous"; + public static final String DEFAULT_TARGET = "anonymous"; + + /** + * The target for the JogDagActions. + * It is equal to datafeed/topic in the context of ingestion jobDag. + * The target could be something else, it's 'JobManager' for job manager actions. + */ + private static final String ACTION_TARGET = "action_target"; @Getter private final Queue actions; private final Reporters reporters; @Getter - private final String name; + private final String target; public JobDagActions(@NonNull final Reporters reporters) { - this(reporters, DEFAULT_NAME); + this(reporters, DEFAULT_TARGET); } - public JobDagActions(@NonNull final Reporters reporters, @NotEmpty final String name) { + public JobDagActions(@NonNull final Reporters reporters, @NotEmpty final String target) { this.actions = new ConcurrentLinkedDeque<>(); - this.name = name; + this.target = target; this.reporters = reporters; } @@ -115,7 +122,7 @@ public boolean execute(final boolean dagSuccess) { try { actionSuccess.set(future.get()); } catch (Exception e) { - log.error("Error running JobDagAction {} for {}:", action.getClass(), this.getName(), e); + log.error("Error running JobDagAction {} for {}:", action.getClass(), this.getTarget(), e); actionSuccess.set(false); successful.set(false); } @@ -130,12 +137,14 @@ public boolean execute(final boolean dagSuccess) { private void reportExecuteTime(@NonNull final IJobDagAction action, final long timeInMillis) { final LongMetric timeMetric = new LongMetric(TIME_METRIC, TimeUnit.MILLISECONDS.toSeconds(timeInMillis)); timeMetric.addTags(action.getMetricTags()); + timeMetric.addTag(ACTION_TARGET, this.getTarget()); this.reporters.getReporters().stream().forEach(r -> r.gauge(timeMetric)); } private void reportActionStatus(@NonNull final IJobDagAction action, final boolean isSuccess) { final LongMetric resultMetric = new LongMetric(RESULT_METRIC, isSuccess ? RESULT_SUCCESS : RESULT_FAILURE); resultMetric.addTags(action.getMetricTags()); + resultMetric.addTag(ACTION_TARGET, this.getTarget()); this.reporters.getReporters().stream().forEach(r -> r.gauge(resultMetric)); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java index 5f013e0..e2378da 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java @@ -47,7 +47,9 @@ public boolean execute(final boolean success) { this.dataFeedMetrics.gaugeAll(reporter); }); } else { - log.warn("No metrics produced or actions being executed on reporter because errors were encountered"); + this.reporters.getReporters().forEach(reporter -> this.dataFeedMetrics.gauageFailureMetric(reporter)); + log.warn("Other than failure reports " + + "no metrics produced or actions being executed on reporter because errors were encountered"); } return success; } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraMetadataManagerConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraMetadataManagerConfiguration.java new file mode 100644 index 0000000..e407598 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraMetadataManagerConfiguration.java @@ -0,0 +1,112 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.configuration; + +import com.google.common.base.Optional; +import com.google.common.base.Splitter; +import com.uber.marmaray.utilities.StringTypes; +import com.uber.marmaray.utilities.ConfigUtil; +import com.google.common.collect.Lists; + +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * {@link CassandraMetadataManagerConfiguration} contains all the generic metadata information for where Hive is either a source or + * sink for the data pipeline job. All CassandraMetadataManagerConfiguration properties starts with {@link #CASSANDRA_METADATA_MANAGER_PREFIX}. + */ + +@Slf4j +public class CassandraMetadataManagerConfiguration extends MetadataManagerConfiguration { + public static final String CASSANDRA_METADATA_MANAGER_PREFIX = METADATA_MANAGER_PREFIX + "cassandra."; + public static final String USERNAME = CASSANDRA_METADATA_MANAGER_PREFIX + "username"; + public static final String PASSWORD = CASSANDRA_METADATA_MANAGER_PREFIX + "password"; + public static final String KEYSPACE = CASSANDRA_METADATA_MANAGER_PREFIX + "keyspace"; + public static final String TABLE_NAME = CASSANDRA_METADATA_MANAGER_PREFIX + "table_name"; + public static final String CLUSTER = CASSANDRA_METADATA_MANAGER_PREFIX + "cluster"; + + // cassandra setting + public static final String DC_PATH = CASSANDRA_METADATA_MANAGER_PREFIX + "data_center_path"; + public static final String DATACENTER = CASSANDRA_METADATA_MANAGER_PREFIX + "datacenter"; + public static final String NATIVE_TRANSPORT_PORT = CASSANDRA_METADATA_MANAGER_PREFIX + "output.native.port"; + public static final String INITIAL_HOSTS = CASSANDRA_METADATA_MANAGER_PREFIX + "output.thrift.address"; + public static final String DEFAULT_OUTPUT_NATIVE_PORT = "9042"; + public static final String OUTPUT_THRIFT_PORT = CASSANDRA_METADATA_MANAGER_PREFIX + "output.thrift.port"; + public static final String STORAGE_PORT = CASSANDRA_METADATA_MANAGER_PREFIX + "storage.port"; + public static final String SSL_STORAGE_PORT = CASSANDRA_METADATA_MANAGER_PREFIX + "ssl.storage.port"; + public static final String DISABLE_QUERY_UNS = CASSANDRA_METADATA_MANAGER_PREFIX + "disable_uns"; + + private static final Splitter splitter = Splitter.on(StringTypes.COMMA); + + @Getter + protected final List initialHosts; + + @Getter + private final String username; + + @Getter + private final String password; + + @Getter + private final String keyspace; + + @Getter + private final String tableName; + + @Getter + private final String cluster; + + public CassandraMetadataManagerConfiguration(@NonNull final Configuration conf) { + super(conf); + ConfigUtil.checkMandatoryProperties(conf, this.getMandatoryProperties()); + + this.username = conf.getProperty(USERNAME).get(); + this.password = conf.getProperty(PASSWORD).get(); + this.keyspace = conf.getProperty(KEYSPACE).get(); + this.tableName = conf.getProperty(TABLE_NAME).get(); + this.cluster = conf.getProperty(CLUSTER).get(); + + if (conf.getProperty(INITIAL_HOSTS).isPresent()) { + this.initialHosts = splitString(conf.getProperty(INITIAL_HOSTS).get()); + } else { + this.initialHosts = new ArrayList<>(); + } + } + + public Optional getNativePort() { + return getConf().getProperty(NATIVE_TRANSPORT_PORT); + } + + protected List splitString(final String commaSeparatedValues) { + return Lists.newArrayList(splitter.split(commaSeparatedValues)); + } + + public static List getMandatoryProperties() { + return Collections.unmodifiableList( + Arrays.asList( + KEYSPACE, + CLUSTER, + TABLE_NAME + )); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java index 8c0e077..91ed65d 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java @@ -21,6 +21,9 @@ import com.google.common.collect.Lists; import com.uber.marmaray.common.PartitionType; import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.schema.cassandra.ClusterKey; import com.uber.marmaray.utilities.ConfigUtil; import com.uber.marmaray.utilities.SchemaUtil; @@ -30,6 +33,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.hadoop.ConfigHelper; +import parquet.Preconditions; import java.io.Serializable; import java.util.ArrayList; @@ -48,7 +52,7 @@ * to sanitize the values again here. */ @Slf4j -public class CassandraSinkConfiguration implements Serializable { +public class CassandraSinkConfiguration implements Serializable, IMetricable { public static final String CASSANDRA_PREFIX_ONLY = "cassandra."; public static final String CASS_COMMON_PREFIX = Configuration.MARMARAY_PREFIX + CASSANDRA_PREFIX_ONLY; @@ -70,6 +74,7 @@ public class CassandraSinkConfiguration implements Serializable { public static final String SSL_STORAGE_PORT = CASS_COMMON_PREFIX + "ssl.storage.port"; public static final String DISABLE_QUERY_UNS = CASS_COMMON_PREFIX + "disable_uns"; + public static final String DISABLE_QUERY_LANGLEY = CASS_COMMON_PREFIX + "disable_langley"; // *** End of Cassandra Configuration Settings ***1 public static final String DEFAULT_OUTPUT_RPC_PORT = "9160"; @@ -83,13 +88,18 @@ public class CassandraSinkConfiguration implements Serializable { public static final String PARTITION_KEYS = CASS_COMMON_PREFIX + "partition_keys"; public static final String CLUSTERING_KEYS = CASS_COMMON_PREFIX + "clustering_keys"; public static final String PARTITION_TYPE = CASS_COMMON_PREFIX + "partition_type"; + public static final String WRITTEN_TIME = CASS_COMMON_PREFIX + "written_time"; public static final String USERNAME = CASS_COMMON_PREFIX + "username"; public static final String PASSWORD = CASS_COMMON_PREFIX + "password"; public static final String USE_CLIENT_SINK = CASS_COMMON_PREFIX + "use_client_sink"; + public static final String SHOULD_SKIP_INVALID_ROWS = CASS_COMMON_PREFIX + "should_skip_invalid_rows"; // optional field for customers to timestamp their data public static final String TIMESTAMP = CASS_COMMON_PREFIX + SchemaUtil.DISPERSAL_TIMESTAMP; public static final String TIMESTAMP_IS_LONG_TYPE = CASS_COMMON_PREFIX + "timestamp_is_long_type"; + public static final String TIMESTAMP_FIELD_NAME = CASS_COMMON_PREFIX + "timestamp_field_name"; + public static final String DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME = "timestamp"; + public static final boolean DEFAULT_TIMESTAMP_IS_LONG_TYPE = false; // File path containing datacenter info on each machine public static final String DC_PATH = "data_center_path"; @@ -97,7 +107,18 @@ public class CassandraSinkConfiguration implements Serializable { public static final String DATACENTER = CASS_COMMON_PREFIX + "datacenter"; public static final String TIME_TO_LIVE = CASS_COMMON_PREFIX + "time_to_live"; - private static final long DEFAULT_TIME_TO_LIVE = 0L; + + // feature flags + public static final String ENABLE_IGNORE_HOSTS = CASS_COMMON_PREFIX + "enable_ignore_hosts"; + public static final boolean DEFAULT_ENABLE_IGNORE_HOSTS = false; + + public static final long DEFAULT_TIME_TO_LIVE = 0L; + + public static final String MAX_BATCH_SIZE_MB = CASS_COMMON_PREFIX + "max_batch_size_mb"; + public static final long DEFAULT_MAX_BATCH_SIZE_MB = -1; + public static final long DISABLE_BATCH = -1; + public static final String MIN_BATCH_DURATION_SECONDS = CASS_COMMON_PREFIX + "min_batch_duration_seconds"; + public static final long DEFAULT_MIN_BATCH_DURATION_SECONDS = 0; private static final Splitter splitter = Splitter.on(StringTypes.COMMA); @@ -127,9 +148,33 @@ public class CassandraSinkConfiguration implements Serializable { @Getter private final Optional writeTimestamp; + @Getter + private final String timestampFieldName; + @Getter private final boolean timestampIsLongType; + @Getter + private final boolean enableIgnoreHosts; + + @Getter + private final long maxBatchSizeMb; + + @Getter + private final long minBatchDurationSeconds; + + @Getter + private Optional dataFeedMetrics = Optional.absent(); + + @Getter + private final Optional writtenTime; + + public CassandraSinkConfiguration(@NonNull final Configuration conf, + @NonNull final DataFeedMetrics dataFeedMetrics) { + this(conf); + setDataFeedMetrics(dataFeedMetrics); + } + public CassandraSinkConfiguration(@NonNull final Configuration conf) { this.conf = conf; ConfigUtil.checkMandatoryProperties(this.conf, this.getMandatoryProperties()); @@ -151,6 +196,10 @@ public CassandraSinkConfiguration(@NonNull final Configuration conf) { ? initClusterKeys(this.splitString(this.conf.getProperty(CLUSTERING_KEYS).get())) : Collections.EMPTY_LIST; + this.writtenTime = this.conf.getProperty(WRITTEN_TIME).isPresent() + ? Optional.of(this.conf.getProperty(WRITTEN_TIME).get()) + : Optional.absent(); + if (this.conf.getProperty(PARTITION_TYPE).isPresent()) { this.partitionType = PartitionType.valueOf(this.conf.getProperty(PARTITION_TYPE) .get().trim().toUpperCase()); @@ -159,7 +208,26 @@ public CassandraSinkConfiguration(@NonNull final Configuration conf) { } this.writeTimestamp = this.conf.getProperty(TIMESTAMP); - this.timestampIsLongType = this.conf.getBooleanProperty(TIMESTAMP_IS_LONG_TYPE, false); + this.timestampFieldName = this.conf.getProperty(TIMESTAMP_FIELD_NAME, DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); + this.timestampIsLongType = this.conf.getBooleanProperty(TIMESTAMP_IS_LONG_TYPE, DEFAULT_TIMESTAMP_IS_LONG_TYPE); + this.enableIgnoreHosts = this.conf.getBooleanProperty(ENABLE_IGNORE_HOSTS, DEFAULT_ENABLE_IGNORE_HOSTS); + this.maxBatchSizeMb = this.conf.getLongProperty(MAX_BATCH_SIZE_MB, DEFAULT_MAX_BATCH_SIZE_MB); + Preconditions.checkState(this.maxBatchSizeMb > 0 || this.maxBatchSizeMb == DISABLE_BATCH, + String.format("%s must greater than zero or %d", MAX_BATCH_SIZE_MB, DISABLE_BATCH)); + this.minBatchDurationSeconds = this.conf.getLongProperty(MIN_BATCH_DURATION_SECONDS, + DEFAULT_MIN_BATCH_DURATION_SECONDS); + Preconditions.checkState(this.minBatchDurationSeconds >= 0, + String.format("%s must be non-negative", MIN_BATCH_DURATION_SECONDS)); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored } /** @@ -214,6 +282,10 @@ public Boolean getUseClientSink() { return this.getConf().getBooleanProperty(USE_CLIENT_SINK, false); } + public Boolean getShouldSkipInvalidRows() { + return this.getConf().getBooleanProperty(SHOULD_SKIP_INVALID_ROWS, false); + } + public Optional getUserName() { return this.getConf().getProperty(USERNAME); } @@ -263,4 +335,8 @@ protected List splitString(final String commaSeparatedValues) { private List initClusterKeys(final List entries) { return entries.stream().map(entry -> ClusterKey.parse(entry)).collect(Collectors.toList()); } + + public boolean isBatchEnabled() { + return getMaxBatchSizeMb() != DISABLE_BATCH; + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java index 48dcfa0..3ec5514 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java @@ -16,32 +16,30 @@ */ package com.uber.marmaray.common.configuration; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; - -import java.io.InputStream; -import java.io.Serializable; - import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.hibernate.validator.constraints.NotEmpty; import java.io.File; import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Properties; import java.util.Set; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.hibernate.validator.constraints.NotEmpty; - /** * {@link Configuration} will be instantiated from a YAML based file * and contain all the pertinent metadata to initialize and execute @@ -85,7 +83,12 @@ public Configuration(@NonNull final Configuration conf) { this.props.putAll(conf.props); } - public void loadYamlFile(@NonNull final File yamlFile, + @VisibleForTesting + public Configuration(@NonNull final Properties properties) { + this.props.putAll(properties); + } + + private void loadYamlFile(@NonNull final File yamlFile, final Optional scope) { try { final FileSystem localFs = FileSystem.getLocal( @@ -108,7 +111,7 @@ public void loadYamlStream(@NonNull final InputStream yamlStream, final JsonNode scopeOverriddenJsonNode = handleScopeOverriding(scope, jsonNode); parseConfigJson(scopeOverriddenJsonNode, ""); } catch (IOException e) { - final String errorMsg = "Error loading yaml file "; + final String errorMsg = "Error loading config from stream"; log.error(errorMsg, e); throw new JobRuntimeException(errorMsg, e); } @@ -159,6 +162,12 @@ public Map getPropertiesWithPrefix(final String prefix, final bo return properties; } + public Properties getProperties() { + final Properties properties = new Properties(); + this.props.forEach((k, v) -> properties.put(k, v)); + return properties; + } + public static T getProperty(@NonNull final Configuration conf, @NotEmpty final String key, @NonNull final T defaultValue) { if (defaultValue instanceof Integer) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java index 75c6a67..940b883 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java @@ -21,6 +21,7 @@ import com.uber.marmaray.common.FileSinkType; import com.uber.marmaray.common.PartitionType; import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.DispersalLengthType; import com.uber.marmaray.utilities.ConfigUtil; import com.uber.marmaray.utilities.SchemaUtil; import lombok.Getter; @@ -46,8 +47,15 @@ public class FileSinkConfiguration implements Serializable { public static final String PATH_PREFIX = FILE_COMM_PREFIX + "path_prefix"; public static final String FILE_TYPE = FILE_COMM_PREFIX + "file_type"; public static final String DEFAULT_FILE_TYPE = "csv"; + public static final String COMPRESSION = FILE_COMM_PREFIX + "compression"; + public static final Boolean DEFAULT_COMPRESSION = false; + public static final String COMPRESSION_CODEC = FILE_COMM_PREFIX + "compression_codec"; + public static final String DEFAULT_COMPRESSION_CODEC = "lz4"; + public static final String ROW_IDENTIFIER = FILE_COMM_PREFIX + "row_identifier"; + public static final String DEFAULT_ROW_IDENTIFIER = ""; public static final String CSV_COLUMN_HEADER = FILE_COMM_PREFIX + "with_column_header"; public static final Boolean DEFAULT_CSV_COLUMN_HEADER = false; + public static final String DISPERSAL_LENGTH = FILE_COMM_PREFIX + "dispersal_length"; //Expected file size in MegaByte public static final String FILE_SIZE_MEGABYTE = FILE_COMM_PREFIX + "file_size_megabyte"; //Default file size set output to one file. @@ -86,6 +94,10 @@ public class FileSinkConfiguration implements Serializable { @Getter private final String fileType; @Getter + private final boolean compression; + @Getter + private final String compressionCodec; + @Getter private final double fileSizeMegaBytes; @Getter private final boolean columnHeader; @@ -109,6 +121,8 @@ public class FileSinkConfiguration implements Serializable { private final String fileNamePrefix; @Getter private final PartitionType partitionType; + @Getter + private final String rowIdentifier; //aws s3 properties @Getter @@ -126,12 +140,18 @@ public class FileSinkConfiguration implements Serializable { @Getter private final Configuration conf; + @Getter + private final DispersalLengthType dispersalLength; + public FileSinkConfiguration(@NonNull final Configuration conf) throws MissingPropertyException, UnsupportedOperationException { this.conf = conf; ConfigUtil.checkMandatoryProperties(this.conf, this.getMandatoryProperties()); this.path = this.conf.getProperty(FS_PATH, DEFAULT_FS_PATH); this.fileType = this.conf.getProperty(FILE_TYPE, DEFAULT_FILE_TYPE); + this.compression = this.conf.getBooleanProperty(COMPRESSION, DEFAULT_COMPRESSION); + this.compressionCodec = this.conf.getProperty(COMPRESSION_CODEC, DEFAULT_COMPRESSION_CODEC); + this.rowIdentifier = this.conf.getProperty(ROW_IDENTIFIER, DEFAULT_ROW_IDENTIFIER); this.fileSizeMegaBytes = this.conf.getDoubleProperty(FILE_SIZE_MEGABYTE, DEFAULT_FILE_SIZE); this.columnHeader = this.conf.getBooleanProperty(CSV_COLUMN_HEADER, DEFAULT_CSV_COLUMN_HEADER); @@ -205,6 +225,13 @@ public FileSinkConfiguration(@NonNull final Configuration conf) this.writeTimestamp = this.conf.getProperty(TIMESTAMP).get(); this.sourceType = this.conf.getProperty(SOURCE_TYPE).get(); + if (this.conf.getProperty(DISPERSAL_LENGTH).isPresent() + && this.conf.getProperty(DISPERSAL_LENGTH).get().equals("multiple_day")) { + this.dispersalLength = DispersalLengthType.MULTIPLE_DAY; + } else { + this.dispersalLength = DispersalLengthType.SINGLE_DAY; + } + this.fileNamePrefix = String.format("%s_%s_%s_%s", FILE_NAME_PREFIX, this.sourceType, this.sourceNamePrefix, this.writeTimestamp); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSourceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSourceConfiguration.java index 7fac4fc..5346c70 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSourceConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSourceConfiguration.java @@ -17,6 +17,7 @@ package com.uber.marmaray.common.configuration; +import com.google.common.base.Optional; import com.uber.marmaray.utilities.ConfigUtil; import lombok.NonNull; import org.apache.hadoop.fs.FileSystem; @@ -49,16 +50,16 @@ public String getType() { return this.conf.getProperty(TYPE).get().toLowerCase(); } - public String getSchema() { - return this.conf.getProperty(SCHEMA).get(); - } - public List getMandatoryProperties() { - return Arrays.asList(DIRECTORY, TYPE, SCHEMA); + return Arrays.asList(DIRECTORY, TYPE); } public FileSystem getFileSystem() throws IOException { return FileSystem.get(new HadoopConfiguration(this.conf).getHadoopConf()); } + public Optional getSchema() { + return this.conf.getProperty(SCHEMA); + } + } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSMetadataManagerConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSMetadataManagerConfiguration.java new file mode 100644 index 0000000..07e736e --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSMetadataManagerConfiguration.java @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.configuration; + +import lombok.Getter; +import lombok.NonNull; + +/** + * {@link HDFSMetadataManagerConfiguration} contains all the generic metadata information for where Hive is either a source or + * sink for the data pipeline job. All HDFSMetadataManagerConfiguration properties starts with {@link #}. + */ +public class HDFSMetadataManagerConfiguration extends MetadataManagerConfiguration { + public static final String HDFS_METADATA_MANAGER_PREFIX = METADATA_MANAGER_PREFIX + "HDFS."; + public static final String BASE_METADATA_PATH = HDFS_METADATA_MANAGER_PREFIX + "job_metadata"; + + @Getter + private final String baseMetadataPath; + + public HDFSMetadataManagerConfiguration(@NonNull final Configuration conf) { + super(conf); + this.baseMetadataPath = this.getConf().getProperty(BASE_METADATA_PATH).get(); + } +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSSchemaServiceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSSchemaServiceConfiguration.java index 1df3367..e4a155a 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSSchemaServiceConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HDFSSchemaServiceConfiguration.java @@ -27,7 +27,7 @@ public class HDFSSchemaServiceConfiguration implements Serializable { - public static final String HDFS_SCHEMA_SERVICE_PREFIX = Configuration.MARMARAY_PREFIX + "hdfs_schema_service."; + public static final String HDFS_SCHEMA_SERVICE_PREFIX = Configuration.MARMARAY_PREFIX + "hdfs_schema_service"; public static final String PATH = HDFS_SCHEMA_SERVICE_PREFIX + "path"; private final Configuration conf; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java index 3658671..e9236bf 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java @@ -18,6 +18,7 @@ import com.google.common.base.Optional; import com.uber.marmaray.common.PartitionType; +import com.uber.marmaray.common.metadata.StringValue; import com.uber.marmaray.utilities.ConfigUtil; import lombok.Getter; import lombok.NonNull; @@ -35,9 +36,9 @@ public class HiveConfiguration implements Serializable { public static final String HIVE_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hive."; public static final String HIVE_DATA_PATH = HIVE_PROPERTY_PREFIX + "data_path"; public static final String JOB_NAME = HIVE_PROPERTY_PREFIX + "job_name"; - public static final String BASE_METADATA_PATH = HIVE_PROPERTY_PREFIX + "job_metadata"; public static final String PARTITION_KEY_NAME = HIVE_PROPERTY_PREFIX + "partition_key_name"; public static final String PARTITION_TYPE = HIVE_PROPERTY_PREFIX + "partition_type"; + public static final String PARTITION = HIVE_PROPERTY_PREFIX + "partition"; @Getter private final Configuration conf; @@ -52,10 +53,10 @@ public class HiveConfiguration implements Serializable { private final String jobName; @Getter - private final String baseMetadataPath; + private final Optional partitionKeyName; @Getter - private final Optional partitionKeyName; + private final Optional partition; @Getter private final PartitionType partitionType; @@ -66,7 +67,6 @@ public HiveConfiguration(@NonNull final Configuration conf) { this.dataPath = this.getConf().getProperty(HIVE_DATA_PATH).get(); this.jobName = this.getConf().getProperty(JOB_NAME).get(); - this.baseMetadataPath = this.getConf().getProperty(BASE_METADATA_PATH).get(); this.partitionKeyName = this.getConf().getProperty(PARTITION_KEY_NAME); if (this.conf.getProperty(PARTITION_TYPE).isPresent()) { @@ -75,9 +75,12 @@ public HiveConfiguration(@NonNull final Configuration conf) { } else { this.partitionType = PartitionType.NONE; } + + this.partition = this.getConf().getProperty(PARTITION).isPresent() + ? Optional.of(new StringValue(this.getConf().getProperty(PARTITION).get())) : Optional.absent(); } public static List getMandatoryProperties() { - return Collections.unmodifiableList(Arrays.asList(HIVE_DATA_PATH, JOB_NAME, BASE_METADATA_PATH)); + return Collections.unmodifiableList(Arrays.asList(HIVE_DATA_PATH, JOB_NAME)); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java index 71fa778..171f228 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java @@ -23,10 +23,12 @@ import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieMetricsConfig; +import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.exceptions.MissingPropertyException; import com.uber.marmaray.utilities.ConfigUtil; +import com.uber.marmaray.utilities.StringTypes; import lombok.Getter; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -126,6 +128,10 @@ public class HoodieConfiguration implements Serializable { public static final String HOODIE_CLEANER_VERSIONS_RETAINED = HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_versions_retained"; public static final int DEFAULT_HOODIE_CLEANER_VERSIONS_RETAINED = 3; + /** + * Hoodie Data partitioner + */ + public static final String HOODIE_DATA_PARTITIONER = HOODIE_COMMON_PROPERTY_PREFIX + "data_partitioner"; /** * Hoodie compaction small file size */ @@ -137,12 +143,13 @@ public class HoodieConfiguration implements Serializable { */ /** * Range for maximum parquet file size (uncompressed) is between {@link #HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT} - * and 6GB. Default is set to 4GB. Set this value to at least > 2.5 times + * and 6GB. Default is set to 1GB. Set this value to at least > 2.5 times + * TODO: Reduced to int to handle current hoodie version. Should be converted back to long on hoodie upgrade * {@link #HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT}. */ public static final String HOODIE_PARQUET_MAX_FILE_SIZE = HOODIE_COMMON_PROPERTY_PREFIX + "parquet_max_file_size"; - public static final long DEFAULT_HOODIE_PARQUET_MAX_FILE_SIZE = 4 * FileUtils.ONE_GB; + public static final int DEFAULT_HOODIE_PARQUET_MAX_FILE_SIZE = (int) FileUtils.ONE_GB; /** * Hoodie insert split size */ @@ -193,6 +200,14 @@ public class HoodieConfiguration implements Serializable { HOODIE_COMMON_PROPERTY_PREFIX + "rollback_inflight_commits"; public static final boolean DEFAULT_HOODIE_ROLLBACK_INFLIGHT_COMMITS = true; + /** + * Payload class to use. + */ + public static final String HOODIE_PAYLOAD_CLASS_NAME = + HOODIE_COMMON_PROPERTY_PREFIX + "paylaod_class_name"; + // default isn't used to set a vaule, but instead to detect if it was set + public static final String DEFAULT_HOODIE_PAYLOAD_CLASS_NAME = "default"; + @Getter private final Configuration conf; @Getter @@ -242,6 +257,11 @@ public String getHoodieMetricsPrefix() { return this.getConf().getProperty(getTablePropertyKey(HOODIE_METRICS_PREFIX, this.tableKey)).get(); } + public String getHoodieDataPartitioner(@NotEmpty final String defaultDataPartitioner) { + return this.getConf().getProperty(getTablePropertyKey(HOODIE_DATA_PARTITIONER, this.tableKey), + defaultDataPartitioner); + } + /** * @return true if {@link com.uber.hoodie.HoodieWriteClient} should rollback inflight commits from previous write * call. @@ -315,7 +335,9 @@ public int getUpsertParallelism() { public HoodieWriteConfig getHoodieWriteConfig() { final HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder(); try { - builder.forTable(getTableName()); + // This table name is used for sending metrics to graphite by hoodie. It expects table name to be without + // ".". + builder.forTable(getTableName().replaceAll("\\.", StringTypes.UNDERSCORE)); builder.withPath(getBasePath()); final boolean combineBeforeInsert = getProperty(HOODIE_COMBINE_BEFORE_INSERT, DEFAULT_HOODIE_COMBINE_BEFORE_INSERT); @@ -337,8 +359,15 @@ public HoodieWriteConfig getHoodieWriteConfig() { // Hoodie compaction config. final HoodieCompactionConfig.Builder compactionConfigBuilder = HoodieCompactionConfig.newBuilder(); + compactionConfigBuilder.withCleanerPolicy(HoodieCleaningPolicy .valueOf(getProperty(HOODIE_CLEANER_POLICY, DEFAULT_HOODIE_CLEANER_POLICY))); + + // set the payload class if it has been set in configuration + final String payloadClass = getProperty(HOODIE_PAYLOAD_CLASS_NAME, DEFAULT_HOODIE_PAYLOAD_CLASS_NAME); + if (!DEFAULT_HOODIE_PAYLOAD_CLASS_NAME.equals(payloadClass)) { + compactionConfigBuilder.withPayloadClass(payloadClass); + } compactionConfigBuilder.retainCommits( getProperty(HOODIE_CLEANER_COMMITS_RETAINED, DEFAULT_HOODIE_CLEANER_COMMITS_RETAINED)); compactionConfigBuilder.retainFileVersions( @@ -356,6 +385,12 @@ public HoodieWriteConfig getHoodieWriteConfig() { compactionConfigBuilder.withAutoClean(shouldAutoClean()); builder.withCompactionConfig(compactionConfigBuilder.build()); + // Hoodie storage config. + builder.withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize( + getProperty(HOODIE_PARQUET_MAX_FILE_SIZE, DEFAULT_HOODIE_PARQUET_MAX_FILE_SIZE) + ).build()); + // Hoodie index config builder.withIndexConfig(new HoodieIndexConfiguration(getConf(), getTableKey()).configureHoodieIndex()); @@ -389,6 +424,9 @@ public HoodieWriteConfig getHoodieWriteConfig() { // enable tmp directory writes for hoodie. builder.withUseTempFolderCopyOnWriteForCreate(true); + // enabled the renaming for copy detection on merge + builder.withUseTempFolderCopyOnWriteForMerge(true); + return builder.build(); } catch (IllegalArgumentException e) { throw new MissingPropertyException(e.getMessage(), e); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java index d73c8dd..4db0214 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.index.HoodieIndex.IndexType; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.utilities.StringTypes; import lombok.Getter; @@ -48,6 +49,7 @@ public class HoodieIndexConfiguration extends HoodieConfiguration { */ public static final String HOODIE_BLOOM_INDEX = "bloom"; public static final String HOODIE_HBASE_INDEX = "hbase"; + public static final String HOODIE_IN_MEMORY_INDEX = "in_memory"; public static final String HOODIE_HBASE_INDEX_PREFIX = "hbase."; public static final String HOODIE_INDEX_TYPE = HOODIE_INDEX_PROPERTY_PREFIX + "type"; public static final String HOODIE_INDEX_ZKNODE = "zknode."; @@ -70,13 +72,13 @@ public class HoodieIndexConfiguration extends HoodieConfiguration { * Hoodie index QPS fraction */ public static final String HOODIE_INDEX_QPS_FRACTION = HOODIE_INDEX_PROPERTY_PREFIX + "qps_fraction"; - public static final double DEFAULT_HOODIE_INDEX_QPS_FRACTION = 0.125f; + public static final double DEFAULT_HOODIE_INDEX_QPS_FRACTION = 0.002f; /** * Hoodie index max QPS per region server */ public static final String HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER = HOODIE_INDEX_PROPERTY_PREFIX + "max_qps_per_region_server"; - public static final int DEFAULT_HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER = 400; + public static final int DEFAULT_HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER = 32000; public static final String DEFAULT_VERSION = ""; /** @@ -102,6 +104,8 @@ public HoodieIndex.IndexType getHoodieIndexType() { return HoodieIndex.IndexType.BLOOM; } else if (HOODIE_HBASE_INDEX.equals(indexName.toLowerCase())) { return HoodieIndex.IndexType.HBASE; + } else if (HOODIE_IN_MEMORY_INDEX.equals(indexName.toLowerCase())) { + return IndexType.INMEMORY; } else { throw new IllegalStateException("Unsupported index type " + indexName); } @@ -170,6 +174,11 @@ public HoodieIndexConfig configureHoodieIndex() { final String zkZnodeParent = getZkZnodeParent(); createHbaseIndexTableIfNotExists(topicName, quorum, port.toString(), zkZnodeParent, version); + builder + .hbaseIndexGetBatchSize(getHoodieIndexGetBatchSize()) + .hbaseTableName(getHoodieHbaseIndexTableName()) + .hbaseZkPort(port) + .hbaseZkQuorum(quorum); } return builder.build(); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java index bd64947..e870ea5 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java @@ -22,7 +22,9 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import scala.Serializable; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,6 +48,10 @@ public class KafkaConfiguration implements Serializable { public static final String ENABLE_AUTO_COMMIT_VALUE = "false"; public static final String KAFKA_BACKOFF_MS_CONFIG = "retry.backoff.ms"; public static final String DEFAULT_KAFKA_BACKOFF_MS_CONFIG = "20"; + public static final String CLIENT_ID = "client.id"; + public static final String DEFAULT_CLIENT_ID = "marmaray-%s"; + private static int lastClientIdCache = 0; + private static final List clientIdQ = new ArrayList<>(); @Getter private final Configuration conf; @@ -66,12 +72,23 @@ public KafkaConfiguration(@NonNull final Configuration conf) { this.kafkaParams.put(VALUE_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName()); this.kafkaParams.put(ENABLE_AUTO_COMMIT, ENABLE_AUTO_COMMIT_VALUE); // If retry backoff is not set then we would want to reduce it to lower values. Default value is 400ms. - if (!kafkaParams.containsKey(KAFKA_BACKOFF_MS_CONFIG)) { - kafkaParams.put(KAFKA_BACKOFF_MS_CONFIG, DEFAULT_KAFKA_BACKOFF_MS_CONFIG); + if (!this.kafkaParams.containsKey(KAFKA_BACKOFF_MS_CONFIG)) { + this.kafkaParams.put(KAFKA_BACKOFF_MS_CONFIG, DEFAULT_KAFKA_BACKOFF_MS_CONFIG); } } public List getMandatoryProperties() { return Arrays.asList(KAFKA_BROKER_LIST); } + + public static synchronized int getClientId() { + if (clientIdQ.isEmpty()) { + int newClientIds = 1000; + while (newClientIds-- > 0) { + clientIdQ.add(lastClientIdCache++); + } + Collections.shuffle(clientIdQ); + } + return clientIdQ.remove(0); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java index b5525f1..48936e9 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java @@ -16,12 +16,16 @@ */ package com.uber.marmaray.common.configuration; +import com.google.common.base.Optional; +import com.uber.marmaray.common.exceptions.MissingPropertyException; import com.uber.marmaray.utilities.NumberConstants; import lombok.Getter; import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; +import java.util.concurrent.TimeUnit; import java.util.LinkedList; import java.util.List; @@ -30,6 +34,7 @@ * * All properties start with {@link #KAFKA_COMMON_PROPERTY_PREFIX}. */ +@Slf4j public class KafkaSourceConfiguration extends KafkaConfiguration { public static final String KAFKA_PROPERTY_PREFIX = KAFKA_COMMON_PROPERTY_PREFIX + "source."; @@ -38,18 +43,23 @@ public class KafkaSourceConfiguration extends KafkaConfiguration { public static final String KAFKA_MAX_MESSAGES_TO_READ = KAFKA_PROPERTY_PREFIX + "max_messages"; public static final long DEFAULT_KAFKA_MAX_MESSAGES_TO_READ = NumberConstants.ONE_MILLION; public static final String KAFKA_READ_PARALLELISM = KAFKA_PROPERTY_PREFIX + "read_parallelism"; + public static final String USE_PARALLEL_BROKER_READ = KAFKA_PROPERTY_PREFIX + "use_parallel_broker_read"; + public static final boolean DEFAULT_USE_PARALLEL_BROKER_READ = false; /** * It is used for the very first run to set partition offsets for kafka topic. Expected format is "yyyy-MM-dd". */ public static final String KAFKA_START_DATE = KAFKA_PROPERTY_PREFIX + "start_date"; public static final String KAFKA_START_DATE_FORMAT = "yyyy-MM-dd"; + // epoch time in seconds + public static final String KAFKA_START_TIME = KAFKA_PROPERTY_PREFIX + "start_time"; + public static final long MAX_KAFKA_LOOKBACK_SEC = TimeUnit.DAYS.toSeconds(7); @Getter private final String topicName; @Getter private final String clusterName; /** - * start time in millis. (inclusive). + * start time in seconds. (inclusive). */ @Getter private final long startTime; @@ -58,10 +68,27 @@ public KafkaSourceConfiguration(@NonNull final Configuration conf) { super(conf); this.topicName = getConf().getProperty(KAFKA_TOPIC_NAME).get(); this.clusterName = getConf().getProperty(KAFKA_CLUSTER_NAME).get(); - this.startTime = - DateTime.parse(getConf().getProperty(KAFKA_START_DATE).get(), - DateTimeFormat.forPattern(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).withZoneUTC() + + //Try to initialize the start time. "start_date" is legacy, please use start_time moving forward. + final Optional startTimeinSeconds = getConf().getProperty(KAFKA_START_TIME); + final Optional startDate = getConf().getProperty(KAFKA_START_DATE); + + if (startTimeinSeconds.isPresent()) { + this.startTime = Long.valueOf(startTimeinSeconds.get()); + } else if (startDate.isPresent()) { + this.startTime = DateTime.parse( + startDate.get(), + DateTimeFormat.forPattern(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).withZoneUTC() ).toDate().getTime(); + } else { + throw new MissingPropertyException(String.format("property %s OR %s must be specified.", + KAFKA_START_TIME, KAFKA_START_DATE)); + } + + if (this.startTime < (TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis()) - MAX_KAFKA_LOOKBACK_SEC)) { + log.error("Invalid Kafka start time/date ({}) - please specify a more recent start time.", this.startTime); + throw new RuntimeException("Invalid kafka start time/date"); + } } public List getMandatoryProperties() { @@ -69,7 +96,6 @@ public List getMandatoryProperties() { ret.addAll(super.getMandatoryProperties()); ret.add(KAFKA_TOPIC_NAME); ret.add(KAFKA_CLUSTER_NAME); - ret.add(KAFKA_START_DATE); return ret; } @@ -77,6 +103,10 @@ public int getReadParallelism() { return Math.max(1, getConf().getIntProperty(KAFKA_READ_PARALLELISM, 1)); } + public boolean isParallelBrokerReadEnabled() { + return this.getConf().getBooleanProperty(USE_PARALLEL_BROKER_READ, DEFAULT_USE_PARALLEL_BROKER_READ); + } + public long getMaxMessagesToRead() { return getConf().getLongProperty(KAFKA_MAX_MESSAGES_TO_READ, DEFAULT_KAFKA_MAX_MESSAGES_TO_READ); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/MetadataManagerConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/MetadataManagerConfiguration.java new file mode 100644 index 0000000..3d1e2a9 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/MetadataManagerConfiguration.java @@ -0,0 +1,78 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.configuration; + +import com.uber.marmaray.common.MetadataManagerType; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.utilities.ConfigUtil; +import lombok.Getter; +import lombok.NonNull; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * {@link MetadataManagerConfiguration} contains all the generic metadata information for where Hive is either a source or + * sink for the data pipeline job. All MetadataManagerConfiguration properties starts with {@link #METADATA_MANAGER_PREFIX}. + */ +public class MetadataManagerConfiguration implements Serializable, IMetricable { + public static final String METADATA_MANAGER_PREFIX = Configuration.MARMARAY_PREFIX + "metadata_manager."; + public static final String JOB_NAME = METADATA_MANAGER_PREFIX + "job_name"; + public static final String TYPE = METADATA_MANAGER_PREFIX + "type"; + + @Getter + private final Configuration conf; + + @Getter + private final String jobName; + + @Getter + private final MetadataManagerType metadataType; + + public MetadataManagerConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties()); + this.jobName = this.conf.getProperty(MetadataManagerConfiguration.JOB_NAME).get(); + if (this.conf.getProperty(MetadataManagerConfiguration.TYPE).isPresent()) { + this.metadataType = MetadataManagerType.valueOf(this.conf.getProperty(MetadataManagerConfiguration.TYPE) + .get().trim().toUpperCase()); + } else { + this.metadataType = MetadataManagerType.HDFS; + } + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + // Ignored + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // Ignored + } + + public static List getMandatoryProperties() { + return Collections.unmodifiableList(Arrays.asList( + MetadataManagerConfiguration.TYPE, + MetadataManagerConfiguration.JOB_NAME)); + } +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/SparkConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SparkConfiguration.java new file mode 100644 index 0000000..2b93d77 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SparkConfiguration.java @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.configuration; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class SparkConfiguration { + + public static final String SPARK_PROPERTIES_KEY_PREFIX = "spark_properties."; + + public static Map getOverrideSparkProperties( + @NonNull final Configuration configuration) { + final Map map = new HashMap<>(); + final Map sparkProps = configuration + .getPropertiesWithPrefix(SPARK_PROPERTIES_KEY_PREFIX, true); + for (Entry entry : sparkProps.entrySet()) { + log.info("Setting spark key:val {} : {}", entry.getKey(), entry.getValue()); + map.put(entry.getKey(), entry.getValue()); + } + return map; + } + + public static void overrideSparkConfInConfiguration( + @NonNull final Configuration configuration, + @NonNull final Map sparkConfigOverrides) { + for (Entry me : sparkConfigOverrides.entrySet()) { + final String sparkConfigKey = me.getKey(); + final String sparkConfigValue = me.getValue(); + log.info("Overriding spark key:val {} : {}", sparkConfigKey, sparkConfigValue); + configuration.setProperty(SPARK_PROPERTIES_KEY_PREFIX + sparkConfigKey, sparkConfigValue); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java index 42c808d..27ec168 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java @@ -17,6 +17,7 @@ package com.uber.marmaray.common.converters.data; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.common.data.ErrorData; @@ -29,6 +30,9 @@ import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.forkoperator.ForkFunction; import com.uber.marmaray.common.forkoperator.ForkOperator; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IMetricable; import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.ErrorTableUtil; import lombok.Getter; @@ -44,6 +48,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; /** * Implementations of the {@link AbstractDataConverter} interface will convert data records from one schema type @@ -54,12 +59,13 @@ * @param outputDataType */ @Slf4j -public abstract class AbstractDataConverter implements Serializable { +public abstract class AbstractDataConverter implements Serializable, IMetricable { public static final long serialVersionUID = 1L; public static final Integer VALID_RECORD = 0; public static final Integer ERROR_RECORD = 1; private static final String SUCCESS = "SUCCESS"; private static final String FAILURE = "FAILURE"; + private static final String CONVERTER_TAG_NAME = "CONVERTER_NAME"; @Getter @NonNull protected Configuration conf; @@ -81,17 +87,26 @@ public abstract class AbstractDataConverter implements Serializa @NonNull protected ErrorExtractor errorExtractor; + @NonNull + protected Optional topicMetrics = Optional.absent(); + public AbstractDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { this.conf = conf; this.errorExtractor = errorExtractor; } + public void setDataFeedMetrics(@NonNull final DataFeedMetrics topicMetrics) { + this.topicMetrics = Optional.of(topicMetrics); + } + public final RDDWrapper map(@NonNull final JavaRDD data) { final ForkOperator converter = new ForkOperator<>(data.map(r -> RawDataHelper.getRawData(r)), new DataConversionFunction(), this.conf); converter.execute(); - + if (topicMetrics.isPresent()) { + reportMetrics(converter.getRddSize(), converter.getNumRddPartitions(), topicMetrics.get()); + } // Write error records. ErrorTableUtil.writeErrorRecordsToErrorTable(data.context(), this.conf, Optional.absent(), new RDDWrapper<>(converter.getRDD(ERROR_RECORD).map(r -> (ErrorData) r), converter.getCount(ERROR_RECORD)), @@ -103,6 +118,15 @@ public final RDDWrapper map(@NonNull final JavaRDD data) { protected abstract List> convert(@NonNull ID data) throws Exception; + private void reportMetrics(final long rddSize, + final int numPartitions, + @NonNull final DataFeedMetrics topicMetrics) { + final Map tags = ImmutableMap.of(CONVERTER_TAG_NAME, this.getClass().getName()); + topicMetrics.createLongMetric(DataFeedMetricNames.RDD_PARTITION_SIZE, + rddSize / Math.max(1, numPartitions), tags); + topicMetrics.createLongMetric(DataFeedMetricNames.NUM_RDD_PARTITIONS, numPartitions, tags); + } + public class DataConversionFunction extends ForkFunction { public DataConversionFunction() { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java index 7abf969..3b375e9 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.common.converters.data; +import com.datastax.driver.core.Statement; import com.datastax.driver.core.querybuilder.Insert; import com.datastax.driver.core.querybuilder.QueryBuilder; import com.google.common.base.Joiner; @@ -26,17 +27,22 @@ import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.schema.cassandra.CassandraSchema; import com.uber.marmaray.utilities.ByteBufferUtil; import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.GenericRecordUtil; import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.TimestampInfo; +import lombok.Getter; import lombok.NonNull; import lombok.Setter; import org.apache.avro.Schema; import org.apache.cassandra.db.marshal.LongType; -import com.datastax.driver.core.Statement; import java.nio.ByteBuffer; import java.util.Collections; @@ -53,7 +59,6 @@ public class CassandraSinkCQLDataConverter extends SinkDataConverter { private static final long serialVersionUID = 1L; - private final String inputSchemaJson; @Setter @@ -62,6 +67,9 @@ public class CassandraSinkCQLDataConverter extends SinkDataConverter dataFeedMetrics = Optional.absent(); + /* * The fields to convert are defined in the job configuration from the user. This can be all or a subset of fields * from the schema. @@ -73,9 +81,7 @@ public class CassandraSinkCQLDataConverter extends SinkDataConverter requiredFields; - private Optional inputSchema = Optional.absent(); - private final TimestampInfo timestampInfo; /** @@ -103,6 +109,16 @@ public CassandraSinkCQLDataConverter(@NonNull final Schema inputSchema, this.tableName = conf.getProperty(CassandraSinkConfiguration.TABLE_NAME, ""); } + @Override + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + @Override public List> convert(final AvroPayload avroPayload) throws Exception { final Insert insertStatement = QueryBuilder.insertInto(keyspaceName, tableName); @@ -119,43 +135,67 @@ public List> convert(final AvroPayload a final Object rawData = avroPayload.getData().get(field.name()); if (rawData != null) { + // support timestamp field + if (SchemaUtil.isTimestampSchema(field.schema())) { + final Long longData = SchemaUtil.decodeTimestamp(rawData).getTime(); + insertStatement.value(field.name(), longData); - final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema()) + } else { + + final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema()) ? GenericRecordUtil.getNonNull(field.schema()) : field.schema(); - final Schema.Type type = nonNullSchema.getType(); - - switch (type) { - case BOOLEAN: - final Boolean boolData = (Boolean) rawData; - insertStatement.value(field.name(), boolData); - break; - case INT: - final Integer intData = (Integer) rawData; - insertStatement.value(field.name(), intData); - break; - case LONG: - final Long longData = (Long) rawData; - insertStatement.value(field.name(), longData); - break; - case DOUBLE: - final Double doubleData = (Double) rawData; - insertStatement.value(field.name(), doubleData); - break; - case STRING: - final String strData = rawData.toString(); - insertStatement.value(field.name(), strData); - break; - case FLOAT: - final Float floatData = (Float) rawData; - insertStatement.value(field.name(), floatData); - break; - // todo(T936057) - add support for non-primitive types - default: - throw new JobRuntimeException("Type " + field.schema().getType() + " not supported"); + final Schema.Type type = nonNullSchema.getType(); + + switch (type) { + case BOOLEAN: + final Boolean boolData = (Boolean) rawData; + insertStatement.value(field.name(), boolData); + break; + case INT: + final Integer intData = (Integer) rawData; + insertStatement.value(field.name(), intData); + break; + case LONG: + final Long longData = (Long) rawData; + insertStatement.value(field.name(), longData); + break; + case DOUBLE: + final Double doubleData = (Double) rawData; + insertStatement.value(field.name(), doubleData); + break; + case STRING: + final String strData = rawData.toString(); + insertStatement.value(field.name(), strData); + break; + case FLOAT: + final Float floatData = (Float) rawData; + insertStatement.value(field.name(), floatData); + break; + case BYTES: + final ByteBuffer byteData = (ByteBuffer) rawData; + insertStatement.value(field.name(), byteData); + break; + // todo(T936057) - add support for non-primitive types + default: + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongMetric( + DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, + ErrorCauseTagNames.NOT_SUPPORTED_FIELD_TYPE)); + } + throw new JobRuntimeException("Type " + field.schema().getType() + " not supported"); + } } } else { if (requiredKeysToFind.contains(field.name())) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_DATA)); + } throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name()); } } @@ -167,11 +207,16 @@ public List> convert(final AvroPayload a final ByteBuffer bb = this.timestampInfo.isSaveAsLongType() ? LongType.instance.decompose(Long.parseLong(this.timestampInfo.getTimestamp().get())) : ByteBufferUtil.wrap(this.timestampInfo.getTimestamp().get()); - insertStatement.value(SchemaUtil.DISPERSAL_TIMESTAMP, bb); + insertStatement.value(timestampInfo.getTimestampFieldName(), bb); } if (!requiredKeysToFind.isEmpty()) { final Joiner joiner = Joiner.on(","); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_FIELD)); + } throw new MissingPropertyException(joiner.join(requiredKeysToFind)); } @@ -192,6 +237,11 @@ private void validate(final Set allFieldsToConvert, final List r .filter(rf -> allFieldsToConvert.contains(this.requiredFields)) .collect(Collectors.toList()); final Joiner joiner = Joiner.on(","); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_FIELD)); + } final String errMsg = String.format("Listed required fields are missing from the list of fields to convert." + " Please check your job configuration. Missing fields are: %s", joiner.join(missingFields)); throw new JobRuntimeException(errMsg); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java index 5fe15f0..41ff0b0 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java @@ -18,11 +18,18 @@ import com.google.common.base.Joiner; import com.google.common.base.Optional; +import com.google.common.collect.Maps; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.schema.cassandra.CassandraDataField; import com.uber.marmaray.common.schema.cassandra.CassandraPayload; import com.uber.marmaray.common.schema.cassandra.CassandraSchema; @@ -30,9 +37,13 @@ import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.GenericRecordUtil; import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.TimeUnitUtil; import com.uber.marmaray.utilities.TimestampInfo; +import lombok.Getter; import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.cassandra.db.marshal.BooleanType; import org.apache.cassandra.db.marshal.DoubleType; import org.apache.cassandra.db.marshal.FloatType; @@ -40,9 +51,11 @@ import org.apache.cassandra.db.marshal.LongType; import java.nio.ByteBuffer; +import java.sql.Timestamp; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -53,27 +66,35 @@ * class will return a RDD of Cassandra payloads to the caller. All keys & values will be serialized as * ByteBuffers as required by Cassandra. All strings are encoded using UTF-8. */ +@Slf4j public class CassandraSinkDataConverter extends SinkDataConverter { private static final long serialVersionUID = 1L; - + private static final String orderTimestampFieldName = "orderTsField"; private final String inputSchemaJson; - /* * The fields to convert are defined in the job configuration from the user. This can be all or a subset of fields * from the schema. */ private final Optional> fieldsToConvert; - /* * The required fields that must be populated in the schema. These keys form the primary/partition/clustering * keys in the Cassandra schema and are defined in the job configuration. */ private final List requiredFields; - - private Optional inputSchema = Optional.absent(); - private final TimestampInfo timestampInfo; + private final CassandraSinkConfiguration cassandraConf; + private final Optional writtenTime; + @Getter + private Optional dataFeedMetrics = Optional.absent(); + + /** + * This map (and schema) are created once per executor in the {@link #convert(AvroPayload)} method. + * It cannot be initialized in constructor due to Schema.Field not being serializable. + * This map caches per field information to avoid recalculations on each row. + */ + private Optional> fieldInfoMap = Optional.absent(); + private Optional inputSchemaAvro = Optional.absent(); /** * This constructor gives the option to only convert certain fields from the schema @@ -85,6 +106,7 @@ public class CassandraSinkDataConverter extends SinkDataConverter> fieldsToConvert, + @NonNull final Optional writtenTime, @NonNull final List requiredFields, @NonNull final TimestampInfo timestampInfo, @NonNull final ErrorExtractor errorExtractor) { @@ -92,73 +114,69 @@ public CassandraSinkDataConverter(@NonNull final Schema inputSchema, if (fieldsToConvert.isPresent()) { validate(fieldsToConvert.get(), requiredFields); } - this.inputSchemaJson = inputSchema.toString(); this.requiredFields = Collections.unmodifiableList(requiredFields); this.fieldsToConvert = fieldsToConvert; + this.writtenTime = writtenTime; this.timestampInfo = timestampInfo; + this.cassandraConf = new CassandraSinkConfiguration(conf); + this.inputSchemaJson = inputSchema.toString(); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); } @Override - public List> convert(final AvroPayload avroPayload) - throws Exception { + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + @Override + public List> convert(@NonNull final AvroPayload avroPayload) { final CassandraPayload row = new CassandraPayload(); final Set requiredKeysToFind = new HashSet<>(this.requiredFields); - if (!this.inputSchema.isPresent()) { - this.inputSchema = Optional.of(new Schema.Parser().parse(inputSchemaJson)); + if (!fieldInfoMap.isPresent()) { + this.inputSchemaAvro = Optional.of(new Schema.Parser().parse(inputSchemaJson)); + this.fieldInfoMap = Optional.of(createFieldInfoMap(inputSchemaAvro.get().getFields())); } - for (int i = 0; i < this.inputSchema.get().getFields().size(); i++) { - final Schema.Field field = this.inputSchema.get().getFields().get(i); + // Record the order column value for USING TIMESTAMP field. + Optional lastbb = Optional.absent(); + Optional lastFieldFname = Optional.absent(); + + final GenericRecord genericRecord = avroPayload.getData(); + for (int i = 0; i < this.inputSchemaAvro.get().getFields().size(); i++) { + final Schema.Field field = this.inputSchemaAvro.get().getFields().get(i); if (!this.fieldsToConvert.isPresent() || this.fieldsToConvert.isPresent() && this.fieldsToConvert.get().contains(field.name().toLowerCase())) { - final Object rawData = avroPayload.getData().get(field.name()); + final Object rawData = genericRecord.get(field.name()); final ByteBuffer bb; if (rawData != null) { - - final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema()) - ? GenericRecordUtil.getNonNull(field.schema()) - : field.schema(); - final Schema.Type type = nonNullSchema.getType(); - - switch (type) { - case BOOLEAN: - final Boolean boolData = (Boolean) rawData; - bb = BooleanType.instance.decompose(boolData); - break; - case INT: - final Integer intData = (Integer) rawData; - bb = Int32Type.instance.decompose(intData); - break; - case LONG: - final Long longData = (Long) rawData; - bb = LongType.instance.decompose(longData); - break; - case DOUBLE: - final Double doubleData = (Double) rawData; - bb = DoubleType.instance.decompose(doubleData); - break; - case STRING: - final String strData = rawData.toString(); - bb = ByteBufferUtil.wrap(strData); - break; - case FLOAT: - final Float floatData = (Float) rawData; - bb = FloatType.instance.decompose(floatData); - break; - // todo(T936057) - add support for non-primitive types - default: - throw new JobRuntimeException("Type " + field.schema().getType() + " not supported"); + bb = getByteBuffer(field, rawData, this.dataFeedMetrics); + if (this.writtenTime.isPresent() && field.name().equals(this.writtenTime.get())) { + lastFieldFname = Optional.fromNullable(field.name()); + lastbb = recordWrittenTime(getNonNullType(field), rawData); } } else { if (requiredKeysToFind.contains(field.name())) { - throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name()); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_DATA)); + } + continue; } bb = null; } - requiredKeysToFind.remove(field.name()); + if (requiredKeysToFind.contains(field.name()) + && bb.limit() != 0) { + requiredKeysToFind.remove(field.name()); + } row.addField(new CassandraDataField(ByteBufferUtil.wrap(field.name()), bb)); } } @@ -168,17 +186,114 @@ public List> convert(final AvroPa ? LongType.instance.decompose(Long.parseLong(this.timestampInfo.getTimestamp().get())) : ByteBufferUtil.wrap(this.timestampInfo.getTimestamp().get()); row.addField( - new CassandraDataField(ByteBufferUtil.wrap(SchemaUtil.DISPERSAL_TIMESTAMP), bb)); + new CassandraDataField(ByteBufferUtil.wrap(this.timestampInfo.getTimestampFieldName()), bb)); } if (!requiredKeysToFind.isEmpty()) { - final Joiner joiner = Joiner.on(","); - throw new MissingPropertyException(joiner.join(requiredKeysToFind)); + final String missingKeys = Joiner.on(",").join(requiredKeysToFind); + if (this.cassandraConf.getShouldSkipInvalidRows()) { + return Collections.singletonList( + new ConverterResult<>( + avroPayload, + String.format("Required keys are missing. Keys: %s", missingKeys))); + } + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_FIELD)); + } + throw new MissingPropertyException(missingKeys); + } + + // Concatenate orderTimestamp value with tailing zero to meet the length of microseconds. + if (lastFieldFname.isPresent() && lastbb.isPresent()) { + row.addField(new CassandraDataField(ByteBufferUtil.wrap(orderTimestampFieldName), + LongType.instance.decompose(Long.parseLong(TimeUnitUtil.convertToMicroSeconds(lastbb.get()))))); } return Collections.singletonList(new ConverterResult<>(row)); } + @NonNull + private Map createFieldInfoMap(@NonNull final List fields) { + + final Map fieldInfoMap = Maps.newHashMapWithExpectedSize(fields.size()); + for (final Schema.Field field : fields) { + final Schema.Type type = getNonNullType(field); + final boolean isTimestampField = SchemaUtil.isTimestampSchema(field.schema()); + final FieldInfo fieldInfo = new FieldInfo(type, isTimestampField); + fieldInfoMap.put(field, fieldInfo); + } + return fieldInfoMap; + } + + /** + * Utility method to load the byte buffer from the message + * @param field Schema.Field loaded from the object (this is how we know how to convert) + * @param rawData the item extracted from the object + * @return ByteBuffer of the bytes + */ + private ByteBuffer getByteBuffer(@NonNull final Schema.Field field, @NonNull final Object rawData, + @NonNull final Optional metrics) { + final ByteBuffer bb; + final FieldInfo fieldInfo = fieldInfoMap.get().get(field); + final Schema.Type type = fieldInfo.getNonNullType(); + if (fieldInfo.isTimestampField()) { + final Timestamp ts = SchemaUtil.decodeTimestamp(rawData); + final long longData = ts.getTime(); + bb = LongType.instance.decompose(longData); + } else { + switch (type) { + case BOOLEAN: + final Boolean boolData = (Boolean) rawData; + bb = BooleanType.instance.decompose(boolData); + break; + case INT: + final Integer intData = (Integer) rawData; + bb = Int32Type.instance.decompose(intData); + break; + case LONG: + final Long longData = (Long) rawData; + bb = LongType.instance.decompose(longData); + break; + case DOUBLE: + final Double doubleData = (Double) rawData; + bb = DoubleType.instance.decompose(doubleData); + break; + case STRING: + final String strData = rawData.toString(); + bb = ByteBufferUtil.wrap(strData); + break; + case FLOAT: + final Float floatData = (Float) rawData; + bb = FloatType.instance.decompose(floatData); + break; + case BYTES: + bb = (ByteBuffer) rawData; + break; + // todo(T936057) - add support for non-primitive types + default: + if (metrics.isPresent()) { + metrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, + ErrorCauseTagNames.NOT_SUPPORTED_FIELD_TYPE)); + } + throw new JobRuntimeException(String.format("Type %s not supported for field %s", + field.schema().getType(), field.name())); + } + } + return bb; + } + + private static Schema.Type getNonNullType(@NonNull final Schema.Field field) { + final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema()) + ? GenericRecordUtil.getNonNull(field.schema()) + : field.schema(); + return nonNullSchema.getType(); + } + /** * Not all the fields in the Avro Schema will be converted to Cassandra fields. * @@ -186,16 +301,59 @@ public List> convert(final AvroPa * @param allFieldsToConvert * @param requiredFields */ - private void validate(final Set allFieldsToConvert, final List requiredFields) { + private void validate(@NonNull final Set allFieldsToConvert, @NonNull final List requiredFields) { if (!allFieldsToConvert.containsAll(requiredFields)) { final List missingFields = requiredFields. stream() .filter(rf -> allFieldsToConvert.contains(this.requiredFields)) .collect(Collectors.toList()); final Joiner joiner = Joiner.on(","); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.MISSING_FIELD)); + } final String errMsg = String.format("Listed required fields are missing from the list of fields to convert." + " Please check your job configuration. Missing fields are: %s", joiner.join(missingFields)); throw new JobRuntimeException(errMsg); } } + + //TODO: Support Hive timestamp type. + private Optional recordWrittenTime(@NonNull final Schema.Type type, @NonNull final Object rawData) { + switch (type) { + case LONG: + return Optional.fromNullable((Long) rawData); + case STRING: + return Optional.fromNullable(Long.parseLong(rawData.toString())); + default: + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, + ErrorCauseTagNames.NOT_SUPPORTED_FIELD_TYPE)); + } + throw new JobRuntimeException("Order column type " + type + + " not supported. Only LONG and STRING type are supported."); + } + } + + private class FieldInfo { + final Schema.Type nonNullType; + final boolean isTimestampField; + + FieldInfo(@NonNull final Schema.Type nonNullType, final boolean isTimestampField) { + this.nonNullType = nonNullType; + this.isTimestampField = isTimestampField; + } + + Schema.Type getNonNullType() { + return nonNullType; + } + + boolean isTimestampField() { + return isTimestampField; + } + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataCSVConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataCSVConverter.java new file mode 100644 index 0000000..a3a4d2d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataCSVConverter.java @@ -0,0 +1,135 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.google.common.base.Preconditions; +import com.opencsv.CSVWriter; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.utilities.ErrorExtractor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaPairRDD; +import scala.Tuple2; + +import java.io.StringWriter; +import java.util.Collections; +import java.util.List; + +/** + * {@link FileSinkDataCSVConverter} extends {@link FileSinkDataConverter} + * This class converts data from intermediate Avro schema to string with csv format. + * This class is only to be used where the sink of the data migration is FileSink. + * The main convertAll method of this class will return a RDD of String with csv format to caller. + * The getHeader method will return a String of column header for the csv file. + */ + +@Slf4j +public class FileSinkDataCSVConverter extends FileSinkDataConverter { + public static final String CSV = "csv"; + public final char separator; + public final String fileType; + + public FileSinkDataCSVConverter(@NonNull final Configuration conf, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + final FileSinkConfiguration fsConf = new FileSinkConfiguration(conf); + this.fileType = fsConf.getFileType(); + this.separator = fsConf.getSeparator(); + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + + /** + * This method converts RDD of AvroPayload data to RDD of String with specified file type. + * Currently supports csv file only. + * @param data + * @return + * @throws UnsupportedOperationException + */ + @Override + public JavaPairRDD convertAll(@NonNull final JavaRDD data) { + try { + Preconditions.checkArgument(!data.isEmpty()); + } catch (Exception e) { + if (this.topicMetrics.isPresent()) { + this.topicMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.NO_DATA)); + } + log.error("No data to convert"); + throw e; + } + final String rowKey = data.first().getData().getSchema().getName(); + final JavaPairRDD lines = data.mapToPair(row -> { + final String line = this.convert(row).get(0).getSuccessData().get().getData(); + log.debug("Line: {}", line); + return new Tuple2<>(rowKey, line); + }); + return lines; + } + + @Override + public List> convert(@NonNull final AvroPayload data) + throws UnsupportedOperationException { + final String line; + if (this.fileType.equals(CSV)) { + final GenericRecord r = data.getData(); + final String[] tmp = r.getSchema().getFields().stream().map(f -> getValuesString(r, f)) + .toArray(String[]::new); + final StringWriter sw = new StringWriter(); + final CSVWriter writer = new CSVWriter(sw + , this.separator, '\"', '\\', ""); + writer.writeNext(tmp, false); + line = sw.toString(); + } else { + final String errorMessage = String.format("Format %s not supported yet.", this.fileType); + throw new UnsupportedOperationException(errorMessage); + } + return Collections.singletonList(new ConverterResult<>(line)); + } + + /** + * This methods get the column header of data. + * It specifically works for file type: csv. + * @param data + * @return String of column header separated by separator. + */ + @Override + public String getHeader(@NonNull final JavaRDD data) { + final AvroPayload line = data.first(); + final String[] headList + = line.getData().getSchema().getFields().stream().map(f -> f.name()).toArray(String[]::new); + final StringWriter sw = new StringWriter(); + final CSVWriter writer = new CSVWriter(sw + , this.separator, '\"', '\\', ""); + writer.writeNext(headList, false); + return sw.toString(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java index 7736235..386feac 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java @@ -17,21 +17,23 @@ package com.uber.marmaray.common.converters.data; -import com.opencsv.CSVWriter; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; -import com.uber.marmaray.common.configuration.FileSinkConfiguration; -import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.GenericRecordUtil; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.StringTypes; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.codec.binary.Hex; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import java.io.StringWriter; -import java.util.List; -import java.util.Collections; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; /** * {@link FileSinkDataConverter} extends {@link SinkDataConverter} @@ -41,69 +43,31 @@ * The getHeader method will return a String of column header for the csv file. */ @Slf4j -public class FileSinkDataConverter extends SinkDataConverter { - public static final String CSV = "csv"; - public final String fileType; - public final char separator; +public abstract class FileSinkDataConverter extends SinkDataConverter { + public static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS Z"; public FileSinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { super(conf, errorExtractor); - final FileSinkConfiguration fsConf = new FileSinkConfiguration(conf); - this.fileType = fsConf.getFileType(); - this.separator = fsConf.getSeparator(); } - /** - * This method converts RDD of AvroPayload data to RDD of String with specified file type. - * Currently supports csv file only. - * @param data - * @return - * @throws UnsupportedOperationException - */ - public JavaRDD convertAll(@NonNull final JavaRDD data) throws UnsupportedOperationException { - final JavaRDD lines = data.map(row -> { - final String line = this.convert(row).get(0).getSuccessData().get().getData(); - log.debug("Line: {}", line); - return line; - }); - return lines; - } + public abstract JavaPairRDD convertAll(@NonNull final JavaRDD data); + public abstract String getHeader(@NonNull final JavaRDD data); - @Override - public List> convert(@NonNull final AvroPayload data) - throws UnsupportedOperationException { - String line = ""; - if (this.fileType.equals(this.CSV)) { - final GenericRecord r = data.getData(); - final String [] tmp = r.getSchema().getFields().stream().map(f ->r.get(f.name()) - .toString()).toArray(String[]::new); - final StringWriter sw = new StringWriter(); - final CSVWriter writer = new CSVWriter(sw - , this.separator, '\"', '\\', ""); - writer.writeNext(tmp, false); - line = sw.toString(); + protected String getValuesString(@NonNull final GenericRecord record, @NonNull final Schema.Field field) { + final Object rawData = record.get(field.name()); + final Schema schema = field.schema(); + final Schema nonNullSchema = GenericRecordUtil.isOptional(schema) ? GenericRecordUtil.getNonNull(schema) + : schema; + if (rawData == null) { + return StringTypes.EMPTY; + } else if (SchemaUtil.isTimestampSchema(schema)) { + final Timestamp ts = SchemaUtil.decodeTimestamp(record.get(field.name())); + return new SimpleDateFormat(DATE_FORMAT).format(ts); + } else if (Schema.Type.BYTES.equals(nonNullSchema.getType())) { + final byte[] bytes = ((ByteBuffer) record.get(field.name())).array(); + return String.format("0x%s", Hex.encodeHexString(bytes)); } else { - //Todo:Add more file type options. - final String errorMessage = "Format " + this.fileType + " not supported yet."; - throw new UnsupportedOperationException(errorMessage); + return record.get(field.name()).toString(); } - return Collections.singletonList(new ConverterResult<>(line)); - } - - /** - * This methods get the column header of data. - * It specifically works for file type: csv. - * @param data - * @return String of column header separated by separator. - */ - public String getHeader(@NonNull final JavaRDD data) { - final AvroPayload line = data.first(); - final String[] headList - = line.getData().getSchema().getFields().stream().map(f -> f.name()).toArray(String[]::new); - final StringWriter sw = new StringWriter(); - final CSVWriter writer = new CSVWriter(sw - , this.separator, '\"', '\\', ""); - writer.writeNext(headList, false); - return sw.toString(); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverterFactory.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverterFactory.java new file mode 100644 index 0000000..eb82320 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverterFactory.java @@ -0,0 +1,42 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.utilities.ErrorExtractor; +import lombok.NonNull; + +/** + * {@link FileSinkDataConverterFactory + * + */ +public class FileSinkDataConverterFactory { + protected FileSinkDataConverterFactory() { + } + + public static FileSinkDataConverter createFileSinkDataConverter(@NonNull final Configuration conf) { + final FileSinkConfiguration fileConfig = new FileSinkConfiguration(conf); + if (fileConfig.getFileType().equals("sequence")) { + return new FileSinkDataJSONConverter(conf, new ErrorExtractor()); + } else { + return new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + } + } +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataJSONConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataJSONConverter.java new file mode 100644 index 0000000..3ab30e8 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataJSONConverter.java @@ -0,0 +1,135 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.utilities.ErrorExtractor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.json.JSONObject; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import scala.Tuple2; + +/** + * {@link FileSinkDataJSONConverter} extends {@link SinkDataConverter} + */ +@Slf4j +public class FileSinkDataJSONConverter extends FileSinkDataConverter { + public final String fileType; + public final String row_identifier; + + public FileSinkDataJSONConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + final FileSinkConfiguration fsConf = new FileSinkConfiguration(conf); + this.fileType = fsConf.getFileType(); + this.row_identifier = fsConf.getRowIdentifier(); + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + + /** + * This API generates RDDPair for saving different file format(sequence, json,...) + * @param data + * @return + */ + @Override + public JavaPairRDD convertAll(@NonNull final JavaRDD data) { + try { + Preconditions.checkNotNull(data.first()); + } catch (Exception e) { + if (this.topicMetrics.isPresent()) { + this.topicMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.NO_DATA)); + } + log.error("No data to convert"); + throw e; + } + final String key = data.first().getData().getSchema().getName(); + if (this.row_identifier.isEmpty()) { + log.warn("row identifier is missing, schema name {} will be is used", key); + } else { + final Object field = data.first().getField(this.row_identifier); + if (field == null) { + final String errorMessage = String.format( + "specified row identifier field : {} does not exist", row_identifier); + if (this.topicMetrics.isPresent()) { + this.topicMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.EMPTY_FIELD)); + + } + throw new UnsupportedOperationException(errorMessage); + } else { + log.info("schema field {} is used for row identifier.", this.row_identifier); + } + } + final JavaPairRDD lines = data.mapToPair(row -> { + final String line = convert(row).get(0).getSuccessData().get().getData(); + final String rowKey = this.row_identifier.isEmpty() ? key + : row.getData().get(this.row_identifier).toString(); + return new Tuple2<>(rowKey, line); + }); + return lines; + } + + @Override + public List> convert(@NonNull final AvroPayload data) + throws UnsupportedOperationException { + final Map jsonMap = getJSONMap(data); + final JSONObject jsonObject = new JSONObject(jsonMap); + return Collections.singletonList(new ConverterResult<>(jsonObject.toString())); + } + + private Map getJSONMap(@NonNull final AvroPayload data) { + Map jsonMap = new HashMap<>(); + final GenericRecord record = data.getData(); + final List fields = record.getSchema().getFields(); + fields.forEach(field -> jsonMap.put(field.name(), getValuesString(record, field))); + return jsonMap; + } + + /** + * No Column Header for json format + * @param data + * @return + */ + @Override + public String getHeader(@NonNull final JavaRDD data) { + return null; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java index 8487b6d..265cedd 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java @@ -23,6 +23,8 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.sinks.hoodie.HoodieSink; import com.uber.marmaray.utilities.ErrorExtractor; @@ -56,6 +58,16 @@ public HoodieSinkDataConverter(@NonNull final Configuration conf, final String s this.errorExtractor = errorExtractor; } + @Override + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + //ignored + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + @Override protected final List>> convert( @NonNull final AvroPayload payload) throws Exception { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java index 11ef4fd..ce838e6 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java @@ -19,17 +19,21 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.schema.ISchemaService.ISchemaServiceReader; import com.uber.marmaray.utilities.ErrorExtractor; import lombok.Getter; import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.function.Function; - import java.util.Collections; import java.util.List; +@Slf4j /** * {@link KafkaSourceDataConverter} extends {@link SourceDataConverter} * @@ -43,7 +47,6 @@ public class KafkaSourceDataConverter extends SourceDataConverter fieldsToCache; - /** * List of {@Link Function} to apply to the record between reading from kafka and * transferring to the ISource @@ -68,9 +71,30 @@ public KafkaSourceDataConverter(@NonNull final ISchemaServiceReader schemaServic this.fieldsToCache = fieldsToCache; } + @Override + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + //ignored + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + @Override public List> convert(@NonNull final byte[] data) throws Exception { - GenericRecord genericRecord = this.schemaServiceReader.read(data); + final GenericRecord genericRecord = this.schemaServiceReader.read(data); + return applyUpdateFunctions(genericRecord); + } + + public List> convert(@NonNull final GenericRecord genericRecord) + throws Exception { + return applyUpdateFunctions(genericRecord); + } + + private List> applyUpdateFunctions(@NonNull final GenericRecord record) + throws Exception { + GenericRecord genericRecord = record; for (Function func : this.updateFunctions) { genericRecord = func.call(genericRecord); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java index 657f402..c62ddc8 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java @@ -16,16 +16,23 @@ */ package com.uber.marmaray.common.converters.data; -import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.SparkUtil; import com.uber.marmaray.utilities.StringTypes; +import lombok.Getter; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; @@ -33,9 +40,12 @@ import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import java.nio.ByteBuffer; +import java.sql.Timestamp; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -55,6 +65,8 @@ public class SparkSourceDataConverter extends SourceDataConverter requiredKeys; private Optional outputSchema = Optional.absent(); + @Getter + private Optional dataFeedMetrics = Optional.absent(); public SparkSourceDataConverter(@NonNull final StructType inputSchema, @NonNull final Schema outputSchema, @@ -68,6 +80,16 @@ public SparkSourceDataConverter(@NonNull final StructType inputSchema, this.requiredKeys = requiredKeys; } + @Override + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + @Override public List> convert(@NonNull final Row row) throws Exception { if (!this.outputSchema.isPresent()) { @@ -82,14 +104,29 @@ public List> convert(@NonNull final Row row) t // todo: think about generalizing this, the pattern is the same for (int i = 0; i < this.fields.length; i++) { - required.remove(this.fields[i].name()); final DataType dt = this.fields[i].dataType(); try { final Object data = row.getAs(this.fields[i].name()); + if (data == null) { + continue; + } + required.remove(this.fields[i].name()); if (supportedDataTypes.contains(dt)) { - gr.put(this.fields[i].name(), data); + // need to handle non-avro datatypes + if (DataTypes.TimestampType.equals(dt)) { + gr.put(this.fields[i].name(), SchemaUtil.encodeTimestamp((Timestamp) data)); + } else if (DataTypes.BinaryType.equals(dt)) { + gr.put(this.fields[i].name(), ByteBuffer.wrap((byte[]) data)); + } else { + gr.put(this.fields[i].name(), data); + } } else { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE_CONVERTER, ErrorCauseTagNames.NOT_SUPPORTED_FIELD_TYPE)); + } throw new JobRuntimeException(dt.toString() + " field type is not supported at this time"); } } catch (final IllegalArgumentException e) { @@ -99,9 +136,22 @@ public List> convert(@NonNull final Row row) t } if (!required.isEmpty()) { - final Joiner joiner = Joiner.on(StringTypes.COMMA); - final String errMsg = String.format("Required fields were missing. Fields: {}", joiner.join(required)); - throw new JobRuntimeException(errMsg); + // TODO: T2701851 Move should skip invalid rows from Cassandra setting + if (this.conf.getBooleanProperty(CassandraSinkConfiguration.SHOULD_SKIP_INVALID_ROWS, false)) { + return Collections.singletonList( + new ConverterResult<>( + row, + String.format("Required keys are missing. Keys: %s", required))); + } else { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE_CONVERTER, ErrorCauseTagNames.MISSING_FIELD)); + } + final String errMsg = String.format("Required fields were missing. Fields: %s", + String.join(StringTypes.COMMA, required)); + throw new JobRuntimeException(errMsg); + } } return Collections.singletonList(new ConverterResult<>(new AvroPayload(gr))); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java index 2b8ee14..1f445c3 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java @@ -19,7 +19,6 @@ import com.google.common.base.Optional; import com.uber.marmaray.common.schema.cassandra.CassandraSchema; import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField; -import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.StringTypes; import com.uber.marmaray.utilities.TimestampInfo; import lombok.Getter; @@ -85,9 +84,9 @@ public CassandraSchema convertToExternalSchema(final Schema commonSchema) { if (this.tsInfo.hasTimestamp()) { if (this.tsInfo.isSaveAsLongType()) { - cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.LONG_TYPE)); + cs.addField(new CassandraSchemaField(tsInfo.getTimestampFieldName(), CassandraSchemaField.LONG_TYPE)); } else { - cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.STRING_TYPE)); + cs.addField(new CassandraSchemaField(tsInfo.getTimestampFieldName(), CassandraSchemaField.STRING_TYPE)); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java index a918cd0..95d555d 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java @@ -17,6 +17,7 @@ package com.uber.marmaray.common.converters.schema; import com.google.common.base.Preconditions; +import com.uber.marmaray.utilities.SchemaUtil; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.spark.sql.types.DataType; @@ -55,44 +56,45 @@ private void addField(final SchemaBuilder.FieldAssembler fieldAssembler, final DataType dt = structField.dataType().defaultConcreteType(); - /** + /* * Can't use switch statement here because this is not a compile time constant expression * and I'd rather compare it to a concrete type rather than use strings * * For now, we just handle atomic, fractional, or integral types. * Todo: Handle Map/Array/Struct types etc */ - final SchemaBuilder.BaseFieldTypeBuilder fieldType = - structField.nullable() ? fieldBuilder.type().nullable() : fieldBuilder.type(); if (dt.equals(DataTypes.TimestampType)) { - fieldType.stringType().noDefault(); - } else if (dt.equals(DataTypes.StringType)) { - fieldType.stringType().noDefault(); - } else if (dt.equals(DataTypes.BooleanType)) { - fieldType.booleanType().noDefault(); - } else if (dt.equals(DataTypes.DateType)) { - fieldType.stringType().noDefault(); - } else if (dt.equals(DataTypes.BinaryType)) { - // validate this is correct - fieldType.bytesType().noDefault(); - } else if (dt.equals(DataTypes.DoubleType)) { - fieldType.doubleType().noDefault(); - } else if (dt.equals(DataTypes.FloatType)) { - fieldType.floatType().noDefault(); - } else if (dt.equals(DataTypes.ByteType)) { - fieldType.bytesType().noDefault(); - } else if (dt.equals(DataTypes.IntegerType)) { - fieldType.intType().noDefault(); - } else if (dt.equals(DataTypes.LongType)) { - fieldType.longType().noDefault(); - } else if (dt.equals(DataTypes.ShortType)) { - // no corresponding short type in DataTypes - // we can make this int and lose no precision - fieldType.intType().noDefault(); + fieldBuilder.type(SchemaUtil.getTimestampSchema(structField.nullable())).noDefault(); } else { - throw new RuntimeException("The field type " + dt + " is not supported"); + final SchemaBuilder.BaseFieldTypeBuilder fieldType = + structField.nullable() ? fieldBuilder.type().nullable() : fieldBuilder.type(); + if (dt.equals(DataTypes.StringType)) { + fieldType.stringType().noDefault(); + } else if (dt.equals(DataTypes.BooleanType)) { + fieldType.booleanType().noDefault(); + } else if (dt.equals(DataTypes.DateType)) { + fieldType.stringType().noDefault(); + } else if (dt.equals(DataTypes.BinaryType)) { + fieldType.bytesType().noDefault(); + } else if (dt.equals(DataTypes.DoubleType)) { + fieldType.doubleType().noDefault(); + } else if (dt.equals(DataTypes.FloatType)) { + fieldType.floatType().noDefault(); + } else if (dt.equals(DataTypes.ByteType)) { + fieldType.bytesType().noDefault(); + } else if (dt.equals(DataTypes.IntegerType)) { + fieldType.intType().noDefault(); + } else if (dt.equals(DataTypes.LongType)) { + fieldType.longType().noDefault(); + } else if (dt.equals(DataTypes.ShortType)) { + // no corresponding short type in DataTypes + // we can make this int and lose no precision + fieldType.intType().noDefault(); + } else { + throw new RuntimeException("The field type " + dt + " is not supported"); + } } - /** + /* * Todo: Handle following types * CalendarIntervalType * StructType diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java index b5f4689..a2581f2 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java @@ -55,6 +55,10 @@ public class ForkOperator implements Serializable { private Optional>> groupRDD = Optional.absent(); @Getter private final StorageLevel persistLevel; + @Getter + private long rddSize; + @Getter + private int numRddPartitions; public ForkOperator(@NonNull final JavaRDD inputRDD, @NonNull final ForkFunction forkFunction, @NonNull final Configuration conf) { @@ -77,12 +81,19 @@ public final void execute() { log.info("#processed records :{} name:{}", processedRecords, forkedData.name()); if (rddInfo.isPresent()) { final long size = rddInfo.get().diskSize() + rddInfo.get().memSize(); + setRddPartitionSize(size, rddInfo.get().numPartitions()); log.info("rddInfo -> name:{} partitions:{} size:{}", forkedData.name(), rddInfo.get().numPartitions(), size); } this.groupRDD = Optional.of(forkedData); } + // set metrics here + private void setRddPartitionSize(final long rddSize, final int numPartitions) { + this.rddSize = rddSize; + this.numRddPartitions = numPartitions; + } + public long getCount(final int filterKey) { return this.forkFunction.getRecordCount(filterKey); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/Dag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/Dag.java new file mode 100644 index 0000000..48d3d05 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/Dag.java @@ -0,0 +1,29 @@ +package com.uber.marmaray.common.job; + +import com.uber.marmaray.common.status.IStatus; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Map; + +public abstract class Dag { + + @NotEmpty @Getter @Setter + private String jobName; + + @NotEmpty @Getter @Setter + private String dataFeedName; + + @Getter @Setter + private Map jobManagerMetadata; + + public Dag(@NonNull final String jobName, @NonNull final String dataFeedName) { + this.dataFeedName = dataFeedName; + this.jobName = jobName; + } + + public abstract IStatus execute(); + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java index db0f105..ae51ca7 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java @@ -26,10 +26,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; /** * {@link ExecutionTimeJobExecutionStrategy} is a {@link IJobExecutionStrategy} that sorts the {@link JobDag}s by @@ -60,9 +62,15 @@ public ExecutionTimeJobExecutionStrategy(@NonNull final JobManagerMetadataTracke } @Override - public List sort(@NonNull final Queue inputJobDags) { - final List result = new ArrayList<>(inputJobDags.size()); + public List sort(@NonNull final Queue inputJobDags) { + final Map initialTopicOrdering = new HashMap<>(); + final AtomicInteger preTopicOrderingCounter = new AtomicInteger(0); + final AtomicInteger postTopicOrderingCounter = new AtomicInteger(0); + inputJobDags.stream().forEach(jobDag -> initialTopicOrdering.put(jobDag.getDataFeedName(), + preTopicOrderingCounter.incrementAndGet())); + final List result = new ArrayList<>(inputJobDags.size()); final long lastExecutionTimeThresholdMillis = TimeUnit.HOURS.toMillis(this.lastExecutionTimeThresholdHours); + log.info("shuffled topic ordering"); inputJobDags.stream().map(dag -> { try { final Optional> contents = this.tracker.get(dag.getDataFeedName()); @@ -82,7 +90,16 @@ public List sort(@NonNull final Queue inputJobDags) { "Unable to get metadata for dag %s : ", dag.getDataFeedName()), e); } }).sorted((o1, o2) -> o2._2().compareTo(o1._2())) - .forEach(tuple -> result.add(tuple._1())); + .forEach( + tuple -> { + log.info("topic ordering for {} changed from {} to {} - with weight {}", + tuple._1().getDataFeedName(), + initialTopicOrdering.get(tuple._1.getDataFeedName()), + postTopicOrderingCounter.incrementAndGet(), + tuple._2); + result.add(tuple._1()); + }); + return result; } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java index 6f82fd5..99a62ba 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java @@ -28,6 +28,6 @@ */ public interface IJobExecutionStrategy { - List sort(@NonNull final Queue inputJobDags); + List sort(@NonNull final Queue inputJobDags); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java index 7f5f1ce..930dcf6 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java @@ -26,8 +26,10 @@ import com.uber.marmaray.common.metadata.IMetadataManager; import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.metrics.LongMetric; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.metrics.TimerMetric; import com.uber.marmaray.common.reporters.Reporters; import com.uber.marmaray.common.sinks.ISink; @@ -35,9 +37,10 @@ import com.uber.marmaray.common.sources.ISource; import com.uber.marmaray.common.sources.IWorkUnitCalculator; import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; import lombok.Getter; import lombok.NonNull; -import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.spark.api.java.JavaRDD; import org.hibernate.validator.constraints.NotEmpty; @@ -46,13 +49,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @Slf4j public class JobDag, C extends IWorkUnitCalculator, - K extends IWorkUnitCalculatorResult> { + K extends IWorkUnitCalculatorResult> extends Dag { public static final String LAST_RUNTIME_METADATA_KEY = "runtime"; public static final String LAST_EXECUTION_METADATA_KEY = "last_execution"; @@ -64,13 +66,7 @@ public class JobDag, C extend private final IMetadataManager metadataManager; @NonNull private final IWorkUnitCalculator workUnitCalculator; - @NonNull @Getter - private final String jobName; - @NonNull @Getter - private final String dataFeedName; - @Getter @Setter - private Map jobManagerMetadata; private final Reporters reporters; private final JobDagActions postJobDagActions; @@ -99,17 +95,37 @@ public JobDag(@NonNull final ISource source, @NotEmpty final String dataFeedName, @NonNull final JobMetrics jobMetrics, @NonNull final Reporters reporters) { + super(jobName, dataFeedName); this.source = source; this.sinkDag = sinkDag; this.metadataManager = metadataManager; this.workUnitCalculator = workUnitCalculator; - this.jobName = jobName; - this.dataFeedName = dataFeedName; this.reporters = reporters; this.postJobDagActions = new JobDagActions(this.reporters, dataFeedName); this.jobMetrics = jobMetrics; - this.dataFeedMetrics = new DataFeedMetrics(this.jobName, - Collections.singletonMap(DataFeedMetrics.DATA_FEED_NAME, this.dataFeedName)); + this.dataFeedMetrics = new DataFeedMetrics(this.getJobName(), + Collections.singletonMap(DataFeedMetrics.DATA_FEED_NAME, this.getDataFeedName())); + } + + // passing datafeed metric from high level job + public JobDag(@NonNull final ISource source, + @NonNull final ISink sink, + @NonNull final IMetadataManager metadataManager, + @NonNull final IWorkUnitCalculator workUnitCalculator, + @NotEmpty final String jobName, + @NotEmpty final String dataFeedName, + @NonNull final JobMetrics jobMetrics, + @NonNull final DataFeedMetrics dataFeedMetrics, + @NonNull final Reporters reporters) { + super(jobName, dataFeedName); + this.source = source; + this.sinkDag = new SingleSinkSubDag(sink); + this.metadataManager = metadataManager; + this.workUnitCalculator = workUnitCalculator; + this.reporters = reporters; + this.postJobDagActions = new JobDagActions(this.reporters, dataFeedName); + this.jobMetrics = jobMetrics; + this.dataFeedMetrics = dataFeedMetrics; } /** @@ -120,23 +136,30 @@ public void addAction(final IJobDagAction action) { this.postJobDagActions.addAction(action); } - public void execute() { - log.info("Starting job dag for {}", this.jobName); + @Override + public IStatus execute() { + + log.info("Starting job dag for {}", this.getJobName()); final AtomicBoolean successful = new AtomicBoolean(true); final ReporterAction reporterAction = new ReporterAction(this.reporters, this.jobMetrics, this.dataFeedMetrics); this.addAction(reporterAction); final TimerMetric timerMetric = this.dataFeedMetrics.createTimerMetric( DataFeedMetricNames.TOTAL_LATENCY_MS, new HashMap<>(), Optional.absent()); + final BaseStatus status = new BaseStatus(); + try { - // set up metrics for downstreams - Arrays.asList(this.workUnitCalculator, this.sinkDag, this.source).forEach(metricable -> { - metricable.setDataFeedMetrics(this.dataFeedMetrics); - metricable.setJobMetrics(this.jobMetrics); - }); + // set up metrics for down streams + Arrays.asList(this.workUnitCalculator, this.sinkDag, this.source, this.metadataManager) + .forEach(metricable -> { + metricable.setDataFeedMetrics(this.dataFeedMetrics); + metricable.setJobMetrics(this.jobMetrics); + }); // initialize previous run state. this.workUnitCalculator.initPreviousRunState(this.metadataManager); // compute work units. final K workUnitCalculatorResult = this.workUnitCalculator.computeWorkUnits(); + final IStatus workUnitCalculatorResultStatus = workUnitCalculatorResult.getStatus(); + status.mergeStatus(workUnitCalculatorResultStatus); log.info("Work unit calculator result :{}", workUnitCalculatorResult); // save run state for next processing this.workUnitCalculator.saveNextRunState(this.metadataManager, workUnitCalculatorResult.getNextRunState()); @@ -154,14 +177,22 @@ public void execute() { try { this.metadataManager.saveChanges(); } catch (IOException e) { - final String msg = "Failed to save metadata changes" + e.getMessage(); + this.dataFeedMetrics.createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.JOB_DAG, ErrorCauseTagNames.SAVE_METADATA)); + final String msg = "Failed to save metadata changes " + e.getMessage(); log.error(msg, e); throw new JobRuntimeException(msg, e); } } catch (Exception e) { + log.error("Failed in JobDag", e); + this.dataFeedMetrics.createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.JOB_DAG, ErrorCauseTagNames.ERROR)); // set failure status if there was an error successful.set(false); - throw e; + status.setStatus(IStatus.Status.FAILURE); + status.addException(e); } finally { // execute all actions at the last minute timerMetric.stop(); @@ -176,6 +207,7 @@ public void execute() { Collections.emptyMap()); this.postJobDagActions.execute(successful.get()); } + return status; } private void reportStatus(final boolean successful) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java index dc46233..f4b3ce5 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java @@ -16,13 +16,13 @@ */ package com.uber.marmaray.common.job; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.uber.marmaray.common.actions.IJobDagAction; import com.uber.marmaray.common.actions.JobDagActions; import com.uber.marmaray.common.configuration.Configuration; -import com.uber.marmaray.common.configuration.HadoopConfiguration; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.exceptions.MetadataException; import com.uber.marmaray.common.metadata.JobManagerMetadataTracker; @@ -32,25 +32,22 @@ import com.uber.marmaray.common.reporters.ConsoleReporter; import com.uber.marmaray.common.reporters.IReporter; import com.uber.marmaray.common.reporters.Reporters; +import com.uber.marmaray.common.spark.SparkFactory; +import com.uber.marmaray.common.status.IStatus; +import com.uber.marmaray.common.status.JobManagerStatus; import com.uber.marmaray.utilities.LockManager; -import com.uber.marmaray.utilities.SparkUtil; -import com.uber.marmaray.utilities.listener.SparkEventListener; import com.uber.marmaray.utilities.listener.SparkJobTracker; import com.uber.marmaray.utilities.listener.TimeoutManager; - import lombok.Getter; import lombok.NonNull; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.avro.mapred.Pair; import org.hibernate.validator.constraints.NotEmpty; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Future; @@ -74,29 +71,25 @@ public final class JobManager { private static final Object lock = new Object(); @NonNull - private final Queue jobDags = new ConcurrentLinkedDeque<>(); + private final Queue jobDags = new ConcurrentLinkedDeque<>(); private final JobDagActions postJobManagerActions; @Getter private final Configuration conf; @NotEmpty private final String appName; - private String appId; - private Optional sparkContext = Optional.absent(); + private final String appId; private final JobLockManager jobLockManager; - @Getter - private final List schemas = new ArrayList<>(); - - @Getter - private final List serializationClasses = new LinkedList<>(); - @Getter private final JobMetrics jobMetrics; @Getter private final Reporters reporters; + @Getter + private final SparkFactory sparkFactory; + @Getter @Setter private JobManagerMetadataTracker tracker; @@ -106,15 +99,26 @@ public final class JobManager { @Setter private Optional jobExecutionStrategy = Optional.absent(); + @Getter + private final JobManagerStatus jobManagerStatus; + private JobManager(@NonNull final Configuration conf, @NotEmpty final String appName, - @NotEmpty final String frequency, final boolean shouldLockFrequency) { + @NotEmpty final String frequency, final boolean shouldLockFrequency, + @NonNull final SparkFactory sparkFactory, @NonNull final Reporters reporters, + @NonNull final JobManagerStatus jobManagerStatus) { + if (sparkFactory.getSparkArgs().getAvroSchemas().isEmpty()) { + log.warn("Schemas must be added to SparkArgs before creating sparkContext. Found no schemas"); + } this.conf = conf; this.appName = appName; this.jobMetrics = new JobMetrics(appName); - this.reporters = new Reporters(); + this.reporters = reporters; + this.sparkFactory = sparkFactory; this.reporters.addReporter(new ConsoleReporter()); this.jobLockManager = new JobLockManager(conf, frequency, shouldLockFrequency); this.postJobManagerActions = new JobDagActions(this.reporters, "jobManager"); + this.appId = sparkFactory.getSparkContext().sc().applicationId(); + this.jobManagerStatus = jobManagerStatus; } /** @@ -123,13 +127,20 @@ private JobManager(@NonNull final Configuration conf, @NotEmpty final String app * @param appName Name of the application, used in the SparkContext * @param frequency name of the frequency, used to lock entire frequencies * @param lockFrequency whether the frequency should be locked + * @param sparkFactory to provide {@link JavaSparkContext} + * @param reporters to report metrics + * @param jobManagerStatus to report status */ - public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName, - @NotEmpty final String frequency, final boolean lockFrequency) { + public static JobManager createJobManager(@NonNull final Configuration conf, + @NotEmpty final String appName, + @NotEmpty final String frequency, final boolean lockFrequency, + @NonNull final SparkFactory sparkFactory, @NonNull final Reporters reporters, + @NonNull final JobManagerStatus jobManagerStatus) { synchronized (lock) { Preconditions.checkState(instance == null, - "JobManager was already created"); - instance = new JobManager(conf, appName, frequency, lockFrequency); + "JobManager was already created"); + instance = new JobManager(conf, appName, frequency, lockFrequency, sparkFactory, + reporters, jobManagerStatus); } return instance; } @@ -139,43 +150,46 @@ public static JobManager createJobManager(@NonNull final Configuration conf, @No * @param conf Configuration for the job manager, used to determine parallelism of execution * @param appName Name of the application, used in the SparkContext * @param frequency name of the frequency, used to lock entire frequencies + * @param lockFrequency whether the frequency should be locked + * @param sparkFactory to provide {@link JavaSparkContext} + * @param reporters to report metrics */ - public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName, - @NotEmpty final String frequency) { - return createJobManager(conf, appName, frequency, DEFAULT_LOCK_FREQUENCY); + public static JobManager createJobManager(@NonNull final Configuration conf, + @NotEmpty final String appName, + @NotEmpty final String frequency, final boolean lockFrequency, + @NonNull final SparkFactory sparkFactory, @NonNull final Reporters reporters) { + return createJobManager(conf, appName, frequency, lockFrequency, sparkFactory, + reporters, new JobManagerStatus()); } /** - * Creates JavaSparkContext if its hasn't been created yet, or returns the instance. {@link #addSchema(Schema)} and - * {@link #addSchemas(Collection)} must not be called once the JavaSparkContext has been created - * @return the JavaSparkContext that will be used to execute the JobDags + * Create the JobManager. Will fail if the job manager has already been created. + * @param conf Configuration for the job manager, used to determine parallelism of execution + * @param appName Name of the application, used in the SparkContext + * @param frequency name of the frequency, used to lock entire frequencies + * @param sparkFactory to provide {@link JavaSparkContext} + * @param reporters to report metrics */ - public JavaSparkContext getOrCreateSparkContext() { - if (!this.sparkContext.isPresent()) { - this.sparkContext = Optional.of(new JavaSparkContext( - SparkUtil.getSparkConf( - this.appName, Optional.of(this.schemas), this.serializationClasses, this.conf))); - this.sparkContext.get().sc().addSparkListener(new SparkEventListener()); - // Adding hadoop configuration to default - this.sparkContext.get().sc().hadoopConfiguration().addResource( - new HadoopConfiguration(conf).getHadoopConf()); - this.appId = this.sparkContext.get().sc().applicationId(); - } - return this.sparkContext.get(); + public static JobManager createJobManager(@NonNull final Configuration conf, + @NotEmpty final String appName, + @NotEmpty final String frequency, @NonNull final SparkFactory sparkFactory, + final Reporters reporters) { + return createJobManager(conf, appName, frequency, DEFAULT_LOCK_FREQUENCY, sparkFactory, + reporters, new JobManagerStatus()); } /** * Execute all registered {@link JobDag}, then perform all registered {@link IJobDagAction} */ public void run() { - final Queue futures = new ConcurrentLinkedDeque<>(); + final Queue>> futures = new ConcurrentLinkedDeque<>(); final AtomicBoolean isSuccess = new AtomicBoolean(true); // ensure the SparkContext has been created - final JavaSparkContext sc = getOrCreateSparkContext(); Preconditions.checkState(!this.jobDags.isEmpty(), "No job dags to execute"); - TimeoutManager.init(this.conf, sc.sc()); + final JavaSparkContext javaSparkContext = sparkFactory.getSparkContext(); + TimeoutManager.init(this.conf, javaSparkContext.sc()); final boolean hasMultipleDags = this.jobDags.size() > 1; - final Queue runtimeJobDagOrder; + final Queue runtimeJobDagOrder; if (hasMultipleDags && this.jobExecutionStrategy.isPresent()) { runtimeJobDagOrder = new ConcurrentLinkedDeque<>(this.jobExecutionStrategy.get().sort(this.jobDags)); } else { @@ -186,44 +200,56 @@ public void run() { runtimeJobDagOrder.forEach(jobDag -> futures.add(ThreadPoolService.submit( () -> { - SparkJobTracker.setJobName(sc.sc(), jobDag.getDataFeedName()); + SparkJobTracker.setJobName(javaSparkContext.sc(), jobDag.getDataFeedName()); if (hasMultipleDags) { - setSparkStageName(sc, jobDag.getDataFeedName()); + setSparkStageName(javaSparkContext, jobDag.getDataFeedName()); } - jobDag.execute(); - return null; + final IStatus status = jobDag.execute(); + this.jobManagerStatus.addJobStatus(jobDag.getJobName(), status); + return new Pair<>(jobDag.getJobName(), status); }, ThreadPoolServiceTier.JOB_DAG_TIER))); + TimeoutManager.getInstance().startMonitorThread(); futures.forEach(future -> { try { - future.get(); + final Optional> result = Optional.fromNullable(future.get()); + IStatus.Status status = result.get().value().getStatus(); + log.info("job dag, name: {}, status: {}", + result.get().key(), status.name()); + if (IStatus.Status.FAILURE.equals(status)) { + log.error("Unsuccessful run, jobdag: {}", result.get().key()); + isSuccess.set(false); + } } catch (Exception e) { log.error("Error running job", e); isSuccess.set(false); + this.jobManagerStatus.setStatus(IStatus.Status.FAILURE); + this.jobManagerStatus.addException(e); } } ); - + if (TimeoutManager.getInstance().getTimedOut()) { + log.error("Time out error while running job."); + isSuccess.set(false); + } + // if we're not reporting success/failure through status, we need to throw an exception on failure if (!isSuccess.get()) { throw new JobRuntimeException("Error while running job. Look at previous log entries for detail"); } } catch (final Throwable t) { + log.error("Failed in JobManager", t); isSuccess.set(false); - throw t; + this.jobManagerStatus.setStatus(IStatus.Status.FAILURE); + if (t instanceof Exception) { + // trap exceptions and add them to the status + this.jobManagerStatus.addException((Exception) t); + } else { + // let errors be thrown + throw t; + } } finally { this.postJobManagerActions.execute(isSuccess.get()); - ThreadPoolService.shutdown(!isSuccess.get()); - if (this.isJobManagerMetadataEnabled()) { - jobDags.forEach(jobDag -> this.getTracker().set(jobDag.getDataFeedName(), - jobDag.getJobManagerMetadata())); - try { - this.getTracker().writeJobManagerMetadata(); - } catch (MetadataException e) { - log.error("Unable to save metadata: {}", e.getMessage()); - } - } - sc.stop(); - this.jobLockManager.stop(); + shutdown(!isSuccess.get()); this.reporters.getReporters().forEach(IReporter::finish); } } @@ -232,7 +258,7 @@ public void run() { * Add {@link JobDag} to be executed on {@link #run()} * @param jobDag JobDag to be added */ - public void addJobDag(@NonNull final JobDag jobDag) { + public void addJobDag(@NonNull final Dag jobDag) { if (jobLockManager.lockDag(jobDag.getJobName(), jobDag.getDataFeedName())) { this.jobDags.add(jobDag); } else { @@ -264,35 +290,29 @@ public void addPostJobManagerActions(@NonNull final Collection serializationClasses) { - Preconditions.checkState(!this.sparkContext.isPresent(), - "Serialization classes must be added before sparkContext is instantiated"); - this.serializationClasses.addAll(serializationClasses); + @VisibleForTesting + public static void reset() { + if (instance != null) { + instance.shutdown(true); + instance = null; + } } - /** - * Add Collection of schemas for registration into {@link JavaSparkContext}. Must not be called after the - * JavaSparkContext has been created. - * @param schemas collection of schemas to register - */ - public void addSchemas(@NonNull final Collection schemas) { - schemas.forEach(this::addSchema); + private void shutdown(final boolean forceShutdown) { + ThreadPoolService.shutdown(forceShutdown); + if (this.isJobManagerMetadataEnabled()) { + this.jobDags.forEach(jobDag -> this.getTracker().set(jobDag.getDataFeedName(), + jobDag.getJobManagerMetadata())); + try { + this.getTracker().writeJobManagerMetadata(); + } catch (MetadataException e) { + log.error("Unable to save metadata: {}", e.getMessage()); + } + } + this.sparkFactory.stop(); + this.jobLockManager.stop(); } private static void setSparkStageName(@NonNull final JavaSparkContext jsc, @NotEmpty final String dataFeedName) { @@ -344,17 +364,17 @@ private JobLockManager(@NonNull final Configuration conf, @NotEmpty final String this.dagTimerMetricMap = new HashMap<>(); } - private boolean lockDag(@NotEmpty final String jobName, @NotEmpty final String dagName) { - final String key = LockManager.getLockKey(DAG_LOCK_KEY, dagName); + private boolean lockDag(@NotEmpty final String jobDagName, @NotEmpty final String dataFeedName) { + final String key = LockManager.getLockKey(DAG_LOCK_KEY, jobDagName + "_" + dataFeedName); final TimerMetric timerMetric = new TimerMetric(JobMetricNames.JOB_DAG_LOCK_TIME_MS, ImmutableMap.of( JOB_FREQUENCY_TAG, jobFrequency, - JOB_NAME_TAG, jobName, - DATA_FEED_TAG, dagName)); + JOB_NAME_TAG, jobDagName, + DATA_FEED_TAG, dataFeedName)); final boolean success = lockManager.lock(key, - String.format("JobDag %s AppId %s", dagName, appId)); + String.format("JobDag %s AppId %s", dataFeedName, appId)); timerMetric.stop(); - dagTimerMetricMap.put(dagName, timerMetric); + dagTimerMetricMap.put(dataFeedName, timerMetric); return success; } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JsonHoodieIngestionJob.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JsonHoodieIngestionJob.java deleted file mode 100644 index 153e683..0000000 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/JsonHoodieIngestionJob.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. - * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated - * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the - * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to - * permit persons to whom the Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all copies or substantial portions - * of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO - * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE - * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, - * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS - * IN THE SOFTWARE. - */ - -package com.uber.marmaray.common.job; - -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import com.google.common.base.Optional; -import com.uber.marmaray.common.configuration.Configuration; -import com.uber.marmaray.common.configuration.FileSourceConfiguration; -import com.uber.marmaray.common.configuration.HoodieConfiguration; -import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter; -import com.uber.marmaray.common.converters.data.TSBasedHoodieSinkDataConverter; -import com.uber.marmaray.common.exceptions.JobRuntimeException; -import com.uber.marmaray.common.metadata.HoodieBasedMetadataManager; -import com.uber.marmaray.common.metadata.IMetadataManager; -import com.uber.marmaray.common.schema.HDFSSchemaService; -import com.uber.marmaray.common.schema.ISchemaService; -import com.uber.marmaray.common.sinks.hoodie.HoodieSink; -import com.uber.marmaray.common.sources.file.FileSource; -import com.uber.marmaray.common.sources.file.FileSourceDataConverter; -import com.uber.marmaray.common.sources.file.FileWorkUnitCalculator; -import com.uber.marmaray.common.sources.file.JSONFileSourceDataConverter; -import com.uber.marmaray.utilities.FSUtils; -import com.uber.marmaray.utilities.JsonSourceConverterErrorExtractor; -import com.uber.marmaray.utilities.StringTypes; -import lombok.Getter; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.spark.api.java.JavaSparkContext; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import static com.uber.marmaray.utilities.ErrorTableUtil.ERROR_SCHEMA_IDENTIFIER; -import static com.uber.marmaray.utilities.ErrorTableUtil.ERROR_TABLE_KEY; -import static com.uber.marmaray.utilities.ErrorTableUtil.TABLE_KEY; -import static com.uber.marmaray.utilities.SparkUtil.SPARK_PROPERTIES_KEY_PREFIX; - -/** - * End to end job to load data from JSON files on HDFS to a hoodie table. Example of combining - * {@link com.uber.marmaray.common.sources.ISource}, {@link com.uber.marmaray.common.sinks.ISink}, - * into a {@link JobDag} that can be run to load data. - */ -@Slf4j -public class JsonHoodieIngestionJob { - - public static final String APP_NAME = "MarmarayJsonHoodieIngestion"; - public static final String ONLY_TABLE = "only_table"; - public static final String ERROR_TABLE = "error_table"; - public static final String ERROR_SCHEMA_NAME = "errorSchema"; - - public static void main(final String[] args) throws Exception { - final JsonHoodieIngestionCommandLineOptions cmd = new JsonHoodieIngestionCommandLineOptions(args); - - final Configuration conf = getConfiguration(cmd); - - final JobManager jobManager = JobManager.createJobManager(conf, APP_NAME, StringTypes.EMPTY, false); - - final ISchemaService schemaService = new HDFSSchemaService(conf); - final String schemaName = conf.getProperty(FileSourceConfiguration.SCHEMA).get(); - final Schema schema = schemaService.getSchema(schemaName); - jobManager.addSchema(schema); - final Schema errorSchema = schemaService.getSchema(ERROR_SCHEMA_NAME); - - jobManager.getConf().setProperty(SPARK_PROPERTIES_KEY_PREFIX + ERROR_SCHEMA_IDENTIFIER, errorSchema.toString()); - jobManager.getConf().setProperty(SPARK_PROPERTIES_KEY_PREFIX + TABLE_KEY, ONLY_TABLE); - jobManager.getConf().setProperty(SPARK_PROPERTIES_KEY_PREFIX + ERROR_TABLE_KEY, ERROR_TABLE); - final JavaSparkContext jsc = jobManager.getOrCreateSparkContext(); - - final FileSourceDataConverter sourceDataConverter = - new JSONFileSourceDataConverter(conf, new JsonSourceConverterErrorExtractor(), schema); - final FileSource fileSource = new FileSource(new FileSourceConfiguration(conf), jsc, sourceDataConverter); - final HoodieSinkDataConverter sinkDataConverter = - new TSBasedHoodieSinkDataConverter(conf, "firstName", "timestamp", TimeUnit.SECONDS); - final HoodieConfiguration hoodieConf = HoodieConfiguration.newBuilder(conf, ONLY_TABLE) - .withSchema(schema.toString()) - .build(); - final IMetadataManager metadataMgr = - new HoodieBasedMetadataManager(hoodieConf, new AtomicBoolean(true), jsc); - final HoodieSink hoodieSink = new HoodieSink( - hoodieConf, - sinkDataConverter, - jsc, - HoodieSink.HoodieSinkOp.INSERT, - metadataMgr); - final FileWorkUnitCalculator workUnitCalculator = new FileWorkUnitCalculator(new FileSourceConfiguration(conf)); - - final JobDag jobDag = new JobDag( - fileSource, - hoodieSink, - metadataMgr, - workUnitCalculator, - APP_NAME, - schemaName, - jobManager.getJobMetrics(), - jobManager.getReporters()); - - jobManager.addJobDag(jobDag); - jobManager.run(); - } - - private static Configuration getConfiguration(@NonNull final JsonHoodieIngestionCommandLineOptions cmd) { - final Configuration conf = new Configuration(); - try { - final FileSystem fs = FSUtils.getFs(conf); - final Path dataFeedConfFile = new Path(cmd.getConfFile()); - log.info("Loading configuration from {}", dataFeedConfFile.toString()); - conf.loadYamlStream(fs.open(dataFeedConfFile), Optional.absent()); - } catch (IOException e) { - final String errorMessage = String.format("Unable to find configuration for %s", cmd.getConfFile()); - log.error(errorMessage); - throw new JobRuntimeException(errorMessage, e); - } - return conf; - } - - private static final class JsonHoodieIngestionCommandLineOptions { - @Getter - @Parameter(names = {"--configurationFile", "-c"}, description = "path to configuration file", required = true) - private String confFile; - - private JsonHoodieIngestionCommandLineOptions(@NonNull final String[] args) { - final JCommander commander = new JCommander(this); - commander.parse(args); - } - } - -} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java index 8821a28..3f22600 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java @@ -76,9 +76,12 @@ public final class ThreadPoolService { private ThreadPoolService(final int numThreads, final int reservedJobDagThreads, final int reservedActionsThreads) { log.info("Starting thread pool service numThreads:{} numJobDagThreads:{}", numThreads, reservedJobDagThreads); - Preconditions.checkState(numThreads > 0 && reservedJobDagThreads > 0 && reservedActionsThreads > 0, - String.format("Number of threads should be positive: total: %d, jobDag: %d, actions: %d", - numThreads, reservedJobDagThreads, reservedActionsThreads)); + Preconditions.checkState(numThreads > 0 , String.format( + "numThreads should be positive: %d", numThreads)); + Preconditions.checkState(reservedJobDagThreads > 0, String.format( + "reservedJobDagThreads should be positive: %d", reservedJobDagThreads)); + Preconditions.checkState(reservedActionsThreads > 0, String.format( + "reservedActionThreads should be positive: %d", reservedActionsThreads)); Preconditions.checkState(numThreads >= reservedJobDagThreads + reservedActionsThreads, String.format( "Total threads must be at least equal to reserved threads: total: %d, jobDag: %d, actions: %d ", diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/CassandraBasedMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/CassandraBasedMetadataManager.java new file mode 100644 index 0000000..e8d3cfa --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/CassandraBasedMetadataManager.java @@ -0,0 +1,366 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.metadata; + +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.Row; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.exceptions.InvalidQueryException; + +import java.util.Map; +import java.util.List; +import java.util.Set; +import java.util.HashMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.stream.Collectors; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.configuration.CassandraMetadataManagerConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.exceptions.MetadataException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.common.schema.cassandra.CassandraMetadataSchemaManager; +import com.uber.marmaray.common.schema.cassandra.CassandraSchema; +import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField; +import com.uber.marmaray.common.schema.cassandra.ClusterKey; +import com.uber.marmaray.utilities.MapUtil; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.parquet.Strings; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +@Slf4j +/** + * {@link CassandraBasedMetadataManager} implements the {@link IMetadataManager} interface. + * It is a cassandra based metadata manager for HDFS files. + */ + +public class CassandraBasedMetadataManager implements IMetadataManager { + + private static final List fields = Collections.unmodifiableList(Arrays.asList( + new CassandraSchemaField("job", CassandraSchemaField.STRING_TYPE), + new CassandraSchemaField("time_stamp", CassandraSchemaField.STRING_TYPE), + new CassandraSchemaField("checkpoint", CassandraSchemaField.STRING_TYPE))); + + private static final int maxTimestampCount = 5; + + protected final CassandraSchema schema; + protected final CassandraMetadataSchemaManager schemaManager; + private Optional> metadataMap = Optional.absent(); + private int numTimestamps; + private Optional oldestTimestamp; + + @Getter + private final AtomicBoolean shouldSaveChanges; + + @Getter + private final String job; + + @Getter + private final CassandraMetadataManagerConfiguration config; + + @Getter + private Optional dataFeedMetrics = Optional.absent(); + + /** + * initialize metadata manager table + */ + public CassandraBasedMetadataManager( + @NonNull final CassandraMetadataManagerConfiguration cassandraMetadataManagerConfig, + @NonNull final AtomicBoolean shouldSaveChanges) throws IOException { + this.config = cassandraMetadataManagerConfig; + this.job = this.config.getJobName(); + + this.shouldSaveChanges = shouldSaveChanges; + this.numTimestamps = 0; + this.oldestTimestamp = Optional.absent(); + + this.schema = new CassandraSchema(this.config.getKeyspace(), this.config.getTableName(), this.fields); + this.schemaManager = new CassandraMetadataSchemaManager( + this.schema, + Arrays.asList("job"), + Arrays.asList(new ClusterKey("time_stamp", ClusterKey.Order.DESC)), + Optional.absent()); + } + + private Map getMetadataMap() { + if (!this.metadataMap.isPresent()) { + try { + this.metadataMap = Optional.of(generateMetaDataMap()); + } catch (IOException e) { + log.error("Failed in loading cassandra based metadata manager", e); + throw new JobRuntimeException(e); + } + } + return this.metadataMap.get(); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + /** + * set operation applies on generated temporarily + * map from Cassandra queries + * @param key + * @param value + * @throws MetadataException + */ + @Override + public void set(@NotEmpty final String key, @NonNull final StringValue value) throws MetadataException { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); + getMetadataMap().put(key, value); + } + + /** + * remove operation on map + * @param key + * @return + */ + @Override + public Optional remove(@NotEmpty final String key) { + return Optional.fromNullable(getMetadataMap().remove(key)); + } + + /** + * get operation from map + * @param key + * @return + * @throws MetadataException + */ + @Override + public Optional get(@NotEmpty final String key) throws MetadataException { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); + return getMetadataMap().containsKey(key) ? Optional.of(getMetadataMap().get(key)) : Optional.absent(); + } + + /** + * @return + * Returns all keys + */ + @Override + public Set getAllKeys() { + return getMetadataMap().keySet(); + } + + /** + * Upon a successful job, this method keeps the latest checkpoint + * and drops the oldest checkpoint if there are more than maxTimestampCount for + * the job. + * + * @return + * @throws IOException + */ + @Override + public void saveChanges() { + if (this.shouldSaveChanges.compareAndSet(true, false)) { + log.info("Saving checkpoint information to Cassandra"); + } else { + log.info("Checkpoint info is already saved. Not saving it again."); + return; + } + + log.info("Connecting to Cassandara cluster"); + try (final Cluster cluster = getClusterBuilder().build(); + final Session session = cluster.connect()) { + + /** + * oldest checkpoint is present only if maxTimestampCount checkpoints + * are already present for current job + */ + if (this.oldestTimestamp.isPresent()) { + //delete older checkpoint before inserting the new one + final String deleteCmd = this.schemaManager.generateDeleteOldestCheckpoint( + this.job, this.oldestTimestamp); + log.info("Removing the oldest checkpoint with command: {}", deleteCmd); + cmdExec(session, deleteCmd); + } + + final Long currentTime = System.currentTimeMillis(); + final String checkpoint = serializeMetadataMap(); + String cassandraCols = String.format("job, time_stamp, checkpoint"); + String cassandraVals = String.format("'%s', '%s', '%s'", + this.job, currentTime.toString(), checkpoint); + + final String insertCmd = this.schemaManager.generateInsertStmt(cassandraCols, cassandraVals); + log.info("Inserting {} to cassandra with command: {}", cassandraVals, insertCmd); + cmdExec(session, insertCmd); + log.info("Write back {} line", getMetadataMap().size()); + } + } + + public Map generateMetaDataMap() throws IOException { + log.info("Attempting to generate metadata map from Cassandra queries"); + + log.info("Connecting to Cassandara cluster"); + try (final Cluster cluster = getClusterBuilder().build(); + final Session session = cluster.connect()) { + + final String setKeySpaceCmd = "USE marmaray;"; + log.info("setting keyspace with Cassandra command: {}", setKeySpaceCmd); + cmdExec(session, setKeySpaceCmd); + // check if table exists + ResultSet results; + log.info("Attempting to getting column names"); + + try { + final String columnNameQuery = this.schemaManager.getColumnsFromTableQuery(); + log.info("Getting column names with table query: {}", columnNameQuery); + results = cmdExec(session, columnNameQuery); + } catch (InvalidQueryException e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags(ModuleTagNames.METADATA_MANAGER, + ErrorCauseTagNames.CASSANDRA_QUERY)); + } + final String columnNameFromCFQuery = this.schemaManager.getColumnsFromColumnFamilyQuery(); + log.error("Saw an InvalidQueryException. Getting column names using column families: {}", + columnNameFromCFQuery); + results = cmdExec(session, columnNameFromCFQuery); + } + + final Map columns = results.all() + .stream() + .collect(Collectors.toMap(r -> r.getString("column_name"), r -> r.getString("type"))); + + log.info("Found columns: {}", columns.toString()); + if (columns.isEmpty()) { + final String createTable = this.schemaManager.generateCreateTableStmt(); + //TODO : do we consider not existing table as an error? + log.info("No existing columns found. Creating table with Cassandra command: {}", createTable); + cmdExec(session, this.schemaManager.generateCreateTableStmt()); + log.info("Returning empty metadata map"); + return new HashMap(); + } else { + // this created new columns based on local specified schema + // as of now we dont not expect this part to get exercised + log.info("Generating alter table statements for any columns not found"); + this.schemaManager.generateAlterTableStmt(columns).forEach(stmt -> { + log.info("Altering Cassandra table with command: {}", stmt); + cmdExec(session, stmt); + }); + + final String jobQuery = this.schemaManager.generateSelectJob(this.job, maxTimestampCount); + log.info("Creating metadata map from Cassandra query: {}", jobQuery); + ResultSet checkpoints = cmdExec(session, jobQuery); + + final List rows = checkpoints.all(); + if (rows.isEmpty()) { + log.info("No result was found, returning empty metadata map"); + return new HashMap(); + } else { + this.numTimestamps = rows.size(); + log.info("Number of founded checkpoints for job {} is: {}", this.job, this.numTimestamps); + if (this.numTimestamps == maxTimestampCount) { + this.oldestTimestamp = Optional.of((String) rows.get(rows.size() - 1).getObject("time_stamp")); + log.info("Keeping oldest checkpoint {}", this.oldestTimestamp.get()); + } else if (this.numTimestamps > maxTimestampCount) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags(ModuleTagNames.METADATA_MANAGER, + ErrorCauseTagNames.ERROR)); + } + } + HashMap metadata = new HashMap(); + final Row row = rows.get(0); + metadata = deserializeMetadata((String) row.getObject("checkpoint")); + log.info("Returning checkpoint"); + return metadata; + } + } + } + } + + public void deleteAllMetadataOfJob(@NonNull final String jobName) { + try (final Cluster cluster = getClusterBuilder().build(); + final Session session = cluster.connect()) { + final String deleteCmd = this.schemaManager.generateDeleteJob(jobName); + log.info("Deleteing all the checkpoints of job: {} with command: {}", jobName, deleteCmd); + cmdExec(session, deleteCmd); + } + } + + private Cluster.Builder getClusterBuilder() { + final Cluster.Builder builder = Cluster.builder().withClusterName(this.config.getCluster()); + builder.withCredentials(this.config.getUsername(), this.config.getPassword()); + this.config.getInitialHosts().stream().forEach(host -> builder.addContactPoint(host)); + if (this.config.getNativePort().isPresent()) { + builder.withPort(Integer.parseInt(this.config.getNativePort().get())); + } else { + builder.withPort(Integer.parseInt(CassandraMetadataManagerConfiguration.DEFAULT_OUTPUT_NATIVE_PORT)); + } + builder.withQueryOptions(new QueryOptions().setConsistencyLevel(ConsistencyLevel.QUORUM)); + return builder; + } + + private String serializeMetadataMap() { + HashMap metadata = new HashMap(); + for (final Map.Entry entry : getMetadataMap().entrySet()) { + metadata.put(entry.getKey(), entry.getValue().getValue()); + } + String checkpoints = MapUtil.serializeMap(metadata); + return checkpoints; + } + + private HashMap deserializeMetadata(@NotEmpty final String rawMetadata) { + HashMap metadata = new HashMap(); + Map meta = MapUtil.deserializeMap(rawMetadata); + for (final Map.Entry entry : meta.entrySet()) { + log.info("deserialize to metadata map ({} : {})", entry.getKey(), entry.getValue()); + metadata.put(entry.getKey(), new StringValue(entry.getValue())); + } + return metadata; + } + + private ResultSet cmdExec(@NotEmpty final Session session, @NotEmpty final String command) { + ResultSet result; + try { + result = session.execute(command); + } catch (Exception e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags(ModuleTagNames.METADATA_MANAGER, + ErrorCauseTagNames.EXEC_CASSANDRA_CMD)); + } + log.error("Exception: {}", e); + throw new JobRuntimeException(e); + } + return result; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java index 0980312..1b06988 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java @@ -46,12 +46,11 @@ public class HDFSDatePartitionManager extends HDFSPartitionManager { private final Optional startDate; public HDFSDatePartitionManager(@NotEmpty final String metadataKey, - @NotEmpty final String genericBaseMetadataPath, @NotEmpty final String genericBaseDataPath, @NotEmpty final String partitionKeyName, @NonNull final Optional startDate, @NonNull final FileSystem fileSystem) throws IOException { - super(metadataKey, genericBaseMetadataPath, genericBaseDataPath, fileSystem); + super(metadataKey, genericBaseDataPath, fileSystem); this.partitionKeyName = partitionKeyName + StringTypes.EQUAL; this.hasPartitionKeyInHDFSPartitionPath = hasPartitionKeyNameInPartition(); this.startDate = startDate; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java index 960816a..c896a70 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java @@ -26,6 +26,11 @@ import com.google.common.base.Preconditions; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.exceptions.MetadataException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.utilities.FSUtils; import lombok.Getter; import lombok.NonNull; @@ -68,7 +73,7 @@ public class HDFSMetadataManager implements IMetadataManager { // Using a thread-safe HashMap doesn't really provide any protection against jobs from other or same // customers running jobs against the same metadata directory. We eventually want to take locks on // a directory (possivly via ZooKeeper) so only one job can operate at a time per job name. - private final Map metadataMap; + private Optional> metadataMap = Optional.absent(); @NonNull private final FileSystem fileSystem; @@ -76,6 +81,9 @@ public class HDFSMetadataManager implements IMetadataManager { @NotEmpty @Getter private final String baseMetadataPath; + @Getter + private Optional dataFeedMetrics = Optional.absent(); + /* * If it is able to update {@link #shouldSaveChanges} from true to false; then only it will create new * metadata file and will save information in it. @@ -89,29 +97,54 @@ public HDFSMetadataManager(@NonNull final FileSystem fs, @NotEmpty final String this.fileSystem = fs; this.baseMetadataPath = baseMetadataPath; this.shouldSaveChanges = shouldSaveChanges; - this.metadataMap = loadMetadata(); + if (!fs.exists(new Path(this.baseMetadataPath))) { + // Ensuring that directories are created in case they are not found. + fs.mkdirs(new Path(this.baseMetadataPath)); + } + } + + private Map getMetadataMap() { + if (!this.metadataMap.isPresent()) { + try { + this.metadataMap = Optional.of(loadMetadata()); + } catch (IOException e) { + log.error("Failed in loading HDFS based metadata manager", e); + throw new JobRuntimeException(e); + } + } + return this.metadataMap.get(); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored } @Override public void set(@NotEmpty final String key, @NonNull final StringValue value) throws MetadataException { Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); - this.metadataMap.put(key, value); + getMetadataMap().put(key, value); } @Override public Optional remove(@NotEmpty final String key) { - return Optional.fromNullable(this.metadataMap.remove(key)); + return Optional.fromNullable(getMetadataMap().remove(key)); } @Override public Optional get(@NotEmpty final String key) throws MetadataException { Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); - return this.metadataMap.containsKey(key) ? Optional.of(this.metadataMap.get(key)) : Optional.absent(); + return getMetadataMap().containsKey(key) ? Optional.of(getMetadataMap().get(key)) : Optional.absent(); } @Override public Set getAllKeys() { - return this.metadataMap.keySet(); + return getMetadataMap().keySet(); } /** @@ -161,6 +194,11 @@ private void pruneMetadataFiles() { } } } catch (final IOException e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.METADATA_MANAGER, ErrorCauseTagNames.SAVE_METADATA)); + } final String errMsg = String.format("IOException occurred while pruning metadata files. Message: %s", e.getMessage()); log.warn(errMsg); @@ -183,6 +221,11 @@ private void writeMetadataFile() { log.info("Saving metadata to: {}", fileLocation); this.fileSystem.rename(new Path(tmpFileLocation), new Path(fileLocation)); } catch (final IOException e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.METADATA_MANAGER, ErrorCauseTagNames.SAVE_METADATA)); + } final String errMsg = String.format("IOException occurred while saving changes. Message: %s", e.getMessage()); throw new MetadataException(errMsg, e); @@ -242,8 +285,8 @@ public Map loadMetadata(final Path path) throws IOException private void serialize(final ObjectOutputStream out) throws IOException { out.writeInt(SERIALIZATION_VERSION); - out.writeInt(this.metadataMap.size()); - for (final Map.Entry entry : this.metadataMap.entrySet()) { + out.writeInt(getMetadataMap().size()); + for (final Map.Entry entry : getMetadataMap().entrySet()) { log.info("Serializing key: {} and value: {}", entry.getKey(), entry.getValue().getValue()); out.writeUTF(entry.getKey()); out.writeUTF(entry.getValue().getValue()); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java index d1466b8..cbb7ea9 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java @@ -52,7 +52,6 @@ public class HDFSPartitionManager { private final boolean isSinglePartition; public HDFSPartitionManager(@NotEmpty final String metadataKey, - @NotEmpty final String genericBaseMetadataPath, @NotEmpty final String baseDataPath, @NonNull final FileSystem fileSystem) throws IOException { this.metadataKey = metadataKey; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java index c0633a2..d65f8a6 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java @@ -17,12 +17,6 @@ package com.uber.marmaray.common.metadata; import com.google.common.base.Optional; -import com.uber.marmaray.common.configuration.HadoopConfiguration; -import com.uber.marmaray.common.configuration.HoodieConfiguration; -import com.uber.marmaray.common.exceptions.JobRuntimeException; -import com.uber.marmaray.utilities.FSUtils; -import com.uber.marmaray.utilities.HoodieUtil; -import com.uber.marmaray.utilities.MapUtil; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieAvroPayload; @@ -30,6 +24,22 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.marmaray.common.configuration.HadoopConfiguration; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.utilities.FSUtils; +import com.uber.marmaray.utilities.HoodieUtil; +import com.uber.marmaray.utilities.MapUtil; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.FileSystem; +import org.apache.parquet.Strings; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.hibernate.validator.constraints.NotEmpty; import java.io.IOException; import java.util.ArrayList; @@ -40,15 +50,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import lombok.Getter; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.hadoop.fs.FileSystem; -import org.apache.parquet.Strings; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.hibernate.validator.constraints.NotEmpty; - /** * It should be used if metadata information needs to be stored in hoodie. It uses hoodie commit file to store * metadata information. In cases where where job has hoodie sink then it will have no-op for saveChanges(). @@ -62,7 +63,7 @@ public class HoodieBasedMetadataManager implements IMetadataManager private final HoodieConfiguration hoodieConf; private final AtomicBoolean saveChanges; private transient Optional jsc = Optional.absent(); - private final Map metadataMap; + private Optional> metadataMap = Optional.absent(); /** * @param shouldSaveChanges {@link AtomicBoolean} which {@link #saveChanges} will use to determine if @@ -75,7 +76,23 @@ public HoodieBasedMetadataManager(@NonNull final HoodieConfiguration hoodieConf, this.hoodieConf = hoodieConf; this.saveChanges = shouldSaveChanges; this.jsc = Optional.of(jsc); - this.metadataMap = readMetadataInfo(this.hoodieConf); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + // ignored + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + private Map getMetadataMap() { + if (!this.metadataMap.isPresent()) { + this.metadataMap = Optional.of(readMetadataInfo(this.hoodieConf)); + } + return this.metadataMap.get(); } /** @@ -86,7 +103,7 @@ public void set(@NotEmpty final String key, @NonNull final StringValue value) { throw new JobRuntimeException( String.format("Metadata manager changes are already saved.key:%s:value%s", key, value)); } - this.metadataMap.put(key, value.getValue()); + getMetadataMap().put(key, value.getValue()); } /** @@ -97,7 +114,7 @@ public void set(@NotEmpty final String key, @NonNull final StringValue value) { */ @Override public Optional remove(@NotEmpty final String key) { - final String val = this.metadataMap.remove(key); + final String val = getMetadataMap().remove(key); return val == null ? Optional.absent() : Optional.of(new StringValue(val)); } @@ -105,7 +122,7 @@ public Optional remove(@NotEmpty final String key) { * Returns given metadata key. */ public Optional get(@NotEmpty final String key) { - final String val = this.metadataMap.get(key); + final String val = getMetadataMap().get(key); return val == null ? Optional.absent() : Optional.of(new StringValue(val)); } @@ -114,14 +131,14 @@ public Optional get(@NotEmpty final String key) { */ @Override public Set getAllKeys() { - return this.metadataMap.keySet(); + return getMetadataMap().keySet(); } /** * Returns immutable map of metadata key-value pairs. */ public Map getAll() { - return Collections.unmodifiableMap(this.metadataMap); + return Collections.unmodifiableMap(getMetadataMap()); } public AtomicBoolean shouldSaveChanges() { @@ -158,7 +175,7 @@ public void saveChanges() { */ public HashMap getMetadataInfo() { final HashMap map = new HashMap<>(); - map.put(HOODIE_METADATA_KEY, MapUtil.serializeMap(this.metadataMap)); + map.put(HOODIE_METADATA_KEY, MapUtil.serializeMap(getMetadataMap())); return map; } @@ -169,7 +186,7 @@ public HashMap getMetadataInfo() { private static Map readMetadataInfo( @NonNull final HoodieConfiguration hoodieConf) { try { - final FileSystem fs = FSUtils.getFs(hoodieConf.getConf()); + final FileSystem fs = FSUtils.getFs(hoodieConf.getConf(), Optional.of(hoodieConf.getBasePath())); HoodieUtil.initHoodieDataset(fs, hoodieConf); final HoodieTableMetaClient hoodieTableMetaClient = new HoodieTableMetaClient(new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java index f442f3c..11a2501 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java @@ -17,6 +17,9 @@ package com.uber.marmaray.common.metadata; import com.google.common.base.Optional; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.metrics.JobMetrics; import lombok.NonNull; import java.io.IOException; @@ -32,10 +35,16 @@ * while other metadata managers will write through the changes when {@link IMetadataManager#set(String, AbstractValue)} * is invoked. Please see the comments and details for each implementation for further details. */ -public interface IMetadataManager extends Serializable { +public interface IMetadataManager extends Serializable, IMetricable { void set(@NonNull final String key, @NonNull final T value); Optional remove(@NonNull final String key); Optional get(@NonNull final String key); void saveChanges() throws IOException; Set getAllKeys(); + + @Override + void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics); + + @Override + void setJobMetrics(@NonNull final JobMetrics jobMetrics); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java index 4d3b9b8..e4b07b8 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java @@ -62,7 +62,8 @@ public JobManagerMetadataTracker(@NonNull final Configuration config) throws IOE throw new MissingPropertyException("Base Path for HDFS JobManager Metadata Tracker is missing."); } this.metadataManager = - new HDFSMetadataManager(FSUtils.getFs(config), basePath.get(), new AtomicBoolean(true)); + new HDFSMetadataManager(FSUtils.getFs(config, basePath), basePath.get(), + new AtomicBoolean(true)); this.shouldSaveChanges = new AtomicBoolean(true); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/MultiMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/MultiMetadataManager.java new file mode 100644 index 0000000..2fe495d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/MultiMetadataManager.java @@ -0,0 +1,190 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.metadata; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.uber.marmaray.common.exceptions.MetadataException; + +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import com.google.common.base.Optional; +import org.apache.parquet.Strings; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Set; + +@Slf4j +/** + * {@link MultiMetadataManager} implements the {@link IMetadataManager} interface. + * It is a cassandra based metadata manager for HDFS files. + */ + +public class MultiMetadataManager implements IMetadataManager { + + private final List> metadataManagersList; + private Optional> metadataMap = Optional.absent(); + + @Getter + private final AtomicBoolean shouldSaveChanges; + + /** + * initialize metadata manager table + */ + public MultiMetadataManager(@NonNull final List> metadataManagersList, + @NonNull final AtomicBoolean shouldSaveChanges) { + + this.metadataManagersList = ImmutableList.copyOf(metadataManagersList); + this.shouldSaveChanges = shouldSaveChanges; + } + + private Map getMetadataMap() { + if (!this.metadataMap.isPresent()) { + this.metadataMap = Optional.of(loadMetadata()); + } + return this.metadataMap.get(); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.metadataManagersList.forEach(metadataManager -> + metadataManager.setDataFeedMetrics(dataFeedMetrics)); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + this.metadataManagersList.forEach(metadataManager -> + metadataManager.setJobMetrics(jobMetrics)); + } + + /** + * set operation applies on generated temporarily + * map from Cassandra queries + * @param key + * @param value + * @throws MetadataException + */ + @Override + public void set(@NotEmpty final String key, @NonNull final StringValue value) throws MetadataException { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); + this.metadataManagersList.forEach(metadataManager -> metadataManager.set(key, value)); + getMetadataMap().put(key, value); + } + + /** + * remove operation on map + * @param key + * @return + */ + @Override + public Optional remove(@NotEmpty final String key) { + this.metadataManagersList.forEach(metadataManager -> metadataManager.remove(key)); + return Optional.fromNullable(getMetadataMap().remove(key)); + } + + /** + * get operation from map + * @param key + * @return + * @throws MetadataException + */ + @Override + public Optional get(@NotEmpty final String key) throws MetadataException { + return getMetadataMap().containsKey(key) ? Optional.of(getMetadataMap().get(key)) : Optional.absent(); + } + + /** + * @return + * Returns all keys + */ + @Override + public Set getAllKeys() { + return getMetadataMap().keySet(); + } + + /** + * Upon a successful job, this method keeps the latest checkpoint. + * @return + * @throws IOException + */ + @Override + public void saveChanges() { + if (this.shouldSaveChanges.compareAndSet(true, false)) { + log.info("Saving checkpoint information."); + } else { + log.info("Checkpoint info is already saved. Not saving it again."); + return; + } + + /** update all children metadata managers with recent checkpoints*/ + getMetadataMap().forEach((key, value) -> { + this.metadataManagersList.forEach(metadataManager -> metadataManager.set(key, value)); + }); + + /** save checkpoints in all children metadata managers*/ + log.info("Save changes in all metadata managers"); + this.metadataManagersList.forEach(metadataManager -> { + try { + metadataManager.saveChanges(); + } catch (IOException e) { + throw new MetadataException("Unable to save JobManager Metadata", e); + } + }); + } + + /** + * load metadata map + * @return latest metadata captured from all metadata managers + */ + @VisibleForTesting + private Map loadMetadata() { + log.info("load all metadata managers"); + + HashMap metadata = new HashMap(); + this.metadataManagersList.forEach(metadataManager -> { + log.info("metadata manager : {}", metadataManager.toString()); + metadataManager.getAllKeys().forEach(key -> { + final Optional metadataManagerValue = metadataManager.get(key); + log.info("metadata: key: {}, value: {}", key, metadataManagerValue.get()); + if (metadata.containsKey(key)) { + final StringValue currentValue = metadata.get(key); + int compareResult = metadataManagerValue.get().toString() + .compareTo(currentValue.toString()); + if (compareResult != 0) { + log.info("metadata mismatch in child manager!"); + if (compareResult > 0) { + metadata.put(key, metadataManagerValue.get()); + } + } + } else { + metadata.put(key, metadataManagerValue.get()); + } + }); + }); + return metadata; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.java index 98e57c2..dffe815 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.java @@ -17,6 +17,8 @@ package com.uber.marmaray.common.metadata; import com.google.common.base.Optional; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; import lombok.NonNull; import org.hibernate.validator.constraints.NotEmpty; @@ -25,6 +27,15 @@ import java.util.Set; public class NoOpMetadataManager implements IMetadataManager { + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + } + @Override public void set(@NotEmpty final String key, @NonNull final StringValue value) { // no-op diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraMetric.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraMetric.java new file mode 100644 index 0000000..dd26a00 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraMetric.java @@ -0,0 +1,17 @@ +package com.uber.marmaray.common.metrics; + +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; + +import java.util.Map; + +public class CassandraMetric { + + public static final String TABLE_NAME_TAG = "tableName"; + + public static Map createTableNameTags(@NonNull final CassandraSinkConfiguration cassandraConf) { + return DataFeedMetrics.createAdditionalTags(TABLE_NAME_TAG, + cassandraConf.getKeyspace() + StringTypes.UNDERSCORE + cassandraConf.getTableName()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java index 6599c3c..e08d4e1 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java @@ -17,16 +17,34 @@ package com.uber.marmaray.common.metrics; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.HashMap; +import java.util.Map; /** * Known {@link DataFeedMetrics} names. */ +@Slf4j public final class DataFeedMetricNames { // metrics in context of the job public static final String RUNTIME = "runtime"; public static final String SIZE_BYTES = "size_bytes"; public static final String CURRENT_STATUS = "current_status"; + // read input metrics + public static final String TOTAL_READ_SIZE = "total_read_size"; + public static final String AVERAGE_INPUT_RECORD_SIZE = "average_input_record_size"; + public static final String MAX_INPUT_RECORD_SIZE = "max_input_record_size"; + public static final String TOTAL_READ_SIZE_PER_PARTITION = "total_read_size_per_partition"; + public static final String NUM_INPUT_PARTITIONS = "num_input_partitions"; + + public static final String NUM_RDD_PARTITIONS = "num_rdd_partitions"; + public static final String RDD_PARTITION_SIZE = "rdd_partition_size"; + public static final String TOTAL_FILE_COUNT = "total_file_count"; + public static final String TOTAL_WRITE_SIZE = "total_write_size"; + public static final String AVAILABLE_ROWCOUNT = "available_rowcount"; public static final String INPUT_ROWCOUNT = "input_rowcount"; public static final String OUTPUT_ROWCOUNT = "output_rowcount"; @@ -58,8 +76,26 @@ public final class DataFeedMetricNames { public static final String CONVERT_SCHEMA_LATENCY_MS = "convert_schema_latency_ms"; public static final String TOTAL_LATENCY_MS = "total_latency_ms"; + // Error Exposure Metric + public static final String MARMARAY_JOB_ERROR = "marmaray_job_error"; + public static final String MARMARAY_JOB_CONFIG_ERROR = "marmaray_job_config_error"; + + // TAGS + private static final String MODULE = "module"; + private static final String CAUSE = "cause"; + private DataFeedMetricNames() { throw new JobRuntimeException("This class should never be instantiated"); } + + public static Map getErrorModuleCauseTags( + @NotEmpty final String module, @NotEmpty final String cause) { + final Map tags = new HashMap<>(); + tags.put(MODULE, module); + tags.put(CAUSE, cause); + // TODO : remove this info + log.info("added tags, module {}, cause {}", module, cause); + return tags; + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java index 54bc3e1..d259a19 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java @@ -25,8 +25,10 @@ import lombok.NonNull; import org.hibernate.validator.constraints.NotEmpty; +import java.io.Serializable; import java.time.Instant; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -35,12 +37,20 @@ * and {@link DataFeedMetrics#createTimerMetric}, and reported using {@link DataFeedMetrics#gaugeAll} */ @Getter -public class DataFeedMetrics implements Reportable { +public class DataFeedMetrics implements Reportable, Serializable { + public static final String DATA_FEED_NAME = "datafeed"; + private static final String JOB_TAG = "job"; + @Getter private final Set metricSet = Sets.newHashSet(); + + @Getter + private final Set failureMetricSet = Sets.newHashSet(); + private final Map baseTags; + @NotEmpty private final String jobName; @@ -61,6 +71,26 @@ public LongMetric createLongMetric(@NonNull final String metricName, return metric; } + public LongMetric createLongMetric(@NonNull final String metricName, + final long metricValue) { + return createLongMetric(metricName, metricValue, new HashMap<>()); + } + + public LongMetric createLongFailureMetric(@NonNull final String metricName, + final long metricValue, + @NonNull final Map additionalTags) { + final LongMetric metric = new LongMetric(metricName, metricValue); + metric.addTags(getBaseTags()); + metric.addTags(additionalTags); + this.failureMetricSet.add(metric); + return metric; + } + + public LongMetric createLongFailureMetric(@NonNull final String metricName, + final long metricValue) { + return createLongFailureMetric(metricName, metricValue, new HashMap<>()); + } + public TimerMetric createTimerMetric(@NonNull final String metricName) { final TimerMetric metric = new TimerMetric(metricName); metric.addTags(getBaseTags()); @@ -80,7 +110,22 @@ public TimerMetric createTimerMetric(@NonNull final String metricName, return metric; } - public void gaugeAll(final IReporter reporter) { + public static Map createAdditionalTags(@NonNull final String key, @NonNull final String val) { + final Map additionalTag = new HashMap<>(); + additionalTag.put(key, val); + return additionalTag; + } + + public void gauageFailureMetric(final IReporter reporter) { + this.failureMetricSet.forEach(reporter::gauge); + } + + public void guageNonFailureMetric(final IReporter reporter) { this.metricSet.forEach(reporter::gauge); } + + public void gaugeAll(final IReporter reporter) { + guageNonFailureMetric(reporter); + gauageFailureMetric(reporter); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java index 4e589f4..624b588 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java @@ -17,15 +17,17 @@ package com.uber.marmaray.common.metrics; import lombok.Getter; +import lombok.ToString; import org.hibernate.validator.constraints.NotEmpty; /** * {@link Double} implementation of {@link Metric} */ @Getter +@ToString public class DoubleMetric extends Metric { public DoubleMetric(final String metricName) { - super(metricName); + this("metric-type", 0.0); } public DoubleMetric(@NotEmpty final String metricName, final double metricValue) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/ErrorCauseTagNames.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ErrorCauseTagNames.java new file mode 100644 index 0000000..b942151 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ErrorCauseTagNames.java @@ -0,0 +1,85 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.metrics; + + import com.uber.marmaray.common.exceptions.JobRuntimeException; + +public final class ErrorCauseTagNames { + + public static final String FILE_PATH = "file_path"; + public static final String MEMORY = "memory"; + public static final String PERMISSON = "permission"; + public static final String SCHEMA = "schema"; + public static final String NO_DATA = "no_data"; + + public static final String ERROR = "error"; + public static final String CONFIG_ERROR = "config_error"; + + // FS + public static final String FS_UTIL = "fs_util"; + public static final String FS_UTIL_DELETE = "fs_util_delete"; + + // SOURCE + public static final String NO_FILE = "file_does_not_exist"; + public static final String EMPTY_PATH = "no_file_in_data_path"; + + // SINK + public static final String COMPRESSION = "compression"; + public static final String OUTPUT_FILE_FORMAT = "output_file_format"; + public static final String CALCULATE_SAMPLE_SIZE = "calculate_sample_size"; + public static final String WRITE_FILE = "write_file"; + public static final String WRITE_TO_SINK = "write_to_sink"; + public static final String WRITE_TO_SINK_CSV = "write_to_sink_csv"; + public static final String WRITE_TO_SINK_SEQ = "write_to_sink_sequence"; + public static final String WRITE_TO_CASSANDRA = "write_to_sink_cassandra"; + public static final String CONNECT_TO_SINK = "connect_to_sink"; + + //AWS + public static final String AWS = "aws"; + public static final String UPLOAD = "upload_to_s3"; + + // metadata + public static final String SAVE_METADATA = "saving_metadata"; + public static final String LOAD_METADATA = "loading_metadata"; + + // cassandra + public static final String CASSANDRA = "cassandra"; + public static final String CASSANDRA_QUERY = "cassandra_query"; + public static final String EXEC_CASSANDRA_CMD = "exec_cassandra_cmd"; + + //converter + public static final String DATA_CONVERSION = "data_conversion"; + public static final String NOT_SUPPORTED_FIELD_TYPE = "not_supported_field_type"; + public static final String MISSING_FIELD = "missing_field"; + public static final String MISSING_DATA = "missing_data"; + public static final String EMPTY_FIELD = "empty_field"; + + // schema manager + public static final String CLUSTER_KEY = "cluster_key_missing/not_found"; + public static final String PARTITION_KEY = "partition_key_missing/not_found"; + public static final String KEYSPACE = "keyspace_is_missing"; + public static final String TABLE = "table_name_is_missing"; + public static final String SCHEMA_FIELD = "schema_field_is_missing"; + public static final String PARTITION_CLUSTERING_KEYS_OVERLAP = "partition_clustering_keys_overlap"; + + // job manager + public static final String TIME_OUT = "time_out"; + + private ErrorCauseTagNames() { + throw new JobRuntimeException("Class should never be instantiated"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java index 1a9ee9b..b430458 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java @@ -17,12 +17,14 @@ package com.uber.marmaray.common.metrics; import lombok.Getter; +import lombok.ToString; import org.hibernate.validator.constraints.NotEmpty; /** * {@link Long} implementation of {@link Metric} */ @Getter +@ToString public class LongMetric extends Metric { public LongMetric(final String metricName) { super(metricName); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java index 4ed8df3..8e6c564 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java @@ -23,6 +23,7 @@ import lombok.NonNull; import org.hibernate.validator.constraints.NotEmpty; +import java.io.Serializable; import java.util.Map; import static java.util.Objects.isNull; @@ -33,7 +34,7 @@ * to a UtilTable (todo) */ @Getter -public abstract class Metric { +public abstract class Metric implements Serializable { @NotEmpty protected String metricName; protected T metricValue; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/ModuleTagNames.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ModuleTagNames.java new file mode 100644 index 0000000..b3371ae --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ModuleTagNames.java @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.metrics; + + import com.uber.marmaray.common.exceptions.JobRuntimeException; + +public final class ModuleTagNames { + public static final String SOURCE = "source"; + public static final String SINK = "sink"; + public static final String SCHEMA_MANAGER = "schema_manager"; + public static final String SOURCE_CONVERTER = "source_converter"; + public static final String SINK_CONVERTER = "sink_converter"; + public static final String SUB_DAG = "sub_dag"; + public static final String WORK_UNIT_CALCULATOR = "work_unit_calc"; + public static final String JOB_MANAGER = "job_manager"; + public static final String JOB_DAG = "job_dag"; + public static final String METADATA_MANAGER = "metadata_manager"; + public static final String SINK_CONFIGURATION = "sink_configuration"; + public static final String CONFIGURATION = "config"; + + private ModuleTagNames() { + throw new JobRuntimeException("Class should never be instantiated"); + } +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java index 1161236..ec14f97 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import lombok.Getter; import lombok.NonNull; +import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.hibernate.validator.constraints.NotEmpty; @@ -34,6 +35,7 @@ */ @Slf4j @Getter +@ToString public class TimerMetric extends LongMetric { private final Instant startTime; private Instant endTime; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/reporters/IKafkaDataLossReporter.java b/marmaray/src/main/java/com/uber/marmaray/common/reporters/IKafkaDataLossReporter.java new file mode 100644 index 0000000..cde2b7a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/IKafkaDataLossReporter.java @@ -0,0 +1,11 @@ +package com.uber.marmaray.common.reporters; + +import org.hibernate.validator.constraints.NotEmpty; + +/** + * {@link IKafkaDataLossReporter} reports Kafka data loss + */ +public interface IKafkaDataLossReporter { + void reportDataLoss(@NotEmpty final String kafkaTopicName, + final long totalNumberOfMessagesLost); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java b/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java index 0c23b88..ebbaeb4 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java @@ -19,6 +19,7 @@ import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.metrics.Metric; import lombok.Getter; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import java.util.Queue; @@ -48,6 +49,12 @@ public void report(final Metric m) { this.reporters.forEach(r -> r.gauge(m)); } + public void report(@NonNull final Metric ... metrics) { + for (Metric m : metrics) { + report(m); + } + } + public void finish() { if (!isFinished) { this.reporters.forEach(r -> r.finish()); diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraMetadataSchemaManager.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraMetadataSchemaManager.java new file mode 100644 index 0000000..5df9552 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraMetadataSchemaManager.java @@ -0,0 +1,98 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.schema.cassandra; + +import com.google.common.base.Optional; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.List; + +@Slf4j +public class CassandraMetadataSchemaManager extends CassandraSchemaManager { + + public CassandraMetadataSchemaManager(@NonNull final CassandraSchema schema, + @NonNull final List partitionKeys, + @NonNull final List clusteringKeys, + @NonNull final Optional ttl) { + super(schema, partitionKeys, clusteringKeys, ttl, Optional.absent(), Optional.absent(), false); + } + + /** + * @return + * Returns insert statement to add row + */ + public String generateInsertStmt(@NotEmpty final String key, @NonNull final String value) { + final String ttlStr = this.ttl.isPresent() ? "USING TTL " + this.ttl.get().toString() : StringTypes.EMPTY; + + return String.format("INSERT INTO %s.%s ( %s ) VALUES ( %s ) %s", + this.schema.getKeySpace(), + this.schema.getTableName(), + key, + value, + ttlStr); + } + + /** + * @return + * Delete command to remove a job + */ + public String generateDeleteJob(@NotEmpty final String key) { + return String.format("DELETE FROM %s.%s WHERE job='%s'", + this.schema.getKeySpace(), + this.schema.getTableName(), + key); + } + + /** + * @return + * Cassandra command to delete oldest checkpoint + */ + public String generateDeleteOldestCheckpoint(@NotEmpty final String key, + @NotEmpty final Optional oldestTimestamp) { + return String.format("DELETE FROM %s.%s WHERE job='%s' and time_stamp='%s'", + this.schema.getKeySpace(), + this.schema.getTableName(), + key, + oldestTimestamp.get()); + } + + /** + * @return + * Returns Select Cassandra command to query metadata manager + */ + public String generateSelectJob(@NotEmpty final String key, @NotEmpty final int limit) { + return String.format("SELECT * FROM %s.%s WHERE job='%s' LIMIT %s;", + this.schema.getKeySpace(), + this.schema.getTableName(), + key, + limit); + } + + /** + * @return + * Returns drop table cassandra command + * this resets our metadata manager + */ + public String generateDropTable() { + return String.format("DROP TABLE %s.%s", + this.schema.getKeySpace(), + this.schema.getTableName()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.java index 3161f2d..18ab0b5 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.java @@ -19,11 +19,11 @@ import com.google.common.base.Joiner; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.utilities.GenericRecordUtil; +import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.StringTypes; import lombok.AllArgsConstructor; import lombok.Getter; import org.apache.avro.Schema; -import org.apache.spark.sql.execution.columnar.LONG; import org.hibernate.validator.constraints.NotEmpty; import java.io.Serializable; @@ -36,6 +36,8 @@ public final class CassandraSchemaField implements Serializable { public static final String FLOAT_TYPE = "float"; public static final String BOOLEAN_TYPE = "boolean"; public static final String DOUBLE_TYPE = "double"; + public static final String TIMESTAMP_TYPE = "timestamp"; + public static final String BINARY_TYPE = "blob"; private static final Joiner joiner = Joiner.on(StringTypes.COMMA); @@ -57,6 +59,10 @@ public static String convertFromAvroType(final Schema schema) { : schema; final Schema.Type type = nonNullSchema.getType(); + if (SchemaUtil.isTimestampSchema(schema)) { + return TIMESTAMP_TYPE; + } + switch (type) { case STRING: return STRING_TYPE; @@ -70,6 +76,8 @@ public static String convertFromAvroType(final Schema schema) { return BOOLEAN_TYPE; case DOUBLE: return DOUBLE_TYPE; + case BYTES: + return BINARY_TYPE; default: // todo T935985: Support more complex types from Schema.Type throw new JobRuntimeException("Type " + type + " is not supported for conversion. Field(s): " diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaManager.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaManager.java new file mode 100644 index 0000000..1a57d73 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaManager.java @@ -0,0 +1,394 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.schema.cassandra; + +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.common.schema.ISinkSchemaManager; +import com.uber.marmaray.utilities.StringTypes; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@Slf4j +public class CassandraSchemaManager implements ISinkSchemaManager, Serializable, IMetricable { + private static final Joiner joiner = Joiner.on(",").skipNulls(); + private static final String COLUMN_NAME = "column_name"; + private static final String TYPE = "type"; + private static final String SYSTEM_SCHEMA_COLS = "system_schema.columns"; + private static final String KEYSPACE_NAME = "keyspace_name"; + + @Getter + protected transient Optional dataFeedMetrics = Optional.absent(); + + @Getter + final Boolean validSchema; + + /** + * todo - consider including partition & clustering keys in schema itself. This isn't done here + * because it gives us greater flexibility to have different partitioning/clustering schemes + * based on a single set of fields in Cassandra Schema. We also have validation code to ensure all + * partitioning/clustering keys are actually a valid field. Keeping this open to change in the future. + */ + @Getter + final CassandraSchema schema; + final List partitionKeys; + final List clusteringKeys; + final Optional ttl; + final Optional timestamp; + final Boolean orderRequired; + + // todo T928813: consider adding support for index creation + + public CassandraSchemaManager(@NonNull final CassandraSchema schema, + @NonNull final List partitionKeys, + @NonNull final List clusteringKeys, + @NonNull final Optional ttl, + @NonNull final Optional dataFeedMetrics, + @NonNull final Optional timestamp, + final boolean orderRequired) { + this.schema = schema; + this.partitionKeys = partitionKeys; + this.clusteringKeys = clusteringKeys; + this.ttl = ttl; + this.dataFeedMetrics = dataFeedMetrics; + this.validSchema = validateSchema(); + this.timestamp = timestamp; + this.orderRequired = orderRequired; + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + // set during construction + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + public String generateCreateTableStmt() { + final String fields = generateFieldsSyntax(); + final String primaryKeyStmt = generatePrimaryKeySyntax(); + final String clusteringOrder = generateClusteringOrderSyntax(); + final String createStatement = String.format("CREATE TABLE IF NOT EXISTS %s.%s (%s, %s) %s", + this.schema.getKeySpace(), + this.schema.getTableName(), + fields, + primaryKeyStmt, + clusteringOrder); + log.info("Generated table schema is {}", createStatement); + return createStatement; + } + + /** + * The schema is the source of truth here and if any of the schema field names aren't found + * in the existing columns we need to alter that table and update the schema + * + * @param existingColumns + * @return List of strings, one for each column that needs to be added + */ + public List generateAlterTableStmt(final Map existingColumns) { + final Map fields = this.schema.getFields().stream() + .collect(Collectors.toMap(CassandraSchemaField::getFieldName, CassandraSchemaField::getType)); + log.info("Existing field names in schema: {}", Arrays.toString(fields.keySet().toArray())); + final List missingCols = new ArrayList<>(); + fields.forEach((fieldName, fieldType) -> { + if (existingColumns.containsKey(fieldName)) { + if (!existingColumns.get(fieldName).equals(fieldType)) { + throw new IllegalStateException(String.format( + "Incompatible field types for %s: sink has %s and source has %s", + fieldName, existingColumns.get(fieldName), fieldType)); + } + } else { + missingCols.add(fieldName); + } + } + ); + log.info("Missing columns (if any): {}", Arrays.toString(missingCols.toArray())); + return this.schema.getFields().stream() + .filter(field -> missingCols.contains(field.getFieldName())) + .map(field -> String.format("ALTER TABLE %s.%s ADD %s", + this.schema.getKeySpace(), this.schema.getTableName(), field.toString())) + .collect(Collectors.toList()); + } + + /** + * @return + * Insert statement used for sstable loading using the fields and ttl value + */ + public String generateInsertStmt() { + final String fields = this.schema.getFields() + .stream() + .map(f -> StringTypes.SPACE + f.getFieldName()) + .collect(Collectors.joining(",")); + + final String values = this.schema.getFields() + .stream() + .map(f -> "?") + .collect(Collectors.joining(",")); + + final String ttlStr = this.ttl.isPresent() ? "USING TTL " + this.ttl.get().toString() : StringTypes.EMPTY; + + if (this.orderRequired) { + final String timestampStr = (this.ttl.isPresent() ? " AND " : " USING ") + "TIMESTAMP "; + + return String.format("INSERT INTO %s.%s ( %s ) VALUES ( %s ) %s %s ?", + this.schema.getKeySpace(), + this.schema.getTableName(), + fields, + values, + ttlStr, + timestampStr); + } else { + final String timestampStr = this.timestamp.isPresent() + ? ((this.ttl.isPresent() ? " AND " : " USING ") + "TIMESTAMP " + this.timestamp.get().toString()) + : StringTypes.EMPTY; + + return String.format("INSERT INTO %s.%s ( %s ) VALUES ( %s ) %s %s", + this.schema.getKeySpace(), + this.schema.getTableName(), + fields, + values, + ttlStr, + timestampStr); + } + } + + /** + * @return + * Returns all the column names for a specific Cassandra table + */ + public String getColumnsFromTableQuery() { + return String.format("SELECT %s, %s FROM %s WHERE %s = '%s' " + + "AND table_name = '%s'", + COLUMN_NAME, + TYPE, + SYSTEM_SCHEMA_COLS, + KEYSPACE_NAME, + this.schema.getKeySpace(), + this.schema.getTableName()); + } + + /** + * @return + * Returns all the column names for a specific column family + */ + public String getColumnsFromColumnFamilyQuery() { + return String.format("SELECT %s, %s FROM %s WHERE %s = '%s' " + + "AND columnfamily_name = '%s'", + COLUMN_NAME, + TYPE, + SYSTEM_SCHEMA_COLS, + KEYSPACE_NAME, + this.schema.getKeySpace(), + this.schema.getTableName()); + } + + /** + * @return + * Returns a comma separated list of all the fields in the schema + */ + private String generateFieldsSyntax() { + final List fields = this.schema.getFields().stream() + .map(field -> field.toString()).collect(Collectors.toList()); + return joiner.join(fields); + } + + /** + * @return + * Returns the syntax to define a primary key in a create table statement + */ + private String generatePrimaryKeySyntax() { + final List clusterKeyNames = + this.clusteringKeys.stream().map(key -> key.name).collect(Collectors.toList()); + return String.format("PRIMARY KEY ((%s)%s)", joiner.join(this.partitionKeys), + clusterKeyNames.isEmpty() ? StringTypes.EMPTY : "," + joiner.join(clusterKeyNames)); + } + + /** + * @return + * Returns the optional clustering order syntax if it needs to be created. The clustering key should + * also define if it is in ascending (default) or descending order + */ + private String generateClusteringOrderSyntax() { + if (this.clusteringKeys.isEmpty()) { + return StringTypes.EMPTY; + } + + final List clusterOrder = + this.clusteringKeys.stream().map(key -> key.toString()).collect(Collectors.toList()); + + return String.format("WITH CLUSTERING ORDER BY (%s)", joiner.join(clusterOrder)); + } + + /** + * Validates all the private member variables and ensures that we can successfully create a cassandra table + * If any of the conditions are not met we throw an exception + *

+ * We assume that field names are case insensitive so we handle that accordingly + * + * @throws IllegalStateException + */ + private Boolean validateSchema() { + + try { + Preconditions.checkState(!Strings.isNullOrEmpty(this.schema.getKeySpace())); + } catch (Exception e) { + log.error("Keyspace is missing"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.KEYSPACE)); + } + return false; + } + + try { + Preconditions.checkState(!Strings.isNullOrEmpty(this.schema.getTableName())); + } catch (Exception e) { + log.error("Table name is missing"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.TABLE)); + } + return false; + } + + try { + Preconditions.checkState(this.schema.getFields() != null && !this.schema.getFields().isEmpty()); + } catch (Exception e) { + log.error("Schema fields missing"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.SCHEMA_FIELD)); + } + return false; + } + + try { + Preconditions.checkState(this.partitionKeys != null && !this.partitionKeys.isEmpty()); + } catch (Exception e) { + log.error("Partition key(s) missing"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.PARTITION_KEY)); + } + return false; + } + + try { + Preconditions.checkNotNull(this.clusteringKeys); + } catch (Exception e) { + log.error("Clustering keys is null"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.CLUSTER_KEY)); + } + return false; + } + + final List lowerCasePartitionKeys = this.partitionKeys + .stream() + .map(p -> p.toLowerCase()) + .collect(Collectors.toList()); + + // Partition keys and clustering keys should always be completely independent lists + final List clusterKeyNames = + this.clusteringKeys + .stream() + .map(key -> key.name.toLowerCase()) + .collect(Collectors.toList()); + + final List duplicateKeys = + lowerCasePartitionKeys + .stream() + .filter(p -> clusterKeyNames.contains(p.toLowerCase())) + .collect(Collectors.toList()); + try { + Preconditions.checkState(duplicateKeys.isEmpty(), "Partition and clustering keys should have no overlap"); + } catch (Exception e) { + log.error("Partition and clustering keys should have no overlap"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.PARTITION_CLUSTERING_KEYS_OVERLAP)); + } + return false; + } + + // Each partition key should be found in fields + final List fieldNames = + this.schema.getFields() + .stream() + .map(f -> f.getFieldName().toLowerCase()) + .collect(Collectors.toList()); + + log.info("Field names found: {}", fieldNames.size()); + fieldNames.stream().forEach(f -> log.info("Schema field: {}", f)); + + try { + Preconditions.checkState(fieldNames.containsAll(lowerCasePartitionKeys)); + } catch (Exception e) { + log.error("One or more of your partition keys (%s) were not found in the available schema fields (%s)", + lowerCasePartitionKeys, fieldNames); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.PARTITION_KEY)); + } + return false; + } + + // Each clustering key should also be found in fields + if (!this.clusteringKeys.isEmpty()) { + try { + Preconditions.checkState(fieldNames.containsAll(clusterKeyNames)); + } catch (Exception e) { + log.error("Clustering keys not found in field names to disperse"); + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SCHEMA_MANAGER, ErrorCauseTagNames.CLUSTER_KEY)); + } + return false; + } + } + return true; + } + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java index f7f8e1f..e9ad4a4 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java @@ -16,239 +16,29 @@ */ package com.uber.marmaray.common.schema.cassandra; -import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.uber.marmaray.common.schema.ISinkSchemaManager; -import com.uber.marmaray.utilities.StringTypes; -import lombok.Getter; +import com.uber.marmaray.common.metrics.DataFeedMetrics; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; - -import java.io.Serializable; -import java.util.Arrays; import java.util.List; -import java.util.stream.Collectors; @Slf4j -public class CassandraSinkSchemaManager implements ISinkSchemaManager, Serializable { - private static final Joiner joiner = Joiner.on(",").skipNulls(); - private static final String COLUMN_NAME = "column_name"; - private static final String SYSTEM_SCHEMA_COLS = "system_schema.columns"; - private static final String KEYSPACE_NAME = "keyspace_name"; - - /** - * todo - consider including partition & clustering keys in schema itself. This isn't done here - * because it gives us greater flexibility to have different partitioning/clustering schemes - * based on a single set of fields in Cassandra Schema. We also have validation code to ensure all - * partitioning/clustering keys are actually a valid field. Keeping this open to change in the future. - */ - @Getter - final CassandraSchema schema; - final List partitionKeys; - final List clusteringKeys; - final Optional ttl; - - // todo T928813: consider adding support for index creation +public class CassandraSinkSchemaManager extends CassandraSchemaManager { public CassandraSinkSchemaManager(@NonNull final CassandraSchema schema, @NonNull final List partitionKeys, @NonNull final List clusteringKeys, - @NonNull final Optional ttl) { - this.schema = schema; - this.partitionKeys = partitionKeys; - this.clusteringKeys = clusteringKeys; - this.ttl = ttl; - - validateSchema(); + @NonNull final Optional ttl, + @NonNull final Optional dataFeedMetrics, + @NonNull final Optional timestamp, + final boolean orderRequired) { + super(schema, partitionKeys, clusteringKeys, ttl, dataFeedMetrics, timestamp, orderRequired); } public CassandraSinkSchemaManager(@NonNull final CassandraSchema schema, @NonNull final List partitionKeys, - @NonNull final List clusteringKeys) { - this(schema, partitionKeys, clusteringKeys, Optional.absent()); - } - - public String generateCreateTableStmt() { - final String fields = generateFieldsSyntax(); - final String primaryKeyStmt = generatePrimaryKeySyntax(); - final String clusteringOrder = generateClusteringOrderSyntax(); - final String createStatement = String.format("CREATE TABLE IF NOT EXISTS %s.%s (%s, %s) %s", - this.schema.getKeySpace(), - this.schema.getTableName(), - fields, - primaryKeyStmt, - clusteringOrder); - log.info("Generated table schema is {}", createStatement); - return createStatement; - } - - /** - * The schema is the source of truth here and if any of the schema field names aren't found - * in the existing columns we need to alter that table and update the schema - * - * @param existingColumns - * @return List of strings, one for each column that needs to be added - */ - public List generateAlterTableStmt(final List existingColumns) { - final List fieldNames = this.schema.getFields().stream() - .map(field -> field.getFieldName()).collect(Collectors.toList()); - log.info("Existing field names in schema: {}", Arrays.toString(fieldNames.toArray())); - final List missingCols = fieldNames.stream() - .filter(field -> !existingColumns.contains(field)) - .collect(Collectors.toList()); - log.info("Missing columns (if any): {}", Arrays.toString(missingCols.toArray())); - return this.schema.getFields().stream() - .filter(field -> missingCols.contains(field.getFieldName())) - .map(field -> String.format("ALTER TABLE %s.%s ADD %s", - this.schema.getKeySpace(), this.schema.getTableName(), field.toString())) - .collect(Collectors.toList()); - } - - /** - * @return - * Insert statement used for sstable loading using the fields and ttl value - */ - public String generateInsertStmt() { - final String fields = this.schema.getFields() - .stream() - .map(f -> StringTypes.SPACE + f.getFieldName()) - .collect(Collectors.joining(",")); - - final String values = this.schema.getFields() - .stream() - .map(f -> "?") - .collect(Collectors.joining(",")); - - final String ttlStr = this.ttl.isPresent() ? "USING TTL " + this.ttl.get().toString() : StringTypes.EMPTY; - - return String.format("INSERT INTO %s.%s ( %s ) VALUES ( %s ) %s", - this.schema.getKeySpace(), - this.schema.getTableName(), - fields, - values, - ttlStr); - } - - /** - * @return - * Returns all the column names for a specific Cassandra table - */ - public String getColumnNamesFromTableQuery() { - return String.format("SELECT %s FROM %s WHERE %s = '%s' " - + "AND table_name = '%s'", - COLUMN_NAME, - SYSTEM_SCHEMA_COLS, - KEYSPACE_NAME, - this.schema.getKeySpace(), - this.schema.getTableName()); - } - - /** - * @return - * Returns all the column names for a specific column family - */ - public String getColumnNamesFromColumnFamilyQuery() { - return String.format("SELECT %s FROM %s WHERE %s = '%s' " - + "AND columnfamily_name = '%s'", - COLUMN_NAME, - SYSTEM_SCHEMA_COLS, - KEYSPACE_NAME, - this.schema.getKeySpace(), - this.schema.getTableName()); - } - - /** - * @return - * Returns a comma separated list of all the fields in the schema - */ - private String generateFieldsSyntax() { - final List fields = this.schema.getFields().stream() - .map(field -> field.toString()).collect(Collectors.toList()); - return joiner.join(fields); - } - - /** - * @return - * Returns the syntax to define a primary key in a create table statement - */ - private String generatePrimaryKeySyntax() { - final List clusterKeyNames = - this.clusteringKeys.stream().map(key -> key.name).collect(Collectors.toList()); - return String.format("PRIMARY KEY ((%s)%s)", joiner.join(this.partitionKeys), - clusterKeyNames.isEmpty() ? StringTypes.EMPTY : "," + joiner.join(clusterKeyNames)); - } - - /** - * @return - * Returns the optional clustering order syntax if it needs to be created. The clustering key should - * also define if it is in ascending (default) or descending order - */ - private String generateClusteringOrderSyntax() { - if (this.clusteringKeys.isEmpty()) { - return StringTypes.EMPTY; - } - - final List clusterOrder = - this.clusteringKeys.stream().map(key -> key.toString()).collect(Collectors.toList()); - - return String.format("WITH CLUSTERING ORDER BY (%s)", joiner.join(clusterOrder)); - } - - /** - * Validates all the private member variables and ensures that we can successfully create a cassandra table - * If any of the conditions are not met we throw an exception - *

- * We assume that field names are case insensitive so we handle that accordingly - * - * @throws IllegalStateException - */ - private void validateSchema() { - Preconditions.checkState(!Strings.isNullOrEmpty(this.schema.getKeySpace()), "Keyspace is missing"); - Preconditions.checkState(!Strings.isNullOrEmpty(this.schema.getTableName()), "Table name is missing"); - Preconditions.checkState(this.schema.getFields() != null && !this.schema.getFields().isEmpty(), - "Schema fields missing"); - Preconditions.checkState(this.partitionKeys != null && !this.partitionKeys.isEmpty(), - "Partition key(s) missing"); - Preconditions.checkNotNull(this.clusteringKeys, "Clustering keys is null"); - - final List lowerCasePartitionKeys = this.partitionKeys - .stream() - .map(p -> p.toLowerCase()) - .collect(Collectors.toList()); - - // Partition keys and clustering keys should always be completely independent lists - final List clusterKeyNames = - this.clusteringKeys - .stream() - .map(key -> key.name.toLowerCase()) - .collect(Collectors.toList()); - - final List duplicateKeys = - lowerCasePartitionKeys - .stream() - .filter(p -> clusterKeyNames.contains(p.toLowerCase())) - .collect(Collectors.toList()); - Preconditions.checkState(duplicateKeys.isEmpty(), "Partition and clustering keys should have no overlap"); - - // Each partition key should be found in fields - final List fieldNames = - this.schema.getFields() - .stream() - .map(f -> f.getFieldName().toLowerCase()) - .collect(Collectors.toList()); - - log.info("Field names found: {}", fieldNames.size()); - fieldNames.stream().forEach(f -> log.info("Schema field: {}", f)); - - Preconditions.checkState(fieldNames.containsAll(lowerCasePartitionKeys), - "One or more of your partition keys were not found in the available schema fields"); - - // Each clustering key should also be found in fields - if (!clusteringKeys.isEmpty()) { - Preconditions.checkState(fieldNames.containsAll(clusterKeyNames), - "Clustering keys not found in field names to disperse"); - } + @NonNull final List clusteringKeys, + @NonNull final Optional timestamp) { + super(schema, partitionKeys, clusteringKeys, Optional.absent(), Optional.absent(), timestamp, false); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java index 0c208f8..ca82844 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java @@ -45,7 +45,7 @@ public enum Order { } public String toString() { - return name + StringTypes.SPACE + order; + return this.name + StringTypes.SPACE + this.order; } /* @@ -61,5 +61,9 @@ public static ClusterKey parse(@NotEmpty final String value) { return new ClusterKey(tokens.get(0), Order.valueOf(tokens.get(1))); } } + + public boolean isClusterKeyColumn(@NotEmpty final String name) { + return this.name.equals(name); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java index 8125bc6..ca72718 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java @@ -20,10 +20,6 @@ import com.uber.marmaray.common.metadata.IMetadataManager; import com.uber.marmaray.common.metadata.StringValue; import com.uber.marmaray.utilities.MapUtil; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.Map; -import java.util.Queue; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NonNull; @@ -31,6 +27,11 @@ import lombok.extern.slf4j.Slf4j; import org.hibernate.validator.constraints.NotEmpty; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; + /** * Convenience class for managing {@link SinkStat} for {@link #MAX_HISTORY_SIZE} runs. */ @@ -39,7 +40,7 @@ @ToString public class SinkStatManager { - public static final int MAX_HISTORY_SIZE = 256; + public static final int MAX_HISTORY_SIZE = 8; public static final String METAKEY = "sink-stat-%s"; @Getter @@ -97,6 +98,7 @@ public long getAvgRecordSize() { numEntries += 1; } } + log.info("tableName:{}:avgRecordSize:{}:numEntries:{}", this.tableName, avgRecordSize, numEntries); return avgRecordSize / Math.max(numEntries, 1); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java index 1db8edf..9167514 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java @@ -25,16 +25,15 @@ import com.uber.marmaray.common.data.RDDWrapper; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; -import com.uber.marmaray.utilities.StringTypes; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; -import java.util.HashMap; -import java.util.Map; - /** * {@link CassandraClientSink} implements the {@link CassandraSink} interface for a Cassandra sink. The AvroPayload RDD * will be forked into valid & invalid records based on set criteria. @@ -53,6 +52,12 @@ public CassandraClientSink(@NonNull final CassandraSinkCQLDataConverter converte this.converter = converter; } + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + super.setDataFeedMetrics(dataFeedMetrics); + this.converter.setDataFeedMetrics(dataFeedMetrics); + } + @Override public void write(@NonNull final JavaRDD data) { @@ -79,40 +84,39 @@ public void write(@NonNull final JavaRDD data) { * failure and thinking the dispersal job has succeeded when no data has actually been dispersed and * checkpoints have been persisted to indicate the partition was successfully dispersed. */ - final String errMsg = String.join("No data was found to disperse. As a safeguard, we are failing " + if (this.tableMetrics.isPresent()) { + this.tableMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, DataFeedMetricNames + .getErrorModuleCauseTags(ModuleTagNames.SINK, ErrorCauseTagNames.NO_DATA)); + } + final String errMsg = String.join("" + , "No data was found to disperse. As a safeguard, we are failing " , "the job explicitly. Please check your data and ensure records are valid and " , "partition and clustering keys are populated. If your partition has empty data you will have to " , "delete it to proceed. Otherwise, please contact your support team for troubleshoooting"); throw new JobRuntimeException(errMsg); } - // TODO: Figure out how to calculate the data size generated from CQL statements. - if (this.tableMetrics.isPresent()) { - final Map tags = new HashMap<>(); - tags.put(TABLE_NAME_TAG, this.conf.getKeyspace() + StringTypes.UNDERSCORE + this.conf.getTableName()); - this.tableMetrics.get() - .createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, payloadWrapper.getCount(), tags); - } - - final String clusterName = this.conf.getClusterName(); final String keyspaceName = this.conf.getKeyspace(); - JavaRDD writtenRdd = payloadWrapper.getData().mapPartitions(iter -> { - final Cluster.Builder builder = Cluster.builder().withClusterName(clusterName); - if (this.conf.getNativePort().isPresent()) { - builder.withPort(Integer.parseInt(this.conf.getNativePort().get())); - } else { - builder.withPort(Integer.parseInt(CassandraSinkConfiguration.DEFAULT_OUTPUT_NATIVE_PORT)); - } - this.conf.getInitialHosts().forEach(builder::addContactPoint); + final JavaRDD writtenRdd = payloadWrapper.getData().mapPartitions( + iter -> { + final Cluster.Builder builder = this.getBuilder(this.conf.getHadoopConf()); try (final Cluster cluster = builder.build(); final Session session = cluster.connect(keyspaceName)) { while (iter.hasNext()) { - Statement statement = iter.next(); + final Statement statement = iter.next(); try { session.execute(statement); } catch (Exception e) { + if (this.tableMetrics.isPresent()) { + this.tableMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, + 1, + DataFeedMetricNames.getErrorModuleCauseTags(ModuleTagNames.SINK, + ErrorCauseTagNames.EXEC_CASSANDRA_CMD)); + } log.error("Exception: {}", e); throw new JobRuntimeException(e); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java index 901b0cc..89ac723 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java @@ -16,16 +16,29 @@ */ package com.uber.marmaray.common.sinks.cassandra; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; +import com.google.common.primitives.Ints; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.converters.data.CassandraSinkDataConverter; +import com.uber.marmaray.common.data.ForkData; import com.uber.marmaray.common.data.RDDWrapper; import com.uber.marmaray.common.exceptions.JobRuntimeException; -import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.forkoperator.ForkFunction; +import com.uber.marmaray.common.forkoperator.ForkOperator; +import com.uber.marmaray.common.metrics.CassandraMetric; import com.uber.marmaray.common.metrics.CassandraPayloadRDDSizeEstimator; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.common.schema.cassandra.CassandraDataField; import com.uber.marmaray.common.schema.cassandra.CassandraPayload; import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; +import com.uber.marmaray.common.schema.cassandra.ClusterKey; +import com.uber.marmaray.utilities.ByteBufferUtil; +import com.uber.marmaray.utilities.SizeUnit; import com.uber.marmaray.utilities.StringTypes; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -36,18 +49,25 @@ import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; +import parquet.Preconditions; import scala.Tuple2; +import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; -import java.util.HashMap; +import java.util.Collections; import java.util.List; -import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.cassandra.hadoop.cql3.CqlBulkRecordWriter.IGNORE_HOSTS; /** * {@link CassandraSSTableSink} extends the {@link CassandraSink} class for a Cassandra sink. The AvroPayload RDD * will be forked into valid & invalid records based on set criteria. - * + *

* Valid records will then be written to the Cassandra backend */ @Slf4j @@ -70,15 +90,34 @@ public CassandraSSTableSink(@NonNull final CassandraSinkDataConverter converter, this.converter = converter; } + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + log.info("set datafeed metric + converter"); + super.setDataFeedMetrics(dataFeedMetrics); + this.converter.setDataFeedMetrics(dataFeedMetrics); + } + @Override public void write(@NonNull final JavaRDD data) { final Configuration hadoopConf = this.conf.getHadoopConf(); + setIgnoredHosts(hadoopConf); + log.info("Setting up Cassandra Table"); this.setupCassandraTable(hadoopConf); log.info("Converting data to cassandra payload"); - final RDDWrapper payloadWrapper = this.converter.map(data); + final RDDWrapper payloadWrapper; + if (this.conf.isBatchEnabled()) { + payloadWrapper = this.sort(this.converter.map(data)); + } else { + payloadWrapper = this.converter.map(data); + } + + final CassandraPayloadRDDSizeEstimator cassandraPayloadRddSizeEstimator = + new CassandraPayloadRDDSizeEstimator(); + + final long estimatedPayloadSize = cassandraPayloadRddSizeEstimator.estimateTotalSize(payloadWrapper); if (payloadWrapper.getCount() == 0) { /* @@ -93,7 +132,13 @@ public void write(@NonNull final JavaRDD data) { * failure and thinking the dispersal job has succeeded when no data has actually been dispersed and * checkpoints have been persisted to indicate the partition was successfully dispersed. */ - final String errMsg = String.join("No data was found to disperse. As a safeguard, we are failing " + if (this.tableMetrics.isPresent()) { + this.tableMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, DataFeedMetricNames + .getErrorModuleCauseTags(ModuleTagNames.SINK, ErrorCauseTagNames.NO_DATA)); + } + final String errMsg = String.join("" + , "No data was found to disperse. As a safeguard, we are failing " , "the job explicitly. Please check your data and ensure records are valid and " , "partition and clustering keys are populated. If your partition has empty data you will have to " , "delete it to proceed. Otherwise, please contact your support team for troubleshoooting"); @@ -101,21 +146,69 @@ public void write(@NonNull final JavaRDD data) { } if (this.tableMetrics.isPresent()) { - final Map tags = new HashMap<>(); - tags.put(TABLE_NAME_TAG, this.conf.getKeyspace() + StringTypes.UNDERSCORE + this.conf.getTableName()); - this.tableMetrics.get() - .createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, payloadWrapper.getCount(), tags); - - final CassandraPayloadRDDSizeEstimator cassandraPayloadRddSizeEstimator = - new CassandraPayloadRDDSizeEstimator(); this.tableMetrics.get() .createLongMetric(DataFeedMetricNames.OUTPUT_BYTE_SIZE, - cassandraPayloadRddSizeEstimator.estimateTotalSize(payloadWrapper), - tags); + estimatedPayloadSize, + CassandraMetric.createTableNameTags(this.conf)); } - final JavaRDD cassandraRecords = payloadWrapper.getData(); + if (this.conf.isBatchEnabled()) { + final int numBatches = getNumBatches(estimatedPayloadSize); + final long recordsPerBatch = (long) Math.ceil((double) payloadWrapper.getCount() / numBatches); + log.info("Running with {} batches of {} records", numBatches, recordsPerBatch); + final JavaRDD> indexedRdd = + payloadWrapper.getData().zipWithIndex().map(r -> r); + final ForkFunction> forkFunction = + new ForkFunction>() { + @Override + protected List>> process( + final Tuple2 record) { + return Collections.singletonList(new ForkData<>( + Collections.singletonList((int) (record._2().intValue() / recordsPerBatch)), record)); + } + }; + final ForkOperator> forkOperator = + new ForkOperator<>(indexedRdd, forkFunction, this.conf.getConf()); + final List batchList = IntStream.range(0, numBatches).boxed().collect(Collectors.toList()); + final long sleepDurationMillis = TimeUnit.SECONDS.toMillis(this.conf.getMinBatchDurationSeconds()); + forkFunction.registerKeys(batchList); + forkOperator.execute(); + batchList.forEach( + index -> { + log.info("Processing batch {}", index); + final long startTime = System.currentTimeMillis(); + final JavaRDD> indexedFork = forkOperator.getRDD(index); + final JavaRDD indexedData = indexedFork.map(Tuple2::_1); + saveCassandraPayloads(hadoopConf, indexedData.coalesce(1)); + final long duration = System.currentTimeMillis() - startTime; + final long millisToSleep = Math.max(0, sleepDurationMillis - duration); + log.info("Complete, sleeping for {} ms", millisToSleep); + try { + Thread.sleep(millisToSleep); + } catch (InterruptedException e) { + throw new JobRuntimeException("Interrupted while sleeping between batches", e); + } + } + ); + } else { + saveCassandraPayloads(hadoopConf, payloadWrapper.getData()); + } + } + + @VisibleForTesting + int getNumBatches(final double estimatedPayloadSize) { + Preconditions.checkState(this.conf.isBatchEnabled(), "Batching isn't enabled; no batches to get."); + final int numBatches = + (int) Math.ceil(estimatedPayloadSize / SizeUnit.MEGABYTES.toBytes(this.conf.getMaxBatchSizeMb())); + // account for large disparity in the records + return Math.max(1, numBatches); + } + /* + * Actually save the cassandra payloads per batch + */ + private void saveCassandraPayloads(@NonNull final Configuration hadoopConf, + @NonNull final JavaRDD cassandraRecords) { log.info("Initializing system properties"); this.initSystemProperties(cassandraRecords); @@ -130,22 +223,107 @@ public void write(@NonNull final JavaRDD data) { cassandraRecords.mapToPair(f -> new Tuple2<>(generateDummyValue(), f.convertData())); log.info("Creating and saving sstables"); - byteBufferRDDs.saveAsNewAPIHadoopFile(TMP_FILE_PATH, - keyClass, - // Due to type erasure of parameterized types just use List here - List.class, - outputFormatClass, - hadoopConf); - + try { + byteBufferRDDs.saveAsNewAPIHadoopFile(TMP_FILE_PATH, + keyClass, + // Due to type erasure of parameterized types just use List here + List.class, + outputFormatClass, + hadoopConf); + } catch (Exception e) { + if (this.tableMetrics.isPresent()) { + this.tableMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, DataFeedMetricNames + .getErrorModuleCauseTags(ModuleTagNames.SINK, ErrorCauseTagNames.WRITE_TO_CASSANDRA)); + } + log.error("Error while writing to cassandra SSTable", e); + throw new JobRuntimeException(e); + } log.info("Finished write process"); } + @VisibleForTesting + void setIgnoredHosts(final Configuration hadoopConf) { + if (this.conf.isEnableIgnoreHosts()) { + final String ignoredHosts = computeIgnoredHosts(); + log.info("Ignoring hosts: {}", ignoredHosts); + hadoopConf.set(IGNORE_HOSTS, ignoredHosts); + } + } + + /** + * Method to compute a list of hosts to ignore, in order to not ship the SSTables to those hosts + * @return comma-separated string of hosts + */ + protected String computeIgnoredHosts() { + return StringTypes.EMPTY; + } + + private RDDWrapper sort(@NonNull final RDDWrapper wrapper) { + final JavaRDD data = wrapper.getData(); + final CassandraPayload firstRecord = data.first(); + final List sortOrder = computeSortOrder(firstRecord); + final JavaRDD sortedData = data.sortBy( + cassandraPayload -> { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final List values = cassandraPayload.convertData(); + sortOrder.forEach( + i -> { + try { + baos.write(values.get(i).array()); + } catch (IOException e) { + throw new JobRuntimeException("Unable to produce sort key for record", e); + } + } + ); + return ByteBuffer.wrap(baos.toByteArray()); + }, + true, + data.getNumPartitions()); + return new RDDWrapper<>(sortedData, wrapper.getCount()); + } + + /** + * Determine which numerical fields should be sorted, and in which order so we can quickly access the fields + * from the CassandraPayload + * + * Order should be partition keys, followed by clustering keys. If the example parquet has + * a, b, c, d, e fields, and the partition keys are c, a and the clustering keys are e, b, should return + * {2, 0, 4, 1} so we can quickly + * @param record sample record we can use to determine key ordering + * @return List of indicies of the ByteBuffers to use for sorting. + */ + @VisibleForTesting + List computeSortOrder(@NonNull final CassandraPayload record) { + // map sort order -> index in fields list + final int[] sortOrder = new int[this.conf.getPartitionKeys().size() + this.conf.getClusteringKeys().size()]; + final int partitionKeySize = this.conf.getPartitionKeys().size(); + final List fields = record.getData(); + for (int i = 0; i < fields.size(); i++) { + final String columnName = ByteBufferUtil.convertToString(fields.get(i).getColumnKey()); + final int partitionKeyIndex = this.conf.getPartitionKeys().indexOf(columnName); + if (partitionKeyIndex != -1) { + sortOrder[partitionKeyIndex] = i; + } else { + final List clusterKeys = this.conf.getClusteringKeys(); + for (int j = 0; j < this.conf.getClusteringKeys().size(); j++) { + if (clusterKeys.get(j).isClusterKeyColumn(columnName)) { + sortOrder[j + partitionKeySize] = i; + break; + } + } + } + } + return Ints.asList(sortOrder); + } + private void initSystemProperties(@NonNull final JavaRDD cassandraRecords) { if (this.conf.getStoragePort().isPresent() || this.conf.getSSLStoragePort().isPresent()) { final Optional storagePort = this.conf.getStoragePort(); final Optional sslStoragePort = this.conf.getSSLStoragePort(); - cassandraRecords.foreach(cr -> { + cassandraRecords.foreachPartition( + partition -> { if (storagePort.isPresent()) { System.setProperty("cassandra.storage_port", storagePort.get()); } @@ -153,7 +331,8 @@ private void initSystemProperties(@NonNull final JavaRDD cassa if (sslStoragePort.isPresent()) { System.setProperty("cassandra.ssl_storage_port", sslStoragePort.get()); } - }); + } + ); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java index 82ad7cd..57fc4a5 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java @@ -20,11 +20,15 @@ import com.datastax.driver.core.ResultSet; import com.datastax.driver.core.Session; import com.datastax.driver.core.exceptions.InvalidQueryException; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Strings; import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; import com.uber.marmaray.common.sinks.ISink; import lombok.NonNull; @@ -32,9 +36,11 @@ import org.apache.cassandra.hadoop.ConfigHelper; import org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat; import org.apache.hadoop.conf.Configuration; +import org.hibernate.validator.constraints.NotEmpty; import java.io.Serializable; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -45,9 +51,6 @@ */ @Slf4j public abstract class CassandraSink implements ISink, Serializable { - - public static final String TABLE_NAME_TAG = "tableName"; - protected final CassandraSinkConfiguration conf; protected final CassandraSinkSchemaManager schemaManager; protected transient Optional tableMetrics = Optional.absent(); @@ -74,22 +77,7 @@ public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { * @param hadoopConf */ void setupCassandraTable(@NonNull final Configuration hadoopConf) { - ConfigHelper.setOutputRpcPort(hadoopConf, CassandraSinkConfiguration.DEFAULT_OUTPUT_RPC_PORT); - final Cluster.Builder builder = Cluster.builder().withClusterName(this.conf.getClusterName()); - if (!Strings.isNullOrEmpty(ConfigHelper.getOutputKeyspaceUserName(hadoopConf)) - && !Strings.isNullOrEmpty(ConfigHelper.getOutputKeyspacePassword(hadoopConf))) { - builder.withCredentials( - ConfigHelper.getOutputKeyspaceUserName(hadoopConf), - ConfigHelper.getOutputKeyspacePassword(hadoopConf) - ); - } - this.conf.getInitialHosts().stream().forEach(host -> builder.addContactPoint(host)); - - if (this.conf.getNativePort().isPresent()) { - builder.withPort(Integer.parseInt(this.conf.getNativePort().get())); - } else { - builder.withPort(Integer.parseInt(CassandraSinkConfiguration.DEFAULT_OUTPUT_NATIVE_PORT)); - } + final Cluster.Builder builder = getBuilder(hadoopConf); final String keySpace = this.conf.getKeyspace(); @@ -101,31 +89,30 @@ void setupCassandraTable(@NonNull final Configuration hadoopConf) { log.info("Attempting to getting column names"); try { - final String columnNameQuery = schemaManager.getColumnNamesFromTableQuery(); + final String columnNameQuery = this.schemaManager.getColumnsFromTableQuery(); log.info("Getting column names with table query: {}", columnNameQuery); - results = session.execute(columnNameQuery); + results = cmdExec(session, columnNameQuery); } catch (InvalidQueryException e) { - final String columnNameFromCFQuery = schemaManager.getColumnNamesFromColumnFamilyQuery(); + final String columnNameFromCFQuery = this.schemaManager.getColumnsFromColumnFamilyQuery(); log.error("Saw an InvalidQueryException. Getting column names using column families: {}", columnNameFromCFQuery); - results = session.execute(columnNameFromCFQuery); + results = cmdExec(session, columnNameFromCFQuery); } - final List columnNames = results.all() + final Map columns = results.all() .stream() - .map(r -> r.getString("column_name")) - .collect(Collectors.toList()); + .collect(Collectors.toMap(r -> r.getString("column_name"), r -> r.getString("type"))); - if (columnNames.isEmpty()) { + if (columns.isEmpty()) { log.info("No existing columns found. Executing create table statement: {}", this.schemaManager.generateCreateTableStmt()); - session.execute(this.schemaManager.generateCreateTableStmt()); + cmdExec(session, this.schemaManager.generateCreateTableStmt()); log.info("Create table statement executed"); } else { log.info("Generating alter table statements for any columns not found"); - this.schemaManager.generateAlterTableStmt(columnNames).forEach(stmt -> { + this.schemaManager.generateAlterTableStmt(columns).forEach(stmt -> { log.info("Executing statement: {}", stmt); - session.execute(stmt); + cmdExec(session, stmt); }); } } @@ -136,14 +123,56 @@ void setupCassandraTable(@NonNull final Configuration hadoopConf) { this.conf.getKeyspace(), this.conf.getTableName()); - log.info("Using table schema: {}", schemaManager.generateCreateTableStmt()); + log.info("Using table schema: {}", this.schemaManager.generateCreateTableStmt()); CqlBulkOutputFormat.setTableSchema(hadoopConf, this.conf.getTableName(), - schemaManager.generateCreateTableStmt()); + this.schemaManager.generateCreateTableStmt()); - log.info("Using insert statement: {}", schemaManager.generateInsertStmt()); + log.info("Using insert statement: {}", this.schemaManager.generateInsertStmt()); CqlBulkOutputFormat.setTableInsertStatement(hadoopConf, this.conf.getTableName(), - schemaManager.generateInsertStmt()); + this.schemaManager.generateInsertStmt()); + } + + protected Cluster.Builder getBuilder(@NonNull final Configuration hadoopConf) { + return getBuilder(hadoopConf, this.conf.getInitialHosts()); + } + + @VisibleForTesting + public Cluster.Builder getBuilder(@NonNull final Configuration hadoopConf, + @NonNull final List initialHosts) { + ConfigHelper.setOutputRpcPort(hadoopConf, CassandraSinkConfiguration.DEFAULT_OUTPUT_RPC_PORT); + final Cluster.Builder builder = Cluster.builder().withClusterName(this.conf.getClusterName()); + if (!Strings.isNullOrEmpty(ConfigHelper.getOutputKeyspaceUserName(hadoopConf)) + && !Strings.isNullOrEmpty(ConfigHelper.getOutputKeyspacePassword(hadoopConf))) { + builder.withCredentials( + ConfigHelper.getOutputKeyspaceUserName(hadoopConf), + ConfigHelper.getOutputKeyspacePassword(hadoopConf) + ); + } + initialHosts.forEach(builder::addContactPoint); + + if (this.conf.getNativePort().isPresent()) { + builder.withPort(Integer.parseInt(this.conf.getNativePort().get())); + } else { + builder.withPort(Integer.parseInt(CassandraSinkConfiguration.DEFAULT_OUTPUT_NATIVE_PORT)); + } + return builder; + } + + protected ResultSet cmdExec(@NotEmpty final Session session, @NotEmpty final String command) { + ResultSet result; + try { + result = session.execute(command); + } catch (Exception e) { + if (this.tableMetrics.isPresent()) { + this.tableMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, DataFeedMetricNames + .getErrorModuleCauseTags(ModuleTagNames.SINK, ErrorCauseTagNames.EXEC_CASSANDRA_CMD)); + } + log.error("Error while running a cassandra command."); + result = null; + } + return result; } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java index 4957521..a6ca472 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java @@ -32,27 +32,38 @@ import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.util.IOUtils; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.DispersalLengthType; import com.uber.marmaray.common.DispersalType; import com.uber.marmaray.common.configuration.AwsConfiguration; import com.uber.marmaray.common.configuration.FileSinkConfiguration; import com.uber.marmaray.common.configuration.HadoopConfiguration; import com.uber.marmaray.common.converters.data.FileSinkDataConverter; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; +import org.hibernate.validator.constraints.NotEmpty; import org.spark_project.guava.annotations.VisibleForTesting; import java.io.IOException; import java.io.InputStream; +import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.Date; /** * {@link AwsFileSink} implements {@link FileSink} interface to build a FileSink - * that first convert data to String with csv format + * that first convert data to String using sink converter {csv, sequence, ....} * and then save to Aws bucket with config defined in {@link AwsConfiguration} */ @Slf4j @@ -91,14 +102,19 @@ protected AmazonS3 getS3Connection() { * @param path source file path * @param partNum partition number of the file */ - private void uploadFileToS3(@NonNull final FileSystem fileSystem, - @NonNull final Path path, @NonNull final int partNum) { + private void uploadFileToS3(@NonNull final FileSystem fileSystem, @NonNull final Path path, + @NonNull final int partNum, final Date date) { byte[] contentBytes = new byte [0]; log.info("Start upload file to S3 with partition num: {}", partNum); log.info("Start calculating file bytes."); try (final InputStream input = fileSystem.open(path)) { contentBytes = IOUtils.toByteArray(input); - } catch (IOException e) { + } catch (Exception e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.UPLOAD)); + } log.error("Failed while reading bytes from source path with message %s", e.getMessage()); throw new JobRuntimeException(e); } @@ -107,15 +123,38 @@ private void uploadFileToS3(@NonNull final FileSystem fileSystem, log.info("Uploading from {} to S3 bucket {}/{}", path.toString() , this.awsConf.getBucketName(), this.awsConf.getObjectKey()); try (final InputStream inputStream = fileSystem.open(path)) { - final String objKey = String.format("%s_%0" + this.digitNum + "d", this.awsConf.getS3FilePrefix(), partNum); - final PutObjectRequest request = new PutObjectRequest(this.awsConf.getBucketName(), + String objKey = ""; + if (this.conf.getDispersalLength().equals(DispersalLengthType.MULTIPLE_DAY)) { + objKey = String.format("%s/%s_%0" + this.digitNum + "d", + String.join("/", this.awsConf.getObjectKey(), + String.valueOf(date.getYear() + 1900), + String.valueOf(date.getMonth() + 1), + String.valueOf(date.getDate())), + this.conf.getFileNamePrefix(), partNum); + } else { + objKey = String.format("%s_%0" + this.digitNum + "d", + this.awsConf.getS3FilePrefix(), partNum); + } + log.info("s3 object key: {}", objKey); + final PutObjectRequest request = new PutObjectRequest( + this.awsConf.getBucketName(), objKey, inputStream, metadata); this.s3Client.putObject(request); } catch (AmazonServiceException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.UPLOAD)); + } log.error("Failed while putObject to bucket %s with message %s" , this.awsConf.getBucketName(), e.getErrorMessage()); throw new JobRuntimeException(e); } catch (IOException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.UPLOAD)); + } log.error("Failed while open source path with %s", e.getMessage()); throw new JobRuntimeException(e); } @@ -126,7 +165,7 @@ private void uploadFileToS3(@NonNull final FileSystem fileSystem, * If the {@link FileSinkConfiguration#dispersalType} is OVERWRITE, * it will overwrite existing files with prefix {@link AwsConfiguration #objectKey} in {@link AwsConfiguration #bucketName} * Then save converted and repartitioned data to temporary path {@link FileSinkConfiguration#fullPath} - * And finally upload each file in that path to aws s3 bucket with {@link AwsFileSink#uploadFileToS3(FileSystem, Path, int)} + * And finally upload each file in that path to aws s3 bucket with {@link AwsFileSink#uploadFileToS3(FileSystem, Path, int, Date)} * * @param data data to upload to aws s3 */ @@ -141,12 +180,32 @@ public void write(@NonNull final JavaRDD data) { fs.delete(destPath, true); } } catch (IOException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.FS_UTIL_DELETE)); + } log.error("Job failure while deleting temporary path {} before s3 sink write" , this.awsConf.getSourcePath()); throw new JobRuntimeException(e); } + + if (this.conf.getDispersalLength().equals(DispersalLengthType.MULTIPLE_DAY)) { + final Map> dateRepartitionedData = dateRepartition(data); + dateRepartitionedData.forEach((day, partition) -> { + log.info("multiple_day, write for day: {}", day); + super.write(partition); + writeToS3(day); + }); + } else { + super.write(data); + writeToS3(null); + } + } + + private void writeToS3(final Date date) { + //Write data to temporary path - super.write(data); final Path destPath = new Path(this.awsConf.getFileSystemPrefix()); log.info("Start to load file system object for intermediate storage."); try { @@ -202,7 +261,7 @@ public void write(@NonNull final JavaRDD data) { final Path path = s.getPath(); final String fileName = path.getName(); if (!fileName.equals(SUCCESS) && !fileName.endsWith(CRC)) { - this.uploadFileToS3(fileSystem, path, partitionId); + this.uploadFileToS3(fileSystem, path, partitionId, date); partitionId += 1; } } @@ -211,14 +270,61 @@ public void write(@NonNull final JavaRDD data) { fileSystem.delete(temporaryOutput, true); log.info("Finished deleting temporary output path: {}", this.awsConf.getSourcePath()); } catch (IOException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.UPLOAD)); + } log.error("Failed Job while writing " - + "to s3 bucket %s with error message: %s", this.awsConf.getBucketName(), e.getMessage()); + + "to s3 bucket {} with error message: {}", this.awsConf.getBucketName(), e.getMessage()); throw new JobRuntimeException(e); } } + /** + * This API uses Heatpipe timestamp and split original RDD to multiple RDD + * based on the message dates + * @param data + * @return map of date to RDD + */ + private Map> dateRepartition(@NonNull final JavaRDD data) { + + log.info("Date based data repartitoning "); + //collect all dates + Set dates = new HashSet<>(); + data.collect().forEach(line -> { + final long timestamp = (long) line.getField("Hadoop_timestamp"); + final Date msgDay = new Date(timestamp); + final Date day = new Date(msgDay.getYear(), msgDay.getMonth(), msgDay.getDate()); + dates.add(day); + }); + log.info("number of partitions based on dates: {}", dates.size()); + log.info("dates: {}", dates.toString()); + + // group messages based on grouped dates + Map> groupedMessages = new HashMap<>(); + dates.forEach(day -> { + JavaRDD message = data.filter(msg -> { + final long timestamp = (long) msg.getField("Hadoop_timestamp"); + final Date msgDay = new Date(timestamp); + final boolean result = isSameDate(day, msgDay); + log.debug("\t filtering day: {}, message day: {}, \t filter result: {}", + day.toInstant(), msgDay.toString(), result); + return result; + }); + groupedMessages.put(day, message); + }); + + return groupedMessages; + } + @VisibleForTesting protected AmazonS3 getS3Client() { return this.s3Client; } + + private static boolean isSameDate(@NotEmpty final Date dateA, @NotEmpty final Date dateB) { + final SimpleDateFormat fmt = new SimpleDateFormat("yyyyMMdd"); + return fmt.format(dateA).equals(fmt.format(dateB)); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java index 1f1c0f5..42c3e68 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java @@ -19,18 +19,27 @@ import com.google.common.base.Optional; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.FileSinkType; import com.uber.marmaray.common.configuration.FileSinkConfiguration; import com.uber.marmaray.common.converters.data.FileSinkDataConverter; import com.uber.marmaray.common.data.RDDWrapper; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.sinks.ISink; + import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections.IteratorUtils; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; + +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import java.io.Serializable; @@ -38,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import scala.Tuple2; /** * {@link FileSink} implements the {@link ISink} interface for a File sink. @@ -64,7 +74,7 @@ public FileSink(@NonNull final FileSinkConfiguration conf, @NonNull final FileSi @Override public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { this.dataFeedMetrics = Optional.of(dataFeedMetrics); - + this.converter.setDataFeedMetrics(dataFeedMetrics); } @Override @@ -81,30 +91,93 @@ public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { * @param data data to write to sink */ @Override - public void write(@NonNull final JavaRDD data) { - final JavaRDD convertedData = this.converter.convertAll(data); - final int partNum = getRepartitionNum(convertedData); + public void write(@NonNull final JavaRDD data) throws UnsupportedOperationException { + + log.info("Start write {} to path {}", this.conf.getFileType(), this.conf.getFullPath()); + final JavaPairRDD convertedData = this.converter.convertAll(data); + final JavaRDD tmp = convertedData.map(message -> message._2()); + final int partNum = getRepartitionNum(tmp); final int desiredDigit = (int) Math.floor(Math.log10(partNum) + 1); this.digitNum = desiredDigit > DEFAULT_DIGIT_NUM ? desiredDigit : DEFAULT_DIGIT_NUM; - final JavaRDD dataRepartitioned = convertedData.repartition(partNum); - final JavaRDD dataToWrite; - if (this.conf.isColumnHeader()) { - final String header = this.converter.getHeader(data); - dataToWrite = addColumnHeader(header, dataRepartitioned); + + if (this.conf.getFileType().equals("csv")) { + final JavaRDD dataRepartitioned = tmp.repartition(partNum); + final JavaRDD dataToWrite; + if (this.conf.isColumnHeader()) { + final String header = this.converter.getHeader(data); + dataToWrite = addColumnHeader(header, dataRepartitioned); + } else { + dataToWrite = dataRepartitioned; + } + if (dataFeedMetrics.isPresent()) { + final Map tags = new HashMap<>(); + tags.put(SINK_INFO_TAG, this.conf.getSinkType().name()); + final RDDWrapper dataWrapper = new RDDWrapper<>(dataToWrite); + final long totalRows = dataWrapper.getCount(); + this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, + totalRows, tags); + } + log.info("Start write {} to path {}", this.conf.getFileType(), this.conf.getFullPath()); + // based on converter type saving should change + try { + dataToWrite.saveAsTextFile(this.conf.getFullPath()); + log.info("Finished save to path: {}.", this.conf.getFullPath()); + } catch (Exception e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, + 1, DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.WRITE_TO_SINK_CSV)); + } + final String sink = this.conf.getSinkType() == FileSinkType.valueOf("S3") ? "s3" : "hdfs"; + log.error("Error while writing to {} ", sink, e); + } + } else if (this.conf.getFileType().equals("sequence")) { + final JavaPairRDD dataToRepartition = convertedData + .mapToPair(message -> + new Tuple2<>(new Text(message._1()), new Text(message._2()))); + final JavaPairRDD dataToWrite = dataToRepartition.repartition(partNum); + + Configuration savingConf = new Configuration(); + if (this.conf.isCompression()) { + savingConf.set("mapreduce.output.fileoutputformat.compress", "true"); + if (this.conf.getCompressionCodec().equals("lz4")) { + log.info("Save sequence file with LZ4 compression"); + savingConf.set("mapreduce.output.fileoutputformat.compress.codec", + "org.apache.hadoop.io.compress.Lz4Codec"); + } else { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, + 1, DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.COMPRESSION)); + } + final String errorMessage = String.format("Compression codec {} not supported", + this.conf.getCompressionCodec()); + throw new UnsupportedOperationException(errorMessage); + } + } + try { + dataToWrite.saveAsNewAPIHadoopFile(this.conf.getFullPath(), + Text.class, Text.class, SequenceFileOutputFormat.class, savingConf); + log.info("Finished save to path: {}.", this.conf.getFullPath()); + } catch (Exception e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, + 1, DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.WRITE_TO_SINK_SEQ)); + } + final String sink = this.conf.getSinkType() == FileSinkType.valueOf("S3") ? "s3" : "hdfs"; + log.error("Error while writing to {} ", sink, e); + throw new JobRuntimeException(e); + } } else { - dataToWrite = dataRepartitioned; - } - if (dataFeedMetrics.isPresent()) { - final Map tags = new HashMap<>(); - tags.put(SINK_INFO_TAG, this.conf.getSinkType().name()); - final RDDWrapper dataWrapper = new RDDWrapper<>(dataToWrite); - final long totalRows = dataWrapper.getCount(); - this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, - totalRows, tags); + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.OUTPUT_FILE_FORMAT)); + } + final String errorMessage = String.format("Format %s not supported yet.", this.conf.getFileType()); + throw new UnsupportedOperationException(errorMessage); } - log.info("Start write {} to path {}", this.conf.getFileType(), this.conf.getFullPath()); - dataToWrite.saveAsTextFile(this.conf.getFullPath()); - log.info("Finished save to path: {}.", this.conf.getFullPath()); } /** @@ -195,6 +268,11 @@ protected long getSampleSizeInBytes(@NonNull final JavaRDD data) { } log.debug("Sample size in bytes: {}", size); } catch (UnsupportedEncodingException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.CALCULATE_SAMPLE_SIZE)); + } log.error("Failed while calculating sample size: {}", e.getMessage()); throw new JobRuntimeException(e); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java index c80f413..8189ba4 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java @@ -23,6 +23,9 @@ import com.uber.marmaray.common.converters.data.FileSinkDataConverter; import com.uber.marmaray.common.configuration.FileSinkConfiguration; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.fs.FileStatus; @@ -65,6 +68,11 @@ public void write(@NonNull final JavaRDD data) { fileSystem.delete(dataFolder, true); } } catch (IOException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.FS_UTIL)); + } log.error("Exception: {}", e.getMessage()); throw new JobRuntimeException(e); } @@ -93,6 +101,11 @@ public void write(@NonNull final JavaRDD data) { fileSystem.delete(new Path(this.conf.getFullPath()), true); log.info("Finished write files to hdfs path: {}", this.conf.getFullPath()); } catch (IOException e) { + if (dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK, ErrorCauseTagNames.WRITE_FILE)); + } log.error("Exception: {}", e); throw new JobRuntimeException(e); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java index 6b79342..7dc97b5 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.common.sinks.hoodie; +import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.exception.HoodieInsertException; @@ -24,14 +25,12 @@ import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.metadata.IMetadataManager; - +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - import java.util.Map; /** @@ -48,7 +47,7 @@ public HoodieErrorSink(@NonNull final HoodieConfiguration hoodieConf, @NonNull final HoodieSinkOp op, @NonNull final IMetadataManager metadataMgr, final boolean shouldSaveChangesInFuture) { - super(hoodieConf, hoodieSinkDataConverter, jsc, op, metadataMgr, shouldSaveChangesInFuture); + super(hoodieConf, hoodieSinkDataConverter, jsc, op, metadataMgr, shouldSaveChangesInFuture, Optional.absent()); } public void writeRecordsAndErrors(@NonNull final HoodieWriteResult result) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java index 2a91e7c..2d6316a 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java @@ -41,6 +41,7 @@ import com.uber.marmaray.common.sinks.ISink; import com.uber.marmaray.common.sinks.SinkStatManager; import com.uber.marmaray.common.sinks.SinkStatManager.SinkStat; +import com.uber.marmaray.common.sinks.hoodie.partitioner.DefaultHoodieDataPartitioner; import com.uber.marmaray.utilities.ErrorTableUtil; import com.uber.marmaray.utilities.FSUtils; import com.uber.marmaray.utilities.HoodieSinkErrorExtractor; @@ -74,6 +75,7 @@ @Slf4j public class HoodieSink implements ISink, scala.Serializable { + private static final String TABLE_NAME = "table_name"; private final HoodieConfiguration hoodieConf; // It is used for generating HoodieKey from AvroPayload. private final HoodieSinkDataConverter hoodieSinkDataConverter; @@ -92,12 +94,13 @@ public class HoodieSink implements ISink, scala.Serializable { private final SinkStatManager sinkStatMgr; private final boolean shouldSaveChangesInFuture; + @Getter @Setter @NonNull /** * If set then it is used for sorting records during {@link HoodieWriteClientWrapper#bulkInsert(JavaRDD, String)} */ - private Optional bulkInsertPartitioner = Optional.absent(); + private final UserDefinedBulkInsertPartitioner bulkInsertPartitioner; @Setter @NonNull @@ -114,8 +117,9 @@ public HoodieSink(@NonNull final HoodieConfiguration hoodieConf, @NonNull final HoodieSinkDataConverter hoodieSinkDataConverter, @NonNull final JavaSparkContext jsc, @NonNull final HoodieSinkOp op, - @NonNull final IMetadataManager metadataMgr) { - this(hoodieConf, hoodieSinkDataConverter, jsc, op, metadataMgr, false); + @NonNull final IMetadataManager metadataMgr, + @NonNull final Optional defaultDataPartitioner) { + this(hoodieConf, hoodieSinkDataConverter, jsc, op, metadataMgr, false, defaultDataPartitioner); } public HoodieSink(@NonNull final HoodieConfiguration hoodieConf, @@ -123,7 +127,8 @@ public HoodieSink(@NonNull final HoodieConfiguration hoodieConf, @NonNull final JavaSparkContext jsc, @NonNull final HoodieSinkOp op, @NonNull final IMetadataManager metadataMgr, - final boolean shouldSaveChangesInFuture) { + final boolean shouldSaveChangesInFuture, + @NonNull final Optional defaultDataPartitioner) { this.hoodieConf = hoodieConf; this.hoodieSinkDataConverter = hoodieSinkDataConverter; this.jsc = jsc; @@ -132,6 +137,7 @@ public HoodieSink(@NonNull final HoodieConfiguration hoodieConf, this.sinkStatMgr = new SinkStatManager(this.hoodieConf.getTableName(), this.metadataMgr); this.sinkStatMgr.init(); this.shouldSaveChangesInFuture = shouldSaveChangesInFuture; + this.bulkInsertPartitioner = getDataPartitioner(this.hoodieConf, defaultDataPartitioner); } public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { @@ -184,7 +190,8 @@ public void write(@NonNull final RDDWrapper> h */ protected void initDataset() { try { - HoodieUtil.initHoodieDataset(FSUtils.getFs(this.hoodieConf.getConf()), this.hoodieConf); + HoodieUtil.initHoodieDataset(FSUtils.getFs(this.hoodieConf.getConf(), + Optional.of(this.hoodieConf.getBasePath())), this.hoodieConf); } catch (IOException e) { log.error("Error initializing hoodie dataset.", e); throw new JobRuntimeException("Could not initialize hoodie dataset", e); @@ -339,6 +346,8 @@ private void logWriteMetrics(final Optional> writesStatuses private void updateSinkStat(final Optional> writesStatuses) { if (writesStatuses.isPresent()) { final LongAccumulator avgRecordSizeCounter = writesStatuses.get().rdd().sparkContext().longAccumulator(); + final LongAccumulator fileCount = writesStatuses.get().rdd().sparkContext().longAccumulator(); + final LongAccumulator totalSize = writesStatuses.get().rdd().sparkContext().longAccumulator(); writesStatuses.get().foreach( writeStatus -> { final long writeBytes = writeStatus.getStat().getTotalWriteBytes(); @@ -347,6 +356,8 @@ private void updateSinkStat(final Optional> writesStatuses) if (writeBytes > 0 && numInserts > 0) { avgRecordSizeCounter.add(writeBytes / numInserts); } + fileCount.add(1); + totalSize.add(writeBytes); } ); final long avgRecordSize = (int) avgRecordSizeCounter.avg(); @@ -354,6 +365,12 @@ private void updateSinkStat(final Optional> writesStatuses) log.info("Updating Sink Stat manager : avgRecordSize : {}", avgRecordSize); this.sinkStatMgr.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Long.toString(avgRecordSize)); } + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.TOTAL_FILE_COUNT, fileCount.value(), + this.dataFeedMetricsTags); + this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.TOTAL_WRITE_SIZE, totalSize.value(), + this.dataFeedMetricsTags); + } } this.sinkStatMgr.persist(); } @@ -431,6 +448,22 @@ private JavaRDD> dedupRecords(@NonNull final H return writeClient.filterExists(hoodieRecords).persist(StorageLevel.DISK_ONLY()); } + /** + * Use {@link HoodieConfiguration#HOODIE_DATA_PARTITIONER} for setting desired data partitioner. It will be used in + * hoodie's bulk insert flow for repartitioning input records before creating new parquet files. For more details + * see {@link UserDefinedBulkInsertPartitioner}. + */ + public static UserDefinedBulkInsertPartitioner getDataPartitioner(@NonNull final HoodieConfiguration hoodieConf, + @NonNull final Optional defaultDataPartitioner) { + try { + return (UserDefinedBulkInsertPartitioner) Class.forName(hoodieConf.getHoodieDataPartitioner( + defaultDataPartitioner.isPresent() ? defaultDataPartitioner.get() + : DefaultHoodieDataPartitioner.class.getName())).newInstance(); + } catch (InstantiationException | IllegalAccessException | ClassNotFoundException | ClassCastException e) { + throw new JobRuntimeException("exception in initializing data partitioner", e); + } + } + /** * This class is a wrapper around hoodie write client to enable testing. */ @@ -441,7 +474,7 @@ public static class HoodieWriteClientWrapper implements Serializable, Closeable @VisibleForTesting @Getter private final HoodieWriteClient hoodieWriteClient; - private final Optional bulkInsertPartitioner; + private final UserDefinedBulkInsertPartitioner bulkInsertPartitioner; public String startCommit() { return this.hoodieWriteClient.startCommit(); @@ -463,12 +496,7 @@ public JavaRDD insert(@NonNull final JavaRDD bulkInsert(@NonNull final JavaRDD> records, @NotEmpty final String commitTime) { - if (this.bulkInsertPartitioner.isPresent()) { - return this.hoodieWriteClient.bulkInsert(records, commitTime, - Option.apply(this.bulkInsertPartitioner.get())); - } else { - return this.hoodieWriteClient.bulkInsert(records, commitTime); - } + return this.hoodieWriteClient.bulkInsert(records, commitTime, Option.apply(this.bulkInsertPartitioner)); } public JavaRDD upsert(@NonNull final JavaRDD> records, diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/partitioner/DefaultHoodieDataPartitioner.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/partitioner/DefaultHoodieDataPartitioner.java new file mode 100644 index 0000000..4d1054f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/partitioner/DefaultHoodieDataPartitioner.java @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.sinks.hoodie.partitioner; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner; +import lombok.NonNull; +import org.apache.spark.api.java.JavaRDD; +import scala.Serializable; + +/** + * {@link DefaultHoodieDataPartitioner} is used for sorting the records to ensure that all records belonging to single + * partition are grouped together. for more information also see {@link UserDefinedBulkInsertPartitioner}. + */ +public class DefaultHoodieDataPartitioner implements UserDefinedBulkInsertPartitioner, + Serializable { + + @Override + public JavaRDD> repartitionRecords( + @NonNull final JavaRDD> javaRDD, final int outputPartitions) { + return javaRDD.sortBy( + v1 -> String.format("%s %s", v1.getPartitionPath(), v1.getRecordKey()), true, outputPartitions); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java index efb4f0d..18ce5c6 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java @@ -19,6 +19,8 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.metrics.IMetricable; import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; import org.apache.spark.api.java.JavaRDD; /** @@ -31,4 +33,8 @@ public interface ISource getData(K k); + + default IStatus getStatus() { + return new BaseStatus(); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java index f054933..a03bf65 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java @@ -21,6 +21,9 @@ import com.uber.marmaray.common.metrics.IChargebackCalculator; import com.uber.marmaray.common.metrics.IMetricable; import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; + import java.util.List; /** @@ -83,5 +86,12 @@ interface IWorkUnitCalculatorResult { * Returns {@link IRunState} for next run. */ S getNextRunState(); + + /** + * Returns {@link IStatus} for this run. + */ + default IStatus getStatus() { + return new BaseStatus(); + } } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/file/FileSource.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/file/FileSource.java index 8996f89..21d29e9 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/file/FileSource.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/file/FileSource.java @@ -17,6 +17,7 @@ package com.uber.marmaray.common.sources.file; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.FileSourceConfiguration; @@ -40,7 +41,7 @@ public class FileSource implements ISource { private final FileSourceConfiguration conf; - private final JavaSparkContext jsc; + private final Optional jsc; private final FileSourceDataConverter converter; @Override @@ -64,7 +65,7 @@ public JavaRDD getData(@NonNull final FileWorkUnitCalculator.FileWo .map(LocatedFileStatus::getPath) .map(Path::toString) .collect(Collectors.joining(",")); - final RDD fileRows = this.jsc.sc().textFile(filesToRead, 1); + final RDD fileRows = this.jsc.get().sc().textFile(filesToRead, 1); return this.converter.map(fileRows.toJavaRDD()).getData(); } catch (IOException e) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/file/JSONFileSourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/file/JSONFileSourceDataConverter.java index e60aafc..5d0ae66 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/file/JSONFileSourceDataConverter.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/file/JSONFileSourceDataConverter.java @@ -21,6 +21,8 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.utilities.ErrorExtractor; import lombok.NonNull; import org.apache.avro.Schema; @@ -48,6 +50,16 @@ public Schema getSchema() { return this.schema; } + @Override + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + //ignored + } + + @Override + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + @Override protected List> convert(@NonNull final String data) throws Exception { try { @@ -55,7 +67,7 @@ protected List> convert(@NonNull final Stri return Collections.singletonList( new ConverterResult(new AvroPayload(gr))); } catch (MercifulJsonConverter.JsonConversionException e) { - return Collections.singletonList(new ConverterResult(data, e.toString())); + return Collections.singletonList(new ConverterResult(data, e.getMessage())); } } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java index 976cecb..7864416 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java @@ -16,11 +16,16 @@ */ package com.uber.marmaray.common.sources.hive; +import com.google.common.base.Optional; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.HiveSourceConfiguration; import com.uber.marmaray.common.converters.data.SparkSourceDataConverter; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.sources.ISource; import lombok.AllArgsConstructor; import lombok.Getter; @@ -34,7 +39,6 @@ import parquet.Preconditions; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; @Slf4j @@ -48,10 +52,28 @@ public class HiveSource implements ISource dataFeedMetrics = Optional.absent(); + + public HiveSource(@NonNull final HiveSourceConfiguration hiveConf, + @NonNull final SQLContext sqlContext, + @NonNull final SparkSourceDataConverter converter) { + this.hiveConf = hiveConf; + this.sqlContext = sqlContext; + this.converter = converter; + this.inputCount = 0; + } + + @Override public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { - // ignored + this.converter.setDataFeedMetrics(dataFeedMetrics); + this.dataFeedMetrics = Optional.of(dataFeedMetrics); } + @Override public void setJobMetrics(final JobMetrics jobMetrics) { // ignored } @@ -70,22 +92,26 @@ public JavaRDD getData(@NonNull final ParquetWorkUnitCalculatorResu final String hdfsPath = new Path(this.hiveConf.getDataPath(), workUnits.get(0)).toString(); log.info("Reading data from path: {}", hdfsPath); - - final Dataset data = this.sqlContext.read().parquet(hdfsPath); + final Dataset data; + try { + data = this.sqlContext.read().parquet(hdfsPath); + this.inputCount = data.count(); + } catch (Exception e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE, ErrorCauseTagNames.FILE_PATH)); + } + log.error(String.format("Error reading from source path"), e); + throw new JobRuntimeException(e); + } final int numPartitions = calculateHiveNumPartitions(data); - log.info("Using {} partitions", numPartitions); - final JavaRDD hiveRawData = data + return this.converter.map(data .coalesce(numPartitions) - .javaRDD() - .flatMap(row -> { - final List payloads = new ArrayList<>(); - this.converter.convert(row).forEach(d -> payloads.add(d.getSuccessData().get().getData())); - return payloads.iterator(); - }); - return hiveRawData; + .javaRDD()).getData(); } private int calculateHiveNumPartitions(@NonNull final Dataset data) { diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java index c78b080..dc93d8a 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java @@ -25,17 +25,22 @@ import com.uber.marmaray.common.metadata.IMetadataManager; import com.uber.marmaray.common.metadata.MetadataConstants; import com.uber.marmaray.common.metadata.StringValue; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; import com.uber.marmaray.common.metrics.IChargebackCalculator; import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.ModuleTagNames; import com.uber.marmaray.common.sources.IWorkUnitCalculator; import lombok.Getter; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.fs.FileSystem; + import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; @Slf4j public class ParquetWorkUnitCalculator implements @@ -50,19 +55,20 @@ public class ParquetWorkUnitCalculator implements @Getter private final HiveSourceConfiguration hiveConf; + @Getter + private Optional dataFeedMetrics = Optional.absent(); + public ParquetWorkUnitCalculator(@NonNull final HiveSourceConfiguration hiveConf, @NonNull final FileSystem fs) throws IOException { this.hiveConf = hiveConf; final PartitionType partitionType = hiveConf.getPartitionType(); - log.info("Create partition manger with partition type: {}", partitionType); + log.info("Create partition manager with partition type: {}", partitionType); if (partitionType.equals(PartitionType.NONE) || partitionType.equals(PartitionType.NORMAL)) { // create partition manager internally this.partitionManager = new HDFSPartitionManager(hiveConf.getJobName(), - hiveConf.getBaseMetadataPath(), hiveConf.getDataPath(), fs); } else if (partitionType.equals(PartitionType.DATE)) { this.partitionManager = new HDFSDatePartitionManager(hiveConf.getJobName(), - hiveConf.getBaseMetadataPath(), hiveConf.getDataPath(), hiveConf.getPartitionKeyName().get(), getHiveConf().getStartDate(), @@ -77,6 +83,9 @@ public ParquetWorkUnitCalculator(@NonNull final HiveSourceConfiguration hiveConf public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { // ignored, no need to compute data feed level metrics for now. // Data is either dispersed or not and row count is tracked as job level metric + + // use datafeedMetric to expose errors + this.dataFeedMetrics = Optional.of(dataFeedMetrics); } @Override public void setJobMetrics(final JobMetrics jobMetrics) { @@ -90,10 +99,29 @@ public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { @Override public void initPreviousRunState(@NonNull final IMetadataManager metadataManager) { try { - final Optional latestCheckpoint = metadataManager.get(MetadataConstants.CHECKPOINT_KEY); - log.info("Get latest change point: {}", latestCheckpoint); - this.nextPartition = this.partitionManager.getNextPartition(latestCheckpoint); + final Optional latestCheckpoint; + // Backfill dispersal given a specific partition value + if (this.hiveConf.getPartition().isPresent()) { + latestCheckpoint = this.hiveConf.getPartition(); + final List partitionList = this.partitionManager.getExistingPartitions() + .stream() + .filter(partition -> partition.contains(latestCheckpoint.get().getValue())) + .collect(Collectors.toList()); + this.nextPartition = partitionList.isEmpty() ? Optional.absent() : Optional.of(partitionList.get(0)); + } else { + latestCheckpoint = metadataManager.get(MetadataConstants.CHECKPOINT_KEY); + this.nextPartition = this.partitionManager.getNextPartition(latestCheckpoint); + } + + log.info("Get latest change point: {}", + latestCheckpoint.isPresent() ? latestCheckpoint.get().getValue() : Optional.absent()); } catch (final IOException e) { + if (this.dataFeedMetrics.isPresent()) { + this.dataFeedMetrics.get() + .createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags(ModuleTagNames.WORK_UNIT_CALCULATOR, + ErrorCauseTagNames.NO_DATA)); + } throw new JobRuntimeException("Unable to get the next partition. Error message: " + this.nextPartition, e); } } @@ -110,13 +138,12 @@ public void saveNextRunState(@NonNull final IMetadataManager metada * Until we add Cassandra metadata information, we assume explicitly this is a HDFSPartitionManager. * Todo: T898695 - Implement metadata manager using Cassandra backend */ - if (!this.nextPartition.isPresent()) { log.warn("No partition was found to process. Reusing latest checkpoint if exists as checkpoint key"); return; } - if (partitionManager.isSinglePartition()) { + if (this.partitionManager.isSinglePartition()) { log.info("Single partition manager, save next partition {} in metadata manager", this.partitionManager); metadataManager.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(this.nextPartition.get())); } else { @@ -147,8 +174,8 @@ public ParquetWorkUnitCalculatorResult computeWorkUnits() { * value from next run state will be persisted in the metadata as a checkpoint. */ final HiveRunState nextRunState = new HiveRunState(this.nextPartition); - final List workUnits = nextPartition.isPresent() ? Collections.singletonList(nextPartition.get()) - : Collections.EMPTY_LIST; + final List workUnits = this.nextPartition.isPresent() + ? Collections.singletonList(this.nextPartition.get()) : Collections.EMPTY_LIST; return new ParquetWorkUnitCalculatorResult(workUnits, nextRunState); } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/IKafkaOffsetSelector.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/IKafkaOffsetSelector.java new file mode 100644 index 0000000..2ad4187 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/IKafkaOffsetSelector.java @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2019 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.sources.kafka; + +import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; +import lombok.NonNull; +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; +import java.util.Set; + +public interface IKafkaOffsetSelector { + Map getPartitionOffsets(@NonNull final KafkaSourceConfiguration kafkaConf, + @NonNull final Set topicPartitions, + @NonNull final Map earliestLeaderOffsets, + @NonNull final Map latestLeaderOffsets); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java index 614cc7a..872fd18 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java @@ -30,15 +30,15 @@ * reading data from kafka topic. See {@link KafkaWorkUnitCalculator} for how it is used. */ @Slf4j -public class KafkaBootstrapOffsetSelector { +public class KafkaBootstrapOffsetSelector implements IKafkaOffsetSelector { /** * @return per partition offsets to start reading kafka data from. */ public Map getPartitionOffsets(@NonNull final KafkaSourceConfiguration kafkaConf, - @NonNull final Set topicPartitions, - @NonNull final Map earliestLeaderOffsets, - @NonNull final Map latestLeaderOffsets) { + @NonNull final Set topicPartitions, + @NonNull final Map earliestLeaderOffsets, + @NonNull final Map latestLeaderOffsets) { final Map ret = new HashMap<>(); latestLeaderOffsets.entrySet().stream().forEach(e -> ret.put(e.getKey().partition(), e.getValue())); return ret; diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaOffsetResetter.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaOffsetResetter.java new file mode 100644 index 0000000..e87fa5a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaOffsetResetter.java @@ -0,0 +1,30 @@ +package com.uber.marmaray.common.sources.kafka; + +import com.uber.marmaray.common.reporters.IKafkaDataLossReporter; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; + +/** + * {@link KafkaOffsetResetter} holds the logic and state to reset Kafka offsets when there is data loss. + */ +public class KafkaOffsetResetter { + @NonNull + @Getter + private final IKafkaOffsetSelector offsetSelector; + + @NonNull + @Getter + @Setter + private IKafkaDataLossReporter kafkaDataLossReporter = new LogBasedKafkaDataLossReporter(); + + public KafkaOffsetResetter(final IKafkaOffsetSelector offsetSelector) { + this.offsetSelector = offsetSelector; + } + + public KafkaOffsetResetter(final IKafkaOffsetSelector offsetSelector, + final IKafkaDataLossReporter kafkaDataLossReporter) { + this.offsetSelector = offsetSelector; + this.kafkaDataLossReporter = kafkaDataLossReporter; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java index 21ae2c1..9fe7d65 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java @@ -20,37 +20,52 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.KafkaConfiguration; import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; import com.uber.marmaray.common.converters.data.KafkaSourceDataConverter; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; import com.uber.marmaray.common.metrics.DataFeedMetrics; import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.sources.ISource; import com.uber.marmaray.common.sources.kafka.KafkaWorkUnitCalculator.KafkaWorkUnitCalculatorResult; import com.uber.marmaray.utilities.KafkaUtil; +import com.uber.marmaray.utilities.LongAccumulator; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NonNull; import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.spark.Partition; import org.apache.spark.Partitioner; +import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.api.java.function.VoidFunction; -import org.apache.spark.streaming.kafka010.KafkaUtils; -import org.apache.spark.streaming.kafka010.LocationStrategies; +import org.apache.spark.rdd.RDD; +import org.apache.spark.streaming.kafka010.KafkaRDD; +import org.apache.spark.streaming.kafka010.KafkaUtils$; import org.apache.spark.streaming.kafka010.OffsetRange; import org.hibernate.validator.constraints.NotEmpty; import scala.Serializable; import scala.Tuple2; +import scala.collection.Seq; +import scala.collection.mutable.ArrayBuffer; +import scala.reflect.ClassTag$; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Queue; import java.util.TreeMap; +import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * It gets work units from {@link KafkaWorkUnitCalculatorResult} as a list of {@link OffsetRange}, reads messages from @@ -60,6 +75,7 @@ @AllArgsConstructor public class KafkaSource implements ISource, Serializable { + public static final int MIN_PARTITIONS_TO_ENABLE_PARALLEL_BROKER_READ = 256; @Getter private final KafkaSourceConfiguration conf; @Getter @@ -71,9 +87,23 @@ public class KafkaSource implements ISource> filterRecordHandler; + @NonNull + private Optional topicMetrics = Optional.absent(); + + public KafkaSource(@NonNull final KafkaSourceConfiguration conf, + @NonNull final Optional jsc, + @NonNull final KafkaSourceDataConverter dataConverter, + @NonNull final Optional> startDateFilterFunction, + @NonNull final Optional> filterRecordHandler) { + this.conf = conf; + this.jsc = jsc; + this.dataConverter = dataConverter; + this.startDateFilterFunction = startDateFilterFunction; + this.filterRecordHandler = filterRecordHandler; + } public void setDataFeedMetrics(@NonNull final DataFeedMetrics topicMetrics) { - // ignored + this.topicMetrics = Optional.of(topicMetrics); } public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { @@ -95,16 +125,106 @@ public JavaRDD getData(@NonNull final KafkaWorkUnitCalculatorResult } else { readParallelism = workUnits.size(); } - Map> kafkaPartitionOffsetToSparkPartitionMap + final LongAccumulator totalDataReadInBytes = new LongAccumulator("totalDataRead"); + getJsc().get().sc().register(totalDataReadInBytes); + final int numPartitions = workUnits.stream().map(r -> r.partition()).collect(Collectors.toSet()).size(); + final Map> kafkaPartitionOffsetToSparkPartitionMap = getKafkaPartitionOffsetToOutputSparkPartitionMap(this.conf.getTopicName(), workUnits, readParallelism); log.info("using partition offset mapping topic={} : mapping={}", this.conf.getTopicName(), kafkaPartitionOffsetToSparkPartitionMap); - final JavaRDD kafkaData = KafkaUtils.createRDD( - this.jsc.get(), - KafkaUtil.getKafkaParams(this.conf), - workUnits.toArray(new OffsetRange[0]), - LocationStrategies.PreferConsistent()) + final JavaRDD kafkaDataRead = isParallelBrokerReadEnabled(numPartitions) + ? readWithMultiReaderPerPartition(workUnits, kafkaPartitionOffsetToSparkPartitionMap) + : readWithOneReaderPerPartition(workUnits, readParallelism, kafkaPartitionOffsetToSparkPartitionMap); + final JavaRDD kafkaData = kafkaDataRead.map(val -> { + totalDataReadInBytes.add((long) val.length); + return val; + } + ); + if (topicMetrics.isPresent()) { + this.dataConverter.setDataFeedMetrics(this.topicMetrics.get()); + } + final JavaRDD inputRDD = this.dataConverter.map(kafkaData).getData(); + reportReadMetrics(this.topicMetrics, totalDataReadInBytes, numPartitions); + if (this.startDateFilterFunction.isPresent()) { + return inputRDD.filter( + record -> { + if (!this.startDateFilterFunction.get().call(record)) { + if (this.filterRecordHandler.isPresent()) { + this.filterRecordHandler.get().call(record); + } + return false; + } + return true; + }); + } + return inputRDD; + } + + public boolean isParallelBrokerReadEnabled(final int numPartitions) { + final boolean isParallelBrokerReadEnabled = + this.conf.isParallelBrokerReadEnabled() || numPartitions >= MIN_PARTITIONS_TO_ENABLE_PARALLEL_BROKER_READ; + log.info("topicName:{} usingParallelBrokerReader:{} partitions:{}", this.conf.getTopicName(), + isParallelBrokerReadEnabled, numPartitions); + return isParallelBrokerReadEnabled; + } + + /** + * It is used for reading kafka data from kafka brokers by splitting single partition read request into multiple + * small requests such that single kafka consumer has very less data to read and it can avoid potential shuffle + * stage used by {@link #readWithOneReaderPerPartition(List, int, Map)} internally. + */ + public JavaRDD readWithMultiReaderPerPartition( + @NonNull final List workUnits, + @NonNull final Map> kafkaPartitionOffsetToSparkPartitionMap) { + final List newWorkUnits = new LinkedList<>(); + for (final OffsetRange workUnit : workUnits) { + if (kafkaPartitionOffsetToSparkPartitionMap.containsKey(workUnit.partition())) { + /* + Here we are trying to recreate OffsetRanges for a given topic-partition identified by workUnit. + Let us assume that original workUnit to read from kafka was like this. + workUnit -> [partition=1,startOffset=12,endOffset=100] + Let us say it gets split into 4 (due to read_parallelism) (with 22 messages each) then this is how + it will look like in kafkaPartitionOffsetToSparkPartitionMap + {1,{12->0,34->1,56->2,78->3}}. + + "kafkaPartitionOffsetToSparkPartitionMap" holds Offset to outputSparkPartition mapping for every + kafka topic's partition. + Top level map's key is kafka topic's partition number. + For inner map; + key -> kafka partition offset to start new output spark partition + value -> output spark partition's number. + + What this means is for partition 1 create 4 ranges as follows. + [partition=1,startOffset=12,endOffset=34] + [partition=1,startOffset=34,endOffset=56] + [partition=1,startOffset=56,endOffset=78] + [partition=1,startOffset=78,endOffset=100] + */ + long previousOffset = -1; + for (Map.Entry newRange + : kafkaPartitionOffsetToSparkPartitionMap.get(workUnit.partition()).entrySet()) { + if (previousOffset > -1) { + newWorkUnits.add( + OffsetRange.create(workUnit.topicPartition(), previousOffset, newRange.getKey())); + } + previousOffset = newRange.getKey(); + } + Preconditions.checkState(previousOffset > -1, "unexpected offset range value"); + if (previousOffset < workUnit.untilOffset()) { + newWorkUnits + .add(OffsetRange.create(workUnit.topicPartition(), previousOffset, workUnit.untilOffset())); + } + } + } + return readKafkaData(newWorkUnits).map(e -> e.value()); + } + + public JavaRDD readWithOneReaderPerPartition( + @NonNull final List workUnits, + final int readParallelism, + @NonNull final Map> kafkaPartitionOffsetToSparkPartitionMap) { + return readKafkaData(workUnits) .mapToPair( new PairFunction, Integer, byte[]>() { int lastSparkPartition = -1; @@ -135,44 +255,95 @@ public int getPartition(final Object key) { } }) .values(); + } - final JavaRDD inputRDD = this.dataConverter.map(kafkaData).getData(); + @VisibleForTesting + protected JavaRDD> readKafkaData(@NonNull final List workUnits) { + log.info("kafka read workunits #: {} workunits: {}", workUnits.size(), workUnits); + final HashMap kafkaParams = new HashMap<>(KafkaUtil.getKafkaParams(this.conf)); + KafkaUtils$.MODULE$.fixKafkaParams(kafkaParams); + // We shuffle work units so that read requests for single topic+partition are evenly spread across tasks in + // this read stage so that we won't hit the same topic+partition from all tasks. + Collections.shuffle(workUnits); + final RDD> kafkaRDD = new KafkaRDD( + this.jsc.get().sc(), + kafkaParams, + workUnits.toArray(new OffsetRange[0]), + Collections.emptyMap(), + true) { + // It is overridden to ensure that we don't pin topic+partition consumer to only one executor. This allows + // us to do parallel reads from kafka brokers. + @Override + public Seq getPreferredLocations(final Partition thePart) { + return new ArrayBuffer<>(); + } - if (this.startDateFilterFunction.isPresent()) { - return inputRDD.filter( - record -> { - if (!this.startDateFilterFunction.get().call(record)) { - if (this.filterRecordHandler.isPresent()) { - this.filterRecordHandler.get().call(record); - } - return false; - } - return true; - }); + // We are updating client.id on executor per task to ensure we assign unique ids for it. + @Override + public scala.collection.Iterator> compute(final Partition thePart, + final TaskContext context) { + super.kafkaParams().put(KafkaConfiguration.CLIENT_ID, String + .format(KafkaConfiguration.DEFAULT_CLIENT_ID, + KafkaConfiguration.getClientId())); + return super.compute(thePart, context); + } + }; + return new JavaRDD<>(kafkaRDD, ClassTag$.MODULE$.apply(ConsumerRecord.class)); + } + + /** + * report read size metrics per partition and total along with individual record stats like average size + */ + private static void reportReadMetrics(@NonNull final Optional topicMetrics, + @NonNull final LongAccumulator totalDataReadInBytes, + final long totalPartitions) { + if (topicMetrics.isPresent()) { + topicMetrics.get().createLongMetric( + DataFeedMetricNames.TOTAL_READ_SIZE, totalDataReadInBytes.getSum(), new HashMap<>()); + topicMetrics.get().createLongMetric( + DataFeedMetricNames.TOTAL_READ_SIZE_PER_PARTITION, + totalDataReadInBytes.getSum() / totalPartitions, new HashMap<>()); + topicMetrics.get().createLongMetric(DataFeedMetricNames.AVERAGE_INPUT_RECORD_SIZE, + (long) totalDataReadInBytes.getAvg(), new HashMap<>()); + topicMetrics.get().createLongMetric(DataFeedMetricNames.MAX_INPUT_RECORD_SIZE, + totalDataReadInBytes.getMax(), new HashMap<>()); + topicMetrics.get().createLongMetric(DataFeedMetricNames.NUM_INPUT_PARTITIONS, + totalPartitions, new HashMap<>()); } - return inputRDD; } /** * It maps kafka partition's offset ranges to output spark partitions such that each every output spark partition * gets equal number of messages. */ + private static Map> getKafkaPartitionOffsetToOutputSparkPartitionMap( + @NotEmpty final String topicName, @NonNull final List offsetRanges, final int readParallelism) { + + final List outputSparkPartitions = new ArrayList<>(readParallelism); + IntStream.range(0, readParallelism).forEach(i -> outputSparkPartitions.add(i)); + // We shuffle outputSparkPartitionIds so that reducers won't hit same shuffle service while running + // "read_parllelism" tasks. + Collections.shuffle(outputSparkPartitions); + final Queue availableSparkPartitions = new LinkedList<>(outputSparkPartitions); + return getKafkaPartitionOffsetToOutputSparkPartitionMap(topicName, offsetRanges, availableSparkPartitions, + readParallelism); + } + @VisibleForTesting public static Map> getKafkaPartitionOffsetToOutputSparkPartitionMap( - @NotEmpty final String topicName, @NonNull final List offsetRanges, final int readParallelism) { + @NotEmpty final String topicName, @NonNull final List offsetRanges, + @NonNull final Queue availableSparkPartitions, final int readParallelism) { long totalMessages = 0; for (final OffsetRange offsetRange: offsetRanges) { totalMessages += (offsetRange.untilOffset() - offsetRange.fromOffset()); } final long messagesPerPartition = (long) Math.max(Math.ceil(totalMessages * 1.0 / readParallelism), 1); log.info("total Messages for {} :{}", topicName, totalMessages); - final Map> retMap = new HashMap<>(); - Iterator offsetRangesI = offsetRanges.iterator(); - OffsetRange currentOffsetRange = offsetRangesI.next(); - int outputSparkPartition = 0; long remainingSparkPartitionMessageCapacity = messagesPerPartition; - + final Map> retMap = new HashMap<>(); + final Iterator offsetRangesI = offsetRanges.iterator(); + OffsetRange currentOffsetRange = offsetRangesI.next(); while (true) { long currentOffsetRangeMsgCnt = currentOffsetRange.untilOffset() - currentOffsetRange.fromOffset(); if (currentOffsetRangeMsgCnt == 0) { @@ -188,9 +359,11 @@ public static Map> getKafkaPartitionOffsetToOutp } if (remainingSparkPartitionMessageCapacity == 0) { remainingSparkPartitionMessageCapacity = messagesPerPartition; - outputSparkPartition++; + availableSparkPartitions.poll(); } - retMap.get(currentOffsetRange.partition()).put(currentOffsetRange.fromOffset(), outputSparkPartition); + Preconditions.checkState(!availableSparkPartitions.isEmpty(), "missing output spark partitions"); + retMap.get(currentOffsetRange.partition()).put(currentOffsetRange.fromOffset(), + availableSparkPartitions.peek()); if (currentOffsetRangeMsgCnt < remainingSparkPartitionMessageCapacity) { remainingSparkPartitionMessageCapacity -= currentOffsetRangeMsgCnt; @@ -205,9 +378,6 @@ public static Map> getKafkaPartitionOffsetToOutp remainingSparkPartitionMessageCapacity = 0; } } - Preconditions.checkState(outputSparkPartition < readParallelism, - String.format("number of spark partitions can't be larger than read parallelism : found :%s expected<:%s", - outputSparkPartition, readParallelism)); return retMap; } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java index 9dfb65e..3b39dfe 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java @@ -16,7 +16,6 @@ */ package com.uber.marmaray.common.sources.kafka; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; import com.uber.marmaray.common.exceptions.JobRuntimeException; @@ -29,6 +28,8 @@ import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.sources.IWorkUnitCalculator; import com.uber.marmaray.common.sources.kafka.KafkaWorkUnitCalculator.KafkaWorkUnitCalculatorResult; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; import com.uber.marmaray.utilities.StringTypes; import lombok.AllArgsConstructor; import lombok.EqualsAndHashCode; @@ -41,6 +42,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.spark.streaming.kafka010.OffsetRange; import org.hibernate.validator.constraints.NotEmpty; +import parquet.Preconditions; import java.util.ArrayList; import java.util.Collections; @@ -75,9 +77,16 @@ public class KafkaWorkUnitCalculator implements IWorkUnitCalculator kafkaOffsetResetter = Optional.absent(); + @Getter private Optional previousRunState = Optional.absent(); @@ -126,8 +135,12 @@ public void initPreviousRunState(@NonNull final IMetadataManager me this.previousRunState = Optional.of(new KafkaRunState(metadata)); } - @VisibleForTesting - String getTopicSpecificMetadataKey(@NotEmpty final String topicName) { + /** + * Get the specific metadata name for kafka that we'll be using + * @param topicName the name of the topic to get the metadata for + * @return the processed name + */ + public static String getTopicSpecificMetadataKey(@NotEmpty final String topicName) { return String.format("%s%s:", KAFKA_METADATA_WITH_SEPARATOR, topicName); } @@ -164,21 +177,25 @@ public KafkaWorkUnitCalculatorResult computeWorkUnits() { if (oldPartitionOffsets.isEmpty()) { // If it's a first run then initialize new partitions with latest partition offsets. return new KafkaWorkUnitCalculatorResult(Collections.emptyList(), - new KafkaRunState(this.offsetSelector.getPartitionOffsets(this.conf, - latestLeaderOffsets.keySet(), earliestLeaderOffsets, latestLeaderOffsets))); + new KafkaRunState(this.offsetSelector.getPartitionOffsets( + this.conf, + latestLeaderOffsets.keySet(), + earliestLeaderOffsets, + latestLeaderOffsets))); } - updatePartitionStartOffsets(oldPartitionOffsets, earliestLeaderOffsets, latestLeaderOffsets); + final Map newPartitionOffsets = + updatePartitionStartOffsets(oldPartitionOffsets, earliestLeaderOffsets, latestLeaderOffsets); // compute new messages per partition. long totalNewMessages = 0; final List partitionMessages = new ArrayList<>(latestLeaderOffsets.size()); for (Entry entry : latestLeaderOffsets.entrySet()) { - if (!oldPartitionOffsets.containsKey(entry.getKey().partition())) { + if (!newPartitionOffsets.containsKey(entry.getKey().partition())) { log.error("Unable to find offsets for topic {} partition {}", entry.getKey().topic(), entry.getKey().partition()); continue; } - final long messages = entry.getValue() - oldPartitionOffsets.get(entry.getKey().partition()); + final long messages = entry.getValue() - newPartitionOffsets.get(entry.getKey().partition()); log.debug("topicPartition:{}:messages:{}:latestOffset:{}", entry.getKey(), messages, entry.getValue()); if (messages == 0) { continue; @@ -189,10 +206,10 @@ public KafkaWorkUnitCalculatorResult computeWorkUnits() { if (partitionMessages.isEmpty()) { // No messges to read. log.info("No new offsets are found. :{}", topicName); - return new KafkaWorkUnitCalculatorResult(Collections.emptyList(), new KafkaRunState(oldPartitionOffsets)); + return new KafkaWorkUnitCalculatorResult(Collections.emptyList(), new KafkaRunState(newPartitionOffsets)); } final List workUnits = - calculatePartitionOffsetRangesToRead(partitionMessages, oldPartitionOffsets, + calculatePartitionOffsetRangesToRead(partitionMessages, newPartitionOffsets, totalNewMessages); // compute run state for the next run. final KafkaRunState nextRunState = createNextRunState(workUnits); @@ -263,36 +280,86 @@ private KafkaRunState createNextRunState(@NonNull final List workUn return new KafkaRunState(partitionOffsets); } + private Map buildResetPartitionOffsetMap(@NonNull final Map earliestTPOffsets, + @NonNull final Map latestTPOffsets) { + + Preconditions.checkState(kafkaOffsetResetter.isPresent(), "KafkaOffsetResetter should be present " + + "when this method is called"); + final KafkaOffsetResetter kafkaOffsetResetter = this.kafkaOffsetResetter.get(); + return kafkaOffsetResetter.getOffsetSelector().getPartitionOffsets(this.conf, + latestTPOffsets.keySet(), earliestTPOffsets, latestTPOffsets); + + } + + private long computeDifferenceInMessages(@NonNull final Map startPartitionOffsetMap, + @NonNull final Map endPartitionOffsetMap) { + long totalMessagesLost = 0; + for (Entry entry : endPartitionOffsetMap.entrySet()) { + if (!startPartitionOffsetMap.containsKey(entry.getKey())) { + // this partition does not exist in the original offsetMap + totalMessagesLost += entry.getValue(); + continue; + } + totalMessagesLost += entry.getValue() - startPartitionOffsetMap.get(entry.getKey()); + } + return totalMessagesLost; + } + + private void reportDataLoss(@NonNull final Map originalPartitionOffsetMap, + @NonNull final Map resetPartitionOffsetMap) { + Preconditions.checkState(kafkaOffsetResetter.isPresent(), "KafkaOffsetResetter should be present " + + "when this method is called"); + final KafkaOffsetResetter kafkaOffsetResetter = this.kafkaOffsetResetter.get(); + final long totalMessagesLost = computeDifferenceInMessages(originalPartitionOffsetMap, resetPartitionOffsetMap); + kafkaOffsetResetter.getKafkaDataLossReporter().reportDataLoss(this.conf.getTopicName(), + totalMessagesLost); + } + + private Map handleDataLossAndMaybeResetOffsets(@NonNull final Map + earliestTPOffsets, + @NonNull final Map + latestTPOffsets, + @NonNull final TopicPartition topicPartition, + @NonNull final Map partitionOffsetMap, + final long lossStartOffset, + final long lossEndOffset) { + final String errMsg = String.format("DATA_LOSS:MISSED_KAFKA_MESSAGES:topic:%s:partition:%d:" + + "startOffset:%d:endOffset:%d", topicPartition.topic(), + topicPartition.partition(), lossStartOffset, lossEndOffset); + log.error(errMsg); + if (kafkaOffsetResetter.isPresent()) { + final Map resultPartitionMap = buildResetPartitionOffsetMap(earliestTPOffsets, + latestTPOffsets); + reportDataLoss(partitionOffsetMap, resultPartitionMap); + return resultPartitionMap; + } + throw new JobRuntimeException(errMsg); + } + /* It will discover new topic partitions and raise an alert if we lost messages for a topic-partition. */ - private void updatePartitionStartOffsets(@NonNull final Map partitionOffsetMap, - @NonNull final Map earliestTPOffsets, - @NonNull final Map latestTPOffsets) { + private Map updatePartitionStartOffsets(@NonNull final Map partitionOffsetMap, + @NonNull final Map earliestTPOffsets, + @NonNull final Map latestTPOffsets) { if (!partitionOffsetMap.isEmpty()) { - earliestTPOffsets.entrySet().stream().forEach( - entry -> { - final TopicPartition topicPartition = entry.getKey(); - if (!partitionOffsetMap.containsKey(topicPartition.partition())) { - // New partition is found. - log.info("Found new partition for topic:{}:partition:{}", topicPartition.topic(), + for (Entry entry : earliestTPOffsets.entrySet()) { + final TopicPartition topicPartition = entry.getKey(); + if (!partitionOffsetMap.containsKey(topicPartition.partition())) { + // New partition is found. + log.info("Found new partition for topic:{}:partition:{}", topicPartition.topic(), topicPartition.partition()); - partitionOffsetMap.put(topicPartition.partition(), entry.getValue()); - } else if (entry.getValue() > partitionOffsetMap.get(topicPartition.partition())) { - // TODO(omkar): raise an alert? - final String errMsg = - String.format( - "DATA_LOSS:MISSED_KAFKA_MESSAGES:topic:%s:partition:%d:startOffset:%d:endOffset:%d", - topicPartition.topic(), topicPartition.partition(), - partitionOffsetMap.get(topicPartition.partition()), entry.getValue()); - log.error(errMsg); - throw new JobRuntimeException(errMsg); - } + partitionOffsetMap.put(topicPartition.partition(), entry.getValue()); + } else if (entry.getValue() > partitionOffsetMap.get(topicPartition.partition())) { + // data loss detected + return handleDataLossAndMaybeResetOffsets(earliestTPOffsets, latestTPOffsets, + topicPartition, partitionOffsetMap, + partitionOffsetMap.get(topicPartition.partition()), entry.getValue()); } - ); + } } + return partitionOffsetMap; } - /* Creates metrics for the current execution based on the source. */ @@ -393,6 +460,15 @@ public String toString() { .append(workUnit.untilOffset()).append(";")); return sb.toString(); } + + @Override + public IStatus getStatus() { + final BaseStatus status = new BaseStatus(); + if (!this.hasWorkUnits()) { + status.setStatus(IStatus.Status.IN_PROGRESS); + } + return status; + } } // Helper class used for counting and sorting number of messages per partition. diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/LogBasedKafkaDataLossReporter.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/LogBasedKafkaDataLossReporter.java new file mode 100644 index 0000000..12a10ed --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/LogBasedKafkaDataLossReporter.java @@ -0,0 +1,14 @@ +package com.uber.marmaray.common.sources.kafka; + +import com.uber.marmaray.common.reporters.IKafkaDataLossReporter; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +@Slf4j +public class LogBasedKafkaDataLossReporter implements IKafkaDataLossReporter { + + public void reportDataLoss(@NotEmpty final String kafkaTopicName, final long totalNumberOfMessagesLost) { + log.info("Kafka topic hitting loss: {} . Num messages lost: {}.", + kafkaTopicName, totalNumberOfMessagesLost); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/spark/MarmarayKryoSerializer.java b/marmaray/src/main/java/com/uber/marmaray/common/spark/MarmarayKryoSerializer.java new file mode 100644 index 0000000..1141553 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/spark/MarmarayKryoSerializer.java @@ -0,0 +1,70 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.spark; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.factories.ReflectionSerializerFactory; +import com.esotericsoftware.kryo.factories.SerializerFactory; +import com.esotericsoftware.kryo.serializers.FieldSerializer; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Optional; +import lombok.NonNull; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.KryoSerializer; + +/** + * {@link MarmarayKryoSerializer} is a helper class for registering custom serializer instead of + * using KryoSerializer where KryoSerializer is unable to serialize certain objects. + * To use this while running spark job; set "spark.serializer" spark config with + * {@link MarmarayKryoSerializer}'s absolute class name. + */ +public class MarmarayKryoSerializer extends KryoSerializer { + + public MarmarayKryoSerializer(@NonNull final SparkConf sparkConf) { + super(sparkConf); + } + + @Override + public Kryo newKryo() { + final Kryo kryo = super.newKryo(); + kryo.setDefaultSerializer( + new SerializerFactory() { + private final SerializerFactory defaultSerializer = + new ReflectionSerializerFactory(FieldSerializer.class); + @Override + public Serializer makeSerializer(@NonNull final Kryo kryo, @NonNull final Class type) { + final Optional serializer = MarmarayKryoSerializer.this.getSerializer(type); + if (serializer.isPresent()) { + return serializer.get(); + } + return this.defaultSerializer.makeSerializer(kryo, type); + } + } + ); + return kryo; + } + + /** + * @param type classType for which we need a serializer + * @return It returns custom serializer for given class otherwise will return {@link Optional#absent()} + */ + protected Optional getSerializer(@NonNull final Class type) { + return Throwable.class.isAssignableFrom(type) ? Optional.of(new JavaSerializer()) : Optional.absent(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java index 97a8597..0d439e1 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java @@ -17,17 +17,16 @@ package com.uber.marmaray.common.spark; -import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; import java.util.List; -import java.util.Map; -import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NonNull; +import lombok.RequiredArgsConstructor; import lombok.ToString; import org.apache.avro.Schema; @ToString -@AllArgsConstructor +@RequiredArgsConstructor public class SparkArgs { /** @@ -35,24 +34,29 @@ public class SparkArgs { */ @Getter @NonNull - private final Optional> avroSchemas; + private final List avroSchemas; /** * User serialization classes to be added for kryo serialization */ @Getter @NonNull private final List userSerializationClasses; + /** - * Other spark properties provided to override defaults + * Configuration object */ @Getter @NonNull - private final Map overrideSparkProperties; + private final Configuration configuration; /** - * Hadoop Configuration to be added as a resource to SparkContext + * Flag to indicate whether Hive support is needed for SparkSession. + * It is set to false by default. Use {@link #enableHiveSupport()} to turn it on. */ @Getter - @NonNull - private final org.apache.hadoop.conf.Configuration hadoopConfiguration; + private boolean hiveSupportEnabled = false; + + public void enableHiveSupport() { + this.hiveSupportEnabled = true; + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java index 65d6d86..7e700c7 100644 --- a/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java +++ b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java @@ -24,10 +24,12 @@ import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.configuration.ErrorTableConfiguration; +import com.uber.marmaray.common.configuration.HadoopConfiguration; import com.uber.marmaray.common.configuration.HiveConfiguration; import com.uber.marmaray.common.configuration.HiveSourceConfiguration; import com.uber.marmaray.common.configuration.HoodieConfiguration; import com.uber.marmaray.common.configuration.KafkaConfiguration; +import com.uber.marmaray.common.configuration.SparkConfiguration; import com.uber.marmaray.common.converters.converterresult.ConverterResult; import com.uber.marmaray.common.converters.data.AbstractDataConverter; import com.uber.marmaray.common.data.BinaryRawData; @@ -48,14 +50,9 @@ import com.uber.marmaray.utilities.SparkUtil; import com.uber.marmaray.utilities.TimestampInfo; import com.uber.marmaray.utilities.listener.SparkEventListener; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; +import lombok.Getter; import lombok.NonNull; +import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.avro.util.Utf8; import org.apache.spark.SparkConf; @@ -67,43 +64,52 @@ import scala.collection.JavaConverters; import scala.collection.mutable.WrappedArray.ofRef; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + /** * {@link SparkFactory} is responsible for creating any Spark related resource such as * {@link JavaSparkContext} or {@link SparkSession}, or even {@link SparkConf} which is required * to create the former two. Pass this object around in code to access the above mentioned resources. */ @Slf4j +@RequiredArgsConstructor public class SparkFactory { - private Optional sparkSession = Optional.absent(); - + @NonNull + @Getter + private final SparkArgs sparkArgs; + private Optional sparkSessionOptional = Optional.absent(); /** * Uses {@link SparkSession} returned from {@link SparkFactory#getSparkSession} * to create {@link JavaSparkContext}. See {@link SparkFactory#getSparkSession} * for {@link SparkSession} that is retrieved. */ - public synchronized JavaSparkContext getSparkContext( - @NonNull final SparkArgs sparkArgs) { - return new JavaSparkContext(getSparkSession(sparkArgs, false).sparkContext()); + public synchronized JavaSparkContext getSparkContext() { + return new JavaSparkContext(getSparkSession().sparkContext()); } /** * Uses existing {@link SparkSession} if present, else creates a new one */ - public synchronized SparkSession getSparkSession( - @NonNull final SparkArgs sparkArgs, final boolean enableHiveSupport) { - if (this.sparkSession.isPresent()) { - return sparkSession.get(); + public synchronized SparkSession getSparkSession() { + if (this.sparkSessionOptional.isPresent()) { + return this.sparkSessionOptional.get(); } final Builder sparkSessionBuilder = SparkSession.builder(); - if (enableHiveSupport) { + if (this.sparkArgs.isHiveSupportEnabled()) { sparkSessionBuilder.enableHiveSupport(); } - this.sparkSession = Optional.of(sparkSessionBuilder - .config(createSparkConf(sparkArgs)).getOrCreate()); + this.sparkSessionOptional = Optional.of(sparkSessionBuilder + .config(createSparkConf()).getOrCreate()); log.info("Created new SparkSession using {}", sparkArgs); - updateSparkContext(sparkArgs, this.sparkSession.get().sparkContext()); - return this.sparkSession.get(); + updateSparkContext(sparkArgs, this.sparkSessionOptional.get().sparkContext()); + return this.sparkSessionOptional.get(); } /** @@ -111,27 +117,31 @@ public synchronized SparkSession getSparkSession( * registering default/user-input serializable classes and user-input Avro Schemas. * Once {@link SparkContext} is created, we can no longer register serialization classes and Avro schemas. */ - public SparkConf createSparkConf(@NonNull final SparkArgs sparkArgs) { + public SparkConf createSparkConf() { /** * By custom registering classes the full class name of each object * is not stored during serialization which reduces storage space. */ final SparkConf sparkConf = new SparkConf(); - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + sparkConf.set("spark.serializer", MarmarayKryoSerializer.class.getName()); + // We don't want to cache too many connections on kafka consumer so we will be limiting it to 4 per executor + // default value is 64. See org.apache.spark.streaming.kafka010.KafkaRDD for more information + sparkConf.set("spark.streaming.kafka.consumer.cache.initialCapacity", "4"); + sparkConf.set("spark.streaming.kafka.consumer.cache.maxCapacity", "4"); + final List serializableClasses = getDefaultSerializableClasses(); - serializableClasses.addAll(sparkArgs.getUserSerializationClasses()); + serializableClasses.addAll(this.sparkArgs.getUserSerializationClasses()); sparkConf.registerKryoClasses(serializableClasses.toArray(new Class[0])); - if (sparkArgs.getAvroSchemas().isPresent()) { - sparkConf.registerAvroSchemas( - JavaConverters - .iterableAsScalaIterableConverter(sparkArgs.getAvroSchemas().get()) - .asScala() - .toSeq()); - } + sparkConf.registerAvroSchemas( + JavaConverters + .iterableAsScalaIterableConverter(this.sparkArgs.getAvroSchemas()) + .asScala() + .toSeq()); // override spark properties - final Map sparkProps = sparkArgs.getOverrideSparkProperties(); + final Map sparkProps = SparkConfiguration + .getOverrideSparkProperties(this.sparkArgs.getConfiguration()); for (Entry entry : sparkProps.entrySet()) { log.info("Setting spark key:val {} : {}", entry.getKey(), entry.getValue()); sparkConf.set(entry.getKey(), entry.getValue()); @@ -139,6 +149,16 @@ public SparkConf createSparkConf(@NonNull final SparkArgs sparkArgs) { return sparkConf; } + /** + * Stops any existing {@link SparkSession} and removes reference to it + */ + public synchronized void stop() { + if (this.sparkSessionOptional.isPresent()) { + this.sparkSessionOptional.get().stop(); + this.sparkSessionOptional = Optional.absent(); + } + } + /** * Hook for plugging in custom SparkListers */ @@ -151,14 +171,14 @@ private void updateSparkContext(@NonNull final SparkArgs sparkArgs, for (SparkListener sparkListener : getSparkEventListeners()) { sc.addSparkListener(sparkListener); } - sc.hadoopConfiguration().addResource(sparkArgs.getHadoopConfiguration()); + sc.hadoopConfiguration().addResource( + new HadoopConfiguration(sparkArgs.getConfiguration()).getHadoopConf()); } private List getDefaultSerializableClasses() { final List serializableClasses = new LinkedList(Arrays.asList( AbstractDataConverter.class, AbstractValue.class, - AvroPayload.class, ArrayList.class, BinaryRawData.class, CassandraDataField.class, @@ -193,6 +213,7 @@ private List getDefaultSerializableClasses() { Optional.absent().getClass(), Utf8.class, Class.class)); + serializableClasses.addAll(AvroPayload.getSerializationClasses()); SparkUtil.addClassesIfFound(serializableClasses, Arrays.asList( diff --git a/marmaray/src/main/java/com/uber/marmaray/common/status/BaseStatus.java b/marmaray/src/main/java/com/uber/marmaray/common/status/BaseStatus.java new file mode 100644 index 0000000..b748303 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/status/BaseStatus.java @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.status; + +import lombok.Getter; +import lombok.NonNull; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; + +/** + * Base implementation of the {@link IStatus} interface. Defaults to success, but status can be downgraded. + */ +public class BaseStatus implements IStatus { + + @Getter + private Status status = Status.SUCCESS; + @Getter + private final List exceptions = new LinkedList<>(); + + public void setStatus(@NonNull final Status inputStatus) { + if (inputStatus.compareTo(this.status) > 0) { + this.status = inputStatus; + } + } + + public void addException(@NonNull final Exception t) { + this.exceptions.add(t); + } + + public void addExceptions(@NonNull final Collection throwableCollection) { + throwableCollection.forEach(this::addException); + } + + public void mergeStatus(@NonNull final IStatus inputStatus) { + setStatus(inputStatus.getStatus()); + this.addExceptions(inputStatus.getExceptions()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/status/IStatus.java b/marmaray/src/main/java/com/uber/marmaray/common/status/IStatus.java new file mode 100644 index 0000000..e0b9882 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/status/IStatus.java @@ -0,0 +1,42 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.status; + +import java.util.List; + +/** + * Return the status of a particular component. Status will be {@link Status#SUCCESS} if step is complete, + * {@link Status#IN_PROGRESS} if there will be more work to do, and {@link Status#FAILURE} if the component has an + * error. Status can be downgraded, but never upgraded. + */ +public interface IStatus { + + enum Status { + SUCCESS, + IN_PROGRESS, + FAILURE + } + + /** + * Return the status of this particular run. + */ + Status getStatus(); + + List getExceptions(); + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/status/JobManagerStatus.java b/marmaray/src/main/java/com/uber/marmaray/common/status/JobManagerStatus.java new file mode 100644 index 0000000..bf20bb8 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/status/JobManagerStatus.java @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.status; + +import lombok.Getter; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Implemenation of {@link IStatus} designed for return by {@link com.uber.marmaray.common.job.JobManager}. It maintains + * a map of IStatus, one for each {@link com.uber.marmaray.common.job.JobDag}, or other collection of statuses to store. + */ +public class JobManagerStatus extends BaseStatus { + + @Getter + private final Map jobStatuses = new HashMap<>(); + + @Override + public List getExceptions() { + final List jobExceptions = this.jobStatuses.values().stream() + .map(IStatus::getExceptions) + .flatMap(List::stream) + .collect(Collectors.toList()); + jobExceptions.addAll(super.getExceptions()); + return jobExceptions; + } + + public void addJobStatus(@NotEmpty final String name, @NonNull final IStatus status) { + this.jobStatuses.put(name, status); + this.setStatus(status.getStatus()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java index 9723f10..b416abf 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.PrintWriter; +import java.util.concurrent.TimeUnit; + import lombok.NonNull; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -37,12 +39,14 @@ * {@link CommandLineUtil} provides utility methods to interact with the command line */ public class CommandLineUtil { - public static String executeCommand(@NotEmpty final String cmd) { + public static String executeCommand(@NotEmpty final String cmd, + final long timeout, + @NonNull final TimeUnit unit) { final StringBuffer outputBuffer = new StringBuffer(); try { final Process process = Runtime.getRuntime().exec(cmd); - process.waitFor(); + process.waitFor(timeout, unit); String line; try (final BufferedReader br = new BufferedReader(new InputStreamReader(process.getInputStream()))) { @@ -52,6 +56,7 @@ public static String executeCommand(@NotEmpty final String cmd) { } } catch (IOException | InterruptedException e) { throw new JobRuntimeException("Exception occurred while executing command: " + cmd + + " , timeout: " + String.valueOf(timeout) + ", unit: " + unit.name() + " Error Message: " + e.getMessage(), e); } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java index 267a6c4..f866958 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java @@ -46,7 +46,7 @@ public static void checkMandatoryProperties(@NonNull final Configuration conf, mandatoryProps.stream().forEach( prop -> { if (!conf.getProperty(prop).isPresent()) { - log.error("Missing property:{}", prop); + log.error("Missing property:{} existing conf:{}", prop, conf); throw new MissingPropertyException("property:" + prop); } }); diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java index 4322c0d..0a0446f 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java @@ -38,6 +38,7 @@ import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; import org.hibernate.validator.constraints.NotEmpty; import lombok.NonNull; @@ -47,8 +48,10 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; +import java.util.Random; import java.util.concurrent.atomic.AtomicBoolean; +import static com.uber.marmaray.common.configuration.SparkConfiguration.SPARK_PROPERTIES_KEY_PREFIX; import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT; /** @@ -62,12 +65,13 @@ public final class ErrorTableUtil { public static final String HADOOP_ERROR_EXCEPTION = "hadoop_error_exception"; public static final String HADOOP_CHANGELOG_COLUMNS = "Hadoop_Changelog_Columns"; public static final String HADOOP_APPLICATION_ID = "hadoop_application_id"; - public static final String HOODIE_RECORD_KEY = "Hoodie_record_key_constant"; + public static final String HOODIE_RECORD_KEY = "Hoodie_record_key_constant_%s"; public static final String ERROR_SCHEMA_IDENTIFIER = "spark.user.ERROR_SCHEMA"; public static final String ERROR_TABLE_SUFFIX = "_error"; public static final String TABLE_KEY = "spark.user.table_key"; public static final String ERROR_TABLE_KEY = "spark.user.error_table_key"; - + public static final Integer ERROR_ROW_KEY_SUFFIX_MAX = 256; + public static final Integer ERROR_TABLE_RANDOM_SEED_VALUE = 1; /** * Default flag to control whether error table metrics is enabled */ @@ -122,11 +126,19 @@ public static void writeErrorRecordsToErrorTable(@NonNull final SparkContext sc, JavaRDD errorRecords = errorData.getData().map(error -> generateGenericErrorRecord( errorExtractor, errorTableSchema, error, applicationId)); - JavaRDD> hoodieRecords = errorRecords.map(record -> - { - final HoodieKey hoodieKey = new HoodieKey(HOODIE_RECORD_KEY, partitionPath); - HoodieErrorPayload payload = new HoodieErrorPayload(record); - return new HoodieRecord<>(hoodieKey, payload); + JavaRDD> hoodieRecords = errorRecords.map( + new Function>() { + + final Random randomRowKeySuffixGenerator = new Random(ERROR_TABLE_RANDOM_SEED_VALUE); + + @Override + public HoodieRecord call(final GenericRecord genericRecord) { + final HoodieKey hoodieKey = new HoodieKey( + String.format(HOODIE_RECORD_KEY, + randomRowKeySuffixGenerator.nextInt(ERROR_ROW_KEY_SUFFIX_MAX)), partitionPath); + HoodieErrorPayload payload = new HoodieErrorPayload(genericRecord); + return new HoodieRecord<>(hoodieKey, payload); + } } ); @@ -143,12 +155,24 @@ public static void writeErrorRecordsToErrorTable(@NonNull final SparkContext sc, public static void initErrorTableDataset(@NonNull final Configuration conf, @NotEmpty final String errorTableName) throws IOException { final ErrorTableConfiguration errorTableConf = new ErrorTableConfiguration(conf); - final HoodieConfiguration hoodieConf = HoodieConfiguration.newBuilder(errorTableName) + final HoodieConfiguration hoodieConf = HoodieConfiguration.newBuilder(conf, errorTableName) .withBasePath(errorTableConf.getDestPath().toString()) .withTableName(errorTableName) .enableMetrics(false) .build(); - HoodieUtil.initHoodieDataset(FSUtils.getFs(conf), hoodieConf); + HoodieUtil.initHoodieDataset(FSUtils.getFs(conf, Optional.of(hoodieConf.getBasePath())), hoodieConf); + } + + public static void addErrorSchemaConfiguration( + @NonNull final Configuration configuration, @NonNull final Schema errorSchema, + @NotEmpty final String tableKey, @NotEmpty final String errorTableKey) { + // Add Error schema, target table and error table for spark conf to be retrieved later + configuration.setProperty( + SPARK_PROPERTIES_KEY_PREFIX + ERROR_SCHEMA_IDENTIFIER, errorSchema.toString()); + configuration.setProperty( + SPARK_PROPERTIES_KEY_PREFIX + TABLE_KEY, tableKey); + configuration.setProperty( + SPARK_PROPERTIES_KEY_PREFIX + ERROR_TABLE_KEY, errorTableKey); } private static GenericRecord generateGenericErrorRecord(@NonNull final ErrorExtractor errorExtractor, diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java b/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java index d41950e..790f224 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java @@ -19,6 +19,7 @@ import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.configuration.HadoopConfiguration; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.google.common.base.Optional; import java.io.IOException; import java.util.Arrays; import java.util.Comparator; @@ -44,10 +45,21 @@ public class FSUtils { Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName())); /** - * It returns FileSystem based on fs.defaultFS property defined in conf. + * It returns FileSystem based on Hadoop configuration which is built from the conf passed and an optional path */ - public static FileSystem getFs(final Configuration conf) throws IOException { - return FileSystem.get((new HadoopConfiguration(conf)).getHadoopConf()); + public static FileSystem getFs(@NonNull final Configuration conf, + @NonNull final Optional path) throws IOException { + return getFs(new HadoopConfiguration(conf).getHadoopConf(), path); + } + + /** + * It returns FileSystem based on Hadoop Configuration and the Optional Path param + */ + public static FileSystem getFs(@NonNull final org.apache.hadoop.conf.Configuration hadoopConf, + @NonNull final Optional path) throws IOException { + final FileSystem fs = path.isPresent() ? new Path(path.get()).getFileSystem(hadoopConf) + : FileSystem.get(hadoopConf); + return fs; } public static void deleteHDFSMetadataFiles(@NonNull final FileStatus[] fileStatuses, diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java index c312a7c..870767f 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.utilities; +import com.google.common.base.Optional; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableConfig; @@ -53,7 +54,8 @@ public static void initHoodieDataset(@NonNull final FileSystem fs, HoodieTableConfig.HOODIE_PROPERTIES_FILE); if (!fs.exists(hoodiePropertiesFile)) { HoodieTableMetaClient - .initializePathAsHoodieDataset(FSUtils.getFs(hoodieConf.getConf()), + .initializePathAsHoodieDataset(FSUtils.getFs(hoodieConf.getConf(), + Optional.of(hoodieConf.getBasePath())), hoodieConf.getBasePath(), hoodieConf.getHoodieInitProperties()); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java index 7f8ef99..43bdd78 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java @@ -17,6 +17,8 @@ package com.uber.marmaray.utilities; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.status.IStatus; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.hibernate.validator.constraints.NotEmpty; @@ -36,6 +38,7 @@ private JobUtil() { throw new JobRuntimeException("This utility class should not be instantiated"); } + // TODO: clean up datacenter configs public static String getDataCenterForJob(@NotEmpty final String dcPath) throws IOException { log.info("Looking up datacenter information in: {}", dcPath); final File dcFile = new File(dcPath); @@ -45,4 +48,14 @@ public static String getDataCenterForJob(@NotEmpty final String dcPath) throws I return scanner.next(); } } + + public static void raiseExceptionIfStatusFailed(@NonNull final IStatus status) { + if (IStatus.Status.FAILURE.equals(status.getStatus())) { + if (status.getExceptions().isEmpty()) { + throw new JobRuntimeException("Job has failed"); + } else { + throw new JobRuntimeException(status.getExceptions().get(0)); + } + } + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java index 1e9941c..ed726f6 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java @@ -27,11 +27,6 @@ @Slf4j public class KafkaSourceConverterErrorExtractor extends ErrorExtractor { - @Override - public String getRowKey(@NonNull final RawData rawdata) { - return DEFAULT_ROW_KEY; - } - @Override public String getChangeLogColumns(@NonNull final RawData rawdata) { return DEFAULT_CHANGELOG_COLUMNS; diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java index 0112a55..f557496 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java @@ -76,8 +76,13 @@ public static Map getTopicPartitionOffsets(@NonNull final final AtomicInteger attemptNumber = new AtomicInteger(0); verifyTopicPartitions(kafkaConsumer, topicName, topicPartitions); final Callable fetchOffsetTask = () -> { - log.info("requesting topicPartitions for {} - % success {}/{} - attemptNumber - {}", topicName, - partitionOffsets.size(), topicPartitions.size(), attemptNumber.incrementAndGet()); + log.info( + "requesting topicPartitions for {} - % success {}/{} - attemptNumber - {}", + topicName, + partitionOffsets.size(), + topicPartitions.size(), + attemptNumber.incrementAndGet() + ); topicPartitions.stream().forEach( tp -> { try { @@ -119,8 +124,8 @@ public static Map getTopicPartitionOffsets(@NonNull final * Helper method to verify that given kafka topic has all passed in topicPartitions. */ public static void verifyTopicPartitions(@NonNull final KafkaConsumer kafkaConsumer, - @NotEmpty final String topicName, @NonNull final Set topicPartitions) { - Set partitions = new HashSet<>(); + @NotEmpty final String topicName, @NonNull final Set topicPartitions) { + final Set partitions = new HashSet<>(); topicPartitions.stream().forEach( tp -> { partitions.add(tp.partition()); @@ -225,4 +230,5 @@ public static Map getKafkaParams(@NonNull final KafkaConfigurati ); return newKafkaParams; } + } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/LongAccumulator.java b/marmaray/src/main/java/com/uber/marmaray/utilities/LongAccumulator.java new file mode 100644 index 0000000..c678944 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/LongAccumulator.java @@ -0,0 +1,84 @@ +package com.uber.marmaray.utilities; + +import org.apache.spark.util.AccumulatorV2; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import java.io.Serializable; + +@Slf4j +public final class LongAccumulator extends AccumulatorV2 implements Serializable { + @Getter + private Long max; + @Getter + private Long sum; + @Getter + private Long count; + @Getter + private String name; + + public LongAccumulator(@NonNull final String name) { + this.name = name; + this.max = Long.MIN_VALUE; + this.sum = 0L; + this.count = 0L; + } + + @Override + public void add(@NonNull final Long v) { + this.sum += v; + this.count += 1; + this.max = Math.max(max, v); + } + + @Override + public void reset() { + this.sum = 0L; + this.count = 0L; + this.max = 0L; + } + + public double getAvg() { + return this.sum / this.count; + } + + @Override + public LongAccumulator copy() { + final LongAccumulator longAccumulator = new LongAccumulator(this.getName()); + longAccumulator.sum = this.getSum(); + longAccumulator.count = this.getCount(); + longAccumulator.max = this.getMax(); + return longAccumulator; + } + + @Override + public boolean isZero() { + return this.getSum() == 0L && this.getCount() == 0L; + } + + @Override + public Long value() { + return this.getSum(); + } + + @Override + public void merge(@NonNull final AccumulatorV2 other) { + if (other instanceof LongAccumulator) { + this.sum += ((LongAccumulator) other).getSum(); + this.count += ((LongAccumulator) other).getCount(); + this.max = Math.max(this.getMax(), ((LongAccumulator) other).getMax()); + } else { + final String warnMsg = String.format("Cannot merge {} with {}", + this.getClass().getName(), other.getClass().getName()); + throw new UnsupportedOperationException(warnMsg); + } + } + + @Override + public String toString() { + return String.format("name: %s, value: %s, count: %s, max:%s", + this.getName(), this.getSum(), this.getCount(), this.getMax()); + } + + // TODO add percentile support for this accumulator to better understand distribution +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java index 697f8f8..3a034ff 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java @@ -17,9 +17,16 @@ package com.uber.marmaray.utilities; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -34,6 +41,7 @@ import org.hibernate.validator.constraints.NotEmpty; import java.io.IOException; +import java.sql.Timestamp; /** * {@link SchemaUtil} defines utility methods for working with schemas @@ -42,6 +50,8 @@ public final class SchemaUtil { public static final String DISPERSAL_TIMESTAMP = "dispersal_timestamp"; + public static final String TIMESTAMP_PROPERTY = "timestamp"; + public static final String TRUE = "true"; private SchemaUtil() { throw new JobRuntimeException("This utility class should never be instantiated"); @@ -58,11 +68,44 @@ private SchemaUtil() { * @throws IOException */ public static StructType generateSchemaFromParquet(@NonNull final FileSystem fs, - @NotEmpty final String parquetDir) throws IOException { - - log.info("Searching {} for parquet files", parquetDir); - - final FileStatus[] fileStatuses = fs.listStatus(new Path(parquetDir)); + @NotEmpty final String parquetDir, + @NonNull final Optional dataFeedMetrics) + throws IOException { + + log.info("Searching: {} for parquet files", parquetDir); + // TODO : wrong path causes list status failure + final FileStatus[] fileStatuses; + final Path filePath = new Path(parquetDir); + if (fs.exists(filePath)) { + try { + fileStatuses = fs.listStatus(filePath); + } catch (Exception e) { + log.error("Failed listing files to read, possible permission issue"); + if (dataFeedMetrics.isPresent()) { + dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_CONFIG_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE, ErrorCauseTagNames.PERMISSON)); + } + throw new JobRuntimeException("Failed at listing hive files", e); + } + } else { + log.error("Data file path does not exist"); + if (dataFeedMetrics.isPresent()) { + dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_CONFIG_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE, ErrorCauseTagNames.NO_FILE)); + } + throw new JobRuntimeException("path does not exist: " + parquetDir); + } + if (fileStatuses.length == 0) { + log.error("Data file path is empty"); + if (dataFeedMetrics.isPresent()) { + dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_CONFIG_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE, ErrorCauseTagNames.EMPTY_PATH)); + } + throw new JobRuntimeException("empty path:" + parquetDir); + } Optional parquetFilePath = Optional.absent(); @@ -81,12 +124,19 @@ public static StructType generateSchemaFromParquet(@NonNull final FileSystem fs, log.info("Reading schema data from : {}", lastFile.getPath().toString()); parquetFilePath = Optional.of(lastFile.getPath()); } else { - throw new JobRuntimeException("Encountered a directory where there should only be files. Path: " - + lastFile.getPath().toString()); + // support multiple partitions + log.info("Found another directory {}.", lastFile.getPath().getName()); + return generateSchemaFromParquet(fs, lastFile.getPath().toString(), dataFeedMetrics); } } if (!parquetFilePath.isPresent()) { + log.error("No file was found in path: {}", parquetDir); + if (dataFeedMetrics.isPresent()) { + dataFeedMetrics.get().createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_CONFIG_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SOURCE, ErrorCauseTagNames.EMPTY_PATH)); + } throw new JobRuntimeException("No files were found in path: " + parquetDir); } @@ -100,4 +150,51 @@ public static StructType generateSchemaFromParquet(@NonNull final FileSystem fs, final StructType structType = converter.convert(messageType); return structType; } + + /** + * This utility will determine the internal version of timestamp. + * Required as avro 1.7.7 doesn't support timestamps natively. + * @param nullable if field is nullable + * @return schema definition for timestamp + */ + public static Schema getTimestampSchema(final boolean nullable) { + if (nullable) { + return SchemaBuilder.builder().nullable().longBuilder().prop(TIMESTAMP_PROPERTY, TRUE).endLong(); + } else { + return SchemaBuilder.builder().longBuilder().prop(TIMESTAMP_PROPERTY, TRUE).endLong(); + + } + } + + /** + * This utility will verify if the schema is representing a timestamp field. + * Required as avro 1.7.7 doesn't support timestamps natively. + * @param schema schema to verify + * @return true if schema is representing a timetsamp + */ + public static boolean isTimestampSchema(@NonNull final Schema schema) { + final Schema nonNullSchema = GenericRecordUtil.isOptional(schema) ? GenericRecordUtil.getNonNull(schema) + : schema; + return Schema.Type.LONG.equals(nonNullSchema.getType()) + && TRUE.equals(nonNullSchema.getProp(TIMESTAMP_PROPERTY)); + } + + /** + * Encode {@link Timestamp} into our internal format, ms since epoch + * @param timestamp The timestamp to encode + * @return the value that's encoded + */ + public static Object encodeTimestamp(@NonNull final Timestamp timestamp) { + return timestamp.getTime(); + } + + /** + * Decode timestamp from our internal format, ms since epoch + * @param objectTs the encoded value + * @return the decoded {@link Timestamp} object + */ + public static Timestamp decodeTimestamp(@NonNull final Object objectTs) { + Preconditions.checkArgument(objectTs instanceof Long, "Invalid object to decode"); + return new Timestamp((Long) objectTs); + } } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/SizeUnit.java b/marmaray/src/main/java/com/uber/marmaray/utilities/SizeUnit.java new file mode 100644 index 0000000..1004cdd --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/SizeUnit.java @@ -0,0 +1,51 @@ +package com.uber.marmaray.utilities; + +/** + * Conversion class for size units (bits and bytes) using 1024 as a factor between the levels. + * Similar to {@link java.util.concurrent.TimeUnit}. + * + * Note that conversions may overflow Long when going from extremely large coarse units to finer ones. + */ +public enum SizeUnit { + + BITS(Constants.BIT_FACTOR), + BYTES(Constants.BYTE_FACTOR), + KILOBYTES(Constants.KILOBYTE_FACTOR), + MEGABYTES(Constants.MEGABYTE_FACTOR), + GIGABYTES(Constants.GIGABYTE_FACTOR); + + private final long factor; + + SizeUnit(final long factor) { + this.factor = factor; + } + + public long toBits(final long input) { + return this.factor * input; + } + + public long toBytes(final long input) { + return this.factor * input / Constants.BYTE_FACTOR; + } + + public long toKilobytes(final long input) { + return this.factor * input / Constants.KILOBYTE_FACTOR; + } + + public long toMegabytes(final long input) { + return this.factor * input / Constants.MEGABYTE_FACTOR; + } + + public long toGigabytes(final long input) { + return this.factor * input / Constants.GIGABYTE_FACTOR; + } + + private static final class Constants { + public static final int UNIT_SEPARATOR = 1024; + public static final long BIT_FACTOR = 1; + public static final long BYTE_FACTOR = 8; + public static final long KILOBYTE_FACTOR = BYTE_FACTOR * UNIT_SEPARATOR; + public static final long MEGABYTE_FACTOR = KILOBYTE_FACTOR * UNIT_SEPARATOR; + public static final long GIGABYTE_FACTOR = MEGABYTE_FACTOR * UNIT_SEPARATOR; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java index 19bb9ce..57ecf85 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java @@ -18,60 +18,23 @@ import com.google.common.base.Optional; import com.google.common.collect.Sets; -import com.uber.hoodie.common.model.HoodieKey; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.marmaray.common.AvroPayload; -import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; -import com.uber.marmaray.common.configuration.Configuration; -import com.uber.marmaray.common.configuration.ErrorTableConfiguration; -import com.uber.marmaray.common.configuration.HiveConfiguration; -import com.uber.marmaray.common.configuration.HiveSourceConfiguration; -import com.uber.marmaray.common.configuration.HoodieConfiguration; -import com.uber.marmaray.common.configuration.KafkaConfiguration; -import com.uber.marmaray.common.converters.converterresult.ConverterResult; -import com.uber.marmaray.common.converters.data.AbstractDataConverter; -import com.uber.marmaray.common.data.BinaryRawData; -import com.uber.marmaray.common.data.ErrorData; -import com.uber.marmaray.common.data.ForkData; -import com.uber.marmaray.common.data.RDDWrapper; -import com.uber.marmaray.common.data.RawData; -import com.uber.marmaray.common.data.ValidData; -import com.uber.marmaray.common.dataset.UtilRecord; import com.uber.marmaray.common.exceptions.JobRuntimeException; -import com.uber.marmaray.common.forkoperator.ForkOperator; -import com.uber.marmaray.common.metadata.AbstractValue; -import com.uber.marmaray.common.schema.cassandra.CassandraDataField; -import com.uber.marmaray.common.schema.cassandra.CassandraSchema; -import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField; -import com.uber.marmaray.common.schema.cassandra.ClusterKey; -import com.uber.marmaray.common.sinks.hoodie.HoodieSink; -import com.uber.marmaray.common.sinks.hoodie.HoodieWriteStatus; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; -import org.apache.avro.util.Utf8; -import org.apache.spark.SparkConf; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.SparkContext; import org.apache.spark.SparkEnv; -import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.storage.RDDInfo; -import org.hibernate.validator.constraints.NotEmpty; -import scala.collection.JavaConverters; +import scala.reflect.ClassManifestFactory; import scala.reflect.ClassTag; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.Map.Entry; import java.util.Set; /** @@ -79,94 +42,16 @@ */ @Slf4j public final class SparkUtil { + public static final ClassTag GENERIC_RECORD_CLASS_TAG = + ClassManifestFactory.fromClass(GenericRecord.class); + public static final ClassTag OBJECT_CLASS_TAG = ClassManifestFactory.fromClass(Object.class); - public static final String SPARK_PROPERTIES_KEY_PREFIX = "spark_properties."; private static ThreadLocal serializerInstance = new ThreadLocal<>(); private SparkUtil() { throw new JobRuntimeException("This utility class should never be instantiated"); } - /** - * @param avroSchemas avro schemas to be added to spark context for serialization - * @param userSerializationClasses serialization classes to be added for kryo serialization - * @param configuration config class to read and apply spark properties if present - */ - public static SparkConf getSparkConf(@NotEmpty final String appName, - @NonNull final Optional> avroSchemas, - @NonNull final List userSerializationClasses, - @NonNull final Configuration configuration) { - final SparkConf sparkConf = new SparkConf().setAppName(appName); - - /** - * By custom registering classes the full class name of each object is not stored during serialization - * which reduces storage space. - * - * Note: We don't have a way to enforce new classes which need to be serialized are added to this list. - * We should think about adding a hook to ensure this list is current. - */ - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - final List serializableClasses = new LinkedList(Arrays.asList( - AbstractDataConverter.class, - AbstractValue.class, - AvroPayload.class, - ArrayList.class, - BinaryRawData.class, - CassandraDataField.class, - CassandraSchema.class, - CassandraSchemaField.class, - CassandraSinkConfiguration.class, - ClusterKey.class, - Configuration.class, - ConverterResult.class, - ErrorTableConfiguration.class, - ErrorData.class, - ForkData.class, - ForkOperator.class, - HiveConfiguration.class, - HiveSourceConfiguration.class, - HoodieConfiguration.class, - HoodieSink.class, - HoodieRecord.class, - HoodieKey.class, - HoodieWriteStatus.class, - KafkaConfiguration.class, - java.util.Optional.class, - Optional.class, - RawData.class, - RDDWrapper.class, - scala.collection.mutable.WrappedArray.ofRef.class, - Object[].class, - TimestampInfo.class, - UtilRecord.class, - ValidData.class, - HashMap.class, - Optional.absent().getClass(), - Utf8.class, - Class.class)); - addClassesIfFound(serializableClasses, - Arrays.asList("com.google.common.base.Present", - "scala.reflect.ClassTag$$anon$1")); - serializableClasses.addAll(userSerializationClasses); - sparkConf.registerKryoClasses(serializableClasses.toArray(new Class[0])); - - if (avroSchemas.isPresent()) { - sparkConf.registerAvroSchemas( - JavaConverters.iterableAsScalaIterableConverter(avroSchemas.get()) - .asScala() - .toSeq()); - } - - // override spark properties - final Map sparkProps = configuration - .getPropertiesWithPrefix(SPARK_PROPERTIES_KEY_PREFIX, true); - for (Entry entry : sparkProps.entrySet()) { - log.info("Setting spark key:val {} : {}", entry.getKey(), entry.getValue()); - sparkConf.set(entry.getKey(), entry.getValue()); - } - return sparkConf; - } - public static void addClassesIfFound(@NonNull final List serializableClasses, @NonNull final List classList) { for (final String className : classList) { @@ -181,7 +66,7 @@ public static void addClassesIfFound(@NonNull final List serializableClas public static Set getSupportedDataTypes() { return Collections.unmodifiableSet(Sets.newHashSet(DataTypes.StringType, DataTypes.IntegerType, DataTypes.LongType, DataTypes.BooleanType, DataTypes.DoubleType, - DataTypes.ShortType, DataTypes.FloatType)); + DataTypes.ShortType, DataTypes.FloatType, DataTypes.TimestampType, DataTypes.BinaryType)); } /** @@ -190,18 +75,25 @@ public static Set getSupportedDataTypes() { */ public static SerializerInstance getSerializerInstance() { if (serializerInstance.get() == null) { - serializerInstance.set(new KryoSerializer(SparkEnv.get().conf()).newInstance()); + serializerInstance.set(SparkEnv.get().serializer().newInstance()); } return serializerInstance.get(); } - public static > T deserialize(@NonNull final byte[] serializedRecord, + public static > T deserialize(final byte[] serializedRecord, @NonNull final K classTag) { + if (serializedRecord == null) { + return null; + } return getSerializerInstance().deserialize(ByteBuffer.wrap(serializedRecord), classTag); } - public static > byte[] serialize(@NonNull final T record, @NonNull final K classTag) { - return getSerializerInstance().serialize(record, classTag).array(); + public static > byte[] serialize(final T record, @NonNull final K classTag) { + if (record == null) { + return null; + } + final byte[] serializedData = getSerializerInstance().serialize(record, classTag).array(); + return serializedData; } public static Optional getRddInfo(@NonNull final SparkContext sc, final int rddId) { @@ -231,4 +123,5 @@ public static SparkSession getOrCreateSparkSession() { log.info(sparkWarning); return SparkSession.builder().getOrCreate(); } + } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/TimeUnitUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/TimeUnitUtil.java new file mode 100644 index 0000000..ec2a766 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/TimeUnitUtil.java @@ -0,0 +1,31 @@ +package com.uber.marmaray.utilities; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +public class TimeUnitUtil { + + private static final Long MICRO_SEC_LOWER_BOUND = (long) Math.pow(10, 15); + private static final Long MICRO_SEC_HIGHER_BOUND = (long) Math.pow(10, 18); + + /** + * ConvertToMicroSeconds + * @param num can only be in these time units [sec, milliseconds, microseconds, nanoseconds] + * @return microseconds + */ + public static String convertToMicroSeconds(final Long num) { + int computeTimes = 4; + Long val = num; + while (computeTimes-- > 0) { + if (val.compareTo(MICRO_SEC_HIGHER_BOUND) >= 0) { + val /= 1000; + } else if (val.compareTo(MICRO_SEC_LOWER_BOUND) < 0) { + val *= 1000; + } else { + return String.valueOf(val); + } + } + + throw new JobRuntimeException("Input timestamp doesn't have expected time unit. " + + "We accept only seconds/milliseconds/microseconds/nanoseconds.]"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java b/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java index f52fa73..f2b8b6b 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java @@ -17,6 +17,7 @@ package com.uber.marmaray.utilities; import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.ToString; @@ -35,11 +36,17 @@ public class TimestampInfo implements Serializable { @Getter final boolean saveAsLongType; + @Getter + final String timestampFieldName; + public boolean hasTimestamp() { return this.timestamp.isPresent(); } public static TimestampInfo generateEmptyTimestampInfo() { - return new TimestampInfo(Optional.absent(), false); + return new TimestampInfo( + Optional.absent(), + false, + CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); } } diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java b/marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java index b2a0b8a..8cf1dfe 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java @@ -28,13 +28,16 @@ public class CassandraClusterInfo { @Getter - private final Optional sslStoragePort; + @Setter + private Optional sslStoragePort; @Getter - private final Optional rpcPort; + @Setter + private Optional rpcPort; @Getter - private final Optional storagePort; + @Setter + private Optional storagePort; @Getter @Setter diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java index f188779..28a6545 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.utilities.listener; +import lombok.extern.slf4j.Slf4j; import org.apache.spark.scheduler.SparkListener; import org.apache.spark.scheduler.SparkListenerApplicationEnd; import org.apache.spark.scheduler.SparkListenerStageCompleted; @@ -25,19 +26,18 @@ import org.apache.spark.scheduler.StageInfo; import org.apache.spark.scheduler.TaskInfo; -import lombok.extern.slf4j.Slf4j; - @Slf4j public class SparkEventListener extends SparkListener { @Override public void onStageSubmitted(final SparkListenerStageSubmitted stageSubmitted) { + TimeoutManager.getInstance().stageStarted(stageSubmitted.stageInfo().stageId()); SparkJobTracker.recordStageInfo(stageSubmitted.stageInfo(), stageSubmitted.properties()); } @Override public void onStageCompleted(final SparkListenerStageCompleted stageCompleted) { - TimeoutManager.getInstance().setLastEventTime(stageCompleted.stageInfo().stageId()); + TimeoutManager.getInstance().stageFinished(stageCompleted.stageInfo().stageId()); final StageInfo stageInfo = stageCompleted.stageInfo(); if (stageInfo.completionTime().isDefined() && stageInfo.submissionTime().isDefined()) { SparkJobTracker.recordStageTime(stageInfo, @@ -52,13 +52,13 @@ public void onStageCompleted(final SparkListenerStageCompleted stageCompleted) { @Override public void onTaskEnd(final SparkListenerTaskEnd taskEnd) { final TaskInfo taskInfo = taskEnd.taskInfo(); - TimeoutManager.getInstance().setLastEventTime(taskEnd.stageId()); + TimeoutManager.getInstance().taskFinished(taskEnd.stageId()); SparkJobTracker.recordTaskTime(taskEnd.stageId(), taskInfo.finishTime() - taskInfo.launchTime()); } @Override public void onTaskStart(final SparkListenerTaskStart taskStart) { - TimeoutManager.getInstance().setLastEventTime(taskStart.stageId()); + TimeoutManager.getInstance().taskStarted(taskStart.stageId()); } @Override diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java index f99b0b7..e0696d3 100644 --- a/marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java @@ -16,20 +16,25 @@ */ package com.uber.marmaray.utilities.listener; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.Getter; import lombok.NonNull; +import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.spark.SparkContext; import org.apache.spark.scheduler.Stage; import scala.collection.Iterator; -import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; /** * TimeoutManager @@ -54,18 +59,24 @@ public final class TimeoutManager { private static TimeoutManager instance = null; + @Getter + private static Boolean timedOut; + private final long jobTimeoutMillis; private final long stageTimeoutMillis; private final SparkContext sc; private final long startTime; - private final Map lastActiveTime = new ConcurrentHashMap<>(); + @VisibleForTesting + @Getter + private final Map lastActiveTime = new ConcurrentHashMap<>(); private TimeoutManager(final int jobTimeoutInMins, final int stageStalledInMins, @NonNull final SparkContext sc) { this.jobTimeoutMillis = TimeUnit.MINUTES.toMillis(jobTimeoutInMins); this.stageTimeoutMillis = TimeUnit.MINUTES.toMillis(stageStalledInMins); this.sc = sc; this.startTime = getCurrentTime(); + this.timedOut = false; log.info("Initializing TimeoutManager, job_timeout = {}ms, stage_timeout = {}ms", this.jobTimeoutMillis, this.stageTimeoutMillis); } @@ -89,6 +100,10 @@ public static synchronized void init(@NonNull final Configuration conf, @NonNull } } + public static synchronized void close() { + instance = null; + } + public void startMonitorThread() { log.info("Start timeout monitoring..."); final Thread monitor = new Thread(() -> monitorTimeout()); @@ -105,15 +120,17 @@ private void monitorTimeout() { if (jobTimeout()) { log.error("The spark job is taking longer than {} ms. Cancelling all jobs...", this.jobTimeoutMillis); + this.timedOut = true; this.sc.cancelAllJobs(); throw new TimeoutException("The spark job is timing out"); } final List stalledStages = this.stalledStages(); if (stalledStages.size() > 0) { - for (Stage stage: stalledStages) { + for (Stage stage : stalledStages) { log.error("Cancelling stage {}-{} and its related jobs due to inactivity... details: {}", stage.id(), stage.name(), stage.details()); + this.timedOut = true; this.sc.cancelStage(stage.id()); } } @@ -125,8 +142,28 @@ private void monitorTimeout() { } } - public void setLastEventTime(final int id) { - lastActiveTime.put(id, getCurrentTime()); + public void stageFinished(final int stageId) { + this.lastActiveTime.remove(stageId); + } + + public void stageStarted(final int stageId) { + this.lastActiveTime.put(stageId, new StageActivityTracker()); + } + + public void taskStarted(final int stageId) { + Optional stageTracker = Optional.fromNullable(lastActiveTime.get(stageId)); + if (!stageTracker.isPresent()) { + stageStarted(stageId); + stageTracker = Optional.fromNullable(lastActiveTime.get(stageId)); + } + stageTracker.get().taskStarted(); + } + + public void taskFinished(final int stageId) { + final Optional stageTracker = Optional.fromNullable(lastActiveTime.get(stageId)); + if (stageTracker.isPresent()) { + stageTracker.get().taskFinished(); + } } public boolean jobTimeout() { @@ -134,14 +171,15 @@ public boolean jobTimeout() { } public List stalledStages() { - final List stalledStages = new ArrayList<>(); + final List stalledStages = new LinkedList<>(); final long currentTime = getCurrentTime(); final Iterator stageItr = sc.dagScheduler().runningStages().iterator(); while (stageItr.hasNext()) { final Stage stage = stageItr.next(); final int stageId = stage.id(); - if (lastActiveTime.containsKey(stageId)) { - if (currentTime - lastActiveTime.get(stageId) > stageTimeoutMillis) { + final Optional stageTracker = Optional.fromNullable(lastActiveTime.get(stageId)); + if (stageTracker.isPresent() && stageTracker.get().getRunningTasks().get() > 0) { + if (currentTime - lastActiveTime.get(stageId).lastActiveTime > stageTimeoutMillis) { stalledStages.add(stage); } } @@ -152,4 +190,27 @@ public List stalledStages() { private static long getCurrentTime() { return System.currentTimeMillis(); } + + /** + * Helper class for tracking Spark stage's activity. + */ + @Getter + @RequiredArgsConstructor + public static class StageActivityTracker { + @VisibleForTesting + @Getter + private final AtomicInteger runningTasks = new AtomicInteger(0); + private long lastActiveTime = 0L; + + public void taskStarted() { + this.lastActiveTime = getCurrentTime(); + this.runningTasks.incrementAndGet(); + } + + public void taskFinished() { + this.lastActiveTime = getCurrentTime(); + this.runningTasks.decrementAndGet(); + } + + } } diff --git a/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java b/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java index 2c165e7..e31f1b7 100644 --- a/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java @@ -20,9 +20,10 @@ import com.google.common.base.Optional; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.configuration.TestConfiguration; -import com.uber.marmaray.utilities.SparkUtil; +import com.uber.marmaray.common.spark.SparkArgs; +import com.uber.marmaray.common.spark.SparkFactory; import java.io.File; -import java.util.Arrays; +import java.util.Collections; import org.apache.spark.SparkConf; import org.junit.Assert; import org.junit.Test; @@ -34,18 +35,23 @@ public void testSparkConfOverride() { final Configuration conf = new Configuration( TestSparkUtil.class.getResourceAsStream("/configWithScopes.yaml"), Optional.of("incremental")); - final SparkConf sparkConf = SparkUtil - .getSparkConf("fooApp", Optional.absent(), Arrays.asList(), conf); + + final SparkConf sparkConf = getSparkConf(conf); Assert.assertEquals("4g", sparkConf.get("spark.executor.memory")); Assert.assertEquals("4g", sparkConf.get("spark.driver.memory")); Assert.assertEquals("100s", sparkConf.get("spark.network.timeout")); } + private SparkConf getSparkConf(final Configuration conf) { + final SparkArgs sparkArgs = new SparkArgs(Collections.emptyList(), Collections.emptyList(), conf); + final SparkFactory sparkFactory = new SparkFactory(sparkArgs); + return sparkFactory.createSparkConf(); + } + @Test public void testSparkConfOverrideDoesNotFailWithoutAnySparkConfDefinitions() { final Configuration conf = new Configuration(new File(TestConfiguration.CONFIG_YAML), Optional.absent()); - SparkUtil - .getSparkConf("fooApp", Optional.absent(), Arrays.asList(), conf); + getSparkConf(conf); } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.java index c201b16..8c8a148 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.java @@ -104,6 +104,20 @@ public void testMissingPartitionKeys() { Assert.fail(); } + @Test(expected = IllegalStateException.class) + public void testZeroMaxBatchSize() { + final Configuration rawConf = getConfig(""); + rawConf.setProperty(CassandraSinkConfiguration.MAX_BATCH_SIZE_MB, "0"); + new CassandraSinkConfiguration(rawConf); + } + + @Test(expected = IllegalStateException.class) + public void testNegativeSleepTime() { + final Configuration rawConf = getConfig(""); + rawConf.setProperty(CassandraSinkConfiguration.MIN_BATCH_DURATION_SECONDS, "-1"); + new CassandraSinkConfiguration(rawConf); + } + private Configuration getConfig(String propToExclude) { final Configuration conf = new Configuration(); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java index d95deb6..aff2321 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java @@ -22,6 +22,8 @@ import java.io.File; import java.io.InputStream; import java.util.Map; +import java.util.Properties; + import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -94,4 +96,18 @@ public void testConfigurationParseFailWithScopeButWithoutScopeOverrideKey() { public void testConfigurationParseWithNonExistentScope() { new Configuration(scopeAwareConfigInputStream, Optional.of("non-existent-scope")); } + + @Test + public void testGetProperties() { + Configuration conf = new Configuration(new File(CONFIG_YAML), Optional.absent()); + Properties properties = conf.getProperties(); + final String newKey = "new_key"; + Assert.assertEquals(10, properties.size()); + properties.put(newKey, "new_value"); + Properties mutatedProperties = conf.getProperties(); + // Tests that the getProperties() returns a copy of the properties and not a + // direct reference to the properties itself + Assert.assertNull(mutatedProperties.getProperty(newKey)); + } + } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java index aae5a66..f83bd74 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java @@ -16,11 +16,16 @@ */ package com.uber.marmaray.common.configuration; +import com.uber.hoodie.common.model.HoodieAvroPayload; +import com.uber.marmaray.common.AvroPayload; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; import org.hibernate.validator.constraints.NotEmpty; import org.junit.Assert; import org.junit.Test; +import sun.security.krb5.Config; import static com.uber.marmaray.common.util.SchemaTestUtil.getSchema; @@ -87,4 +92,31 @@ private static void verifyProperty(@NotEmpty final String tableName, Assert.assertTrue( value.equals(tableValue) && value.getClass() == tableValue.getClass()); } + + @Test + public void testPayloadSettings() { + final String tableName = "myTable"; + final Configuration conf = new Configuration(); + conf.setProperty(HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_TABLE_NAME, tableName), + tableName); + conf.setProperty(HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_BASE_PATH, tableName), + "/my/path"); + final Schema schema = SchemaBuilder.builder() + .record("foo") + .fields().name("bar").type().nullable().stringType().noDefault() + .endRecord(); + conf.setProperty(HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_AVRO_SCHEMA, tableName), + schema.toString()); + conf.setProperty(HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_METRICS_PREFIX, tableName), + "test"); + HoodieConfiguration hoodieConf = new HoodieConfiguration(conf, tableName); + Assert.assertEquals(HoodieAvroPayload.class.getCanonicalName(), + hoodieConf.getHoodieWriteConfig().getProps().getProperty("hoodie.compaction.payload.class")); + conf.setProperty( + HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_PAYLOAD_CLASS_NAME, tableName), + AvroPayload.class.getCanonicalName()); + hoodieConf = new HoodieConfiguration(conf, tableName); + Assert.assertEquals(AvroPayload.class.getCanonicalName(), + hoodieConf.getHoodieWriteConfig().getProps().getProperty("hoodie.compaction.payload.class")); + } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java index 2b3b8f9..ceb2a6a 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java @@ -66,6 +66,7 @@ public void configureHoodieIndex() throws Exception { setHoodieTableProperty(conf, "index.hbase.zknode.path", "/path/to/zk"); setHoodieTableProperty(conf, "index.hbase_index_table", "myHbaseTable"); Assert.assertEquals("HBASE", getHoodiePropertyFromConfig(conf, "hoodie.index.type")); + Assert.assertEquals("myHbaseTable", getHoodiePropertyFromConfig(conf, "hoodie.index.hbase.table")); } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaSourceConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaSourceConfiguration.java new file mode 100644 index 0000000..ba354d7 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaSourceConfiguration.java @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2019 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.configuration; + +import com.uber.marmaray.common.util.KafkaTestHelper; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.junit.Assert; +import org.junit.Test; + +import java.text.SimpleDateFormat; +import java.util.Date; + +public class TestKafkaSourceConfiguration { + + @Test + public void constructorWithStartDate() { + final SimpleDateFormat dateFormat = new SimpleDateFormat(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT); + final String startDate = dateFormat.format(new Date()); + final KafkaSourceConfiguration kafkaConfig = KafkaTestHelper.getKafkaSourceConfiguration( + "topic-name", "broker-address", startDate); + final long expectedStartDateEpoch = DateTime.parse(startDate, DateTimeFormat.forPattern( + KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).withZoneUTC()).toDate().getTime(); + Assert.assertEquals(kafkaConfig.getStartTime(), expectedStartDateEpoch); + } + + @Test + public void constructorWithStartTime() { + // If both start_date and start_time are specified, start_time should take precedence. + final SimpleDateFormat dateFormat = new SimpleDateFormat(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT); + final String startDate = dateFormat.format(new Date()); + final String startDateAsEpoch = String.valueOf(DateTime.parse(startDate, DateTimeFormat.forPattern( + KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).withZoneUTC()).toDate().getTime()); + final String startTime = String.valueOf(Long.valueOf(startDateAsEpoch) - 500); + final KafkaSourceConfiguration kafkaConfig = KafkaTestHelper.getKafkaSourceConfiguration( + "topic-name", "broker-address", startDate, startTime); + Assert.assertEquals(kafkaConfig.getStartTime(), (long) Long.valueOf(startTime)); + } + + @Test(expected = RuntimeException.class) + public void constructorWithInvalidStartDate() { + KafkaTestHelper.getKafkaSourceConfiguration( + "topic-name", "broker-address", "1970-01-01"); + } + + @Test(expected = RuntimeException.class) + public void constructorWithInvalidStartTime() { + final SimpleDateFormat dateFormat = new SimpleDateFormat(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT); + final String startDate = dateFormat.format(new Date()); + // The start_date is valid, but the start_time is not. + KafkaTestHelper.getKafkaSourceConfiguration( + "topic-name", "broker-address", startDate, "1000000"); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java index d456257..547f728 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java @@ -22,6 +22,8 @@ import com.uber.marmaray.common.data.RDDWrapper; import com.uber.marmaray.common.exceptions.InvalidDataException; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; import com.uber.marmaray.common.util.AbstractSparkTest; import com.uber.marmaray.utilities.ErrorExtractor; @@ -80,6 +82,12 @@ private static class MockAbstractDataConverter extends AbstractDataConverter> convert(@NotEmpty final String data) throws Exception { if (SUCCESS.equals(data)) { diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java index 6413307..5e34685 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java @@ -17,7 +17,9 @@ package com.uber.marmaray.common.converters; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.data.CassandraSinkDataConverter; import com.uber.marmaray.common.schema.cassandra.CassandraDataField; @@ -36,6 +38,8 @@ import org.junit.Assert; import org.junit.Test; +import static com.uber.marmaray.common.util.CassandraTestConstants.CONFIGURATION; + public class TestCassandraDataFrameConverter extends AbstractSparkTest { private static final String INT_FIELD = "int_field"; @@ -64,7 +68,8 @@ public void convertCommonSchemaDataToCassandra() { final JavaRDD payloadRDD = this.jsc.get().parallelize(records); final CassandraSinkDataConverter csdc = new CassandraSinkDataConverter(avroSchema, - new Configuration(), + CONFIGURATION, + Optional.absent(), Optional.absent(), Collections.EMPTY_LIST, TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java index d922dd0..6be67d0 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java @@ -25,12 +25,17 @@ import com.uber.marmaray.utilities.TimestampInfo; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; import java.util.HashSet; +import static com.uber.marmaray.common.schema.cassandra.CassandraSchemaField.TIMESTAMP_TYPE; + public class TestCassandraSchemaConverter { @Test public void testConvertCommonToCassandraSchemaNoTimestamp() { @@ -90,7 +95,7 @@ public void testConvertCommonToCassandraSchemaWithStringTimestamp() { .name("field0").type().intType().noDefault() .endRecord(); - final TimestampInfo tsInfo = new TimestampInfo(Optional.of("10000"), false); + final TimestampInfo tsInfo = new TimestampInfo(Optional.of("10000"), false, "testTimestamp"); final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", "testTableName", tsInfo, Optional.absent()); final CassandraSchema cassSchema = converter.convertToExternalSchema(record); @@ -103,12 +108,24 @@ public void testConvertCommonToCassandraSchemaWithStringTimestamp() { SchemaTestUtil.getSchema(CassandraSchemaField.INT_TYPE)), intField.getType()); final CassandraSchemaField timestampField = cassSchema.getFields().get(1); - Assert.assertEquals(SchemaUtil.DISPERSAL_TIMESTAMP, timestampField.getFieldName()); + Assert.assertEquals("testTimestamp", timestampField.getFieldName()); Assert.assertEquals(CassandraSchemaField.convertFromAvroType( SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE)), timestampField.getType()); } + @Test + public void testConvertCommonToCassandraSchemaWithTimestampType() { + final Schema record = SchemaBuilder.record("commonSchema") + .fields() + .name("ts").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .endRecord(); + final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", + "testTableName", Optional.absent()); + final CassandraSchema cassSchema = converter.convertToExternalSchema(record); + Assert.assertEquals(TIMESTAMP_TYPE, cassSchema.getFields().get(0).getType()); + } + @Test public void testConvertCommonToCassandraSchemaRemovesFieldStartingWithUnderscore() { final Schema record = SchemaBuilder.record("commonSchema") diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java index 1321508..11bd722 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java @@ -23,8 +23,12 @@ import com.uber.marmaray.common.converters.schema.DataFrameSchemaConverter; import com.uber.marmaray.common.util.SparkTestUtil; import com.uber.marmaray.utilities.ErrorExtractor; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -40,6 +44,7 @@ import org.junit.Before; import org.junit.Test; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.Collections; import java.util.List; diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java index 5ca044e..b5fbde8 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java @@ -24,6 +24,9 @@ import org.junit.Assert; import org.junit.Test; +import static com.uber.marmaray.utilities.SchemaUtil.TIMESTAMP_PROPERTY; +import static com.uber.marmaray.utilities.SchemaUtil.TRUE; + public class TestDataFrameSchemaConverter { private static final String INT_TYPE = "intType"; @@ -85,7 +88,8 @@ public void testConvertSparkToAvroSchema() { Assert.assertEquals(field.schema().getType(), Schema.Type.BYTES); break; case TIMESTAMP_TYPE: - Assert.assertEquals(field.schema().getType(), Schema.Type.STRING); + Assert.assertEquals(field.schema().getType(), Schema.Type.LONG); + Assert.assertEquals(field.schema().getProp(TIMESTAMP_PROPERTY), TRUE); break; case SHORT_TYPE: Assert.assertEquals(field.schema().getType(), Schema.Type.INT); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkCQLDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkCQLDataConverter.java new file mode 100644 index 0000000..ffcb7ce --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkCQLDataConverter.java @@ -0,0 +1,123 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.datastax.driver.core.Statement; +import com.google.common.base.Optional; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.TimestampInfo; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; + +import static com.uber.marmaray.common.configuration.CassandraSinkConfiguration.KEYSPACE; +import static com.uber.marmaray.common.configuration.CassandraSinkConfiguration.TABLE_NAME; + +public class TestCassandraSinkCQLDataConverter extends AbstractSparkTest { + + private Configuration conf; + + @Before + public void setup() { + this.conf = new Configuration(); + this.conf.setProperty(TABLE_NAME, "myTable"); + this.conf.setProperty(KEYSPACE, "myKeyspace"); + + } + + @Test + public void testInsertNull() throws Exception { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("longVal").type().nullable().longType().noDefault() + .endRecord(); + final CassandraSinkCQLDataConverter converter = new CassandraSinkCQLDataConverter( + schema, this.conf, Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final GenericRecord gr = new GenericData.Record(schema); + final AvroPayload payload = new AvroPayload(gr); + final List> result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + Assert.assertEquals("INSERT INTO myKeyspace.myTable () VALUES ();", + result.get(0).getSuccessData().get().getData().toString()); + } + + @Test + public void testInsertTimestamp() throws Exception { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("ts").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .name("longVal").type().nullable().longType().noDefault() + .endRecord(); + final CassandraSinkCQLDataConverter converter = new CassandraSinkCQLDataConverter( + schema, this.conf, Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + GenericRecord gr = new GenericData.Record(schema); + gr.put("ts", SchemaUtil.encodeTimestamp(new Timestamp(1514844885123L))); + gr.put("longVal", 123456L); + AvroPayload payload = new AvroPayload(gr); + List> result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + Assert.assertEquals("INSERT INTO myKeyspace.myTable (ts,longVal) VALUES (1514844885123,123456);", + result.get(0).getSuccessData().get().getData().toString()); + // test null timestamp + gr = new GenericData.Record(schema); + gr.put("longVal", 123456L); + gr.put("ts", null); + payload = new AvroPayload(gr); + result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + Assert.assertEquals("INSERT INTO myKeyspace.myTable (longVal) VALUES (123456);", + result.get(0).getSuccessData().get().getData().toString()); + } + + @Test + public void testInsertBinary() throws Exception { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("byteField").type().nullable().bytesType().noDefault() + .endRecord(); + final CassandraSinkCQLDataConverter converter = new CassandraSinkCQLDataConverter( + schema, this.conf, Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final GenericRecord gr = new GenericData.Record(schema); + gr.put("byteField", ByteBuffer.wrap(new byte[] {0x1c, 0x2f, 0x01, 0x34})); + final AvroPayload payload = new AvroPayload(gr); + final List> result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + Assert.assertEquals("INSERT INTO myKeyspace.myTable (byteField) VALUES (0x1c2f0134);", + result.get(0).getSuccessData().get().getData().toString()); + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkDataConverter.java new file mode 100644 index 0000000..f47da2f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestCassandraSinkDataConverter.java @@ -0,0 +1,270 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.ErrorCauseTagNames; +import com.uber.marmaray.common.metrics.ModuleTagNames; +import com.uber.marmaray.common.schema.cassandra.CassandraDataField; +import com.uber.marmaray.common.schema.cassandra.CassandraPayload; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.utilities.ByteBufferUtil; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.TimestampInfo; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.cassandra.db.marshal.LongType; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mockito; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static com.uber.marmaray.common.util.CassandraTestConstants.CONFIGURATION; + +public class TestCassandraSinkDataConverter extends AbstractSparkTest { + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testNullInTestWriteAllFieldsMockDataToCassandraWithTimestampConverter() { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("longVal").type().nullable().longType().noDefault() + .name("ts").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .name("longVal2").type().nullable().longType().noDefault() + .name("floatVal").type().nullable().floatType().noDefault() + .endRecord(); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, CONFIGURATION, Optional.absent(), Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final GenericRecord gr = new GenericData.Record(schema); + final long longVal2 = 123456L; + final long ts = 1550944636123L; + final float floatVal = 123456.789f; + gr.put("longVal2", longVal2); + gr.put("ts", ts); + gr.put("floatVal", floatVal); + final AvroPayload payload = new AvroPayload(gr); + final List> result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + final List cassandraData = result.get(0).getSuccessData().get().getData().getData(); + Assert.assertEquals(4, cassandraData.size()); + Assert.assertEquals("longVal", ByteBufferUtil.convertToString(cassandraData.get(0).getColumnKey())); + Assert.assertNull(cassandraData.get(0).getValue()); + Assert.assertEquals("ts", ByteBufferUtil.convertToString(cassandraData.get(1).getColumnKey())); + Assert.assertEquals(ts, cassandraData.get(1).getValue().getLong()); + Assert.assertEquals("longVal2", ByteBufferUtil.convertToString(cassandraData.get(2).getColumnKey())); + Assert.assertEquals(longVal2, cassandraData.get(2).getValue().getLong()); + Assert.assertEquals(floatVal, cassandraData.get(3).getValue().getFloat(), 0.00001); + } + + @Test + public void testNullTimestamp() { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("longVal").type().nullable().longType().noDefault() + .name("ts").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .name("longVal2").type().nullable().longType().noDefault() + .name("floatVal").type().nullable().floatType().noDefault() + .endRecord(); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, CONFIGURATION, Optional.absent(), Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final GenericRecord gr = new GenericData.Record(schema); + final AvroPayload payload = new AvroPayload(gr); + final List> result = converter.convert(payload); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + final List cassandraData = result.get(0).getSuccessData().get().getData().getData(); + Assert.assertEquals(4, cassandraData.size()); + Assert.assertEquals("longVal", ByteBufferUtil.convertToString(cassandraData.get(0).getColumnKey())); + Assert.assertNull(cassandraData.get(0).getValue()); + Assert.assertEquals("ts", ByteBufferUtil.convertToString(cassandraData.get(1).getColumnKey())); + Assert.assertNull(cassandraData.get(1).getValue()); + Assert.assertEquals("longVal2", ByteBufferUtil.convertToString(cassandraData.get(2).getColumnKey())); + Assert.assertNull(cassandraData.get(2).getValue()); + Assert.assertEquals("floatVal", ByteBufferUtil.convertToString(cassandraData.get(3).getColumnKey())); + Assert.assertNull(cassandraData.get(3).getValue()); + } + + @Test + public void testSkipInvalidRow() { + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("primary_key").type().nullable().stringType().noDefault().endRecord(); + final Configuration conf = getConfiguration(); + conf.setProperty(CassandraSinkConfiguration.SHOULD_SKIP_INVALID_ROWS, "true"); + + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.absent(), ImmutableList.of("primary_key"), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + + // test null primary key + final GenericRecord nullKey = new GenericData.Record(schema); + nullKey.put("primary_key", null); + final AvroPayload nullKeyPayload = new AvroPayload(nullKey); + final List> nullKeyResult = converter.convert(nullKeyPayload); + Assert.assertEquals(1, nullKeyResult.size()); + Assert.assertTrue(nullKeyResult.get(0).getErrorData().isPresent()); + Assert.assertEquals("Required keys are missing. Keys: primary_key", + nullKeyResult.get(0).getErrorData().get().getErrMessage()); + + // test empty string primary key + final GenericRecord emptyKey = new GenericData.Record(schema); + emptyKey.put("primary_key", ""); + final AvroPayload emptyKeyPayload = new AvroPayload(emptyKey); + final List> emptyKeyResult = converter.convert(emptyKeyPayload); + Assert.assertEquals(1, emptyKeyResult.size()); + Assert.assertTrue(emptyKeyResult.get(0).getErrorData().isPresent()); + Assert.assertEquals("Required keys are missing. Keys: primary_key", + emptyKeyResult.get(0).getErrorData().get().getErrMessage()); + } + + @Test + public void testByteArray() { + final Configuration conf = getConfiguration(); + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("byteField").type().nullable().bytesType().noDefault() + .endRecord(); + final GenericRecord record = new GenericData.Record(schema); + final ByteBuffer value = ByteBuffer.wrap(new byte[]{0x4a, 0x64, 0x4c}); + record.put("byteField", value); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final List> result = converter.convert(new AvroPayload(record)); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + final List cassandraData = result.get(0).getSuccessData().get().getData().getData(); + Assert.assertEquals(1, cassandraData.size()); + Assert.assertEquals(value, cassandraData.get(0).getValue()); + } + + @Test + public void testLongTypeWrittenTime() { + final Configuration conf = getConfiguration(); + + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("timeField").type().nullable().longType().noDefault() + .endRecord(); + final GenericRecord record = new GenericData.Record(schema); + final Long value = 1934000000000L; + record.put("timeField", value); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.of("timeField"), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final List> result = converter.convert(new AvroPayload(record)); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + final List cassandraData = result.get(0).getSuccessData().get().getData().getData(); + Assert.assertEquals(2, cassandraData.size()); + Assert.assertEquals(value+"", LongType.instance.getString(cassandraData.get(0).getValue())); + Assert.assertEquals(value+"000", LongType.instance.getString(cassandraData.get(1).getValue())); + } + + @Test + public void testStringTypeWrittenTime() { + final Configuration conf = getConfiguration(); + + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("timeField").type().nullable().stringType().noDefault() + .endRecord(); + final GenericRecord record = new GenericData.Record(schema); + final String value = "1934000000000"; + record.put("timeField", value); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.of("timeField"), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + final List> result = converter.convert(new AvroPayload(record)); + Assert.assertEquals(1, result.size()); + Assert.assertTrue(result.get(0).getSuccessData().isPresent()); + final List cassandraData = result.get(0).getSuccessData().get().getData().getData(); + Assert.assertEquals(2, cassandraData.size()); + } + + @Test + public void testNotSupportedTypeWrittenTime() { + this.exception.expect(JobRuntimeException.class); + this.exception.expectMessage("Order column type BYTES not supported. Only LONG and STRING type are supported."); + + final Configuration conf = getConfiguration(); + + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("byteField").type().nullable().bytesType().noDefault() + .endRecord(); + final GenericRecord record = new GenericData.Record(schema); + final ByteBuffer value = ByteBuffer.wrap(new byte[]{0x4a, 0x64, 0x4c}); + record.put("byteField", value); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.of("byteField"), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + converter.setDataFeedMetrics(new DataFeedMetrics("test_job", new HashMap<>())); + converter.convert(new AvroPayload(record)); + } + + @Test + public void testUnsupportedDataType() { + this.exception.expect(JobRuntimeException.class); + this.exception.expectMessage("Type ARRAY not supported"); + + final Configuration conf = getConfiguration(); + final DataFeedMetrics mockMetrics = Mockito.mock(DataFeedMetrics.class); + final Schema schema = SchemaBuilder.builder().record("myRecord").fields() + .name("arrayField").type().array().items().nullable().stringType().noDefault() + .endRecord(); + final GenericRecord record = new GenericData.Record(schema); + record.put("arrayField", Collections.emptyList()); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( + schema, conf, Optional.absent(), Optional.absent(), Collections.emptyList(), + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + converter.setDataFeedMetrics(mockMetrics); + try { + converter.convert(new AvroPayload(record)); + Assert.fail(); + } catch (JobRuntimeException e) { + Mockito.verify(mockMetrics).createLongFailureMetric(DataFeedMetricNames.MARMARAY_JOB_ERROR, 1, + DataFeedMetricNames.getErrorModuleCauseTags( + ModuleTagNames.SINK_CONVERTER, ErrorCauseTagNames.NOT_SUPPORTED_FIELD_TYPE)); + throw e; + } + } + + private Configuration getConfiguration() { + final Configuration conf = new Configuration(); + conf.setProperty(CassandraSinkConfiguration.KEYSPACE, "keyspace"); + conf.setProperty(CassandraSinkConfiguration.TABLE_NAME, "table"); + conf.setProperty(CassandraSinkConfiguration.CLUSTER_NAME, "test-cluster"); + conf.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, "primary_key"); + return conf; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataCSVConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataCSVConverter.java new file mode 100644 index 0000000..827c260 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataCSVConverter.java @@ -0,0 +1,184 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.AvroPayloadUtil; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.SparkException; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.List; +import java.util.TimeZone; + + +@Slf4j +public class TestFileSinkDataCSVConverter extends AbstractSparkTest { + + private static final String SEPARATOR = ","; + private static final String CSV = "csv"; + + @Test + public void testConvertAllWithCsv() { + log.info("Starts Test convert all with csv"); + final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); + final Configuration conf = initConf(SEPARATOR, CSV); + log.info("Starting to convert data."); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + final JavaPairRDD tmpDataConverted = converter.convertAll(payloadData); + final JavaRDD dataConverted = tmpDataConverted.map(message -> message._2()); + int i = 1; + for (String line: dataConverted.collect()){ + Assert.assertEquals(String.valueOf(i) + SEPARATOR + String.valueOf(i) + SEPARATOR + "true", line ); + i = i + 1; + } + } + + @Test + public void testConvertAllWithCsvSpecialChar() { + log.info("Starts Test convert all with csv"); + final String separator = ","; + final JavaRDD payloadData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), 10, StringTypes.EMPTY); + final Configuration conf = initConf(separator, CSV); + log.info("Starting to convert data."); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + final JavaPairRDD tmpDataConverted = converter.convertAll(payloadData); + final JavaRDD dataConverted = tmpDataConverted.map(message -> message._2()); + int i = 1; + for (String line: dataConverted.collect()){ + Assert.assertEquals(String.valueOf(i) + SEPARATOR + "\"" + String.valueOf(i) + "\\\",try\\\\\"" + SEPARATOR + "true", line); + i = i + 1; + } + } + + @Test(expected = SparkException.class) + public void testConvertAllWithJsonNotSupported() { + log.info("Starts Test convert all with json"); + final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); + final Configuration conf = initConf(SEPARATOR, "json"); + log.info("Starting to convert data."); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + final JavaPairRDD tmpDataConverted = converter.convertAll(payloadData); + final JavaRDD dataConverted = tmpDataConverted.map(message -> message._2()); + int i = 1; + for (String line: dataConverted.collect()){ + Assert.assertEquals(String.valueOf(i) + SEPARATOR + String.valueOf(i) + SEPARATOR + "true", line ); + i = i + 1; + } + } + + @Test + public void testConvertTimestamp() { + final TimeZone defaultTZ = TimeZone.getDefault(); + try { + // test is assuming UTC + TimeZone.setDefault(TimeZone.getTimeZone("UTC")); + final Schema schema = SchemaBuilder.builder().record("mySchema").fields() + .name("ts").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .name("longId").type().nullable().longType().noDefault() + .endRecord(); + GenericRecord gr = new GenericData.Record(schema); + gr.put("ts", SchemaUtil.encodeTimestamp(new Timestamp(1514844885123L))); + gr.put("longId", 123456L); + final Configuration conf = initConf(SEPARATOR, CSV); + AvroPayload payloadData = new AvroPayload(gr); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + List> converted = converter.convert(payloadData); + Assert.assertEquals(1, converted.size()); + Assert.assertTrue(converted.get(0).getSuccessData().isPresent()); + Assert.assertEquals("2018-01-01 22:14:45.123 +0000,123456", + converted.get(0).getSuccessData().get().getData()); + gr = new GenericData.Record(schema); + gr.put("longId", 123456L); + payloadData = new AvroPayload(gr); + converted = converter.convert(payloadData); + Assert.assertEquals(1, converted.size()); + Assert.assertTrue(converted.get(0).getSuccessData().isPresent()); + Assert.assertEquals(",123456", + converted.get(0).getSuccessData().get().getData()); + gr = new GenericData.Record(schema); + payloadData = new AvroPayload(gr); + converted = converter.convert(payloadData); + Assert.assertEquals(1, converted.size()); + Assert.assertTrue(converted.get(0).getSuccessData().isPresent()); + Assert.assertEquals(",", + converted.get(0).getSuccessData().get().getData()); + + } finally { + TimeZone.setDefault(defaultTZ); + } + } + + @Test + public void testGetHeaderWithCsv() { + final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); + final Configuration conf = initConf(SEPARATOR, CSV); + log.info("Starting to get data header."); + final FileSinkDataConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + final String header = converter.getHeader(payloadData); + final String resultHeader = "int_field,string_field,boolean_field"; + Assert.assertEquals(resultHeader, header); + log.info("Header: {}", header); + } + + private Configuration initConf(@NonNull final String separator, @NonNull final String fileType) { + final Configuration conf = new Configuration(); + final String filePrefix = this.fileSystem.get().getWorkingDirectory().toString(); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, filePrefix); + conf.setProperty(FileSinkConfiguration.SEPARATOR, separator); + conf.setProperty(FileSinkConfiguration.FILE_TYPE, fileType); + conf.setProperty(FileSinkConfiguration.SOURCE_TYPE, "HDFS"); + conf.setProperty(FileSinkConfiguration.TIMESTAMP, "201808011000"); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, this.fileSystem.get().getWorkingDirectory().toString()); + conf.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test.table"); + return conf; + } + + @Test + public void testBinary() { + final Schema schema = SchemaBuilder.builder().record("mySchema").fields() + .name("byteField").type().nullable().bytesType().noDefault() + .endRecord(); + final GenericRecord gr = new GenericData.Record(schema); + gr.put("byteField", ByteBuffer.wrap(new byte[] {0x12, 0x34, 0x56, 0x78, 0x0a})); + final Configuration conf = initConf(SEPARATOR, CSV); + final AvroPayload payloadData = new AvroPayload(gr); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); + final List> converted = converter.convert(payloadData); + Assert.assertEquals(1, converted.size()); + Assert.assertTrue(converted.get(0).getSuccessData().isPresent()); + Assert.assertEquals("0x123456780a", converted.get(0).getSuccessData().get().getData()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataConverter.java deleted file mode 100644 index 4aa5abe..0000000 --- a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataConverter.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. - * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated - * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the - * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to - * permit persons to whom the Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all copies or substantial portions - * of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO - * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE - * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, - * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS - * IN THE SOFTWARE. - */ - -package com.uber.marmaray.common.converters.data; - -import com.uber.marmaray.common.AvroPayload; -import com.uber.marmaray.common.configuration.Configuration; -import com.uber.marmaray.common.configuration.FileSinkConfiguration; -import com.uber.marmaray.common.util.AbstractSparkTest; -import com.uber.marmaray.common.util.AvroPayloadUtil; -import com.uber.marmaray.utilities.ErrorExtractor; -import com.uber.marmaray.utilities.StringTypes; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.spark.SparkException; -import org.apache.spark.api.java.JavaRDD; -import org.junit.Assert; -import org.junit.Test; - -@Slf4j -public class TestFileSinkDataConverter extends AbstractSparkTest { - @Test - public void testConvertAllWithCsv() { - log.info("Starts Test convert all with csv"); - final String separator = " "; - final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); - final Configuration conf = initConf(separator, "csv"); - log.info("Starting to convert data."); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); - final JavaRDD dataConverted = converter.convertAll(payloadData); - int i = 1; - for (String line: dataConverted.collect()){ - Assert.assertEquals(String.valueOf(i) + separator + String.valueOf(i) + separator + "true", line ); - i = i + 1; - } - } - - @Test - public void testConvertAllWithCsvSpecialChar() { - log.info("Starts Test convert all with csv"); - final String separator = ","; - final JavaRDD payloadData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), 10, StringTypes.EMPTY); - final Configuration conf = initConf(separator, "csv"); - log.info("Starting to convert data."); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); - final JavaRDD dataConverted = converter.convertAll(payloadData); - int i = 1; - for (String line: dataConverted.collect()){ - Assert.assertEquals(String.valueOf(i) + separator + "\"" + String.valueOf(i) + "\\\",try\\\\\"" + separator + "true", line); - i = i + 1; - } - } - - @Test(expected = SparkException.class) - public void testConvertAllWithJsonNotSupported() { - log.info("Starts Test convert all with json"); - final String separator = ","; - final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); - final Configuration conf = initConf(separator, "json"); - log.info("Starting to convert data."); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); - final JavaRDD dataConverted = converter.convertAll(payloadData); - int i = 1; - for (String line: dataConverted.collect()){ - Assert.assertEquals(String.valueOf(i) + separator + String.valueOf(i) + separator + "true", line ); - i = i + 1; - } - } - - @Test - public void testGetHeaderWithCsv() { - final String separator = ","; - final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 10, StringTypes.EMPTY); - final Configuration conf = initConf(separator, "csv"); - log.info("Starting to get data header."); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); - final String header = converter.getHeader(payloadData); - final String resultHeader = "int_field,string_field,boolean_field"; - Assert.assertEquals(resultHeader, header); - log.info("Header: {}", header); - } - - private Configuration initConf(@NonNull final String separator, @NonNull final String fileType) { - final Configuration conf = new Configuration(); - final String filePrefix = this.fileSystem.get().getWorkingDirectory().toString(); - conf.setProperty(FileSinkConfiguration.PATH_PREFIX, filePrefix); - conf.setProperty(FileSinkConfiguration.SEPARATOR, separator); - conf.setProperty(FileSinkConfiguration.FILE_TYPE, fileType); - conf.setProperty(FileSinkConfiguration.SOURCE_TYPE, "HDFS"); - conf.setProperty(FileSinkConfiguration.TIMESTAMP, "201808011000"); - conf.setProperty(FileSinkConfiguration.PATH_PREFIX, this.fileSystem.get().getWorkingDirectory().toString()); - conf.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test.table"); - return conf; - } -} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataJSONConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataJSONConverter.java new file mode 100644 index 0000000..1aae53f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataJSONConverter.java @@ -0,0 +1,70 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.AvroPayloadUtil; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.junit.Assert; +import org.junit.Test; + +@Slf4j +public class TestFileSinkDataJSONConverter extends AbstractSparkTest { + + private static final String SEQUENCE = "sequence"; + + @Test + public void testConvertAll() { + log.info("Starts Test json convert"); + final JavaRDD payloadData = AvroPayloadUtil.generateTestData(this.jsc.get(), 5, StringTypes.EMPTY); + final Configuration conf = initConf(SEQUENCE); + log.info("Starting to convert data."); + final FileSinkDataJSONConverter converter = new FileSinkDataJSONConverter(conf, new ErrorExtractor()); + final JavaPairRDD tmpDataConverted = converter.convertAll(payloadData); + final JavaRDD dataConverted = tmpDataConverted.map(message -> message._2()); + int i = 1; + for (String line: dataConverted.collect()){ + log.info("line {} : {}", i, line); + //samlpe json : {"int_field":"1","boolean_field":"true","string_field":"1"} + Assert.assertEquals("{\"int_field\":\"" + String.valueOf(i) + + "\",\"boolean_field\":\"true\",\"string_field\":\"" + String.valueOf(i) +"\"}", line); + i = i + 1; + } + } + + private Configuration initConf(@NonNull final String fileType) { + final Configuration conf = new Configuration(); + final String filePrefix = this.fileSystem.get().getWorkingDirectory().toString(); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, filePrefix); + conf.setProperty(FileSinkConfiguration.FILE_TYPE, fileType); + conf.setProperty(FileSinkConfiguration.COMPRESSION_CODEC, "lz4"); + conf.setProperty(FileSinkConfiguration.SOURCE_TYPE, "HDFS"); + conf.setProperty(FileSinkConfiguration.TIMESTAMP, "201811280000"); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, this.fileSystem.get().getWorkingDirectory().toString()); + conf.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test.table"); + return conf; + } +} \ No newline at end of file diff --git a/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestSparkSourceDataConverter.java b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestSparkSourceDataConverter.java new file mode 100644 index 0000000..4e183cf --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestSparkSourceDataConverter.java @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.converters.data; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SchemaUtil; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class TestSparkSourceDataConverter extends AbstractSparkTest { + + public static final Schema OUTPUT_SCHEMA = SchemaBuilder.builder().record("commonSchema") + .fields() + .name("field1").type().intType().noDefault() + .name("field2").type().longType().noDefault() + .name("field3").type(SchemaUtil.getTimestampSchema(true)).noDefault() + .name("field4").type().nullable().bytesType().noDefault() + .endRecord(); + public static final StructType INPUT_SCHEMA = new StructType(new StructField[]{ + new StructField("field1", DataTypes.IntegerType, false, null), + new StructField("field2", DataTypes.LongType, false, null), + new StructField("field3", DataTypes.TimestampType, true, null), + new StructField("field4", DataTypes.BinaryType, true, null)}); + + @Test + public void convert() throws Exception { + final byte[] binaryData = new byte[] {0x12, 0x34, 0x56}; + final Row data = new GenericRowWithSchema(new Object[] {1, 2L, new Timestamp(123456789), binaryData}, INPUT_SCHEMA); + final SparkSourceDataConverter converter = new SparkSourceDataConverter(INPUT_SCHEMA, OUTPUT_SCHEMA, new Configuration(), Collections.emptySet(), new ErrorExtractor()); + final List> results = converter.convert(data); + Assert.assertEquals(1, results.size()); + Assert.assertTrue(results.get(0).getSuccessData().isPresent()); + final AvroPayload resultPayload = results.get(0).getSuccessData().get().getData(); + Assert.assertEquals(ByteBuffer.wrap(binaryData), resultPayload.getData().get("field4")); + } + + @Test + public void testNulls() throws Exception { + final Row data = new GenericRowWithSchema(new Object[] {1, 2L, null, null}, INPUT_SCHEMA); + final SparkSourceDataConverter converter = new SparkSourceDataConverter(INPUT_SCHEMA, OUTPUT_SCHEMA, new Configuration(), Collections.emptySet(), new ErrorExtractor()); + final List> results = converter.convert(data); + Assert.assertEquals(1, results.size()); + Assert.assertTrue(results.get(0).getSuccessData().isPresent()); + final AvroPayload resultPayload = results.get(0).getSuccessData().get().getData(); + Assert.assertEquals(1, resultPayload.getData().get("field1")); + Assert.assertEquals(2L, resultPayload.getData().get("field2")); + Assert.assertNull(resultPayload.getData().get("field3")); + Assert.assertNull(resultPayload.getData().get("field4")); + } + + @Test + public void testRequiredFields() throws Exception { + final Row data = new GenericRowWithSchema(new Object[] {1, 2L, null, null}, INPUT_SCHEMA); + final Set requiredFields = new HashSet<>(Arrays.asList("field1", "field4")); + final Configuration conf = new Configuration(); + final SparkSourceDataConverter converter = new SparkSourceDataConverter(INPUT_SCHEMA, OUTPUT_SCHEMA, conf, requiredFields, new ErrorExtractor()); + try { + converter.convert(data); + Assert.fail("conversion should fail due to missing key"); + } catch (JobRuntimeException e) { + Assert.assertTrue(e.getMessage().contains("Required fields were missing.")); + } + conf.setProperty(CassandraSinkConfiguration.SHOULD_SKIP_INVALID_ROWS, "true"); + final List> results = converter.convert(data); + Assert.assertEquals(1, results.size()); + Assert.assertFalse(results.get(0).getSuccessData().isPresent()); + Assert.assertTrue(results.get(0).getErrorData().isPresent()); + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java b/marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java index aee0d90..60bc1ff 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java @@ -59,7 +59,7 @@ public void testSortOrder() throws IOException { final JobManagerMetadataTracker tracker = mock(JobManagerMetadataTracker.class); when(tracker.get(any())).then(new ConfigurationAnswer()); final IJobExecutionStrategy strategy = new ExecutionTimeJobExecutionStrategy(tracker); - final Queue jobDagQueue = new ConcurrentLinkedDeque<>(); + final Queue jobDagQueue = new ConcurrentLinkedDeque<>(); final JobDag jobDag1 = mockJobDag("dag1"); jobDagQueue.add(jobDag1); final JobDag jobDag3 = mockJobDag("dag3"); @@ -70,8 +70,8 @@ public void testSortOrder() throws IOException { jobDagQueue.add(jobDag4); final JobDag jobDag5 = mockJobDag("dag5"); jobDagQueue.add(jobDag5); - final List resultQueue = strategy.sort(jobDagQueue); - final List expectedQueue = new ArrayList<>(jobDagQueue.size()); + final List resultQueue = strategy.sort(jobDagQueue); + final List expectedQueue = new ArrayList<>(jobDagQueue.size()); // first jobs with no history and/or no success in 6 hours expectedQueue.add(jobDag3); expectedQueue.add(jobDag4); @@ -82,7 +82,7 @@ public void testSortOrder() throws IOException { assertListEquals(expectedQueue, resultQueue); } - private void assertListEquals(final List expectedList, final List resultList) { + private void assertListEquals(final List expectedList, final List resultList) { assertEquals(String.format("Size of queues mismatched: %s vs %s", expectedList, resultList) , expectedList.size(), resultList.size()); for (int i = 0; i < expectedList.size(); i++) { diff --git a/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobDag.java b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobDag.java new file mode 100644 index 0000000..5a39c5f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobDag.java @@ -0,0 +1,151 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.job; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.actions.IJobDagAction; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.metadata.IMetadataManager; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.reporters.IReporter; +import com.uber.marmaray.common.reporters.Reporters; +import com.uber.marmaray.common.sinks.ISink; +import com.uber.marmaray.common.sources.ISource; +import com.uber.marmaray.common.sources.IWorkUnitCalculator; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; +import com.uber.marmaray.common.util.AbstractSparkTest; +import org.apache.spark.api.java.JavaRDD; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.io.IOException; +import java.util.HashMap; + +import static com.uber.marmaray.common.util.SchemaTestUtil.getRandomData; +import static com.uber.marmaray.common.util.SchemaTestUtil.getSchema; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class TestJobDag extends AbstractSparkTest { + + @Mock + private ISource mockSource; + + @Mock + private ISink mockSink; + + @Mock + private IMetadataManager mockMetatdataManager; + + @Mock + private IWorkUnitCalculator mockIWorkUnitCalculator; + + @Mock + private IWorkUnitCalculator.IWorkUnitCalculatorResult mockIWorkUnitCalculatorResult; + + @Mock + private JobMetrics mockJobMetrics; + + private Reporters reporters; + + @Mock + private IReporter mockIReporter; + + @Mock + private IJobDagAction mockJobDagAction; + + private static final String TS_KEY = "timestamp"; + private static final String RECORD_KEY = "primaryKey"; + + @Before + public void testSetup() { + this.reporters = new Reporters(); + this.reporters.addReporter(this.mockIReporter); + ThreadPoolService.init(new Configuration()); + } + + @After + public void tearDown() { + ThreadPoolService.shutdown(true); + } + + @Test + public void testSuccessCase() { + final BaseStatus status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + + when(this.mockIWorkUnitCalculatorResult.getStatus()).thenReturn(status); + when(this.mockIWorkUnitCalculator.computeWorkUnits()).thenReturn(this.mockIWorkUnitCalculatorResult); + when(this.mockIWorkUnitCalculatorResult.hasWorkUnits()).thenReturn(true); + + // Spark JavaRDD mock + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final JavaRDD inputRDD = + this.jsc.isPresent() ? this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, + RECORD_KEY, 10)) : null; + when(this.mockSource.getData(any())).thenReturn(inputRDD); + + final Dag jdag = new JobDag(this.mockSource, this.mockSink, this.mockMetatdataManager, + this.mockIWorkUnitCalculator, "test_job_name", + "test_data_feed_name", this.mockJobMetrics, this.reporters); + ((JobDag) jdag).addAction(this.mockJobDagAction); + jdag.setJobManagerMetadata(new HashMap<>()); + + // Verify result is expected + final IStatus res = jdag.execute(); + assertEquals("Job status doesn't return SUCCESS", IStatus.Status.SUCCESS, res.getStatus()); + assertNotNull(((JobDag) jdag).getDataFeedMetrics()); + assertEquals("DataFeedMetric doesn't return expected job name.", "test_job_name", + ((JobDag) jdag).getDataFeedMetrics().getJobName()); + } + + @Test + public void testMetadataManagerException() throws Exception { + final BaseStatus status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + final String exceptionMsg = "Mock metadataManager exception."; + + when(this.mockIWorkUnitCalculatorResult.getStatus()).thenReturn(status); + when(this.mockIWorkUnitCalculator.computeWorkUnits()).thenReturn(this.mockIWorkUnitCalculatorResult); + when(this.mockIWorkUnitCalculatorResult.hasWorkUnits()).thenReturn(false); + // Throw IO exception in metadataManager + doThrow(new IOException(exceptionMsg)).when(this.mockMetatdataManager).saveChanges(); + + final Dag jdag = new JobDag(this.mockSource, this.mockSink, this.mockMetatdataManager, + this.mockIWorkUnitCalculator, "test_job_name", + "test_data_feed_name", this.mockJobMetrics, this.reporters); + ((JobDag) jdag).addAction(this.mockJobDagAction); + + // Verify result is expected + final IStatus res = jdag.execute(); + assertEquals("Job status doesn't return FAILURE", IStatus.Status.FAILURE, res.getStatus()); + assertEquals("Job status has more than one exception.", 1, res.getExceptions().size()); + assertEquals("Job status exception doesn't return expected message.", "Failed to save metadata changes " + exceptionMsg, + res.getExceptions().get(0).getMessage()); + + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobManager.java b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobManager.java new file mode 100644 index 0000000..4385f3d --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobManager.java @@ -0,0 +1,213 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.job; + +import com.google.common.base.Optional; +import com.uber.marmaray.TestSparkUtil; +import com.uber.marmaray.common.actions.IJobDagAction; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.LockManagerConfiguration; +import com.uber.marmaray.common.configuration.SparkConfiguration; +import com.uber.marmaray.common.configuration.ZookeeperConfiguration; +import com.uber.marmaray.common.metadata.JobManagerMetadataTracker; +import com.uber.marmaray.common.reporters.Reporters; +import com.uber.marmaray.common.spark.SparkArgs; +import com.uber.marmaray.common.spark.SparkFactory; +import com.uber.marmaray.common.status.BaseStatus; +import com.uber.marmaray.common.status.IStatus; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.curator.test.TestingServer; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +@Slf4j +public class TestJobManager { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @NonNull + private static final Configuration conf = new Configuration(); + + @Mock + private TestingServer mockZkServer; + + @Mock + private JobDag mockJobDag1, mockJobDag2, mockJobDag3; + + @Mock + private JobManagerMetadataTracker mockJobManagerMetatdataTracker; + + @Mock + private IJobDagAction mockJobDagAction; + + @Before + public void testSetup() throws Exception { + doNothing().when(this.mockZkServer).start(); + conf.setProperty(LockManagerConfiguration.IS_ENABLED, "false"); + conf.setProperty(LockManagerConfiguration.ZK_BASE_PATH, "/////test////lock_manager////"); + conf.setProperty(ZookeeperConfiguration.ZK_QUORUM, "connect string"); + conf.setProperty(ZookeeperConfiguration.ZK_PORT, Integer.toString(2181)); + conf.setProperty(LockManagerConfiguration.ACQUIRE_LOCK_TIME_MS, Integer.toString(10 * 1000)); + conf.setProperty(LockManagerConfiguration.ZK_SESSION_TIMEOUT_MS, Integer.toString(30 * 1000)); + conf.setProperty(LockManagerConfiguration.ZK_CONNECTION_TIMEOUT_MS, Integer.toString(12 * 1000)); + JobManager.reset(); + } + + @Test + public void testEmptyJobDagException() { + final SparkArgs sparkArgs = getSampleMarmaraySparkArgs(); + final SparkFactory sparkFactory = new SparkFactory(sparkArgs); + + final JobManager jobManager = JobManager.createJobManager(this.conf, + "test_app_name", "daily", sparkFactory, new Reporters()); + + this.thrown.expect(RuntimeException.class); + this.thrown.expectMessage("No job dags to execute"); + + jobManager.run(); + sparkFactory.stop(); + } + + @Test + public void testSuccessCase() { + final SparkArgs sparkArgs = getSampleMarmaraySparkArgs(); + final SparkFactory sparkFactory = new SparkFactory(sparkArgs); + + final BaseStatus status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + + when(this.mockJobDag1.getDataFeedName()).thenReturn("data_feed_name1"); + when(this.mockJobDag1.getJobName()).thenReturn("job_name1"); + when(this.mockJobDag2.getDataFeedName()).thenReturn("data_feed_name2"); + when(this.mockJobDag2.getJobName()).thenReturn("job_name2"); + when(this.mockJobDag3.getDataFeedName()).thenReturn("data_feed_name3"); + when(this.mockJobDag3.getJobName()).thenReturn("job_name3"); + when(this.mockJobDag1.execute()).thenReturn(status); + when(this.mockJobDag2.execute()).thenReturn(status); + when(this.mockJobDag3.execute()).thenReturn(status); + when(this.mockJobDag1.getJobManagerMetadata()).thenReturn(new HashMap<>()); + when(this.mockJobDag2.getJobManagerMetadata()).thenReturn(new HashMap<>()); + when(this.mockJobDag3.getJobManagerMetadata()).thenReturn(new HashMap<>()); + doNothing().when(this.mockJobManagerMetatdataTracker).writeJobManagerMetadata(); + + final JobManager jobManager = JobManager.createJobManager(this.conf, + "test_app_name", "daily", false, sparkFactory, new Reporters()); + + jobManager.addJobDag(this.mockJobDag1); + jobManager.addJobDags(Arrays.asList(this.mockJobDag2, this.mockJobDag3)); + jobManager.setJobManagerMetadataEnabled(true); + jobManager.setTracker(this.mockJobManagerMetatdataTracker); + jobManager.addPostJobManagerAction(this.mockJobDagAction); + jobManager.addPostJobManagerActions(new ArrayList<>()); + jobManager.run(); + sparkFactory.stop(); + + // verify + verify(this.mockJobDag1, times(4)).getDataFeedName(); + verify(this.mockJobDag2, times(4)).getDataFeedName(); + verify(this.mockJobDag3, times(4)).getDataFeedName(); + verify(this.mockJobDag1, times(1)).execute(); + verify(this.mockJobDag2, times(1)).execute(); + verify(this.mockJobDag3, times(1)).execute(); + assertEquals("Test doesn't return successful status for jobdag1.", IStatus.Status.SUCCESS, + jobManager.getJobManagerStatus().getJobStatuses().get("job_name1").getStatus()); + assertEquals("Test doesn't return successful status for jobdag2.", IStatus.Status.SUCCESS, + jobManager.getJobManagerStatus().getJobStatuses().get("job_name2").getStatus()); + assertEquals("Test doesn't return successful status for jobdag3.", IStatus.Status.SUCCESS, + jobManager.getJobManagerStatus().getJobStatuses().get("job_name3").getStatus()); + assertNotNull(jobManager.getJobMetrics()); + assertNotNull(jobManager.getReporters()); + assertNotNull(jobManager.getConf()); + assertNotNull(jobManager.getSparkFactory()); + } + + @Test + public void testJobRunException() { + final SparkArgs sparkArgs = getSampleMarmaraySparkArgs(); + final SparkFactory sparkFactory = new SparkFactory(sparkArgs); + + final BaseStatus status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + + when(this.mockJobDag1.getDataFeedName()).thenReturn("data_feed_name1"); + when(this.mockJobDag1.getJobName()).thenReturn("job_name1"); + when(this.mockJobDag1.execute()).thenThrow(new RuntimeException("Mock exception")); + when(this.mockJobDag1.getJobManagerMetadata()).thenReturn(new HashMap<>()); + doNothing().when(this.mockJobManagerMetatdataTracker).writeJobManagerMetadata(); + + final JobManager jobManager = JobManager.createJobManager(this.conf, + "test_app_name", "daily", false, sparkFactory, new Reporters()); + + jobManager.addJobDag(this.mockJobDag1); + jobManager.setJobManagerMetadataEnabled(true); + jobManager.setTracker(this.mockJobManagerMetatdataTracker); + jobManager.addPostJobManagerAction(this.mockJobDagAction); + jobManager.addPostJobManagerActions(new ArrayList<>()); + jobManager.run(); + sparkFactory.stop(); + + // verify + assertEquals("Test doesn't return FAILURE status.", IStatus.Status.FAILURE, + jobManager.getJobManagerStatus().getStatus()); + assertEquals("Test should return only one exception.", 2, + jobManager.getJobManagerStatus().getExceptions().size()); + assertTrue("Test doesn't return expected error message.", + jobManager.getJobManagerStatus().getExceptions().get(0).getMessage().contains("Mock exception")); + } + + private SparkArgs getSampleMarmaraySparkArgs() { + final Schema recordSchema = SchemaBuilder.record("fooRecord").fields().name("abc").type() + .intType().intDefault(0).endRecord(); + + final Map overrideSparkProperties = new HashMap<>(); + overrideSparkProperties.put("spark.master", "local[2]"); + overrideSparkProperties.put("spark.app.name", "foo_bar"); + + final com.uber.marmaray.common.configuration.Configuration conf = + new com.uber.marmaray.common.configuration.Configuration( + TestSparkUtil.class.getResourceAsStream("/config.yaml"), + Optional.absent()); + + SparkConfiguration.overrideSparkConfInConfiguration(conf, overrideSparkProperties); + return new SparkArgs(Arrays.asList(recordSchema), Collections.emptyList(), conf); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java b/marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java index 17088fc..362b8c8 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java @@ -56,26 +56,26 @@ public void testService(){ } // Test for numThreads values. - final AtomicInteger exeptionCount = new AtomicInteger(0); + final AtomicInteger exceptionCount = new AtomicInteger(0); IntStream.of(0, -1).forEach( numThreads -> { try { initService(numThreads, 1, 1); Assert.fail(); } catch (IllegalStateException e) { - exeptionCount.incrementAndGet(); + exceptionCount.incrementAndGet(); } } ); - Assert.assertEquals(2, exeptionCount.get()); + Assert.assertEquals(2, exceptionCount.get()); final int numThreads = 4; final int numJobDagThreads = 2; final int numActionsThreads = 2; initService(numThreads, numJobDagThreads, numActionsThreads); - // re-initialization should fail. + // re-initialization should fail (singleton pattern). try { initService(numThreads, numJobDagThreads, numActionsThreads); Assert.fail(); @@ -85,7 +85,7 @@ public void testService(){ // Test that caller always waits till spawned tasks have finished. final AtomicInteger runningCounter = new AtomicInteger(0); - final int attempts = 10; + final int attempts = 3; final Queue> branchResults = new LinkedList<>(); final Callable task = () -> { @@ -93,7 +93,7 @@ public void testService(){ while (pending-- > 0) { runningCounter.incrementAndGet(); try { - Thread.sleep(100); + Thread.sleep(1); } catch (InterruptedException e) { Assert.fail(); } @@ -124,7 +124,7 @@ public void testService(){ () -> { counter.incrementAndGet(); try { - Thread.sleep(200); + Thread.sleep(1); } catch (InterruptedException e) { log.error("I was interrupted???"); } @@ -144,7 +144,7 @@ public void testService(){ Assert.assertEquals(jobDagCount + actionsCount, results.size()); // make sure they all ran Assert.assertEquals(jobDagCount + actionsCount, counter.get()); - exeptionCount.set(0); + exceptionCount.set(0); while (!results.isEmpty()) { final Future status = results.poll(); try { @@ -153,13 +153,13 @@ public void testService(){ } catch (InterruptedException e) { Assert.fail(); } catch (ExecutionException e) { - exeptionCount.incrementAndGet(); + exceptionCount.incrementAndGet(); // expected. Assert.assertEquals(ArithmeticException.class, e.getCause().getClass()); } } // make sure they all failed - Assert.assertEquals(jobDagCount + actionsCount, exeptionCount.get()); + Assert.assertEquals(jobDagCount + actionsCount, exceptionCount.get()); } @Test diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java index 260138c..1329fd8 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java @@ -23,6 +23,9 @@ import java.util.Map; import java.util.Set; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.hibernate.validator.constraints.NotEmpty; @@ -33,6 +36,14 @@ public class MemoryMetadataManager implements IMetadataManager { private final Map metadataMap = new HashMap<>(); + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + } + @Override public void set(@NotEmpty final String key, @NotEmpty final StringValue value) { metadataMap.put(key, value); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestCassandraBasedMetadataManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestCassandraBasedMetadataManager.java new file mode 100644 index 0000000..03a6303 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestCassandraBasedMetadataManager.java @@ -0,0 +1,112 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.common.metadata; + +import com.uber.marmaray.common.util.CassandraTestConstants; +import com.uber.marmaray.common.configuration.CassandraMetadataManagerConfiguration; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.MetadataManagerConfiguration; +import org.junit.After; +import org.junit.Test; +import org.junit.Before; +import org.apache.thrift.transport.TTransportException; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import org.junit.Assert; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.base.Optional; +import java.io.IOException; +import java.util.Map; +import com.uber.marmaray.common.util.CassandraTestUtil; + + +public class TestCassandraBasedMetadataManager { + + private final static String JOB_NAME = "jobName"; + private CassandraBasedMetadataManager metadataManager; + private Configuration conf = new Configuration(); + CassandraMetadataManagerConfiguration cassandraMetadataManagerConf; + + + @Before + public void setupTest() throws IOException { + try { + CassandraTestUtil.setupCluster(); + } catch (TTransportException | IOException e) { + throw new JobRuntimeException("Error while initializing Cassandra cluster for test: " + e.getMessage()); + } + this.conf.setProperty(MetadataManagerConfiguration.JOB_NAME, JOB_NAME); + this.conf.setProperty(MetadataManagerConfiguration.TYPE, "CASSANDRA"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.CLUSTER, "clusterName"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.NATIVE_TRANSPORT_PORT, "9142"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.INITIAL_HOSTS, CassandraTestConstants.LOCALHOST); + this.conf.setProperty(CassandraMetadataManagerConfiguration.USERNAME, "marmaray"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.PASSWORD, "password"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.TABLE_NAME, "marmaray_metadata_table"); + this.conf.setProperty(CassandraMetadataManagerConfiguration.KEYSPACE, CassandraTestConstants.KEY_SPACE); + this.cassandraMetadataManagerConf = new CassandraMetadataManagerConfiguration(this.conf); + this.metadataManager = new CassandraBasedMetadataManager(cassandraMetadataManagerConf, + new AtomicBoolean(true)); + } + + @After + public void teardownTest() { + CassandraTestUtil.teardownCluster(); + } + + @Test + public void testReadWrite() throws IOException { + this.metadataManager.set(MetadataConstants.CHECKPOINT_KEY, new StringValue("val")); + final Optional readValue = this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY); + Assert.assertTrue(readValue.isPresent()); + Assert.assertTrue(readValue.get().getValue().equals("val")); + + // save metadata map back to Cassandra + this.metadataManager.saveChanges(); + + // reload the metadata from Cassandra and check + final Map metadataMap = this.metadataManager.generateMetaDataMap(); + Assert.assertTrue(metadataMap.get(MetadataConstants.CHECKPOINT_KEY).getValue().equals("val")); + } + + @Test + public void testOverwriteCheckpointValue() throws IOException { + + final StringValue val1 = new StringValue("testVal"); + this.metadataManager.set(MetadataConstants.CHECKPOINT_KEY, val1); + + final StringValue val2 = new StringValue("testVal2"); + this.metadataManager.set(MetadataConstants.CHECKPOINT_KEY, val2); + + final Optional readValue = this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY); + Assert.assertTrue(readValue.isPresent()); + Assert.assertTrue(readValue.get().getValue().equals("testVal2")); + + this.metadataManager.saveChanges(); + + final Map metaDataMap= this.metadataManager.generateMetaDataMap(); + Assert.assertTrue(metaDataMap.get(MetadataConstants.CHECKPOINT_KEY).getValue().equals("testVal2")); + } + + + @Test + public void testDeletionIsPropagated() throws IOException { + this.metadataManager.generateMetaDataMap(); + this.metadataManager.deleteAllMetadataOfJob(JOB_NAME); + final Optional readValue = this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY); + Assert.assertFalse(readValue.isPresent()); + } +} \ No newline at end of file diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java index da1e45e..252f342 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java @@ -43,7 +43,7 @@ public class TestHDFSDatePartitionManager { @Before public void setupTest() throws IOException { - this.fs = FSUtils.getFs(new Configuration()); + this.fs = FSUtils.getFs(new Configuration(), Optional.absent()); } @After @@ -59,7 +59,6 @@ public void testGetNextPartitionWithNonexistentCheckpoint() throws IOException { this.fs.mkdirs(basePath); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.absent(), @@ -84,7 +83,6 @@ public void testGetNextPartitionWithStartDateAndNoCheckpoint() throws IOExceptio final Date startDate = sdf.parse("2017-05-15"); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.of(startDate), @@ -107,7 +105,6 @@ public void testGetNextPartitionWitMultipleDatePartitionsAndNoCheckpoint() this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-05-02")); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.absent(), @@ -137,7 +134,6 @@ public void testGetNextPartitionWithMultipleDatePartitionsAndOneCheckpoint() final StringValue val1 = new StringValue("datestr=2017-05-02"); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.absent(), @@ -169,7 +165,6 @@ public void testGetNextPartitionWithCheckpointLaterThanStartDate() final StringValue val1 = new StringValue("datestr=2017-06-02"); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.of(startDate), @@ -201,7 +196,6 @@ public void testGetNextPartitionWithCheckpointBeforeThanStartDate() final StringValue val1 = new StringValue("datestr=2017-05-02"); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.of(startDate), @@ -230,7 +224,6 @@ public void testGetNextPartitionWithSmallerExistentCheckpoint() throws IOExcepti this.fs.mkdirs(new Path(partition1, FILE1)); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.absent(), @@ -258,7 +251,6 @@ public void testGetNextPartitionWithLargerExistentCheckpoint() throws IOExceptio this.fs.mkdirs(new Path(partition1, FILE1)); final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, DATESTR, Optional.absent(), diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java index 00f173f..18cff33 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java @@ -48,7 +48,7 @@ public void setupTest() throws IOException { config.setProperty(MetadataConstants.JOBMANAGER_METADATA_STORAGE, "hdfs"); config.setProperty(MetadataConstants.JOBMANAGER_METADATA_HDFS_BASEPATH, HDFSTestConstants.JOBMANAGER_BASE_METADATA_BASEPATH); - this.fileSystem = FSUtils.getFs(config); + this.fileSystem = FSUtils.getFs(config, Optional.absent()); final AtomicBoolean condition = new AtomicBoolean(true); basePath = new Path(HDFSTestConstants.JOBMANAGER_BASE_METADATA_BASEPATH); this.tracker = Optional.of(new JobManagerMetadataTracker(config)); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java index f0a8b9b..1c2c3ee 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java @@ -40,7 +40,7 @@ public class TestHDFSMetadataManager { @Before public void setupTest() throws IOException { - this.fileSystem = FSUtils.getFs(new Configuration()); + this.fileSystem = FSUtils.getFs(new Configuration(), Optional.absent()); final AtomicBoolean condition = new AtomicBoolean(true); final String metadataPath = new Path(HDFSTestConstants.BASE_METADATA_PATH, JOB_NAME).toString(); this.metadataManager = new HDFSMetadataManager(this.fileSystem, metadataPath, condition); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java index c927141..2b3ee04 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java @@ -48,7 +48,7 @@ public class TestHDFSPartitionManager { @Before public void setupTest() throws IOException { - this.fileSystem = FSUtils.getFs(new Configuration()); + this.fileSystem = FSUtils.getFs(new Configuration(), Optional.absent()); } @After @@ -69,7 +69,6 @@ public void testGetNextPartitionWithNonExistentCheckpoint() throws InterruptedEx this.fileSystem.create(filePath); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -90,7 +89,6 @@ public void testGetNextPartitionWithOnlyTempFileCheckpoints() throws Interrupted this.fileSystem.create(filePath); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -123,7 +121,6 @@ public void testGetNextPartitionSinglePartition() throws IOException, Interrupte final StringValue val1 = new StringValue(PARTITION1); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -149,7 +146,6 @@ public void testGetNextPartitionMultipleDataPartitions() throws IOException, Int this.fileSystem.create(new Path(partition3Path, FILE1)); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -173,7 +169,6 @@ public void testGetLastCheckpointMultipleCheckpoints() throws IOException, Inter this.fileSystem.create(new Path(partition2Path, FILE1)); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -189,7 +184,6 @@ public void testGetLastCheckpointMultipleCheckpoints() throws IOException, Inter metadataManager.saveChanges(); final HDFSPartitionManager pm2 = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -208,7 +202,6 @@ public void testGetLastCheckpointMultipleCheckpoints() throws IOException, Inter final HDFSPartitionManager pm3 = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -231,7 +224,6 @@ public void testGetNextPartitionCheckpointIsLargerThanPartition() throws Interru final StringValue val1 = new StringValue(PARTITION2); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -259,7 +251,6 @@ public void testGetExistingPartitions() throws IOException { this.fileSystem.create(new Path(partition2Path, FILE1)); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); @@ -276,7 +267,6 @@ public void testGetExistingPartitionsOnlyFilesExist() throws IOException { this.fileSystem.create(partition0File); final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, - HDFSTestConstants.BASE_METADATA_PATH, RAW_DATA_PATH, this.fileSystem); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.java b/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.java index 0e106fa..0fac15b 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.java @@ -80,7 +80,8 @@ public void testCreateTableWithSinglePrimaryKeyAndNoClusteringKey() { final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Collections.singletonList("country_code"), - Collections.EMPTY_LIST); + Collections.EMPTY_LIST, + Optional.absent()); final String createTableStmt = schemaManager.generateCreateTableStmt(); final String expected = "CREATE TABLE IF NOT EXISTS marmaray.crossfit_gyms (country_code text," + "state_province text,city text,capacity int,gym_name text, PRIMARY KEY ((country_code))) "; @@ -98,7 +99,8 @@ public void testCreateTableWithSinglePrimaryKeyAndOneClusteringKeyDesc() { final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Collections.singletonList("country_code"), - Collections.singletonList(new ClusterKey("state_province", ClusterKey.Order.DESC))); + Collections.singletonList(new ClusterKey("state_province", ClusterKey.Order.DESC)), + Optional.absent()); final String createTableStmt = schemaManager.generateCreateTableStmt(); final String expected = "CREATE TABLE IF NOT EXISTS marmaray.crossfit_gyms (country_code text," + "state_province text,city text,capacity int,gym_name text, PRIMARY KEY ((country_code),state_province))" @@ -117,7 +119,8 @@ public void testCreateTableWithMultiplePrimaryKeys() { final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), - Collections.EMPTY_LIST); + Collections.EMPTY_LIST, + Optional.absent()); final String createTableStmt = schemaManager.generateCreateTableStmt(); final String expected = "CREATE TABLE IF NOT EXISTS marmaray.crossfit_gyms (country_code text," + "state_province text,city text,capacity int,gym_name text, PRIMARY KEY ((country_code,state_province))) "; @@ -136,7 +139,8 @@ public void testCreateTableWithMultiplePrimaryKeysAndClusteringKeys() { schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); final String createTableStmt = schemaManager.generateCreateTableStmt(); final String expected = "CREATE TABLE IF NOT EXISTS marmaray.crossfit_gyms (country_code text," + "state_province text,city text,capacity int,gym_name text, PRIMARY KEY ((country_code,state_province)" + @@ -160,10 +164,12 @@ public void testAlterTableWithOneNewColumn() { schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); final List alterTableQueries = - schemaManager.generateAlterTableStmt(fields.stream().map(f -> f.getFieldName()).collect(Collectors.toList())); + schemaManager.generateAlterTableStmt(fields.stream() + .collect(Collectors.toMap(CassandraSchemaField::getFieldName, CassandraSchemaField::getType))); Assert.assertTrue(alterTableQueries.size() == 1); Assert.assertEquals("ALTER TABLE marmaray.crossfit_gyms ADD new_field1 text", alterTableQueries.get(0)); @@ -183,10 +189,12 @@ public void testAlterTableWithOneMultipleColumns() { schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); final List alterTableQueries = - schemaManager.generateAlterTableStmt(fields.stream().map(f -> f.getFieldName()).collect(Collectors.toList())); + schemaManager.generateAlterTableStmt(fields.stream() + .collect(Collectors.toMap(CassandraSchemaField::getFieldName, CassandraSchemaField::getType))); Assert.assertTrue(alterTableQueries.size() == 2); Assert.assertEquals("ALTER TABLE marmaray.crossfit_gyms ADD new_field1 text", alterTableQueries.get(0)); @@ -204,12 +212,13 @@ public void testGenerateColumnNameQueries() { schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); - final String cfQuery = schemaManager.getColumnNamesFromColumnFamilyQuery(); - final String tableQuery = schemaManager.getColumnNamesFromTableQuery(); - Assert.assertEquals("SELECT column_name FROM system_schema.columns WHERE " + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); + final String cfQuery = schemaManager.getColumnsFromColumnFamilyQuery(); + final String tableQuery = schemaManager.getColumnsFromTableQuery(); + Assert.assertEquals("SELECT column_name, type FROM system_schema.columns WHERE " + "keyspace_name = 'marmaray' AND columnfamily_name = 'crossfit_gyms'", cfQuery); - Assert.assertEquals("SELECT column_name FROM system_schema.columns WHERE " + Assert.assertEquals("SELECT column_name, type FROM system_schema.columns WHERE " + "keyspace_name = 'marmaray' AND table_name = 'crossfit_gyms'", tableQuery); } @@ -221,101 +230,136 @@ public void testInsertStatement() { schema, Arrays.asList("country_code", "state_province"), Collections.EMPTY_LIST, - Optional.of(10000L)); + Optional.of(10000L), + Optional.absent(), + Optional.absent(), + false); final String insertStmt = schemaManagerNoTTL.generateInsertStmt(); final String expected = "INSERT INTO marmaray.crossfit_gyms ( country_code, state_province, city, " + "capacity, gym_name ) VALUES ( ?,?,?,?,? ) USING TTL 10000"; - Assert.assertEquals(expected, insertStmt); + Assert.assertEquals(expected, insertStmt.trim()); + final CassandraSinkSchemaManager schemaManagerWithTTL = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), Collections.EMPTY_LIST, - Optional.absent()); + Optional.absent(), + Optional.absent(), + Optional.absent(), + false); final String insertStmtWithTTL = schemaManagerWithTTL.generateInsertStmt(); - Assert.assertEquals(expected.replace("USING TTL 10000", StringTypes.EMPTY), insertStmtWithTTL); + Assert.assertEquals(expected.replace("USING TTL 10000", StringTypes.EMPTY).trim(), insertStmtWithTTL.trim()); } - @Test(expected = IllegalStateException.class) + @Test public void testPartitionAndClusterKeyHaveSameName() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("country_code", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); - Assert.fail(); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testEmptyKeySpace() { final CassandraSchema schema = new CassandraSchema(StringTypes.EMPTY, TABLE, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("country_code", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); - Assert.fail(); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testEmptyTableName() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, StringTypes.EMPTY, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), - Collections.EMPTY_LIST); - Assert.fail(); + Collections.EMPTY_LIST, + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testFieldsIsNull() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, null); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), - Collections.EMPTY_LIST); - Assert.fail(); + Collections.EMPTY_LIST, + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testFieldsIsEmpty() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, Collections.EMPTY_LIST); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Arrays.asList("country_code", "state_province"), - Collections.EMPTY_LIST); - Assert.fail(); + Collections.EMPTY_LIST, + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testPartitionKeysIsEmpty() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Collections.EMPTY_LIST, - Collections.EMPTY_LIST); - Assert.fail(); + Collections.EMPTY_LIST, + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) - public void tesClusterKeyIsNotInFieldNames() { + @Test + public void testClusterKeyIsNotInFieldNames() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Collections.singletonList("country_code"), - Collections.singletonList(new ClusterKey("non_cluster_field_name", ClusterKey.Order.ASC))); - Assert.fail(); + Collections.singletonList(new ClusterKey("non_cluster_field_name", ClusterKey.Order.ASC)), + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); } - @Test(expected = IllegalStateException.class) + @Test public void testPartitionKeyIsNotInFieldNames() { final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( schema, Collections.singletonList("non_field_name"), - Collections.EMPTY_LIST); + Collections.EMPTY_LIST, + Optional.absent()); + Assert.assertEquals(schemaManager.getValidSchema(), false); + } + + @Test(expected = IllegalStateException.class) + public void testSchemaMismatch() { + final List wrongFields = fields.stream() + .map( f -> f.getFieldName().equals("city") + ? new CassandraSchemaField("city", CassandraSchemaField.LONG_TYPE) + : f) + .collect(Collectors.toList()); + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, wrongFields); + final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( + schema, + Arrays.asList("country_code", "state_province"), + Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); + schemaManager.generateAlterTableStmt(fields.stream() + .collect(Collectors.toMap(CassandraSchemaField::getFieldName, CassandraSchemaField::getType))); Assert.fail(); + } private void validateCreateTable(final Session session) { @@ -330,7 +374,7 @@ private void validateCreateTable(final Session session) { Arrays.asList("country_code", "state_province", "city", "capacity", "gym_name"))); } - private void validateAlterTable(final Session session, List newColumns) { + private void validateAlterTable(final Session session, final List newColumns) { final ResultSet results = session.execute(getColumnsQuery); final List columns = results.all() @@ -350,7 +394,8 @@ private void createBasicTable() { schema, Arrays.asList("country_code", "state_province"), Arrays.asList(new ClusterKey("city", ClusterKey.Order.DESC), - new ClusterKey("gym_name", ClusterKey.Order.ASC))); + new ClusterKey("gym_name", ClusterKey.Order.ASC)), + Optional.absent()); final String createTableStmt = schemaManager.generateCreateTableStmt(); try (final Session session = getSession()) { diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java index 272abb2..45c95b0 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java @@ -16,38 +16,22 @@ */ package com.uber.marmaray.common.sinks.cassandra; -import static com.uber.marmaray.common.util.CassandraTestConstants.KEY_SPACE; -import static com.uber.marmaray.common.util.CassandraTestConstants.TABLE; - -import com.datastax.driver.core.Cluster; -import com.datastax.driver.core.Row; -import com.datastax.driver.core.Session; -import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.converters.data.CassandraSinkCQLDataConverter; -import com.uber.marmaray.common.converters.data.CassandraSinkDataConverter; import com.uber.marmaray.common.converters.schema.CassandraSchemaConverter; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.schema.cassandra.CassandraSchema; import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; import com.uber.marmaray.common.schema.cassandra.ClusterKey; -import com.uber.marmaray.common.util.AbstractSparkTest; import com.uber.marmaray.common.util.AvroPayloadUtil; import com.uber.marmaray.common.util.CassandraTestConstants; import com.uber.marmaray.common.util.CassandraTestUtil; import com.uber.marmaray.utilities.ErrorExtractor; -import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.StringTypes; import com.uber.marmaray.utilities.TimestampInfo; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; - import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.spark.SparkException; @@ -58,6 +42,15 @@ import org.junit.Before; import org.junit.Test; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; + +import static com.uber.marmaray.common.util.CassandraTestConstants.KEY_SPACE; +import static com.uber.marmaray.common.util.CassandraTestConstants.TABLE; + @Slf4j public class TestCassandraClientSink extends TestCassandraSinkUtil { @@ -139,7 +132,7 @@ public void testWriteMockErrorDataToCassandra() { final CassandraSchema cassandraSchema = schemaConverter.convertToExternalSchema(avroSchema); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys); + new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys, Optional.absent()); final CassandraSinkConfiguration conf = initializeConfiguration(false, includeTimestamp); final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); @@ -166,7 +159,8 @@ private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { final List requiredFields = Arrays.asList(schemaFields.get(0), schemaFields.get(1)); final Optional timestamp = addLongTimestamp ? Optional.of(TEST_TIMESTAMP) : Optional.absent(); - final TimestampInfo tsInfo = new TimestampInfo(timestamp, true); + final TimestampInfo tsInfo = new TimestampInfo(timestamp, true, + CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); final CassandraSinkCQLDataConverter converter = new CassandraSinkCQLDataConverter(AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY), @@ -182,7 +176,7 @@ private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { final Optional ttl = Optional.of(10000L); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl); + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl, Optional.absent(), Optional.absent(), false); final CassandraSinkConfiguration conf = initializeConfiguration(false, addLongTimestamp); final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); @@ -206,7 +200,8 @@ private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addSt final Schema avroSchema = AvroPayloadUtil.getAvroTestDataSchema(CassandraTestConstants.BOOLEAN_FIELD); final Optional timestamp = addStringTimestamp ? Optional.of(TEST_TIMESTAMP) : Optional.absent(); - final TimestampInfo tsInfo = new TimestampInfo(timestamp, false); + final TimestampInfo tsInfo = new TimestampInfo(timestamp, false, + CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); final CassandraSinkCQLDataConverter converter = new CassandraSinkCQLDataConverter( @@ -220,7 +215,7 @@ private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addSt final CassandraSchema schema = schemaConverter.convertToExternalSchema(avroSchema); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys); + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, Optional.absent()); final CassandraSinkConfiguration conf = initializeConfiguration(true, addStringTimestamp); final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java index 0cbf5fe..9128cfd 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java @@ -16,13 +16,6 @@ */ package com.uber.marmaray.common.sinks.cassandra; -import static com.uber.marmaray.common.util.CassandraTestConstants.KEY_SPACE; -import static com.uber.marmaray.common.util.CassandraTestConstants.TABLE; - -import com.datastax.driver.core.Cluster; -import com.datastax.driver.core.Row; -import com.datastax.driver.core.Session; -import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; @@ -30,31 +23,48 @@ import com.uber.marmaray.common.converters.data.CassandraSinkDataConverter; import com.uber.marmaray.common.converters.schema.CassandraSchemaConverter; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.schema.cassandra.CassandraDataField; +import com.uber.marmaray.common.schema.cassandra.CassandraPayload; import com.uber.marmaray.common.schema.cassandra.CassandraSchema; import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; import com.uber.marmaray.common.schema.cassandra.ClusterKey; -import com.uber.marmaray.common.util.AbstractSparkTest; import com.uber.marmaray.common.util.AvroPayloadUtil; import com.uber.marmaray.common.util.CassandraTestConstants; import com.uber.marmaray.common.util.CassandraTestUtil; +import com.uber.marmaray.utilities.ByteBufferUtil; import com.uber.marmaray.utilities.ErrorExtractor; -import com.uber.marmaray.utilities.SchemaUtil; import com.uber.marmaray.utilities.StringTypes; import com.uber.marmaray.utilities.TimestampInfo; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; +import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.cassandra.hadoop.cql3.CqlBulkRecordWriter; import org.apache.spark.SparkException; import org.apache.spark.api.java.JavaRDD; import org.apache.thrift.transport.TTransportException; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static com.uber.marmaray.common.util.CassandraTestConstants.CONFIGURATION; +import static com.uber.marmaray.common.util.CassandraTestConstants.KEY_SPACE; +import static com.uber.marmaray.common.util.CassandraTestConstants.TABLE; +@Slf4j public class TestCassandraSSTableSink extends TestCassandraSinkUtil { private static final String TEST_TIMESTAMP = "10000"; @@ -123,11 +133,14 @@ public void testWriteMockErrorDataToCassandra() { // Still want to include the INT field in the schema but we don't write data for it final Schema avroSchema = AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY); + final Configuration conf = new Configuration(CONFIGURATION); + conf.setProperty(CassandraSinkConfiguration.MAX_BATCH_SIZE_MB, "1"); final CassandraSinkDataConverter converter = new CassandraSinkDataConverter(avroSchema, - new Configuration(), + conf, Optional.of(new HashSet<>(schemaFields)), + Optional.absent(), requiredFields, TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); @@ -135,16 +148,16 @@ public void testWriteMockErrorDataToCassandra() { final CassandraSchema cassandraSchema = schemaConverter.convertToExternalSchema(avroSchema); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys); - final CassandraSinkConfiguration conf = initializeConfiguration(false, includeTimestamp); + new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys, Optional.absent()); + final CassandraSinkConfiguration sinkConf = initializeConfiguration(false, includeTimestamp); - final CassandraSSTableSink sink = new CassandraSSTableSink(converter, schemaManager, conf); + final CassandraSSTableSink sink = new CassandraSSTableSink(converter, schemaManager, sinkConf); try { sink.write(testData); } catch (Exception e) { Assert.assertEquals(SparkException.class, e.getClass()); Assert.assertEquals(JobRuntimeException.class, e.getCause().getClass()); - Assert.assertEquals(JobRuntimeException.class, e.getCause().getCause().getClass()); + Assert.assertEquals(MissingPropertyException.class, e.getCause().getCause().getClass()); } } @@ -163,12 +176,14 @@ private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { final List requiredFields = Arrays.asList(schemaFields.get(0), schemaFields.get(1)); final Optional timestamp = addLongTimestamp ? Optional.of(TEST_TIMESTAMP) : Optional.absent(); - final TimestampInfo tsInfo = new TimestampInfo(timestamp, true); + final TimestampInfo tsInfo = new TimestampInfo(timestamp, true, + CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); final CassandraSinkDataConverter dataconverter = new CassandraSinkDataConverter(AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY), - new Configuration(), + CONFIGURATION, Optional.of(new HashSet<>(schemaFields)), + Optional.absent(), requiredFields, tsInfo, new ErrorExtractor()); @@ -179,7 +194,7 @@ private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { final Optional ttl = Optional.of(10000L); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl); + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl, Optional.absent(), Optional.absent(), false); final CassandraSinkConfiguration conf = initializeConfiguration(false, addLongTimestamp); final CassandraSSTableSink sink = new CassandraSSTableSink(dataconverter, schemaManager, conf); @@ -203,13 +218,15 @@ private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addSt final Schema avroSchema = AvroPayloadUtil.getAvroTestDataSchema(CassandraTestConstants.BOOLEAN_FIELD); final Optional timestamp = addStringTimestamp ? Optional.of(TEST_TIMESTAMP) : Optional.absent(); - final TimestampInfo tsInfo = new TimestampInfo(timestamp, false); + final TimestampInfo tsInfo = new TimestampInfo(timestamp, false, + CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); final CassandraSinkDataConverter converter = new CassandraSinkDataConverter( avroSchema, - new Configuration(), + CONFIGURATION, Optional.of(new HashSet<>(schemaFields)), + Optional.absent(), requiredFields, tsInfo, new ErrorExtractor()); @@ -217,11 +234,122 @@ private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addSt final CassandraSchema schema = schemaConverter.convertToExternalSchema(avroSchema); final CassandraSinkSchemaManager schemaManager = - new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys); + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, Optional.absent()); final CassandraSinkConfiguration conf = initializeConfiguration(true, addStringTimestamp); final CassandraSSTableSink sink = new CassandraSSTableSink(converter, schemaManager, conf); sink.write(testData); validateCassandraTable(100, true, addStringTimestamp); } + + @Test + public void testIgnoredHosts() { + final Configuration conf = new Configuration(); + conf.setProperty(CassandraSinkConfiguration.ENABLE_IGNORE_HOSTS, "true"); + conf.setProperty(CassandraSinkConfiguration.KEYSPACE, "keyspace"); + conf.setProperty(CassandraSinkConfiguration.TABLE_NAME, "table"); + conf.setProperty(CassandraSinkConfiguration.CLUSTER_NAME, "cluster"); + conf.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, "partition"); + conf.setProperty(CassandraSinkConfiguration.DATACENTER, "dc"); + CassandraSinkConfiguration sinkConf = new CassandraSinkConfiguration(conf); + CassandraSSTableSink sink = Mockito.spy(new CassandraSSTableSink( + Mockito.mock(CassandraSinkDataConverter.class), + Mockito.mock(CassandraSinkSchemaManager.class), + sinkConf)); + final String ignoredHosts = "127.0.0.1"; + Mockito.when(sink.computeIgnoredHosts()).thenReturn(ignoredHosts); + org.apache.hadoop.conf.Configuration hadoopConf = sinkConf.getHadoopConf(); + Assert.assertNull(hadoopConf.get(CqlBulkRecordWriter.IGNORE_HOSTS)); + sink.setIgnoredHosts(hadoopConf); + Assert.assertEquals(ignoredHosts, hadoopConf.get(CqlBulkRecordWriter.IGNORE_HOSTS)); + // test that nothing happens if the flag is disabled + conf.setProperty(CassandraSinkConfiguration.ENABLE_IGNORE_HOSTS, "false"); + sinkConf = new CassandraSinkConfiguration(conf); + hadoopConf = sinkConf.getHadoopConf(); + sink = Mockito.spy(new CassandraSSTableSink( + Mockito.mock(CassandraSinkDataConverter.class), + Mockito.mock(CassandraSinkSchemaManager.class), + sinkConf)); + Assert.assertNull(hadoopConf.get(CqlBulkRecordWriter.IGNORE_HOSTS)); + sink.setIgnoredHosts(hadoopConf); + Assert.assertNull(hadoopConf.get(CqlBulkRecordWriter.IGNORE_HOSTS)); + + } + + @Test + public void testBatching() { + CassandraSinkConfiguration sinkConf = initializeConfiguration(false, false); + final Configuration conf = sinkConf.getConf(); + conf.setProperty(CassandraSinkConfiguration.MAX_BATCH_SIZE_MB, "1"); + sinkConf = new CassandraSinkConfiguration(conf); + + final Schema schema = SchemaBuilder.builder().record("myrecord").fields() + .name(INT_FIELD).type().nullable().intType().noDefault() + .name(STRING_FIELD).type().nullable().stringType().noDefault() + .endRecord(); + final List payloadList = IntStream.range(1, 5).mapToObj( + i -> { + final GenericRecord gr = new GenericData.Record(schema); + gr.put(INT_FIELD, i); + gr.put(STRING_FIELD, String.valueOf(i)); + final AvroPayload payload = new AvroPayload(gr); + return payload; + } + ).collect(Collectors.toList()); + final JavaRDD data = this.jsc.get().parallelize(payloadList); + final CassandraSinkDataConverter converter = new CassandraSinkDataConverter(schema, conf, Optional.absent(), + Optional.absent(), Collections.singletonList(INT_FIELD), TimestampInfo.generateEmptyTimestampInfo(), + new ErrorExtractor()); + final CassandraSchemaConverter cassandraSchemaConverter = new CassandraSchemaConverter( + sinkConf.getKeyspace(), sinkConf.getTableName(), + TimestampInfo.generateEmptyTimestampInfo(), sinkConf.getFilteredColumns()); + final CassandraSchema cassandraSchema = cassandraSchemaConverter.convertToExternalSchema(schema); + + final CassandraSinkSchemaManager manager = new CassandraSinkSchemaManager(cassandraSchema, + Collections.singletonList(INT_FIELD), Collections.emptyList(), Optional.absent(), Optional.absent(), + Optional.absent(), false); + final CassandraSSTableSink sink = new CassandraSSTableSink(converter, manager, sinkConf); + sink.write(data); + validateCassandraTable(4, true, false); + } + + @Test + public void testCountComputation() { + final CassandraSinkConfiguration mockConfiguration = Mockito.mock(CassandraSinkConfiguration.class); + final CassandraSSTableSink mockSink = Mockito.spy( + new CassandraSSTableSink(Mockito.mock(CassandraSinkDataConverter.class), + Mockito.mock(CassandraSinkSchemaManager.class), + mockConfiguration)); + Mockito.when(mockConfiguration.getMaxBatchSizeMb()).thenReturn(1L); + Mockito.when(mockConfiguration.isBatchEnabled()).thenReturn(true); + // test round up + Assert.assertEquals(2, mockSink.getNumBatches(1024 * 1024 * 1.5)); + // test exact value + Assert.assertEquals(1, mockSink.getNumBatches(1024 * 1024)); + Mockito.when(mockConfiguration.getMaxBatchSizeMb()).thenReturn(Long.MAX_VALUE); + Assert.assertEquals(1, mockSink.getNumBatches(1)); + } + + @Test + public void testComputeSortOrder() { + final CassandraSinkConfiguration mockConfiguration = Mockito.mock(CassandraSinkConfiguration.class); + final CassandraSSTableSink sink = new CassandraSSTableSink(Mockito.mock(CassandraSinkDataConverter.class), + Mockito.mock(CassandraSinkSchemaManager.class), + mockConfiguration); + Mockito.when(mockConfiguration.getPartitionKeys()).thenReturn(Arrays.asList("e", "a")); + Mockito.when(mockConfiguration.getClusteringKeys()).thenReturn(Collections.singletonList(new ClusterKey("b", ClusterKey.Order.ASC))); + final CassandraPayload payload = new CassandraPayload(); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("a"), ByteBufferUtil.wrap("aVal"))); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("b"), ByteBufferUtil.wrap("bVal"))); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("c"), ByteBufferUtil.wrap("cVal"))); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("d"), ByteBufferUtil.wrap("dVal"))); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("e"), ByteBufferUtil.wrap("eVal"))); + payload.addField(new CassandraDataField(ByteBufferUtil.wrap("f"), ByteBufferUtil.wrap("fVal"))); + final List sortOrder = sink.computeSortOrder(payload); + Assert.assertEquals(3, sortOrder.size()); + Assert.assertEquals(4, (int) sortOrder.get(0)); + Assert.assertEquals(0, (int) sortOrder.get(1)); + Assert.assertEquals(1, (int) sortOrder.get(2)); + + } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.java index 10730eb..f0cf14f 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.java @@ -51,6 +51,8 @@ public class TestCassandraSinkUtil extends AbstractSparkTest { protected static final String TEST_TIMESTAMP = "10000"; + protected static final String INT_FIELD = "int_field"; + protected static final String STRING_FIELD = "string_field"; protected CassandraSinkConfiguration initializeConfiguration(boolean excludeField, boolean hasTimestampField) { @@ -61,8 +63,8 @@ protected CassandraSinkConfiguration initializeConfiguration(boolean excludeFiel conf.setProperty(CassandraSinkConfiguration.TABLE_NAME, CassandraTestConstants.TABLE); conf.setProperty(CassandraSinkConfiguration.KEYSPACE, CassandraTestConstants.KEY_SPACE); conf.setProperty(CassandraSinkConfiguration.CLUSTER_NAME, "testCluster"); - conf.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, "int_field"); - conf.setProperty(CassandraSinkConfiguration.CLUSTERING_KEYS, "string_field"); + conf.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, INT_FIELD); + conf.setProperty(CassandraSinkConfiguration.CLUSTERING_KEYS, STRING_FIELD); // we always exclude the boolean field for now, can modify this in future to exclude a specific field if (!excludeField) { @@ -98,7 +100,7 @@ protected void validateCassandraTable(final int expectedNumRows, } if (checkTimestampField) { - final Object timestampObj = row.getObject(SchemaUtil.DISPERSAL_TIMESTAMP); + final Object timestampObj = row.getObject(CassandraSinkConfiguration.DEFAULT_DISPERSAL_TIMESTAMP_FIELD_NAME); if (timestampObj instanceof Long) { Assert.assertEquals(Long.parseLong(TEST_TIMESTAMP), timestampObj); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java index 4d7bd32..d074c9d 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java @@ -69,9 +69,9 @@ protected Configuration initConfig(@NonNull final String pathPrefix, @NonNull fi protected Configuration initConfigWithAws(@NonNull final String pathPrefix, @NonNull final String objectKey, @NonNull final String awsLocal, @NonNull final String dispersalType, @NonNull final String timeStamp, @NonNull final String sourceSubPath, - @NonNull final String bucketName) { + @NonNull final String bucketName, @NonNull final String fileType) { final Configuration conf=new Configuration(); - conf.setProperty(FileSinkConfiguration.FILE_TYPE,"csv"); + conf.setProperty(FileSinkConfiguration.FILE_TYPE,fileType); conf.setProperty(FileSinkConfiguration.FILE_SIZE_MEGABYTE,"0.001"); conf.setProperty(FileSinkConfiguration.SEPARATOR,","); conf.setProperty(FileSinkConfiguration.PATH_PREFIX, pathPrefix); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java index 9506b1c..04c4404 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java @@ -23,6 +23,7 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.model.PutObjectRequest; +import com.uber.marmaray.common.converters.data.FileSinkDataConverterFactory; import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.AwsConfiguration; import com.uber.marmaray.common.configuration.Configuration; @@ -30,7 +31,6 @@ import com.uber.marmaray.common.converters.data.FileSinkDataConverter; import com.uber.marmaray.common.exceptions.JobRuntimeException; import com.uber.marmaray.common.util.AvroPayloadUtil; -import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.StringTypes; import io.findify.s3mock.S3Mock; import lombok.NonNull; @@ -91,7 +91,8 @@ public class TestAwsFileSink extends FileSinkTestUtil { private static final String OBJ_KEY_2 = "test2"; private static final String PARENT_DIR = "aws_test"; private static final int NUM_RECORD = 100; - private static final int EXPECTED_PARTITION_NUM = 2; + private static final int EXPECTED_CSV_PARTITION_NUM = 2; + private static final int EXPECTED_JSON_PARTITION_NUM = 7; private static final int EXPECTED_INVOCATIONS = 1; private static final String TIMESTAMP1 = "201808011025"; private static final String TIMESTAMP2 = "201808012025"; @@ -101,6 +102,8 @@ public class TestAwsFileSink extends FileSinkTestUtil { private static final String VERSION = "version"; private static final String OVERWRITE = "overwrite"; private static final String S3_TEST_FOLDER = "s3-tests"; + private static final String CSV = "csv"; + private static final String SEQUENCE = "sequence"; private S3Mock s3mock; private String port; @Rule @@ -141,15 +144,29 @@ public void tearDownTest() throws IOException { @Ignore @Test public void testWriteToS3WithVersion() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, SOURCE_SUB_PATH1, S3_BUCKET_NAME); - final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, SOURCE_SUB_PATH1, S3_BUCKET_NAME); - final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, SOURCE_SUB_PATH2, S3_BUCKET_NAME); + // CSV + final Configuration csvConf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, CSV); + final Configuration csvConf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, CSV); + final Configuration csvConf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_BUCKET_NAME, CSV); final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), NUM_RECORD, StringTypes.EMPTY); - testWriteGeneral(testData, conf1); - testWriteGeneral(testData, conf2); - testWriteGeneral(testData, conf3); + testWriteGeneral(testData, csvConf1); + testWriteGeneral(testData, csvConf2); + testWriteGeneral(testData, csvConf3); + //SEQUENCE + final Configuration seqConf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, SEQUENCE); + final Configuration seqConf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, SEQUENCE); + final Configuration seqConf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_BUCKET_NAME, SEQUENCE); + testWriteGeneral(testData, seqConf1); + testWriteGeneral(testData, seqConf2); + testWriteGeneral(testData, seqConf3); } /* Test bucket name: uber-test @@ -158,9 +175,12 @@ public void testWriteToS3WithVersion() throws IOException { @Ignore @Test public void testWriteToS3WithSpecialCharAndOverWrite() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL1, VERSION, TIMESTAMP1, SOURCE_SUB_PATH1, S3_BUCKET_NAME); - final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL2, VERSION, TIMESTAMP2, SOURCE_SUB_PATH1, S3_BUCKET_NAME); - final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL3, OVERWRITE, TIMESTAMP3, SOURCE_SUB_PATH2, S3_BUCKET_NAME); + final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, CSV); + final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH1, S3_BUCKET_NAME, CSV); + final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_2, LOCAL3, OVERWRITE, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_BUCKET_NAME, CSV); final JavaRDD testData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), NUM_RECORD, StringTypes.EMPTY); @@ -171,24 +191,34 @@ public void testWriteToS3WithSpecialCharAndOverWrite() throws IOException { //The following tests are used for uploading to Mock S3 Client //Todo-T1984925: FIX test error here. - @Ignore @Test public void testWriteToMockS3WithSingleVersion() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME); - testWriteToMockS3General(conf1); + final Configuration csvConf = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, CSV); + testWriteToMockS3General(csvConf); + final Configuration seqConf = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, SEQUENCE); + testWriteToMockS3General(seqConf); } - @Ignore + @Test public void testWriteToMockS3WithSingleOverwrite() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, OVERWRITE, TIMESTAMP1, SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME); - testWriteToMockS3General(conf1); + final Configuration csvConf = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, OVERWRITE, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, CSV); + testWriteToMockS3General(csvConf); + final Configuration seqConf = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, OVERWRITE, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, SEQUENCE); + testWriteToMockS3General(seqConf); } - @Ignore + @Test public void testWriteToMockS3WithMultiVersion() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME); - final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME); - final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME); + final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, CSV); + final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, CSV); + final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, CSV); final List filePrefix = new ArrayList<>(); filePrefix.add(testWriteToMockS3General(conf1)); filePrefix.add(testWriteToMockS3General(conf2)); @@ -196,18 +226,39 @@ public void testWriteToMockS3WithMultiVersion() throws IOException { final AmazonS3 MockClient = getMockS3Connection(); assertTrue(MockClient.doesBucketExistV2(S3_TEST_BUCKET_NAME)); for (String aFilePrefix : filePrefix) { - for (int j = 0; j < EXPECTED_PARTITION_NUM; j++) { + for (int j = 0; j < EXPECTED_CSV_PARTITION_NUM; j++) { + final Boolean objectExist = MockClient.doesObjectExist(S3_TEST_BUCKET_NAME, aFilePrefix + "_0000" + j); + assertTrue(objectExist); + } + } + + final Configuration seqConf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, SEQUENCE); + final Configuration seqConf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, SEQUENCE); + final Configuration seqConf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, VERSION, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, SEQUENCE); + final List seqFilePrefix = new ArrayList<>(); + seqFilePrefix.add(testWriteToMockS3General(seqConf1)); + seqFilePrefix.add(testWriteToMockS3General(seqConf2)); + seqFilePrefix.add(testWriteToMockS3General(seqConf3)); + assertTrue(MockClient.doesBucketExistV2(S3_TEST_BUCKET_NAME)); + for (String aFilePrefix : seqFilePrefix) { + for (int j = 0; j < EXPECTED_JSON_PARTITION_NUM; j++) { final Boolean objectExist = MockClient.doesObjectExist(S3_TEST_BUCKET_NAME, aFilePrefix + "_0000" + j); assertTrue(objectExist); } } } - @Ignore + @Test public void testWriteToMockS3WithMultiOverWrite() throws IOException { - final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME); - final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME); - final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, OVERWRITE, TIMESTAMP3, SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME); + final Configuration conf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, CSV); + final Configuration conf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, CSV); + final Configuration conf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, OVERWRITE, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, CSV); final List filePrefix = new ArrayList(); filePrefix.add(testWriteToMockS3General(conf1)); filePrefix.add(testWriteToMockS3General(conf2)); @@ -215,7 +266,7 @@ public void testWriteToMockS3WithMultiOverWrite() throws IOException { final AmazonS3 MockClient = getMockS3Connection(); assertTrue(MockClient.doesBucketExistV2(S3_TEST_BUCKET_NAME)); for (int i = 0 ; i < filePrefix.size() ; i++) { - for (int j = 0; j < EXPECTED_PARTITION_NUM; j++) { + for (int j = 0; j < EXPECTED_CSV_PARTITION_NUM; j++) { final Boolean objectExist = MockClient.doesObjectExist(S3_TEST_BUCKET_NAME, filePrefix.get(i) + "_0000" + j); if (i == 1) { assertFalse(objectExist); @@ -225,30 +276,58 @@ public void testWriteToMockS3WithMultiOverWrite() throws IOException { } } + final Configuration seqConf1 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL1, VERSION, TIMESTAMP1, + SOURCE_SUB_PATH1, S3_TEST_BUCKET_NAME, SEQUENCE); + final Configuration seqConf2 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL2, VERSION, TIMESTAMP2, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, SEQUENCE); + final Configuration seqConf3 = initConfigWithAws(this.pathPrefix, OBJ_KEY_1, LOCAL3, OVERWRITE, TIMESTAMP3, + SOURCE_SUB_PATH2, S3_TEST_BUCKET_NAME, SEQUENCE); + final List seqFilePrefix = new ArrayList(); + seqFilePrefix.add(testWriteToMockS3General(seqConf1)); + seqFilePrefix.add(testWriteToMockS3General(seqConf2)); + seqFilePrefix.add(testWriteToMockS3General(seqConf3)); + assertTrue(MockClient.doesBucketExistV2(S3_TEST_BUCKET_NAME)); + for (int i = 0 ; i < seqFilePrefix.size() ; i++) { + for (int j = 0; j < EXPECTED_JSON_PARTITION_NUM; j++) { + final Boolean objectExist = MockClient.doesObjectExist(S3_TEST_BUCKET_NAME, filePrefix.get(i) + "_0000" + j); + if (i == 1) { + assertFalse(objectExist); + } else { + assertTrue(objectExist); + } + } + } } private String testWriteToMockS3General(@NonNull final Configuration conf) throws IOException { final JavaRDD testData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), NUM_RECORD, StringTypes.EMPTY); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + final FileSinkDataConverter converter = FileSinkDataConverterFactory.createFileSinkDataConverter(conf); final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); final AwsConfiguration awsConf = new AwsConfiguration(fileConf); final MockAwsFileSink awsMockSink = spy(new MockAwsFileSink(fileConf, converter)); awsMockSink.write(testData); final AmazonS3 MockClient = awsMockSink.getS3Client(); verify(awsMockSink, times(EXPECTED_INVOCATIONS)).write(Matchers.any(JavaRDD.class)); - verify(MockClient, times(EXPECTED_PARTITION_NUM)).putObject(Matchers.any(PutObjectRequest.class)); + final int partitionNumber = fileConf.getFileType().equals(CSV) ? EXPECTED_CSV_PARTITION_NUM : EXPECTED_JSON_PARTITION_NUM; + verify(MockClient, times(partitionNumber)).putObject(Matchers.any(PutObjectRequest.class)); assertTrue(MockClient.doesBucketExistV2(fileConf.getBucketName().get())); - for (int i = 0 ; i < EXPECTED_PARTITION_NUM ; i++) { + for (int i = 0 ; i < EXPECTED_CSV_PARTITION_NUM ; i++) { final Boolean objectExist = MockClient.doesObjectExist(fileConf.getBucketName().get(), awsConf.getS3FilePrefix()+ "_0000" + i); assertTrue(objectExist); } return awsConf.getS3FilePrefix(); } + /** + * test Aws3 client with end to end connection + * @param testData + * @param conf + * @throws IOException + */ private void testWriteGeneral(@NonNull final JavaRDD testData, @NonNull final Configuration conf) throws IOException { - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + final FileSinkDataConverter converter = FileSinkDataConverterFactory.createFileSinkDataConverter(conf); final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); final FileSink awsSink = new AwsFileSink(fileConf, converter); awsSink.write(testData); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java index da48a99..db3e058 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java @@ -20,13 +20,14 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.configuration.FileSinkConfiguration; -import com.uber.marmaray.common.converters.data.FileSinkDataConverter; +import com.uber.marmaray.common.converters.data.FileSinkDataCSVConverter; import com.uber.marmaray.common.util.AvroPayloadUtil; import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.StringTypes; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.junit.After; import org.junit.Before; @@ -56,7 +57,7 @@ public class TestFileSink extends FileSinkTestUtil{ private JavaRDD convertedData2; private JavaRDD testData2; private Configuration conf; - private FileSinkDataConverter converter; + private FileSinkDataCSVConverter converter; private FileSink fileSink; @Before @@ -69,11 +70,14 @@ public void setupTest() { NUM_RECORD2, StringTypes.EMPTY); this.conf = initConfig(pathPrefix, PATH1, COMMA_SEPARATOR, TIMESTAMP1, SOURCE_SUB_PATH1, VERSION); - this.converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + this.converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); this.fileSink = spy(new HdfsFileSink(fileConf, converter)); - this.convertedData1 = this.converter.convertAll(this.testData1); - this.convertedData2 = this.converter.convertAll(this.testData2); + + final JavaPairRDD tmpData1 = this.converter.convertAll(this.testData1); + this.convertedData1 = tmpData1.map(message -> message._2()); + final JavaPairRDD tmpData2 = this.converter.convertAll(this.testData2); + this.convertedData2 = tmpData2.map(message -> message._2()); } @After @@ -90,15 +94,15 @@ public void testGetRepartitionNum() { @Test public void testGetRddSizeNoMoreThanSampleRow() { - final double rddSize = fileSink.getRddSizeInMegaByte(convertedData1); - final long sampleSize = fileSink.getSampleSizeInBytes(convertedData1); + final double rddSize = fileSink.getRddSizeInMegaByte(this.convertedData1); + final long sampleSize = fileSink.getSampleSizeInBytes(this.convertedData1); final double sampleSizeInMB = (double) sampleSize / FileUtils.ONE_MB; assertEquals(rddSize, sampleSizeInMB, 0.1); } @Test public void testGetRddSizeMoreThanSampleRow() { - final double rddSize = fileSink.getRddSizeInMegaByte(convertedData2); + final double rddSize = fileSink.getRddSizeInMegaByte(this.convertedData2); verify(this.fileSink, times(1)).getSampleSizeInBytes(Matchers.any(JavaRDD.class)); final long sampleSize = fileSink.getSampleSizeInBytes(convertedData2); final double sampleSizeInMB = (double) sampleSize / FileUtils.ONE_MB; @@ -109,7 +113,7 @@ public void testGetRddSizeMoreThanSampleRow() { @Test public void testGetSampleSizeInBytes() { - final long sampleSize = fileSink.getSampleSizeInBytes(convertedData1); + final long sampleSize = fileSink.getSampleSizeInBytes(this.convertedData1); assertEquals(EXPECTED_SAMPLE_SIZE, sampleSize); } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java index 7018d49..9614522 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java @@ -20,7 +20,7 @@ import com.uber.marmaray.common.AvroPayload; import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.configuration.FileSinkConfiguration; -import com.uber.marmaray.common.converters.data.FileSinkDataConverter; +import com.uber.marmaray.common.converters.data.FileSinkDataCSVConverter; import com.uber.marmaray.common.util.AvroPayloadUtil; import com.uber.marmaray.utilities.ErrorExtractor; import com.uber.marmaray.utilities.StringTypes; @@ -109,7 +109,7 @@ public void testWriteToCsvWithHeader() throws IOException { StringTypes.EMPTY); final Configuration conf = initConfig(pathPrefix, PATH4, COMMA_SEPARATOR, TIMESTAMP1, SOURCE_SUB_PATH1, VERSION); conf.setProperty(FileSinkConfiguration.CSV_COLUMN_HEADER, "true"); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); final HdfsFileSink hdfsSink = spy(new HdfsFileSink(fileConf, converter)); hdfsSink.write(testData); @@ -134,7 +134,7 @@ private void testWriteToCsvCommon(@NonNull final String pathPrefix, @NonNull fin @NonNull final int partitionNum, @NonNull final String timeStamp, @NonNull final String sourceSubPath, @NonNull final String dispersalType) throws Exception { final Configuration conf = initConfig(pathPrefix, path, separator, timeStamp, sourceSubPath, dispersalType); - final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + final FileSinkDataCSVConverter converter = new FileSinkDataCSVConverter(conf, new ErrorExtractor()); final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); final HdfsFileSink hdfsSink = spy(new HdfsFileSink(fileConf, converter)); hdfsSink.write(testData); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java index 174cf0d..3a0076d 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java @@ -17,6 +17,7 @@ package com.uber.marmaray.common.sinks.hoodie; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.marmaray.common.AvroPayload; @@ -53,6 +54,8 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -85,14 +88,14 @@ class MockHoodieSink extends HoodieSink { public MockHoodieSink(@NonNull final HoodieConfiguration hoodieConf, @NonNull final HoodieSinkDataConverter hoodieKeyGenerator, @NonNull final JavaSparkContext jsc, @NonNull final HoodieSinkOp op) { - super(hoodieConf, hoodieKeyGenerator, jsc, op, new MemoryMetadataManager()); + super(hoodieConf, hoodieKeyGenerator, jsc, op, new MemoryMetadataManager(), Optional.absent()); } public MockHoodieSink(@NonNull final HoodieConfiguration hoodieConf, @NonNull final HoodieSinkDataConverter hoodieKeyGenerator, @NonNull final JavaSparkContext jsc, @NonNull final HoodieSinkOp op, @NonNull final IMetadataManager metadataMgr) { - super(hoodieConf, hoodieKeyGenerator, jsc, op, metadataMgr); + super(hoodieConf, hoodieKeyGenerator, jsc, op, metadataMgr, Optional.absent()); } @Override @@ -122,7 +125,8 @@ public void testUpdateInsertParallelism() { .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); final HoodieSink mockSink = spy(new HoodieSink(hoodieConf, mock(HoodieSinkDataConverter.class), - mock(JavaSparkContext.class), HoodieSink.HoodieSinkOp.NO_OP, new NoOpMetadataManager())); + mock(JavaSparkContext.class), HoodieSink.HoodieSinkOp.NO_OP, new NoOpMetadataManager(), + Optional.absent())); when(mockSink.calculateNewBulkInsertParallelism(anyLong())).thenReturn(18); Assert.assertTrue(mockSink.updateInsertParallelism(1000)); Assert.assertEquals(18, hoodieConf.getInsertParallelism()); @@ -139,7 +143,8 @@ public void testUpdateBulkInsertParallelism() { .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); final HoodieSink mockSink = spy(new HoodieSink(hoodieConf, mock(HoodieSinkDataConverter.class), - mock(JavaSparkContext.class), HoodieSink.HoodieSinkOp.NO_OP, new NoOpMetadataManager())); + mock(JavaSparkContext.class), HoodieSink.HoodieSinkOp.NO_OP, new NoOpMetadataManager(), + Optional.absent())); when(mockSink.calculateNewBulkInsertParallelism(anyLong())).thenReturn(18); Assert.assertTrue(mockSink.updateBulkInsertParallelism(1000)); Assert.assertEquals(18, hoodieConf.getBulkInsertParallelism()); @@ -159,13 +164,16 @@ public void testHoodieSinkWriteInsertWithoutMetadata() throws IOException { final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), INSERT); final JavaRDD inputRDD = this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); - + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); // Trying to write as an INSERT without HoodieBasedMetadataManager. hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper = hoodieSink.getMockWriteClient(); // It should generate exactly one commit file. - Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); /* Expected function calls. 1) startCommit (once). @@ -201,12 +209,15 @@ public void testHoodieSinkWriteUpsertWithoutMetadata() throws IOException { final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), UPSERT); final JavaRDD inputRDD = this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); - + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); // Trying to write as an UPSERT without HoodieBasedMetadataManager. hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper = hoodieSink.getMockWriteClient(); // It should generate exactly one commit file. - Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); /* Expected function calls. 1) startCommit (once). @@ -244,13 +255,16 @@ public void testHoodieSinkWriteInsertWithMetadata() throws IOException { hoodieBasedMetadataManager); final JavaRDD inputRDD = this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); - + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); // Trying to write as an INSERT with HoodieBasedMetadataManager. hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper = hoodieSink.getMockWriteClient(); // It should generate exactly one commit file. - Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); /* Expected function calls. 1) startCommit (once). @@ -287,14 +301,18 @@ public void testHoodieSinkWriteUpsertWithMetadata() throws IOException { hoodieBasedMetadataManager.set("randomKey", new StringValue("randomValue")); final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), UPSERT, hoodieBasedMetadataManager); + final JavaRDD inputRDD = this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); - + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); // Trying to write as an UPSERT with HoodieBasedMetadataManager. hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper = hoodieSink.getMockWriteClient(); // It should generate exactly one commit file. - Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); /* Expected function calls. 1) startCommit (once). @@ -331,12 +349,16 @@ public void testHoodieSinkWriteDedupeInsert() throws IOException { this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), DEDUP_INSERT); + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); // Trying to write as a DEDUP_INSERT without HoodieBasedMetadataManager. hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper = hoodieSink.getMockWriteClient(); // It should generate exactly one commit file. - Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(1, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); /* Expected function calls. 1) startCommit (once). @@ -364,7 +386,8 @@ public void testHoodieSinkWriteDedupeInsert() throws IOException { hoodieSink.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper2 = hoodieSink.getMockWriteClient(); - Assert.assertEquals(2, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + Assert.assertEquals(2, getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))).size()); final ArgumentCaptor rddCaputure = ArgumentCaptor.forClass(JavaRDD.class); verify(hoodieWriteClientWrapper2).insert(rddCaputure.capture(), Matchers.any()); // All records should get filtered out. @@ -395,11 +418,20 @@ public void testHoodieSinkMetrics() throws IOException { hoodieSink.write(inputRDD); final Set ms = dfm.getMetricSet(); + final Map metricMap = new HashMap<>(); + ms.forEach( metric -> { + final String key = metric.getMetricName(); + metricMap.put(key, metric.getMetricValue()); + }); + final Map expected = ArrayUtils.toMap( new Object[][] { {"output_rowcount", successRecordCount.longValue()}, - {"error_rowcount", failedRecordCount.longValue()} + {"error_rowcount", failedRecordCount.longValue()}, + {"total_file_count", (long)metricMap.get("total_file_count")}, + {"total_write_size", (long)metricMap.get("total_write_size")} }); + Assert.assertEquals(expected.size(), ms.size()); ms.forEach( metric -> { final String key = metric.getMetricName(); @@ -421,12 +453,15 @@ public void testUserDefinedCommitTime() throws IOException { this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); final MockHoodieSink hoodieSink1 = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), BULK_INSERT); + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink1.setDataFeedMetrics(dfm); hoodieSink1.write(inputRDD); - final HoodieWriteClientWrapper hoodieWriteClientWrapper1 = hoodieSink1.getMockWriteClient(); Mockito.verify(hoodieWriteClientWrapper1, Mockito.times(1)).startCommit(); - final List commitFilesAfterFirstCommit = getCommitFiles(basePath, FSUtils.getFs(new Configuration())); + final List commitFilesAfterFirstCommit = getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))); Assert.assertEquals(1, commitFilesAfterFirstCommit.size()); final String customCommit = @@ -434,13 +469,15 @@ public void testUserDefinedCommitTime() throws IOException { new Date(new Date().getTime() - TimeUnit.DAYS.toMillis(365))); final MockHoodieSink hoodieSink2 = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), BULK_INSERT); + hoodieSink2.setDataFeedMetrics(dfm); hoodieSink2.setCommitTime(com.google.common.base.Optional.of(customCommit)); hoodieSink2.write(inputRDD); final HoodieWriteClientWrapper hoodieWriteClientWrapper2 = hoodieSink2.getMockWriteClient(); Mockito.verify(hoodieWriteClientWrapper2, Mockito.times(0)).startCommit(); - final List commitFilesAfterSecondCommit = getCommitFiles(basePath, FSUtils.getFs(new Configuration())); + final List commitFilesAfterSecondCommit = getCommitFiles(basePath, FSUtils.getFs(new Configuration(), + Optional.of(basePath))); Assert.assertEquals(2, commitFilesAfterSecondCommit.size()); final String oldCommitTime = commitFilesAfterFirstCommit.get(0); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sources/file/TestFileWorkUnitCalculator.java b/marmaray/src/test/java/com/uber/marmaray/common/sources/file/TestFileWorkUnitCalculator.java index 313cad0..55a0d53 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sources/file/TestFileWorkUnitCalculator.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/file/TestFileWorkUnitCalculator.java @@ -31,7 +31,6 @@ import java.io.FileOutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.util.HashSet; import java.util.Arrays; import java.util.HashSet; import java.util.Set; @@ -44,7 +43,6 @@ public class TestFileWorkUnitCalculator { public void computeWorkUnitsNoJson() throws Exception{ final Configuration conf = new Configuration(); conf.setProperty(FileSourceConfiguration.TYPE, "json"); - conf.setProperty(FileSourceConfiguration.SCHEMA, "{}"); final Path testDir = Files.createTempDirectory(null); try { conf.setProperty(FileSourceConfiguration.DIRECTORY, testDir.toString()); @@ -61,7 +59,6 @@ public void computeWorkUnitsNoJson() throws Exception{ public void computeWorkUnitsNoSuchDirectory() { final Configuration conf = new Configuration(); conf.setProperty(FileSourceConfiguration.TYPE, "json"); - conf.setProperty(FileSourceConfiguration.SCHEMA, "{}"); conf.setProperty(FileSourceConfiguration.DIRECTORY, "path/not/exist"); final FileWorkUnitCalculator workUnitCalculator = new FileWorkUnitCalculator(new FileSourceConfiguration(conf)); final FileWorkUnitCalculator.FileWorkUnitCalculatorResult result = workUnitCalculator.computeWorkUnits(); @@ -79,16 +76,15 @@ public void computeWorkUnitsSuccess() throws Exception { createFile(testDir, csvFile); final Configuration conf = new Configuration(); conf.setProperty(FileSourceConfiguration.TYPE, "json"); - conf.setProperty(FileSourceConfiguration.SCHEMA, "{}"); conf.setProperty(FileSourceConfiguration.DIRECTORY, testDir.toString()); final FileWorkUnitCalculator workUnitCalculator = new FileWorkUnitCalculator(new FileSourceConfiguration(conf)); final FileWorkUnitCalculator.FileWorkUnitCalculatorResult result = workUnitCalculator.computeWorkUnits(); - Assert.assertEquals(2, result.getWorkUnits().size()); final Set expectedResults = new HashSet<>(Arrays.asList(jsonFile1, jsonFile2)); final Set actualResults = result.getWorkUnits().stream() .map(FileStatus::getPath) .map(org.apache.hadoop.fs.Path::getName) .collect(Collectors.toSet()); + Assert.assertEquals(2, result.getWorkUnits().size()); Assert.assertEquals(expectedResults, actualResults); } finally { FileUtils.deleteDirectory(testDir.toFile()); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java index fbf5101..ca4d249 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java @@ -22,7 +22,6 @@ import com.uber.marmaray.common.converters.data.SparkSourceDataConverter; import com.uber.marmaray.common.converters.schema.DataFrameSchemaConverter; import com.uber.marmaray.common.metadata.HDFSMetadataManager; -import com.uber.marmaray.common.metadata.HDFSPartitionManager; import com.uber.marmaray.common.sources.IWorkUnitCalculator; import com.uber.marmaray.common.util.AbstractSparkTest; import com.uber.marmaray.common.util.FileHelperUtil; @@ -69,7 +68,7 @@ public void testReadDataFromParquetFile() throws IOException { final String dataPath = FileHelperUtil.getResourcePath(getClass(), METADATA_KEY); final HiveSourceConfiguration hiveConf = - HiveTestUtil.initializeConfig(JOB_NAME, dataPath, "testMetadataPath"); + HiveTestUtil.initializeConfig(JOB_NAME, dataPath); final SparkSourceDataConverter converter = new SparkSourceDataConverter(dfSchema, avroSchema, hiveConf.getConf(), Sets.newHashSet(LEFT_FIELD, RIGHT_FIELD), new ErrorExtractor()); final HiveSource source = new HiveSource(hiveConf, this.sqlContext.get(), converter); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java index e919998..df8426c 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java @@ -26,7 +26,6 @@ public class TestHiveSourceConfiguration { private static final String JOB_NAME = "jobFoo"; private static final String DEFAULT_DATA_PATH = "dataPath"; - private static final String DEFAULT_METADATA_PATH = "metadataPath"; @Test(expected = MissingPropertyException.class) @@ -51,7 +50,6 @@ public void testBasicConfig() { final HiveSourceConfiguration hiveConfig = new HiveSourceConfiguration(config); Assert.assertEquals(JOB_NAME, hiveConfig.getJobName()); Assert.assertEquals(DEFAULT_DATA_PATH, hiveConfig.getDataPath()); - Assert.assertEquals(DEFAULT_METADATA_PATH, hiveConfig.getBaseMetadataPath()); Assert.assertTrue(hiveConfig.shouldSaveCheckpoint()); } @@ -59,7 +57,6 @@ private static Configuration getValidHiveSourceConfiguration() { final Configuration config = new Configuration(); config.setProperty(HiveSourceConfiguration.JOB_NAME, JOB_NAME); config.setProperty(HiveSourceConfiguration.HIVE_DATA_PATH, DEFAULT_DATA_PATH); - config.setProperty(HiveSourceConfiguration.BASE_METADATA_PATH, DEFAULT_METADATA_PATH); return config; } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java index bd0386e..95ffeba 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java @@ -62,12 +62,11 @@ public class TestParquetWorkUnitCalculator { public void setupTest() throws IOException { this.config = new Configuration(); - this.fileSystem = FSUtils.getFs(this.config); + this.fileSystem = FSUtils.getFs(this.config, Optional.absent()); this.dataPath = FileTestUtil.getTempFolder(); this.metadataPath = FileTestUtil.getTempFolder(); this.config.setProperty(HiveSourceConfiguration.JOB_NAME, JOB_NAME); - this.config.setProperty(HiveSourceConfiguration.BASE_METADATA_PATH, this.metadataPath); this.config.setProperty(HiveSourceConfiguration.HIVE_DATA_PATH, this.dataPath); this.config.setProperty(HiveSourceConfiguration.PARTITION_TYPE, PartitionType.NORMAL.toString()); @@ -88,7 +87,6 @@ public void testComputeWorkUnitsWithNoPrexistentCheckpointsMultiplePartitions() this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_3)); this.partitionManager = new HDFSPartitionManager(JOB_NAME, - this.metadataPath, this.dataPath, this.fileSystem); @@ -102,7 +100,6 @@ public void testComputeWorkUnitsWithNoPrexistentCheckpointsMultiplePartitions() virtuallyProcessPartition(this.partitionManager, this.metadataManager, Optional.absent(), PARTITION_2); final HDFSPartitionManager partitionManager2 = new HDFSPartitionManager(JOB_NAME, - this.metadataPath, this.dataPath, this.fileSystem); @@ -122,7 +119,6 @@ public void testComputeWorkUnitsWithNoPrexistentCheckpointsSinglePartition() thr this.fileSystem.create(new Path(this.dataPath, dataFileName)); this.partitionManager = new HDFSPartitionManager(JOB_NAME, - this.metadataPath, this.dataPath, this.fileSystem); @@ -137,7 +133,6 @@ public void testComputeWorkUnitsWithNoPrexistentCheckpointsSinglePartition() thr // A checkpoint now exists. Now virtually reprocess that single partition explicitly via data path // by initializing and saving run states final HDFSPartitionManager pm2 = new HDFSPartitionManager(JOB_NAME, - this.metadataPath, this.dataPath, this.fileSystem); @@ -170,7 +165,6 @@ public void testComputeWorkUnitsWithExistentCheckpoint() throws IOException { this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_3)); this.partitionManager = new HDFSPartitionManager(JOB_NAME, - this.metadataPath, this.dataPath, this.fileSystem); diff --git a/marmaray/src/test/java/com/uber/marmaray/common/spark/TestMarmarayKryoSerializer.java b/marmaray/src/test/java/com/uber/marmaray/common/spark/TestMarmarayKryoSerializer.java new file mode 100644 index 0000000..b42e053 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/spark/TestMarmarayKryoSerializer.java @@ -0,0 +1,22 @@ +package com.uber.marmaray.common.spark; + +import com.uber.marmaray.common.util.AbstractSparkTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.List; +import java.util.stream.IntStream; + +public class TestMarmarayKryoSerializer extends AbstractSparkTest { + + @Test + public void testExceptionSerialization() { + final List exceptions = new LinkedList<>(); + final int numberOfExceptions = 10; + IntStream.range(0, numberOfExceptions) + .forEach(i -> exceptions.add(new Exception("test-" + i))); + final List exceptionList = this.jsc.get().parallelize(exceptions).map(o -> o).collect(); + Assert.assertEquals(numberOfExceptions, exceptionList.size()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java b/marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java index b8a9514..3f7458d 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java @@ -18,56 +18,42 @@ package com.uber.marmaray.common.spark; import com.google.common.base.Optional; +import com.uber.marmaray.TestSparkUtil; +import com.uber.marmaray.common.configuration.SparkConfiguration; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import lombok.NonNull; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.apache.hadoop.conf.Configuration; import org.apache.spark.SparkContext; import org.apache.spark.sql.SparkSession; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; public class TestSparkFactory { - private Optional sparkFactory = Optional.absent(); - - @Before - public void setup() { - // Since we are caching sparkContext in it. - this.sparkFactory = Optional.of(new SparkFactory()); - } - @Test public void testSparkSessionAndSparkContext() { final SparkArgs sparkArgs = getSampleMarmaraySparkArgs(); - final SparkSession sparkSession = sparkFactory.get() - .getSparkSession(sparkArgs, false); + final SparkFactory sparkFactory = new SparkFactory(sparkArgs); + final SparkSession sparkSession = sparkFactory.getSparkSession(); assertExpectationsOnSparkContext(sparkArgs, sparkSession.sparkContext()); // should re-use existing SparkContext and not fail - final SparkContext sc2 = sparkFactory.get().getSparkContext(sparkArgs).sc(); + final SparkContext sc2 = sparkFactory.getSparkContext().sc(); assertExpectationsOnSparkContext(sparkArgs, sc2); - } - @After - public void tearDown() { - final SparkArgs sparkArgs = getSampleMarmaraySparkArgs(); - // gets existing sc - this.sparkFactory.get().getSparkContext(sparkArgs).sc().stop(); - this.sparkFactory = Optional.absent(); + // tear down + sparkFactory.stop(); } private void assertExpectationsOnSparkContext( @NonNull final SparkArgs sparkArgs, @NonNull final SparkContext sc) { final String registeredAvroSchemaStr = sc.conf().getAvroSchema().head()._2(); - final Schema expectedAvroSchema = sparkArgs.getAvroSchemas().get().get(0); + final Schema expectedAvroSchema = sparkArgs.getAvroSchemas().get(0); Assert.assertEquals(expectedAvroSchema.toString(), registeredAvroSchemaStr); Assert.assertEquals("foo_bar", sc.appName()); Assert.assertEquals("512", sc.hadoopConfiguration().get("mapreduce.map.memory.mb")); @@ -76,16 +62,17 @@ private void assertExpectationsOnSparkContext( private SparkArgs getSampleMarmaraySparkArgs() { final Schema recordSchema = SchemaBuilder.record("fooRecord").fields().name("abc").type() .intType().intDefault(0).endRecord(); - final Optional> schemas = Optional.of(Arrays.asList(recordSchema)); final Map overrideSparkProperties = new HashMap<>(); overrideSparkProperties.put("spark.master", "local[2]"); overrideSparkProperties.put("spark.app.name", "foo_bar"); - final Configuration hadoopConfiguration = new Configuration(); - hadoopConfiguration.set("mapreduce.map.memory.mb", "512"); + final com.uber.marmaray.common.configuration.Configuration conf = + new com.uber.marmaray.common.configuration.Configuration( + TestSparkUtil.class.getResourceAsStream("/config.yaml"), + Optional.absent()); - return new SparkArgs(schemas, Arrays.asList(), - overrideSparkProperties, hadoopConfiguration); + SparkConfiguration.overrideSparkConfInConfiguration(conf, overrideSparkProperties); + return new SparkArgs(Arrays.asList(recordSchema), Collections.emptyList(), conf); } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/status/TestBaseStatus.java b/marmaray/src/test/java/com/uber/marmaray/common/status/TestBaseStatus.java new file mode 100644 index 0000000..afbee77 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/status/TestBaseStatus.java @@ -0,0 +1,81 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.status; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.InvalidPropertiesFormatException; + +public class TestBaseStatus { + + @Test + public void testGetStatus() { + + // Nothing is a success + BaseStatus status = new BaseStatus(); + Assert.assertEquals(IStatus.Status.SUCCESS, status.getStatus()); + + // Test sticky failure + status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + status.setStatus(IStatus.Status.FAILURE); + status.setStatus(IStatus.Status.SUCCESS); + Assert.assertEquals(IStatus.Status.FAILURE, status.getStatus()); + + // Test sticky in_progress + status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + status.setStatus(IStatus.Status.IN_PROGRESS); + status.setStatus(IStatus.Status.SUCCESS); + status.setStatus(IStatus.Status.SUCCESS); + Assert.assertEquals(IStatus.Status.IN_PROGRESS, status.getStatus()); + + // Test one of each, failure has priority + status = new BaseStatus(); + status.setStatus(IStatus.Status.SUCCESS); + status.setStatus(IStatus.Status.FAILURE); + status.setStatus(IStatus.Status.IN_PROGRESS); + status.setStatus(IStatus.Status.SUCCESS); + Assert.assertEquals(IStatus.Status.FAILURE, status.getStatus()); + } + + @Test + public void testGetThrowables() { + // no throwables + BaseStatus status = new BaseStatus(); + Assert.assertEquals(Collections.emptyList(), status.getExceptions()); + + // one throwable + final Exception e1 = new IllegalStateException("This is a test"); + status = new BaseStatus(); + status.addException(e1); + Assert.assertEquals(Collections.singletonList(e1), status.getExceptions()); + + // multiple throwables stay in order + final Exception e2 = new NullPointerException("This is a second test"); + final Exception e3 = new InvalidPropertiesFormatException("This is a third test"); + status = new BaseStatus(); + status.addException(e1); + status.addExceptions(Arrays.asList(e2, e3)); + Assert.assertEquals(Arrays.asList(e1, e2, e3), status.getExceptions()); + + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/status/TestJobManagerStatus.java b/marmaray/src/test/java/com/uber/marmaray/common/status/TestJobManagerStatus.java new file mode 100644 index 0000000..45b70c5 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/status/TestJobManagerStatus.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ + +package com.uber.marmaray.common.status; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class TestJobManagerStatus { + + @Test + public void testStatuses() { + // no status is successful + JobManagerStatus status = new JobManagerStatus(); + Assert.assertEquals(IStatus.Status.SUCCESS, status.getStatus()); + Assert.assertEquals(Collections.emptyMap(), status.getJobStatuses()); + + status.addJobStatus("MyJob", new BaseStatus()); + Assert.assertEquals(IStatus.Status.SUCCESS, status.getStatus()); + Assert.assertEquals(1, status.getJobStatuses().size()); + + final BaseStatus failedStatus = new BaseStatus(); + final Exception e1 = new NullPointerException("Foo was here!"); + failedStatus.setStatus(IStatus.Status.FAILURE); + failedStatus.addException(e1); + status.addJobStatus("MySecondJob", failedStatus); + Assert.assertEquals(Collections.singletonList(e1), status.getExceptions()); + Assert.assertEquals(IStatus.Status.FAILURE, status.getStatus()); + Assert.assertEquals(2, status.getJobStatuses().size()); + + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java b/marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java index 6baad4b..cf2ee3b 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java @@ -46,7 +46,7 @@ public void setupTest() { this.jsc = Optional.of(new JavaSparkContext(this.spark.get().sparkContext())); this.sqlContext = Optional.of(SQLContext.getOrCreate(this.jsc.get().sc())); try { - this.fileSystem = Optional.of(FSUtils.getFs(new Configuration())); + this.fileSystem = Optional.of(FSUtils.getFs(new Configuration(), Optional.absent())); } catch (IOException e) { log.error("Cannot initialize FileSystem object", e); } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.java b/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.java index b3c2490..5ddc785 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.java @@ -16,6 +16,9 @@ */ package com.uber.marmaray.common.util; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.configuration.Configuration; + public class CassandraTestConstants { public static final String KEY_SPACE = "marmaray"; public static final String TABLE = "crossfit_gyms"; @@ -24,4 +27,11 @@ public class CassandraTestConstants { public static final String STRING_FIELD = "string_field"; public static final String BOOLEAN_FIELD = "boolean_field"; public static final int CASSANDRA_PORT = 9142; + public static final Configuration CONFIGURATION = new Configuration(); + static { + CONFIGURATION.setProperty(CassandraSinkConfiguration.KEYSPACE, KEY_SPACE); + CONFIGURATION.setProperty(CassandraSinkConfiguration.TABLE_NAME, TABLE); + CONFIGURATION.setProperty(CassandraSinkConfiguration.CLUSTER_NAME, "test-cluster"); + CONFIGURATION.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, "key_name1,key_name2"); + } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java index b352db6..e17026d 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java @@ -16,9 +16,14 @@ */ package com.uber.marmaray.common.util; +import lombok.NonNull; +import org.apache.commons.io.FileUtils; import org.apache.commons.lang.CharEncoding; +import org.hibernate.validator.constraints.NotEmpty; import java.io.File; +import java.io.IOException; +import java.io.InputStream; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; @@ -30,4 +35,19 @@ public static String getResourcePath(Class clazz, String path) { throw new RuntimeException(e); } } + + public static void copyResourceFileToLocalFile(@NonNull final Class clazz, + @NotEmpty final String resource, + @NonNull final File targetFile) { + try { + final InputStream stream = clazz.getClassLoader().getResourceAsStream(resource); + if (stream == null) { + throw new RuntimeException(String.format("Unable to find resource %s", resource)); + } + FileUtils.copyInputStreamToFile(stream, targetFile); + } catch (IOException e) { + throw new RuntimeException(e); + } + + } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java index 445c258..766615a 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java @@ -25,13 +25,11 @@ private HiveTestUtil() { } public static HiveSourceConfiguration initializeConfig(final String jobName, - final String dataPath, - final String metadataPath) + final String dataPath) { final Configuration config = new Configuration(); config.setProperty(HiveSourceConfiguration.JOB_NAME, jobName); config.setProperty(HiveSourceConfiguration.HIVE_DATA_PATH, dataPath); - config.setProperty(HiveSourceConfiguration.BASE_METADATA_PATH, metadataPath); return new HiveSourceConfiguration(config); } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java b/marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java index 519d1cb..2e7ce41 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java @@ -41,8 +41,10 @@ import java.io.FileNotFoundException; import java.io.Serializable; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Date; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -53,6 +55,7 @@ import static com.uber.marmaray.common.configuration.KafkaConfiguration.KAFKA_BROKER_LIST; import static com.uber.marmaray.common.configuration.KafkaSourceConfiguration.KAFKA_CLUSTER_NAME; import static com.uber.marmaray.common.configuration.KafkaSourceConfiguration.KAFKA_START_DATE; +import static com.uber.marmaray.common.configuration.KafkaSourceConfiguration.KAFKA_START_TIME; import static com.uber.marmaray.common.configuration.KafkaSourceConfiguration.KAFKA_TOPIC_NAME; @Slf4j @@ -60,7 +63,8 @@ public class KafkaTestHelper { public static final String SCHEMA_SERVICE_TEST_LOCAL_PATH = "src/test/resources/schema-service"; public static final String TEST_KAFKA_CLUSTER_NAME = "test-cluster"; - public static final String TEST_KAFKA_START_DATE = "1970-01-01"; + public static final String TEST_KAFKA_START_DATE = + new SimpleDateFormat(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).format(new Date()); private static final String STRING_VALUE_DEFAULT = "value"; private static final Boolean BOOLEAN_VALUE_DEFAULT = true; @@ -97,31 +101,36 @@ public static void publishMessages(@NonNull final KafkaTestUtils kafkaTestUtils, } ); } + producer.send(ScalaUtil.toScalaSet(messageSet).toSeq()); producer.close(); } - public static List> generateMessages(@NonNull final List messageCountList) - throws FileNotFoundException { - final Schema schema = getSchema("test_schema"); + public static List> generateMessages(@NonNull final List messageCountList, + @NonNull final Schema schema, + @NotEmpty final Integer schemaVersion) { final TestKafkaSchemaService schemaService = new TestKafkaSchemaService(); final ISchemaService.ISchemaServiceWriter writer = - schemaService.getWriter("test_schema", 1); - + schemaService.getWriter(schema.getName(), schemaVersion); + return writeMessages(messageCountList, schema, writer); + } + public static List> writeMessages(@NonNull final List messageCountList, + @NonNull final Schema schema, + @NonNull final ISchemaService.ISchemaServiceWriter writer) { final List> ret = new ArrayList<>(messageCountList.size()); messageCountList.stream().forEach( - messageCount -> { - ret.add(KafkaTestHelper.getTestData(schema, messageCount).stream().map( - record -> { - try { - return writer.write(record); - } catch (InvalidDataException e) { - throw new RuntimeException(e); + messageCount -> { + ret.add(KafkaTestHelper.getTestData(schema, messageCount).stream().map( + record -> { + try { + return writer.write(record); + } catch (InvalidDataException e) { + throw new RuntimeException(e); + } } - } ).collect(Collectors.toList())); - } + } ); return ret; } @@ -173,7 +182,6 @@ private static Object getDefaultValue(@NonNull final Schema.Type type) { log.warn("Found unknown type {}, returning null", type.toString()); return null; } - } public static void publishMessagesToKafkaTopics(@NonNull final KafkaTestUtils kafkaTestUtils, @@ -182,10 +190,25 @@ public static void publishMessagesToKafkaTopics(@NonNull final KafkaTestUtils ka Preconditions.checkArgument(!Strings.isNullOrEmpty(topicName)); // Generate and publish messages to kafka. final List partitionNumMessages = Arrays.asList(3, 5, 6, 7, 10, 25, 35, 45); - final List> messages = KafkaTestHelper.generateMessages(partitionNumMessages); + final List> messages = KafkaTestHelper.generateMessages(partitionNumMessages, + getSchema(topicName), 1); KafkaTestHelper.publishMessages(kafkaTestUtils, topicName, messages); } + public static KafkaSourceConfiguration getKafkaSourceConfiguration(@NotEmpty final String topicName, + @NotEmpty final String brokerAddress, + final String startDate, + final String startTime) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(topicName)); + Preconditions.checkArgument(!Strings.isNullOrEmpty(brokerAddress)); + final Configuration conf = new Configuration(); + KafkaTestHelper.setMandatoryConf(conf, + Arrays.asList( + KAFKA_BROKER_LIST, KAFKA_TOPIC_NAME, KAFKA_CLUSTER_NAME, KAFKA_START_DATE, KAFKA_START_TIME), + Arrays.asList(brokerAddress, topicName, TEST_KAFKA_CLUSTER_NAME, startDate, startTime)); + return new KafkaSourceConfiguration(conf); + } + public static KafkaSourceConfiguration getKafkaSourceConfiguration(@NotEmpty final String topicName, @NotEmpty final String brokerAddress, @NotEmpty final String startDate) { Preconditions.checkArgument(!Strings.isNullOrEmpty(topicName)); @@ -196,7 +219,7 @@ public static KafkaSourceConfiguration getKafkaSourceConfiguration(@NotEmpty fin return new KafkaSourceConfiguration(conf); } - public static KafkaSourceConfiguration getKafkaSourceConfiguration(final String topicName, + public static KafkaSourceConfiguration getKafkaSourceConfiguration(final String topicName, final String brokerAddress) { return getKafkaSourceConfiguration(topicName, brokerAddress, TEST_KAFKA_START_DATE); } @@ -235,7 +258,7 @@ private class TestKafkaSchemaServiceWriter implements ISchemaServiceWriter, Seri private class TestKafkaSchemaServiceReader implements ISchemaServiceReader, Serializable { @Override public GenericRecord read(final byte[] buffer) throws InvalidDataException { - return new GenericRecordBuilder(getSchema("test")).build(); + return new GenericRecordBuilder(getSchema("testTopic")).build(); } } } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java index 1c168fa..c4f6e83 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java @@ -16,6 +16,7 @@ */ package com.uber.marmaray.common.util; +import com.uber.marmaray.common.spark.MarmarayKryoSerializer; import lombok.NonNull; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; @@ -34,7 +35,8 @@ public static SparkConf getSparkConf(final String appName) { .set("spark.default.parallelism", "4") // set the SPARK_LOCAL_IP address only for unit tests .set("spark.driver.bindAddress", "127.0.0.1") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.driver.host", "127.0.0.1") + .set("spark.serializer", MarmarayKryoSerializer.class.getName()) .set("spark.kryoserializer.buffer.max", "512m"); } diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java index f24652b..8d112c7 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java @@ -18,6 +18,7 @@ import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.utilities.FSUtils; +import com.google.common.base.Optional; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -34,7 +35,7 @@ public class TestFsUtils { @Before public void initClass() throws IOException { final Configuration conf = new Configuration(); - this.fileSystem = FSUtils.getFs(conf); + this.fileSystem = FSUtils.getFs(conf, Optional.absent()); } @Test diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java index ef5f2fb..e7b243e 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java @@ -17,6 +17,8 @@ package com.uber.marmaray.common.util; import com.uber.marmaray.utilities.FSUtils; +import com.google.common.base.Optional; +import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; @@ -42,7 +44,7 @@ public void setupTest() throws IOException { * We explicitly don't call close() in a tearDownTest() method as the Hadoop FileSystem object is cached * so if multiple threads are accessing can affect others if one thread closes it. */ - this.fileSystem = FSUtils.getFs(new com.uber.marmaray.common.configuration.Configuration()); + this.fileSystem = FSUtils.getFs(new com.uber.marmaray.common.configuration.Configuration(), Optional.absent()); } @Test diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java index c25eb9c..55bb48f 100644 --- a/marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java @@ -19,6 +19,7 @@ import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.utilities.FSUtils; import com.uber.marmaray.utilities.SchemaUtil; +import com.google.common.base.Optional; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.sql.types.StructField; @@ -31,19 +32,29 @@ public class TestSchemaUtil { - private String parquetDataPath; private FileSystem fs; @Before public void setupTest() throws IOException { - this.parquetDataPath = FileHelperUtil.getResourcePath(getClass(), - new Path("testData", "testPartition").toString()); - this.fs = FSUtils.getFs(new Configuration()); + this.fs = FSUtils.getFs(new Configuration(), Optional.absent()); } @Test public void testGenerateSchemaFromParquet() throws IOException { - final StructType structType = SchemaUtil.generateSchemaFromParquet(fs, this.parquetDataPath); + final String parquetDataPath = FileHelperUtil.getResourcePath(getClass(), + new Path("testData", "testPartition").toString()); + final StructType structType = SchemaUtil.generateSchemaFromParquet(this.fs, + parquetDataPath, Optional.absent()); + Assert.assertEquals(2, structType.fields().length); + validate(structType); + } + + @Test + public void testMultiPartitionSchema() throws Exception { + final String parquetDataPath = FileHelperUtil.getResourcePath(getClass(), + new Path("testData", "testPartition1").toString()); + final StructType structType = SchemaUtil.generateSchemaFromParquet(this.fs, + parquetDataPath, Optional.absent()); Assert.assertEquals(2, structType.fields().length); validate(structType); } diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/ResourcesUtils.java b/marmaray/src/test/java/com/uber/marmaray/utilities/ResourcesUtils.java new file mode 100644 index 0000000..bc4e230 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/ResourcesUtils.java @@ -0,0 +1,14 @@ +package com.uber.marmaray.utilities; +import org.hibernate.validator.constraints.NotEmpty; +import java.io.File; + +public class ResourcesUtils { + public static String getTextFromResource(@NotEmpty final String fileName) throws Exception { + return new String(getBytesFromResource(fileName)); + } + + public static byte[] getBytesFromResource(@NotEmpty final String fileName) throws Exception { + final File file = new File(ResourcesUtils.class.getClassLoader().getResource(fileName).toURI()); + return java.nio.file.Files.readAllBytes(file.toPath()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java b/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java index 2c6717f..95c9de6 100644 --- a/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java @@ -17,7 +17,15 @@ package com.uber.marmaray.utilities; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.uber.marmaray.common.configuration.Configuration; import com.uber.marmaray.common.exceptions.JobRuntimeException; +import jodd.io.StringInputStream; +import org.apache.commons.collections.map.SingletonMap; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -29,16 +37,24 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.IntStream; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Matchers.anyLong; import static org.powermock.api.mockito.PowerMockito.mock; import static org.powermock.api.mockito.PowerMockito.mockStatic; import static org.powermock.api.mockito.PowerMockito.when; @@ -54,8 +70,8 @@ public void testRequestTimeout() { final Set topicPartitions = new HashSet(); final List partitionInfos = new LinkedList<>(); final int numPartitions = 5; - final int fetchTimeoutSec = 10; - final int fetchRetryCnt = 3; + final int fetchTimeoutSec = 1; + final int fetchRetryCnt = 2; when(KafkaUtil.getFetchOffsetRetryCnt()).thenReturn(fetchRetryCnt); when(KafkaUtil.getFetchOffsetTimeoutSec()).thenReturn(fetchTimeoutSec); @@ -73,15 +89,15 @@ public void testRequestTimeout() { Matchers.anyString(), Matchers.any(Set.class))).thenCallRealMethod(); when(kafkaConsumer.position(Matchers.any(TopicPartition.class))).thenAnswer( (Answer) invocationOnMock -> { - TopicPartition tp = invocationOnMock.getArgumentAt(0, TopicPartition.class); + final TopicPartition tp = invocationOnMock.getArgumentAt(0, TopicPartition.class); if (!attempts.containsKey(tp.partition())) { attempts.put(tp.partition(), new AtomicInteger(1)); } else { attempts.get(tp.partition()).incrementAndGet(); } if (tp.partition() == numPartitions - 1) { - // just want to ensure that we timeout this request. - Thread.sleep(TimeUnit.SECONDS.toMillis(TimeUnit.SECONDS.toMillis(10 * fetchTimeoutSec))); + // ensure timeout occurs by adding 10 additional ms to max. + Thread.sleep(TimeUnit.SECONDS.toMillis(fetchTimeoutSec) + 10); } return tp.partition() * 2L; } @@ -99,4 +115,6 @@ public void testRequestTimeout() { } ); } + + } diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/TestSizeUnit.java b/marmaray/src/test/java/com/uber/marmaray/utilities/TestSizeUnit.java new file mode 100644 index 0000000..c82eb41 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/TestSizeUnit.java @@ -0,0 +1,54 @@ +package com.uber.marmaray.utilities; + +import org.junit.Assert; +import org.junit.Test; + +public class TestSizeUnit { + + @Test + public void testToBits() { + Assert.assertEquals(17, SizeUnit.BITS.toBits(17)); + Assert.assertEquals(8 * 125, SizeUnit.BYTES.toBits(125)); + Assert.assertEquals(8 * 1024 * 200, SizeUnit.KILOBYTES.toBits(200)); + Assert.assertEquals(3 * 8 * 1024 * 1024, SizeUnit.MEGABYTES.toBits(3)); + Assert.assertEquals(8L * 1024 * 1024 * 1024, SizeUnit.GIGABYTES.toBits(1)); + } + + @Test + public void testToBytes() { + Assert.assertEquals(125, SizeUnit.BITS.toBytes(8 * 125)); + Assert.assertEquals(17, SizeUnit.BYTES.toBytes(17)); + Assert.assertEquals(1024 * 200, SizeUnit.KILOBYTES.toBytes(200)); + Assert.assertEquals(3 * 1024 * 1024, SizeUnit.MEGABYTES.toBytes(3)); + Assert.assertEquals(1024 * 1024 * 1024, SizeUnit.GIGABYTES.toBytes(1)); + } + + @Test + public void testToKilobytes() { + Assert.assertEquals(125, SizeUnit.BITS.toKilobytes(8 * 125 * 1024)); + Assert.assertEquals(17, SizeUnit.BYTES.toKilobytes(17 * 1024)); + Assert.assertEquals(200, SizeUnit.KILOBYTES.toKilobytes(200)); + Assert.assertEquals(3 * 1024, SizeUnit.MEGABYTES.toKilobytes(3)); + Assert.assertEquals(1024 * 1024, SizeUnit.GIGABYTES.toKilobytes(1)); + } + + @Test + public void testToMegabytes() { + Assert.assertEquals(125, SizeUnit.BITS.toMegabytes(8 * 125 * 1024 * 1024)); + Assert.assertEquals(17, SizeUnit.BYTES.toMegabytes(17 * 1024 * 1024)); + Assert.assertEquals(200, SizeUnit.KILOBYTES.toMegabytes(200 * 1024)); + Assert.assertEquals(3, SizeUnit.MEGABYTES.toMegabytes(3)); + Assert.assertEquals(1024, SizeUnit.GIGABYTES.toMegabytes(1)); + } + + @Test + public void testToGigabytes() { + Assert.assertEquals(125, SizeUnit.BITS.toGigabytes(8L * 125 * 1024 * 1024 * 1024)); + Assert.assertEquals(17, SizeUnit.BYTES.toGigabytes(17L * 1024 * 1024 * 1024)); + Assert.assertEquals(200, SizeUnit.KILOBYTES.toGigabytes(200 * 1024 * 1024)); + Assert.assertEquals(3, SizeUnit.MEGABYTES.toGigabytes(3 * 1024)); + Assert.assertEquals(1, SizeUnit.GIGABYTES.toGigabytes(1)); + // test underflow rounds to 0 + Assert.assertEquals(0, SizeUnit.BITS.toGigabytes(300)); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/TestTimeUnitUtil.java b/marmaray/src/test/java/com/uber/marmaray/utilities/TestTimeUnitUtil.java new file mode 100644 index 0000000..7aca962 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/TestTimeUnitUtil.java @@ -0,0 +1,28 @@ +package com.uber.marmaray.utilities; + +import org.junit.Assert; +import org.junit.Test; + +public class TestTimeUnitUtil { + + @Test + public void testConvertToMicroSeconds() { + // millsec -> microsec; + Assert.assertEquals("1551742037000000", TimeUnitUtil.convertToMicroSeconds(1551742037L)); + + // sec -> microsec; + Assert.assertEquals("1551742000000000", TimeUnitUtil.convertToMicroSeconds(1551742L)); + + // microsec -> microsec; + Assert.assertEquals("1551742037895764", TimeUnitUtil.convertToMicroSeconds(1551742037895764L)); + + // nanosec -> microsec; + Assert.assertEquals("1551742037895764", TimeUnitUtil.convertToMicroSeconds(1551742037895764000L)); + + // microsec_higer_bound -> microsec; + Assert.assertEquals("1000000000000000", TimeUnitUtil.convertToMicroSeconds((long) Math.pow(10, 18))); + + // microsec_lower_bound -> microsec; + Assert.assertEquals("1000000000000000", TimeUnitUtil.convertToMicroSeconds((long) Math.pow(10, 15))); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/listener/TestTimeoutManager.java b/marmaray/src/test/java/com/uber/marmaray/utilities/listener/TestTimeoutManager.java new file mode 100644 index 0000000..32b369b --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/listener/TestTimeoutManager.java @@ -0,0 +1,79 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. + * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated + * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the + * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to + * permit persons to whom the Software is furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all copies or substantial portions + * of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO + * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, + * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS + * IN THE SOFTWARE. + */ +package com.uber.marmaray.utilities.listener; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.util.AbstractSparkTest; +import org.junit.Assert; +import org.junit.Test; + +public class TestTimeoutManager extends AbstractSparkTest { + + @Override + public void setupTest() { + super.setupTest(); + TimeoutManager.init(new Configuration(), this.jsc.get().sc()); + } + + @Override + public void teardownTest() { + super.teardownTest(); + TimeoutManager.close(); + } + + @Test + public void testTaskStageEventHandling() { + final TimeoutManager tm = TimeoutManager.getInstance(); + Assert.assertTrue(tm.getLastActiveTime().isEmpty()); + + // No update expected when stage finishes but is not getting tracked. + tm.stageFinished(1); + Assert.assertTrue(tm.getLastActiveTime().isEmpty()); + + // new stage should start getting tracked after stageStart and should be removed from tracking when stage + // finishes. + tm.stageStarted(1); + Assert.assertEquals(1, tm.getLastActiveTime().size()); + // Only stage is started and no tasks are started yet. + Assert.assertEquals(0, tm.getLastActiveTime().get(1).getRunningTasks().get()); + tm.stageFinished(1); + Assert.assertEquals(0, tm.getLastActiveTime().size()); + + // When task starts it should add stage for tracking. + tm.taskStarted(1); + Assert.assertEquals(1, tm.getLastActiveTime().size()); + Assert.assertEquals(1, tm.getLastActiveTime().get(1).getRunningTasks().get()); + // Let us start another task in same stage. It should only increase running tasks. + tm.taskStarted(1); + Assert.assertEquals(1, tm.getLastActiveTime().size()); + Assert.assertEquals(2, tm.getLastActiveTime().get(1).getRunningTasks().get()); + // finishing one task should not remove stage from tracking.. + tm.taskFinished(1); + Assert.assertEquals(1, tm.getLastActiveTime().size()); + Assert.assertEquals(1, tm.getLastActiveTime().get(1).getRunningTasks().get()); + // task finish event may get dropped.. therefore whenever we receive stage finish event we remove stage from + // tracking. + tm.stageFinished(1); + Assert.assertEquals(0, tm.getLastActiveTime().size()); + + // Let us start tasks from different stages.. it should add 2 different stage trackers. + tm.taskStarted(1); + tm.taskStarted(2); + Assert.assertEquals(2, tm.getLastActiveTime().size()); + + } +} diff --git a/marmaray/src/test/resources/config.yaml b/marmaray/src/test/resources/config.yaml index 8509ec8..83f11b9 100644 --- a/marmaray/src/test/resources/config.yaml +++ b/marmaray/src/test/resources/config.yaml @@ -14,3 +14,5 @@ simple: num_of_retries: 3 wait_time_in_ms: 1000 + hadoop: + mapreduce.map.memory.mb: 512 diff --git a/marmaray/src/test/resources/testData/testPartition1/testPartition2/data.parquet b/marmaray/src/test/resources/testData/testPartition1/testPartition2/data.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9d9d72f1068857c5041dba1b928e54f1af8cb5f4 GIT binary patch literal 454 zcmZWmU2DQH6ithRBEF537WxnZEgN)YwSph`pl=R*DBa`OL)s)-Q0vkvI1%;(_M>;T zt{WH<$hqg-r6moYVBH?i+OL($H;b6% z3bzpMxX5a)gsfatgFBTkmS<$@wqyIA;oy$#8py6?80zN4oOs6g-}m0EQPus8YzDH% zh54nKIe#_LyPE-Ngt0E|~rC2lTTRYO~uoZMfmBj3h z2&9mtyaXx(pkyvqyriSI<8S9V8W7iaYbibX4Sz{{D|kO4>*Z?lX|2d!s4_1$Bn$>Y wNFLH%zT%|EQr6>4^wad2_XJNf*&lWzCc-Eh%I-k&NcLqWAJHewCx!|B0KvCv(*OVf literal 0 HcmV?d00001 diff --git a/pom.xml b/pom.xml index 27a3002..11b37f8 100644 --- a/pom.xml +++ b/pom.xml @@ -17,6 +17,7 @@ 3.5 3.0.2 2.5.3 + 2.5.5 1.8 1.8 1.8 @@ -98,10 +99,10 @@ cloudera-repo-releases - https://repository.cloudera.com/artifactory/public/ + https://repository.cloudera.com/artifactory/repo/ - + @@ -187,6 +188,7 @@ maven-assembly-plugin + ${maven-assembly-plugin.version} package