From 6d0cc68eb74c770f2e38e7aa3ae06e7ab4decdc1 Mon Sep 17 00:00:00 2001 From: Danny Chen Date: Wed, 5 Sep 2018 16:10:09 -0700 Subject: [PATCH] Add .gitignore and code which was missed in first commit --- .gitignore | 90 +++ checkstyles/marmaray-0.0.1.xml | 136 ++++ checkstyles/suppressions.xml | 13 + marmaray-tools/pom.xml | 22 + .../src/main/cli/toggleHDFSMetadataFile.py | 62 ++ .../marmaray/tools/HDFSMetadataPrinter.java | 83 ++ .../marmaray/tools/HDFSMetadataPruner.java | 120 +++ marmaray/build.gradle | 18 + marmaray/config/sample.yaml | 45 ++ marmaray/pom.xml | 55 ++ .../com/uber/marmaray/common/AvroPayload.java | 87 +++ .../uber/marmaray/common/DispersalType.java | 29 + .../uber/marmaray/common/FileSinkType.java | 28 + .../marmaray/common/HoodieErrorPayload.java | 62 ++ .../com/uber/marmaray/common/IPayload.java | 27 + .../uber/marmaray/common/PartitionType.java | 29 + .../com/uber/marmaray/common/WorkUnit.java | 27 + .../common/actions/IJobDagAction.java | 54 ++ .../common/actions/JobDagActions.java | 141 ++++ .../common/actions/ReporterAction.java | 59 ++ .../configuration/AwsConfiguration.java | 80 ++ .../CassandraSinkConfiguration.java | 266 +++++++ .../configuration/ConfigScopeResolver.java | 289 +++++++ .../common/configuration/Configuration.java | 227 ++++++ .../ErrorTableConfiguration.java | 123 +++ .../configuration/FileSinkConfiguration.java | 240 ++++++ .../configuration/HadoopConfiguration.java | 51 ++ .../configuration/HiveConfiguration.java | 71 ++ .../HiveSourceConfiguration.java | 67 ++ .../configuration/HoodieConfiguration.java | 524 +++++++++++++ .../HoodieIndexConfiguration.java | 207 +++++ .../configuration/KafkaConfiguration.java | 77 ++ .../KafkaSourceConfiguration.java | 83 ++ .../LockManagerConfiguration.java | 96 +++ .../RetryStrategyConfiguration.java | 54 ++ .../SimpleRetryStrategyConfiguration.java | 49 ++ .../configuration/ZookeeperConfiguration.java | 55 ++ .../converterresult/ConverterResult.java | 77 ++ .../data/AbstractDataConverter.java | 160 ++++ .../data/CassandraSinkCQLDataConverter.java | 200 +++++ .../data/CassandraSinkDataConverter.java | 201 +++++ .../data/DummyHoodieSinkDataConverter.java | 44 ++ .../data/FileSinkDataConverter.java | 109 +++ .../data/HoodieSinkDataConverter.java | 86 +++ .../data/KafkaSourceDataConverter.java | 79 ++ .../converters/data/SinkDataConverter.java | 37 + .../converters/data/SourceDataConverter.java | 36 + .../data/SparkSourceDataConverter.java | 109 +++ .../data/TSBasedHoodieSinkDataConverter.java | 79 ++ .../schema/AbstractSchemaConverter.java | 30 + .../schema/CassandraSchemaConverter.java | 114 +++ .../schema/DataFrameSchemaConverter.java | 105 +++ .../marmaray/common/data/BinaryRawData.java | 34 + .../uber/marmaray/common/data/ErrorData.java | 37 + .../uber/marmaray/common/data/ForkData.java | 40 + .../com/uber/marmaray/common/data/IData.java | 22 + .../uber/marmaray/common/data/RDDWrapper.java | 54 ++ .../uber/marmaray/common/data/RawData.java | 30 + .../marmaray/common/data/RawDataHelper.java | 32 + .../uber/marmaray/common/data/ValidData.java | 26 + .../marmaray/common/dataset/ErrorRecord.java | 51 ++ .../common/dataset/ExceptionRecord.java | 48 ++ .../marmaray/common/dataset/MetricRecord.java | 45 ++ .../marmaray/common/dataset/UtilRecord.java | 41 + .../marmaray/common/dataset/UtilTable.java | 96 +++ .../exceptions/ForkOperationException.java | 31 + .../exceptions/InvalidDataException.java | 38 + .../exceptions/JobRuntimeException.java | 38 + .../common/exceptions/MetadataException.java | 30 + .../exceptions/MissingPropertyException.java | 31 + .../common/exceptions/RetryException.java | 30 + .../common/forkoperator/FilterFunction.java | 47 ++ .../common/forkoperator/ForkFunction.java | 94 +++ .../common/forkoperator/ForkOperator.java | 113 +++ .../uber/marmaray/common/job/DagPayload.java | 35 + .../ExecutionTimeJobExecutionStrategy.java | 88 +++ .../common/job/IJobExecutionStrategy.java | 33 + .../com/uber/marmaray/common/job/Job.java | 46 ++ .../com/uber/marmaray/common/job/JobDag.java | 189 +++++ .../uber/marmaray/common/job/JobManager.java | 368 +++++++++ .../uber/marmaray/common/job/JobSubDag.java | 236 ++++++ .../marmaray/common/job/SingleSinkSubDag.java | 58 ++ .../common/job/ThreadPoolService.java | 424 ++++++++++ .../common/job/ThreadPoolServiceTier.java | 22 + .../common/metadata/AbstractValue.java | 28 + .../metadata/HDFSDatePartitionManager.java | 137 ++++ .../common/metadata/HDFSMetadataManager.java | 278 +++++++ .../common/metadata/HDFSPartitionManager.java | 154 ++++ .../metadata/HoodieBasedMetadataManager.java | 195 +++++ .../common/metadata/IMetadataManager.java | 41 + .../metadata/JobManagerMetadataTracker.java | 125 +++ .../common/metadata/MetadataConstants.java | 36 + .../common/metadata/NoOpMetadataManager.java | 52 ++ .../marmaray/common/metadata/StringValue.java | 36 + .../CassandraPayloadRDDSizeEstimator.java | 42 + .../common/metrics/ChargebackMetricType.java | 26 + .../common/metrics/DataFeedMetricNames.java | 65 ++ .../common/metrics/DataFeedMetrics.java | 86 +++ .../marmaray/common/metrics/DoubleMetric.java | 44 ++ .../common/metrics/IChargebackCalculator.java | 42 + .../marmaray/common/metrics/IMetricable.java | 32 + .../common/metrics/JobMetricNames.java | 33 + .../common/metrics/JobMetricType.java | 35 + .../marmaray/common/metrics/JobMetrics.java | 75 ++ .../marmaray/common/metrics/LongMetric.java | 44 ++ .../uber/marmaray/common/metrics/Metric.java | 76 ++ .../marmaray/common/metrics/TimerMetric.java | 82 ++ .../common/reporters/ConsoleReporter.java | 39 + .../marmaray/common/reporters/IReporter.java | 29 + .../marmaray/common/reporters/Reportable.java | 26 + .../marmaray/common/reporters/Reporters.java | 59 ++ .../retry/IFunctionThrowsException.java | 28 + .../marmaray/common/retry/IRetryStrategy.java | 29 + .../common/retry/RetryableFunction.java | 79 ++ .../common/retry/SimpleRetryStrategy.java | 75 ++ .../common/schema/ISchemaService.java | 81 ++ .../common/schema/ISinkSchemaManager.java | 20 + .../schema/cassandra/CassandraDataField.java | 33 + .../schema/cassandra/CassandraPayload.java | 59 ++ .../schema/cassandra/CassandraSchema.java | 50 ++ .../cassandra/CassandraSchemaField.java | 79 ++ .../cassandra/CassandraSinkSchemaManager.java | 254 ++++++ .../common/schema/cassandra/ClusterKey.java | 65 ++ .../com/uber/marmaray/common/sinks/ISink.java | 36 + .../common/sinks/SinkStatManager.java | 133 ++++ .../sinks/cassandra/CassandraClientSink.java | 128 ++++ .../sinks/cassandra/CassandraSSTableSink.java | 166 ++++ .../common/sinks/cassandra/CassandraSink.java | 149 ++++ .../common/sinks/file/AwsFileSink.java | 224 ++++++ .../marmaray/common/sinks/file/FileSink.java | 203 +++++ .../common/sinks/file/HdfsFileSink.java | 101 +++ .../common/sinks/hoodie/HoodieErrorSink.java | 87 +++ .../common/sinks/hoodie/HoodieSink.java | 528 +++++++++++++ .../sinks/hoodie/HoodieSinkOperations.java | 37 + .../sinks/hoodie/HoodieWriteStatus.java | 44 ++ .../marmaray/common/sources/IRunState.java | 24 + .../uber/marmaray/common/sources/ISource.java | 34 + .../common/sources/IWorkUnitCalculator.java | 87 +++ .../common/sources/hive/HiveRunState.java | 30 + .../common/sources/hive/HiveSource.java | 100 +++ .../hive/ParquetWorkUnitCalculator.java | 141 ++++ .../hive/ParquetWorkUnitCalculatorResult.java | 47 ++ .../kafka/KafkaBootstrapOffsetSelector.java | 46 ++ .../common/sources/kafka/KafkaRunState.java | 50 ++ .../common/sources/kafka/KafkaSource.java | 214 ++++++ .../kafka/KafkaWorkUnitCalculator.java | 433 +++++++++++ .../uber/marmaray/common/spark/SparkArgs.java | 58 ++ .../marmaray/common/spark/SparkFactory.java | 203 +++++ .../marmaray/utilities/ByteBufferUtil.java | 48 ++ .../marmaray/utilities/CommandLineUtil.java | 92 +++ .../uber/marmaray/utilities/ConfigUtil.java | 54 ++ .../marmaray/utilities/ConverterUtil.java | 55 ++ .../com/uber/marmaray/utilities/DateUtil.java | 40 + .../marmaray/utilities/ErrorExtractor.java | 46 ++ .../marmaray/utilities/ErrorTableUtil.java | 173 +++++ .../com/uber/marmaray/utilities/FSUtils.java | 161 ++++ .../marmaray/utilities/GenericRecordUtil.java | 278 +++++++ .../HoodieSinkConverterErrorExtractor.java | 59 ++ .../utilities/HoodieSinkErrorExtractor.java | 62 ++ .../uber/marmaray/utilities/HoodieUtil.java | 78 ++ .../com/uber/marmaray/utilities/JobUtil.java | 48 ++ .../KafkaSourceConverterErrorExtractor.java | 54 ++ .../uber/marmaray/utilities/KafkaUtil.java | 228 ++++++ .../uber/marmaray/utilities/LockManager.java | 214 ++++++ .../com/uber/marmaray/utilities/MapUtil.java | 96 +++ .../marmaray/utilities/NumberConstants.java | 30 + .../uber/marmaray/utilities/ScalaUtil.java | 58 ++ .../uber/marmaray/utilities/SchemaUtil.java | 103 +++ .../uber/marmaray/utilities/SparkUtil.java | 234 ++++++ .../uber/marmaray/utilities/StringTypes.java | 41 + .../uber/marmaray/utilities/StringUtil.java | 44 ++ .../marmaray/utilities/TimestampInfo.java | 45 ++ .../cluster/CassandraClusterInfo.java | 56 ++ .../listener/SparkEventListener.java | 68 ++ .../utilities/listener/SparkJobTracker.java | 224 ++++++ .../utilities/listener/TimeoutManager.java | 155 ++++ .../java/com/uber/marmaray/TestSparkUtil.java | 51 ++ .../common/actions/TestJobDagActions.java | 244 ++++++ .../configuration/TestAwsConfiguration.java | 69 ++ .../TestCassandraSinkConfiguration.java | 151 ++++ .../TestConfigScopeResolver.java | 92 +++ .../configuration/TestConfiguration.java | 97 +++ .../TestErrorTableConfiguration.java | 65 ++ .../TestFileSinkConfiguration.java | 124 +++ .../TestHadoopConfiguration.java | 44 ++ .../TestHoodieConfiguration.java | 90 +++ .../TestHoodieIndexConfiguration.java | 91 +++ .../configuration/TestKafkaConfiguration.java | 67 ++ .../converters/TestAbstractDataConverter.java | 94 +++ .../TestCassandraDataFrameConverter.java | 108 +++ .../TestCassandraSchemaConverter.java | 156 ++++ .../TestDataFrameDataConverter.java | 166 ++++ .../TestDataFrameSchemaConverter.java | 102 +++ .../data/TestFileSinkDataConverter.java | 109 +++ .../marmaray/common/data/TestRDDWrapper.java | 41 + .../common/dataset/TestUtilTable.java | 151 ++++ .../common/forkoperator/TestForkOperator.java | 209 +++++ ...TestExecutionTimeJobExecutionStrategy.java | 113 +++ .../marmaray/common/job/TestJobSubDag.java | 161 ++++ .../common/job/TestThreadPoolService.java | 262 +++++++ .../common/metadata/HDFSTestConstants.java | 25 + .../metadata/MemoryMetadataManager.java | 64 ++ .../TestHDFSDatePartitionManager.java | 237 ++++++ .../TestHDFSJobLevelMetadataTracker.java | 90 +++ .../metadata/TestHDFSMetadataManager.java | 133 ++++ .../metadata/TestHDFSPartitionManager.java | 245 ++++++ .../TestHoodieBasedMetadataManager.java | 149 ++++ .../common/metrics/TestDataFeedMetrics.java | 82 ++ .../common/metrics/TestJobMetrics.java | 76 ++ .../common/metrics/TestTimerMetric.java | 58 ++ .../common/retry/TestRetryableFunction.java | 100 +++ .../TestCassandraSinkSchemaManager.java | 366 +++++++++ .../schema/cassandra/TestClusterKey.java | 47 ++ .../common/sinks/TestSinkStatManager.java | 83 ++ .../cassandra/TestCassandraClientSink.java | 230 ++++++ .../cassandra/TestCassandraSSTableSink.java | 227 ++++++ .../cassandra/TestCassandraSinkUtil.java | 112 +++ .../common/sinks/file/FileSinkTestUtil.java | 94 +++ .../common/sinks/file/TestAwsFileSink.java | 267 +++++++ .../common/sinks/file/TestFileSink.java | 116 +++ .../common/sinks/file/TestHdfsFileSink.java | 153 ++++ .../common/sinks/hoodie/TestHoodieSink.java | 464 +++++++++++ .../common/sources/hive/TestHiveSource.java | 93 +++ .../hive/TestHiveSourceConfiguration.java | 66 ++ .../hive/TestParquetWorkUnitCalculator.java | 174 +++++ .../common/spark/TestSparkFactory.java | 91 +++ .../common/util/AbstractSparkTest.java | 88 +++ .../marmaray/common/util/AvroPayloadUtil.java | 137 ++++ .../common/util/CassandraTestConstants.java | 27 + .../common/util/CassandraTestUtil.java | 57 ++ .../marmaray/common/util/FileHelperUtil.java | 33 + .../common/util/FileSinkConfigTestUtil.java | 107 +++ .../marmaray/common/util/FileTestUtil.java | 39 + .../marmaray/common/util/HiveTestUtil.java | 37 + .../marmaray/common/util/KafkaTestHelper.java | 242 ++++++ .../util/MultiThreadTestCoordinator.java | 39 + .../common/util/ParquetWriterUtil.java | 49 ++ .../marmaray/common/util/SchemaTestUtil.java | 100 +++ .../marmaray/common/util/SparkTestUtil.java | 51 ++ .../common/util/TestConverterUtil.java | 77 ++ .../marmaray/common/util/TestDateUtil.java | 43 ++ .../marmaray/common/util/TestFsUtils.java | 83 ++ .../marmaray/common/util/TestJobUtil.java | 37 + .../marmaray/common/util/TestLockManager.java | 298 +++++++ .../marmaray/common/util/TestMapUtil.java | 39 + .../common/util/TestParquetWriterUtil.java | 80 ++ .../marmaray/common/util/TestSchemaUtil.java | 64 ++ .../marmaray/utilities/TestKafkaUtil.java | 102 +++ marmaray/src/test/resources/cassandra.yaml | 586 ++++++++++++++ marmaray/src/test/resources/config.yaml | 16 + .../src/test/resources/configWithScopes.yaml | 59 ++ .../src/test/resources/datacenter/datacenter | 1 + .../expectedConfigWithBootstrapScope.yaml | 34 + .../expectedConfigWithIncrementalScope.yaml | 33 + .../test/resources/log4j-surefire.properties | 10 + .../test/resources/schemas/StringPair.avsc | 9 + marmaray/src/test/resources/setupTable.cql | 5 + marmaray/src/test/resources/teardownTable.cql | 4 + .../testData/testPartition/data.parquet | Bin 0 -> 454 bytes pom.xml | 724 ++++++++++++++++++ 260 files changed, 26644 insertions(+) create mode 100644 .gitignore create mode 100644 checkstyles/marmaray-0.0.1.xml create mode 100644 checkstyles/suppressions.xml create mode 100644 marmaray-tools/pom.xml create mode 100755 marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py create mode 100644 marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java create mode 100644 marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java create mode 100644 marmaray/build.gradle create mode 100644 marmaray/config/sample.yaml create mode 100644 marmaray/pom.xml create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/IPayload.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/IData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/Job.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/MetadataConstants.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metadata/StringValue.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraPayloadRDDSizeEstimator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/ChargebackMetricType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/IChargebackCalculator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/IMetricable.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricNames.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricType.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetrics.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/reporters/ConsoleReporter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/reporters/IReporter.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/reporters/Reportable.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/retry/IFunctionThrowsException.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/retry/IRetryStrategy.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/retry/RetryableFunction.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/retry/SimpleRetryStrategy.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/ISchemaService.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/ISinkSchemaManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraDataField.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraPayload.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchema.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/ISink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSinkOperations.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieWriteStatus.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/IRunState.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveRunState.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculatorResult.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaRunState.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ByteBufferUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ConverterUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/DateUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ErrorExtractor.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/GenericRecordUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkConverterErrorExtractor.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkErrorExtractor.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/LockManager.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/MapUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/NumberConstants.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/ScalaUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/StringTypes.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/StringUtil.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkJobTracker.java create mode 100644 marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/actions/TestJobDagActions.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestAwsConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfigScopeResolver.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestErrorTableConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestFileSinkConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHadoopConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java create 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/data/TestRDDWrapper.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/dataset/TestUtilTable.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/forkoperator/TestForkOperator.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/job/TestJobSubDag.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/HDFSTestConstants.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHoodieBasedMetadataManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metrics/TestDataFeedMetrics.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metrics/TestJobMetrics.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/metrics/TestTimerMetric.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/retry/TestRetryableFunction.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestClusterKey.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/TestSinkStatManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/AvroPayloadUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/FileSinkConfigTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/FileTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/MultiThreadTestCoordinator.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/ParquetWriterUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/SchemaTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestConverterUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestDateUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestJobUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestLockManager.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestMapUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java create mode 100644 marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java create mode 100644 marmaray/src/test/resources/cassandra.yaml create mode 100644 marmaray/src/test/resources/config.yaml create mode 100644 marmaray/src/test/resources/configWithScopes.yaml create mode 100644 marmaray/src/test/resources/datacenter/datacenter create mode 100644 marmaray/src/test/resources/expectedConfigWithBootstrapScope.yaml create mode 100644 marmaray/src/test/resources/expectedConfigWithIncrementalScope.yaml create mode 100644 marmaray/src/test/resources/log4j-surefire.properties create mode 100644 marmaray/src/test/resources/schemas/StringPair.avsc create mode 100644 marmaray/src/test/resources/setupTable.cql create mode 100644 marmaray/src/test/resources/teardownTable.cql create mode 100644 marmaray/src/test/resources/testData/testPartition/data.parquet create mode 100644 pom.xml diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..82806b8 --- /dev/null +++ b/.gitignore @@ -0,0 +1,90 @@ +# Compiled source # +################### +*.com +*.class +*.dll +*.exe +*.a +*.o +*.so +*.node + +# Node Waf Byproducts # +####################### +.lock-wscript +build/ +autom4te.cache/ + +# Node Modules # +################ +# Better to let npm install these from the package.json defintion +# rather than maintain this manually +node_modules/ + +# Packages # +############ +# it's better to unpack these files and commit the raw source +# git has its own built in compression methods +*.7z +*.dmg +*.gz +*.iso +*.jar +*.rar +*.tar +*.zip + +# Logs and databases # +###################### +*.log +dump.rdb +*.tap + + +# OS generated files # +###################### +.DS_Store? +.DS_Store +ehthumbs.db +Icon? +Thumbs.db + +# thrift generated files # +########################## +generated/ + +# NodeJS Core Dump +core + +# Jenkins build scripts +rt-jenkins/ + +# Coverage Reports +coverage/ + +# local docs, scratchboards +localdocs/ + +# vi temp files +.*.swp + +# intelliJ +.idea/ +*.iml + +# Project specific items (local conf, build dir) +config/local.json +maps-evidence/ +*.lst +classes/ +target/ +*.dat + +# shaded jar pom file +dependency-reduced-pom.xml + +# output of build plugin org.codehaus.mojo build-helper-maven-plugin +test_properties.props + +# gradle generated logs +.gradle diff --git a/checkstyles/marmaray-0.0.1.xml b/checkstyles/marmaray-0.0.1.xml new file mode 100644 index 0000000..fa3cf98 --- /dev/null +++ b/checkstyles/marmaray-0.0.1.xml @@ -0,0 +1,136 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/checkstyles/suppressions.xml b/checkstyles/suppressions.xml new file mode 100644 index 0000000..f516bb7 --- /dev/null +++ b/checkstyles/suppressions.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + diff --git a/marmaray-tools/pom.xml b/marmaray-tools/pom.xml new file mode 100644 index 0000000..1fc5334 --- /dev/null +++ b/marmaray-tools/pom.xml @@ -0,0 +1,22 @@ + + 4.0.0 + + 1.8 + 1.8 + + + com.uber.marmaray + marmaray-base + 1.0-SNAPSHOT + + marmaray-tools + 1.0-SNAPSHOT + + + + com.uber.marmaray + marmaray + 1.0-SNAPSHOT + + + diff --git a/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py b/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py new file mode 100755 index 0000000..2e96826 --- /dev/null +++ b/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py @@ -0,0 +1,62 @@ +""" +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. +""" + +#!/usr/bin/env python +# -*- coding: utf-8 -*- + + +""" +A script to either enable or disable a HDFS metadata file. A metadata file is disabled if the extension ".tmp" +is appended at the end. Enabling a metadata file will remove the same extension. This tool will allow users +to disable metadata files and associated watermarks inside these files used to determine which partition to process. + +Usage: toggleHDFSMetadataFile -f sample_file -d + toggleHDFSMetadataFile -f sample_file.tmp -e +""" +import argparse, sys +import shutil +import logging +from os.path import basename, dirname, splitext, isfile + +def main(args): + logging.basicConfig(level=logging.INFO) + disabled_ext = ".tmp" + + if not isfile(args.file): + logging.error("the file does not exist") + quit() + + if args.disable: + if not args.file.endswith(disabled_ext): + logging.info("Disabling %s", args.file) + shutil.move(args.file, args.file + disabled_ext) + else: + logging.warning("the specified filename already ends with a .tmp extension") + if args.enable: + if args.file.endswith(disabled_ext): + logging.info("Enabling %s ", args.file) + shutil.move(args.file, dirname(args.file) + splitext(basename(args.file))[0]) + else: + logging.error("the file must end with a .tmp extension") + +if __name__ == '__main__': + p= argparse.ArgumentParser() + p.add_argument('-f', '--file', help='File to toggle on/off as metadata file') + p.add_argument('-e', '--enable', help='Enable .tmp file as metadata file', action='store_true') + p.add_argument('-d', '--disable', help='disable metadata file', action='store_true') + args = p.parse_args() + main(args) 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 new file mode 100644 index 0000000..8b13c25 --- /dev/null +++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java @@ -0,0 +1,83 @@ +/* + * 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.tools; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.metadata.HDFSMetadataManager; +import com.uber.marmaray.common.metadata.StringValue; +import com.uber.marmaray.utilities.CommandLineUtil; +import com.uber.marmaray.utilities.FSUtils; +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.util.Map; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +@Slf4j +public class HDFSMetadataPrinter { + private static final String METADATA_FILE_OPTION = "mfile"; + + public static void main(final String[] args) throws ParseException, IOException { + final CommandLineParser parser = new GnuParser(); + final Options options = getCLIOptions(); + final CommandLine cmd; + + try { + cmd = parser.parse(options, args); + } catch (final ParseException e) { + final String cmdLineSyntax = "java -cp [jar_name] com.uber.marmaray.tools.HDFSMetadataPrinter " + + "-m [METADATA_FILE]"; + final String header = "This tools prints out all the metadata contents of a HDFS metadata file."; + final String footer = "For help, please contact the Hadoop Data Platform team"; + CommandLineUtil.printHelp(options, cmdLineSyntax, header, footer); + throw e; + } + + final String metadataFilePath = cmd.getOptionValue(METADATA_FILE_OPTION); + Preconditions.checkState(!Strings.isNullOrEmpty(metadataFilePath)); + + log.info("Printing contents of metadata file: " + metadataFilePath); + + final Configuration conf = new Configuration(); + final FileSystem fs = FSUtils.getFs(conf); + try (final InputStream is = new BufferedInputStream(fs.open(new Path(metadataFilePath)))) { + try (final ObjectInputStream input = new ObjectInputStream(is)) { + final Map metadataMap = HDFSMetadataManager.deserialize(input); + metadataMap.entrySet() + .stream() + .forEach(entry -> + log.info("Key: " + entry.getKey() + " Value: " + entry.getValue().getValue())); + } + } + } + + private static Options getCLIOptions() { + final Options options = new Options(); + options.addOption(CommandLineUtil.generateOption("m", METADATA_FILE_OPTION, true, "HDFS metadata file", true)); + return options; + } +} 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 new file mode 100644 index 0000000..b2bbe90 --- /dev/null +++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java @@ -0,0 +1,120 @@ +/* + * 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.tools; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.metadata.HDFSMetadataManager; +import com.uber.marmaray.utilities.CommandLineUtil; +import com.uber.marmaray.utilities.FSUtils; +import java.io.IOException; +import java.util.Comparator; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import parquet.Preconditions; + +@Slf4j +public class HDFSMetadataPruner { + private static final String HDFS_PATH_LONG_OPTION = "path"; + private static final String NUM_METADATA_FILES_RETAINED_LONG_OPTION = "numFiles"; + private static final String FAKE_DELETE_LONG_OPTION = "fake"; + private static final String HDFS_PATH_SHORT_OPTION = "p"; + private static final String NUM_METADATA_FILES_RETAINED_SHORT_OPTION = "n"; + private static final String FAKE_DELETE_SHORT_OPTION = "f"; + + // Metadata file names in HDFS = nanoseconds since epoch so we can sort by name + private static final Comparator byTimestampedNameAsc = + Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName())); + + // Todo - consider putting main functionality in a static utility method + public static void main(final String[] args) throws ParseException, IOException { + final CommandLineParser parser = new GnuParser(); + final Options options = getCLIOptions(); + final CommandLine cmd; + + try { + cmd = parser.parse(options, args); + } catch (final ParseException e) { + final String cmdLineSyntax = + String.format("java -cp [jar_name] com.uber.marmaray.tools.HDFSMetadataCuller " + + "-%s [METADATA_PATH] -%s [NUM_METADATA_FILES_RETAINED] -%s [FAKE_DELETE_BOOLEAN]", + HDFS_PATH_SHORT_OPTION, NUM_METADATA_FILES_RETAINED_SHORT_OPTION, FAKE_DELETE_SHORT_OPTION); + final String header = "This tool prunes metadata files for an HDFS path by modification time"; + final String footer = "For help, please contact the Hadoop Data Platform team"; + CommandLineUtil.printHelp(options, cmdLineSyntax, header, footer); + throw e; + } + + final Path metadataPath = new Path(cmd.getOptionValue(HDFS_PATH_LONG_OPTION)); + + final int numFilesToRetain = cmd.hasOption(NUM_METADATA_FILES_RETAINED_LONG_OPTION) + ? Integer.parseInt(cmd.getOptionValue(NUM_METADATA_FILES_RETAINED_LONG_OPTION)) + : HDFSMetadataManager.DEFAULT_NUM_METADATA_FILES_TO_RETAIN; + + Preconditions.checkState(numFilesToRetain > 0, "Number of files to retain cannot be <= 0"); + + final boolean fakeDelete = cmd.hasOption(FAKE_DELETE_LONG_OPTION) + ? Boolean.parseBoolean(cmd.getOptionValue(FAKE_DELETE_LONG_OPTION)) + : false; + + final Configuration conf = new Configuration(); + final FileSystem fs = FSUtils.getFs(conf); + + if (fs.isDirectory(metadataPath)) { + final FileStatus[] fileStatuses = fs.listStatus(metadataPath); + + if (fileStatuses.length < numFilesToRetain) { + log.info("No files were deleted. Number of files ({}) is less than number to retain ({})", + fileStatuses.length, numFilesToRetain); + return; + } + + FSUtils.deleteHDFSMetadataFiles(fileStatuses, fs, numFilesToRetain, fakeDelete); + } else { + log.warn("Cannot prune any files, the path {} is not a directory", metadataPath); + } + } + + private static Options getCLIOptions() { + final Options options = new Options(); + options.addOption(CommandLineUtil.generateOption(HDFS_PATH_SHORT_OPTION, + HDFS_PATH_LONG_OPTION, + true, + "HDFS path", + true)); + + options.addOption(CommandLineUtil.generateOption(NUM_METADATA_FILES_RETAINED_SHORT_OPTION, + NUM_METADATA_FILES_RETAINED_LONG_OPTION, + true, + "number of metadata files to retain", + false)); + + options.addOption(CommandLineUtil.generateOption(FAKE_DELETE_SHORT_OPTION, + FAKE_DELETE_LONG_OPTION, + true, + "fake delete", + false)); + return options; + } +} + diff --git a/marmaray/build.gradle b/marmaray/build.gradle new file mode 100644 index 0000000..015d697 --- /dev/null +++ b/marmaray/build.gradle @@ -0,0 +1,18 @@ +plugins { + id 'com.kageiit.jacobo' version '2.0.1' +} + +description = 'translate jacoco to cobertura' + +dependencies { +} + +task jacobo(type: com.kageiit.jacobo.JacoboTask) { + jacocoReport = file("./target/site/jacoco-ut/jacoco.xml") + coberturaReport = file("./target/site/cobertura/coverage.xml") + srcDirs = ["./src/main/java"] +} + +task noop { + // noop task for when tests don't run +} diff --git a/marmaray/config/sample.yaml b/marmaray/config/sample.yaml new file mode 100644 index 0000000..fc3e37c --- /dev/null +++ b/marmaray/config/sample.yaml @@ -0,0 +1,45 @@ +--- +marmaray: + hadoop: + anything: "???" + hive: + dataPath: "" + jobName: "" + source: + saveCheckpoint: false + hoodie: + tables: + cell_table: + table_name: "" + base_path: "" + schema: "" + parallelism: 1024 + row_table: + table_name: "" + base_bath: "" + schema: "" + default: + combine_before_insert: true + combine_before_upsert: true + parallelism: 512 + kafka: + conn: + bootstrap: + servers: "???" + source: + topicName: + maxMessage: + readParallelism: + cassandra: + output: + native.port: "" + thrift.address: "" + keyspace: "keyspace" + tablename: "tableName" + cluster_name: "clusterName" + column_list: "columnList" + partition_keys: "partitionKeys" + clustering_keys: "clusteringKeys" + inputPath: "inputPath" + partitionType: "partitionType" + time_to_live: 0L diff --git a/marmaray/pom.xml b/marmaray/pom.xml new file mode 100644 index 0000000..1d5683d --- /dev/null +++ b/marmaray/pom.xml @@ -0,0 +1,55 @@ + + 4.0.0 + + 2.7.4 + 1.8 + 1.8 + 2.7.1 + + + + + + org.jacoco + jacoco-maven-plugin + + + org.fortasoft + gradle-maven-plugin + + + maven-assembly-plugin + + + + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${jackson.dataformat.yaml} + + + + org.apache.curator + curator-recipes + ${apache.curator} + + + + org.apache.curator + curator-test + ${apache.curator} + test + + + + com.uber.marmaray + marmaray-base + 1.0-SNAPSHOT + + marmaray + 1.0-SNAPSHOT + + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java new file mode 100644 index 0000000..d9214de --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java @@ -0,0 +1,87 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.data.IData; +import com.uber.marmaray.utilities.SparkUtil; +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.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.avro.Schema.Type.RECORD; + +/** + * This class contains the Avro data as payload with the schema + */ +// TODO (T962137) +@ToString +@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; + + 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())); + } + } + } + + 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)); + } + } + + /** + * Avoid calling it to fetch top level record fields. + */ + public GenericRecord getData() { + return SparkUtil.deserialize(this.byteRecord, recordClassTag); + } + + /** + * 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); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java b/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java new file mode 100644 index 0000000..e6094cb --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java @@ -0,0 +1,29 @@ +/* + * 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 DispersalType} defines two dispersal types: + * version: append new file to path with version id + * overwrite: delete old files and then add new file to path + */ +public enum DispersalType { + VERSION, + OVERWRITE +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java b/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java new file mode 100644 index 0000000..f9aa958 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.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 FileSinkType} defines two options of file sink destinations + * 1. HDFS + * 2. S3: aws s3 + */ +public enum FileSinkType { + HDFS, + S3 +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java new file mode 100644 index 0000000..11e9551 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java @@ -0,0 +1,62 @@ +/* + * 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; + +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import lombok.NonNull; + +import java.io.IOException; +import java.util.Optional; + +/** + * {@link HoodieErrorPayload} is a class to represent error payloads written by Hoodie. + */ +public class HoodieErrorPayload implements HoodieRecordPayload { + private final GenericRecord record; + + public HoodieErrorPayload(@NonNull final GenericRecord record) { + this.record = record; + } + + @Override + public Optional getInsertValue(final Schema schema) throws IOException { + final Optional record = getRecord(); + return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema)); + } + + protected Optional getRecord() { + return Optional.of(this.record); + } + + @Override + public HoodieErrorPayload preCombine(final HoodieErrorPayload hoodieErrorPayload) { + throw new JobRuntimeException("Not implemented yet!!"); + } + + @Override + public Optional combineAndGetUpdateValue(final IndexedRecord indexedRecord, final Schema schema) + throws IOException { + throw new JobRuntimeException("Not implemented yet!!"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/IPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/IPayload.java new file mode 100644 index 0000000..49b6470 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/IPayload.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; + +/** + * We explicitly model a generic payload here so that it gives us the flexibility to + * wrap the data with additional metadata as needed + * + * @param data type + */ +public interface IPayload { + D getData(); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java b/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java new file mode 100644 index 0000000..6d5c9f0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java @@ -0,0 +1,29 @@ +/* + * 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 PartitionType} defines partition type for data + * normal: partition by some defined key + * date: partition by date + * none: no partition + */ +public enum PartitionType { + NORMAL, + DATE, + NONE +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java b/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java new file mode 100644 index 0000000..a4793e7 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.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; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +@AllArgsConstructor +public class WorkUnit { + + @Getter + private final String workEntity; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java new file mode 100644 index 0000000..8ec2426 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java @@ -0,0 +1,54 @@ +/* + * 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.actions; + +import java.util.HashMap; +import java.util.Map; + +/** + * {@link IJobDagAction} is interface to determine a generic action to execute. + * {@link JobDagActions} are completely independent and will determine if they should run based on success status. + */ +public interface IJobDagAction { + int DEFAULT_TIMEOUT_SECONDS = 120; + String ACTION_TYPE = "action_type"; + + /** + * Execute the action + * + * @param successful whether the job dag succeeded + * @return true if action succeeded + */ + boolean execute(boolean successful); + + /** + * Timeout to wait for the action to complete + * @return number of seconds to wait for task completion + */ + default int getTimeoutSeconds() { + return DEFAULT_TIMEOUT_SECONDS; + } + + /** + * @return metric tags to be used for reporting metrics. + */ + default Map getMetricTags() { + final Map metricsTags = new HashMap<>(); + metricsTags.put(ACTION_TYPE, this.getClass().getSimpleName()); + return metricsTags; + } +} 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 new file mode 100644 index 0000000..c9767bf --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java @@ -0,0 +1,141 @@ +/* + * 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.actions; + +import com.uber.marmaray.common.job.ThreadPoolService; +import com.uber.marmaray.common.job.ThreadPoolServiceTier; +import com.uber.marmaray.common.metrics.LongMetric; +import com.uber.marmaray.common.reporters.Reporters; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Collection; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_FAILURE; +import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_SUCCESS; + +@Slf4j +/** + * {@link JobDagActions} are actions that are run based on success status. + * This class is completely independent and NOT an implementation of the {@link IJobDagAction} interface + */ +public final class JobDagActions { + public static final String RESULT_METRIC = "result"; + public static final String TIME_METRIC = "execution_time"; + + /** + * Class to hold and execute actions. + * + * Actions are executed in parallel; execution status will not affect the others. + */ + + public static final String DEFAULT_NAME = "anonymous"; + + @Getter + private final Queue actions; + private final Reporters reporters; + + @Getter + private final String name; + + public JobDagActions(@NonNull final Reporters reporters) { + this(reporters, DEFAULT_NAME); + } + + public JobDagActions(@NonNull final Reporters reporters, @NotEmpty final String name) { + this.actions = new ConcurrentLinkedDeque<>(); + this.name = name; + this.reporters = reporters; + } + + /** + * Add an action to the container + * @param action IAction to hold + */ + public void addAction(@NonNull final IJobDagAction action) { + this.actions.add(action); + } + + /** + * Add a collection of actions to the container + * @param actions Collection of IActions to hold + */ + public void addActions(@NonNull final Collection actions) { + this.actions.addAll(actions); + } + + /** + * Execute all of the actions in parallel. + * + * Parallelism is managed by the ThreadPoolService. Throws IllegalStateException if any IAction + * throws an Exception during execution. + * @param dagSuccess whether the actions are responding to a successful dag run or a failed one + */ + public boolean execute(final boolean dagSuccess) { + final AtomicBoolean successful = new AtomicBoolean(true); + final ConcurrentMap, IJobDagAction> futures = new ConcurrentHashMap<>(); + this.actions.forEach(a -> + futures.put(ThreadPoolService.submit(() -> { + final long startTime = System.currentTimeMillis(); + final boolean success; + try { + success = a.execute(dagSuccess); + } finally { + final long endTime = System.currentTimeMillis(); + reportExecuteTime(a, endTime - startTime); + } + return success; + }, ThreadPoolServiceTier.ACTIONS_TIER, a.getTimeoutSeconds()), + a)); + futures.forEach((future, action) -> { + final AtomicBoolean actionSuccess = new AtomicBoolean(true); + try { + actionSuccess.set(future.get()); + } catch (Exception e) { + log.error("Error running JobDagAction {} for {}:", action.getClass(), this.getName(), e); + actionSuccess.set(false); + successful.set(false); + } + reportActionStatus(action, actionSuccess.get()); + }); + if (!successful.get()) { + log.warn("Errors encountered during JobDagActions execution"); + } + return successful.get(); + } + + 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()); + 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()); + 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 new file mode 100644 index 0000000..d33e738 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java @@ -0,0 +1,59 @@ +/* + * 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.actions; + +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.reporters.Reporters; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@AllArgsConstructor +/** + * {@link ReporterAction} is an implemetation of the {@link IJobDagAction} interface and is used to + * report metrics + */ +public class ReporterAction implements IJobDagAction { + + public static int DEFAULT_TIMEOUT_SECONDS = 120; + + @Getter + private final Reporters reporters; + @Getter + private final JobMetrics jobMetrics; + private final DataFeedMetrics dataFeedMetrics; + + @Override + public boolean execute(final boolean success) { + if (success) { + this.reporters.getReporters().forEach(reporter -> { + this.jobMetrics.gaugeAll(reporter); + this.dataFeedMetrics.gaugeAll(reporter); + }); + } else { + log.warn("No metrics produced or actions being executed on reporter because errors were encountered"); + } + return success; + } + + @Override + public int getTimeoutSeconds() { + return DEFAULT_TIMEOUT_SECONDS; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java new file mode 100644 index 0000000..fa83c80 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java @@ -0,0 +1,80 @@ +/* + * 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.utilities.ConfigUtil; +import lombok.Getter; +import lombok.NonNull; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class AwsConfiguration implements Serializable { + @Getter + private final String region; + @Getter + private final String awsAccessKeyId; + @Getter + private final String awsSecretAccessKey; + @Getter + private final String bucketName; + @Getter + private final String objectKey; + @Getter + private final String s3FilePrefix; + @Getter + private final String sourcePath; + @Getter + private final String fileSystemPrefix; + @Getter + private final String pathKey; + + public AwsConfiguration(@NonNull final FileSinkConfiguration conf) { + ConfigUtil.checkMandatoryProperties(conf.getConf(), this.getMandatoryProperties()); + this.region = conf.getAwsRegion().get(); + this.awsAccessKeyId = conf.getAwsAccessKeyId().get(); + this.awsSecretAccessKey = conf.getAwsSecretAccesskey().get(); + this.bucketName = conf.getBucketName().get(); + this.objectKey = conf.getObjectKey().get(); + this.sourcePath = conf.getFullPath(); + this.fileSystemPrefix = conf.getPathPrefix(); + + String s3FilePrefix; + if (conf.getSourcePartitionPath().isPresent()) { + s3FilePrefix = String.format("%s/%s/", this.objectKey, conf.getSourcePartitionPath().get()); + } else { + s3FilePrefix = String.format("%s/", this.objectKey); + } + this.pathKey = s3FilePrefix; + s3FilePrefix += conf.getFileNamePrefix(); + this.s3FilePrefix = s3FilePrefix; + } + + private List getMandatoryProperties() { + return Collections.unmodifiableList( + Arrays.asList( + FileSinkConfiguration.AWS_REGION, + FileSinkConfiguration.BUCKET_NAME, + FileSinkConfiguration.OBJECT_KEY, + FileSinkConfiguration.AWS_ACCESS_KEY_ID, + FileSinkConfiguration.AWS_SECRET_ACCESS_KEY + )); + } +} 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 new file mode 100644 index 0000000..8c0e077 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java @@ -0,0 +1,266 @@ +/* + * 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.google.common.collect.Lists; +import com.uber.marmaray.common.PartitionType; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.schema.cassandra.ClusterKey; +import com.uber.marmaray.utilities.ConfigUtil; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.StringTypes; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.hadoop.ConfigHelper; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link CassandraSinkConfiguration} contains all relevant configuration properties to disperse data to Cassandra. + * Properties include but are not limited to MapReduce job settings, optimization parameters, etc. + * + * We explicitly assume all whitespace characters are trimmed from values in the yaml reader so we don't have + * to sanitize the values again here. + */ +@Slf4j +public class CassandraSinkConfiguration implements Serializable { + + public static final String CASSANDRA_PREFIX_ONLY = "cassandra."; + public static final String CASS_COMMON_PREFIX = Configuration.MARMARAY_PREFIX + CASSANDRA_PREFIX_ONLY; + public static final String HADOOP_COMMON_PREFIX = Configuration.MARMARAY_PREFIX + "hadoop."; + // *** Cassandra Configuration Settings *** + public static final String OUTPUT_THRIFT_PORT = CASS_COMMON_PREFIX + "output.thrift.port"; + + public static final String NATIVE_TRANSPORT_PORT = + HADOOP_COMMON_PREFIX + CASSANDRA_PREFIX_ONLY + "output.native.port"; + // Note: CassandraUnit uses port 9142 so it won't disturb any local default cassandra installation + public static final String DEFAULT_OUTPUT_NATIVE_PORT = "9042"; + + public static final String INITIAL_HOSTS = HADOOP_COMMON_PREFIX + CASSANDRA_PREFIX_ONLY + "output.thrift.address"; + + // this setting is currently not used yet but will eventually be needed for throttling + public static final String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits"; + + public static final String STORAGE_PORT = CASS_COMMON_PREFIX + "storage.port"; + 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"; + + // *** End of Cassandra Configuration Settings ***1 + public static final String DEFAULT_OUTPUT_RPC_PORT = "9160"; + // *** End of Cassandra Configuration Settings *** + + // *** Dispersal Job Settings *** + public static final String KEYSPACE = CASS_COMMON_PREFIX + "keyspace"; + public static final String TABLE_NAME = CASS_COMMON_PREFIX + "tablename"; + public static final String CLUSTER_NAME = CASS_COMMON_PREFIX + "cluster_name"; + public static final String COLUMN_LIST = CASS_COMMON_PREFIX + "column_list"; + 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 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"; + + // 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"; + + // File path containing datacenter info on each machine + public static final String DC_PATH = "data_center_path"; + + 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; + + private static final Splitter splitter = Splitter.on(StringTypes.COMMA); + + @Getter + protected final Configuration conf; + + @Getter + protected final List initialHosts; + + /** + * Optional field. This functionality enables a subset of columns to be dispersed from the data source and + * not forcing the user to either disperse all or none of the data. + * This set must include all partition and clustering keys if defined or the job will fail. + */ + @Getter + private final Optional> filteredColumns; + + @Getter + private final List partitionKeys; + + @Getter + private final List clusteringKeys; + + @Getter + private final PartitionType partitionType; + + @Getter + private final Optional writeTimestamp; + + @Getter + private final boolean timestampIsLongType; + + public CassandraSinkConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + ConfigUtil.checkMandatoryProperties(this.conf, this.getMandatoryProperties()); + + this.partitionKeys = this.splitString(this.conf.getProperty(PARTITION_KEYS).get()); + + if (this.conf.getProperty(INITIAL_HOSTS).isPresent()) { + this.initialHosts = this.splitString(this.conf.getProperty(INITIAL_HOSTS).get()); + } else { + this.initialHosts = new ArrayList<>(); + } + + // Source fields can be case insensitive so we convert everything to lower case for comparing + this.filteredColumns = this.conf.getProperty(COLUMN_LIST).isPresent() + ? Optional.of(new HashSet<>(this.splitString(this.conf.getProperty(COLUMN_LIST).get().toLowerCase()))) + : Optional.absent(); + + this.clusteringKeys = this.conf.getProperty(CLUSTERING_KEYS).isPresent() + ? initClusterKeys(this.splitString(this.conf.getProperty(CLUSTERING_KEYS).get())) + : Collections.EMPTY_LIST; + + if (this.conf.getProperty(PARTITION_TYPE).isPresent()) { + this.partitionType = PartitionType.valueOf(this.conf.getProperty(PARTITION_TYPE) + .get().trim().toUpperCase()); + } else { + this.partitionType = PartitionType.NONE; + } + + this.writeTimestamp = this.conf.getProperty(TIMESTAMP); + this.timestampIsLongType = this.conf.getBooleanProperty(TIMESTAMP_IS_LONG_TYPE, false); + } + + /** + * Returns hadoop configuration. + * Some example of things that could be set are the output rpc port (cassandra.output.thrift.port) + * and other cassandra specific hadoop configuration settings. + */ + public org.apache.hadoop.conf.Configuration getHadoopConf() { + final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + + if (this.getUserName().isPresent() && this.getPassword().isPresent()) { + ConfigHelper.setOutputKeyspaceUserNameAndPassword( + hadoopConf, + this.getUserName().get(), + this.getPassword().get() + ); + } + + if (!this.conf.getProperty(DATACENTER).isPresent()) { + // should never happen, but a sanity check + throw new MissingPropertyException("The datacenter information is missing"); + } + + // By default use Murmur3 Partitioner for now, we may want this to be changed in future + ConfigHelper.setOutputPartitioner(hadoopConf, Murmur3Partitioner.class.getName()); + + log.info("Setting output local DC only to true"); + ConfigHelper.setOutputLocalDCOnly(hadoopConf, true); + + this.conf.getPropertiesWithPrefix(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX, true).forEach( + (key, value) -> { + log.info("hadoop-conf-update:key:[{}]:value:[{}]", key, value); + hadoopConf.set(key, value); + } + ); + return hadoopConf; + } + + public String getKeyspace() { + return this.getConf().getProperty(KEYSPACE).get().trim(); + } + + public String getTableName() { + return this.getConf().getProperty(TABLE_NAME).get().trim(); + } + + public String getClusterName() { + return this.getConf().getProperty(CLUSTER_NAME).get().trim(); + } + + public Boolean getUseClientSink() { + return this.getConf().getBooleanProperty(USE_CLIENT_SINK, false); + } + + public Optional getUserName() { + return this.getConf().getProperty(USERNAME); + } + + public Optional getPassword() { + return this.getConf().getProperty(PASSWORD); + } + + public Optional getSSLStoragePort() { + return this.conf.getProperty(SSL_STORAGE_PORT).isPresent() + ? Optional.of(this.conf.getProperty(SSL_STORAGE_PORT).get()) + : Optional.absent(); + } + + public Optional getStoragePort() { + return this.conf.getProperty(STORAGE_PORT).isPresent() + ? Optional.of(this.conf.getProperty(STORAGE_PORT).get()) + : Optional.absent(); + } + + // TTL is optional. By default if you set it to 0 it means forever (confirm this) + public Optional getTimeToLive() { + return this.conf.getProperty(TIME_TO_LIVE).isPresent() + ? Optional.of(this.conf.getLongProperty(TIME_TO_LIVE, DEFAULT_TIME_TO_LIVE)) + : Optional.absent(); + } + + // TODO - For these optional fields, consider adding a default value + public Optional getNativePort() { + return this.conf.getProperty(NATIVE_TRANSPORT_PORT); + } + + private List getMandatoryProperties() { + return Collections.unmodifiableList( + Arrays.asList( + KEYSPACE, + TABLE_NAME, + CLUSTER_NAME, + PARTITION_KEYS + )); + } + + protected List splitString(final String commaSeparatedValues) { + return Lists.newArrayList(splitter.split(commaSeparatedValues)); + } + + private List initClusterKeys(final List entries) { + return entries.stream().map(entry -> ClusterKey.parse(entry)).collect(Collectors.toList()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java new file mode 100644 index 0000000..1369dd6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java @@ -0,0 +1,289 @@ +/* + * 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.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +/** + *

+ * What qualify as scopes?
+ * Top-level nodes which appear under a tag dedicated to represent + * scope override qualify as scopes. + *

+ *

+ * How is scope override defined?
+ * Scope override definition is defined under a config driven tag dedicated + * for scope override. In the example below in (fig 1), the tag happens to be + * scope_override and scopes are x, y & z. Scopes y and z are the overrider + * scopes and x is the default scope. + *

+ *

+ * How is scope overriding done?
+ * There are not more than 2 scopes participating in a scope override at any time. + * That is, either the user chooses to use the scope y, in which case y overrides x + * OR the user chooses to use scope z, in which case z overrides x + * OR user chooses to not use any scope, in which case no overriding is done at all. + * Also note that y and z are totally unrelated and never participate together in + * scope overriding. Before explaining the logic of scope overriding, please note + * the convention in the example below + *

    + *
  • + * Any node that starts with p represents a primitive node. In other + * words, it represents a config-key and a node that starts with v + * represents the config-value. + *
  • + *
  • + * Nodes x, y, z represent scopes as explained in the previous question. + *
  • + *
  • + * Rest of nodes [a-g] represent internal nodes. + *
  • + *
+ * + * (fig 1) + *
+ *     ├── scope_override
+ *     │   ├── {y = x}
+ *     │   └── {z = x}
+ *     │     
+ *     ├── x
+ *     │   └── a
+ *     │       ├── b
+ *     │       │   ├── c
+ *     │       │   │   ├── d
+ *     │       │   │   │   ├── {p3 = v3_x}
+ *     │       │   │   │   └── {p4 = v4_x}
+ *     │       │   │   └── {p2 = v2_x}
+ *     │       │   └── {p1 = v1_x}
+ *     │       └── e
+ *     │           └── f
+ *     │               └── {p5 = v5_x}
+ *     ├── y
+ *     │   └── a
+ *     │       └── b
+ *     │           ├── c
+ *     │           │   ├── d
+ *     │           │   │   ├── {p3 = v3_y}
+ *     │           │   │   └── {p4 = v4_y}
+ *     │           │   └── {p2 = v2_y}
+ *     │           └── {p1 = v1_y}
+ *     └── z
+ *         └── a
+ *             └── b
+ *                 ├── c
+ *                 │   ├── d
+ *                 │   │   ├── {p3 = v3_z}
+ *                 │   │   └── {p7 = v7_z}
+ *                 │   └── {p2 = v2_z}
+ *                 ├── g
+ *                 │   └── {p6 = v6_z}
+ *                 └── {p1 = v1_z}
+ *
+ * 
+ * + *

+ * Overriding is done by replacing the value of a config-key that appears + * in both the scopes, along the exact same hierarchy. + * + * When scope y overrides x, the final config-key values resolved are below. + * It inherits p5 as is from x. Overrides p1, p2, p3, p4.
+ * (fig 2) + *

+ *    a
+ *    ├── b
+ *    │   ├── c
+ *    │   │   ├── d
+ *    │   │   │   ├── {p3 = v3_y}
+ *    │   │   │   └── {p4 = v4_y}
+ *    │   │   └── {p2 = v2_y}
+ *    │   └── {p1 = v1_y}
+ *    └── e
+ *        └── f
+ *            └── {p5 = v5_x}
+ * 
+ * When scope z overrides x, the final config-key values resolved are below. + * It inherits p5 as is from x. Overrides p1, p2, p3, p4. Retains its p6.
+ * (fig 3) + *
+ *    a
+ *    ├── b
+ *    │   ├── c
+ *    │   │   ├── d
+ *    │   │   │   ├── {p3 = v3_z}
+ *    │   │   │   └── {p4 = v4_z}
+ *    │   │   │   └── {p7 = v7_z}
+ *    │   │   └── {p2 = v2_z}
+ *    │   ├── g
+ *    │   │   └── {p6 = v6_z}
+ *    │   └── {p1 = v1_y}
+ *    └── e
+ *        └── f
+ *            └── {p5 = v5_x}
+ * 
+ *

+ */ +@Slf4j +@RequiredArgsConstructor +public class ConfigScopeResolver { + + private final String scopeOverrideMappingKey; + private Map scopeOverrideMap; + private final ObjectMapper mapper = new ObjectMapper(); + + public JsonNode projectOverrideScopeOverDefault( + @NonNull final Optional scope, @NonNull final JsonNode rootJsonNode) { + if (!scope.isPresent() || !rootJsonNode.isContainerNode()) { + log.info("No scope overriding in effect. " + + "Either scope: {} is absent, or {} is not a container node", + scope, rootJsonNode); + return rootJsonNode; + } + Preconditions.checkState(rootJsonNode.has(scopeOverrideMappingKey), + String.format( + "scopeOverrideMappingKey: %s is not present in config but scoping is expected with scope: %s", + scopeOverrideMappingKey, scope)); + + final ObjectNode root = copyToObjectNode(rootJsonNode); + final JsonNode scopeOverrideDefinitionNodeVal = root.get(scopeOverrideMappingKey); + Preconditions.checkArgument( + scopeOverrideDefinitionNodeVal != null + && scopeOverrideDefinitionNodeVal.isContainerNode(), + String.format("Value for scopePrecedence %s should be a map, got null or primitive: ", + scopeOverrideDefinitionNodeVal)); + this.scopeOverrideMap = mapper.convertValue(scopeOverrideDefinitionNodeVal, Map.class); + log.info("scopeOverrideMap is {} scope is {}", this.scopeOverrideMap, scope.get()); + + Preconditions.checkArgument(scopeOverrideMap.containsKey(scope.get()), + "Un-recognized scope passed for config resolving"); + + root.remove(scopeOverrideMappingKey); + + final String overrideScope = scope.get(); + final String defaultScope = scopeOverrideMap.get(overrideScope); + if (root.has(overrideScope) && root.has(defaultScope)) { + final JsonNode resolvedNode = handleScopeOverride(root, overrideScope, + defaultScope); + root.remove(overrideScope); + root.remove(defaultScope); + final Iterator fieldNamesOfResolvedNode = resolvedNode.fieldNames(); + while (fieldNamesOfResolvedNode.hasNext()) { + final String fieldNameOfResolvedNode = fieldNamesOfResolvedNode.next(); + root.put(fieldNameOfResolvedNode, resolvedNode.get(fieldNameOfResolvedNode)); + } + } else { + log.info("No overriding done for scope combinations as one of them is missing." + + " IsOverrideScopePresent: {}, IsDefaultScopePresent: {} ", + root.has(overrideScope), root.has(defaultScope)); + } + for (final Entry entry : scopeOverrideMap.entrySet()) { + // remove all scope definitions, now that resolving is done + root.remove(entry.getKey()); + root.remove(entry.getValue()); + } + return root; + } + + private JsonNode handleScopeOverride( + @NonNull final ObjectNode root, + @NotEmpty final String overrideScope, + @NotEmpty final String defaultScope) { + final JsonNode overridingNode = root.get(overrideScope); + final JsonNode defaultNode = root.get(defaultScope); + final ObjectNode defaultNodeCopy; + defaultNodeCopy = copyToObjectNode(defaultNode); + // defaultNodeCopy will be updated by projecting overridingNode over it + projectOverrideNodeOverDefaultForField(null, defaultNodeCopy, overridingNode); + return defaultNodeCopy; + + } + + private ObjectNode copyToObjectNode(@NonNull final JsonNode defaultNode) { + final ObjectNode defaultNodeCopy; + try { + defaultNodeCopy = (ObjectNode) mapper + .readTree(mapper.writeValueAsString(defaultNode)); + } catch (IOException e) { + log.error("Got exception", e); + throw new JobRuntimeException(e); + } + return defaultNodeCopy; + } + + private void projectOverrideNodeOverDefaultForField( + @NotEmpty final String fieldName, + @NonNull final JsonNode parentDefaultNode, + @NonNull final JsonNode parentOverridingNode) { + + final JsonNode defaultNode = + (fieldName == null) ? parentDefaultNode : parentDefaultNode.get(fieldName); + final JsonNode overridingNode = + (fieldName == null) ? parentOverridingNode : parentOverridingNode.get(fieldName); + + if (fieldName != null) { + // not first time call to recursion + if (defaultNode == null || overridingNode == null) { + final JsonNode nodeToPutAtFieldName = java.util.Optional.ofNullable(defaultNode) + .orElse(overridingNode); + log.info("Copying fieldName: {} value: {}", fieldName, nodeToPutAtFieldName); + ((ObjectNode) parentDefaultNode).put(fieldName, nodeToPutAtFieldName); + return; + } + Preconditions + .checkState( + (defaultNode.isContainerNode() && overridingNode.isContainerNode()) + || (!defaultNode.isContainerNode() && !overridingNode.isContainerNode()), + "Mismatch in node type between default node: {} and overriding node: {}." + + " One of them is a primitive node", defaultNode, overridingNode); + if (!overridingNode.isContainerNode()) { + // primitive node or TextNode since that is the only primitive node that appears here + // so blindly accept the value of the overriding node + log.info("Using value: {} of override node for fieldName: {}", overridingNode, + fieldName); + ((ObjectNode) parentDefaultNode).put(fieldName, overridingNode); + } else { + // both are container nodes + projectOverAllFields(defaultNode, overridingNode); + } + } else { + // first call to recursion, represents root default node and override node. + projectOverAllFields(defaultNode, overridingNode); + } + } + + private void projectOverAllFields( + @NonNull final JsonNode defaultNode, @NonNull final JsonNode overridingNode) { + final Iterator childFieldNames = overridingNode.fieldNames(); + while (childFieldNames.hasNext()) { + final String childFieldName = childFieldNames.next(); + projectOverrideNodeOverDefaultForField(childFieldName, defaultNode, overridingNode); + } + } +} 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 new file mode 100644 index 0000000..48dcfa0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java @@ -0,0 +1,227 @@ +/* + * 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.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.uber.marmaray.common.exceptions.JobRuntimeException; + +import java.io.File; +import java.io.IOException; +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 + * a data transfer job. Supports scopes and scope based config + * overriding. Refer to documentation of {@link ConfigScopeResolver} + * for more info on scope overriding + */ +@Slf4j +public class Configuration implements Serializable { + + public static final String MARMARAY_PREFIX = "marmaray."; + public static final String SCOPE_OVERRIDE_MAPPING_KEY = "scope_override_map"; + + private final Properties props = new Properties(); + + /** + * @deprecated todo: remove this constructor in a separate diff + * since callers will need to inject scope, so will need change in callers + */ + @Deprecated + public Configuration() { + + } + + public Configuration(@NonNull final File yamlFile, + @NonNull final Optional scope) { + loadYamlFile(yamlFile, scope); + } + + public Configuration(@NonNull final InputStream inputStream, + @NonNull final Optional scope) { + loadYamlStream(inputStream, scope); + } + + /** + * @deprecated todo: remove this constructor in a separate diff + * since callers will need to inject scope, so will need change in callers + */ + @Deprecated + public Configuration(@NonNull final Configuration conf) { + this.props.putAll(conf.props); + } + + public void loadYamlFile(@NonNull final File yamlFile, + final Optional scope) { + try { + final FileSystem localFs = FileSystem.getLocal( + new HadoopConfiguration(new Configuration()).getHadoopConf()); + final InputStream yamlInputStream = localFs.open(new Path(yamlFile.getPath())); + loadYamlStream(yamlInputStream, scope); + } catch (IOException e) { + final String errorMsg = String + .format("Error loading yaml config file %s", yamlFile.getAbsolutePath()); + log.error(errorMsg, e); + throw new JobRuntimeException(errorMsg, e); + } + } + + public void loadYamlStream(@NonNull final InputStream yamlStream, + @NonNull final Optional scope) { + try { + final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); + final JsonNode jsonNode = yamlReader.readTree(yamlStream); + final JsonNode scopeOverriddenJsonNode = handleScopeOverriding(scope, jsonNode); + parseConfigJson(scopeOverriddenJsonNode, ""); + } catch (IOException e) { + final String errorMsg = "Error loading yaml file "; + log.error(errorMsg, e); + throw new JobRuntimeException(errorMsg, e); + } + } + + private JsonNode handleScopeOverriding( + @NonNull final Optional scope, @NonNull final JsonNode jsonNode) { + return new ConfigScopeResolver(SCOPE_OVERRIDE_MAPPING_KEY) + .projectOverrideScopeOverDefault(scope, jsonNode); + } + + public String getProperty(final String key, final String defaultValue) { + return this.props.getProperty(key, defaultValue); + } + + public Optional getProperty(final String key) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); + final String val = this.props.getProperty(key); + return (val == null) ? Optional.absent() : Optional.of(val); + } + + public void setProperty(final String key, final String value) { + this.props.setProperty(key, value); + } + + /** + * Returns properties with or without prefix. + * @param prefix + * @param removePrefix if true it will remove prefix from properties. + * @return {@link Map} with properties. + */ + public Map getPropertiesWithPrefix(final String prefix, final boolean removePrefix) { + Preconditions.checkState(!Strings.isNullOrEmpty(prefix)); + final Map properties = new HashMap<>(); + final int prefixLength = prefix.length(); + this.props.entrySet().forEach( + entry -> { + final String key = (String) entry.getKey(); + if (key.startsWith(prefix)) { + if (removePrefix) { + properties.put(key.substring(prefixLength), entry.getValue().toString()); + } else { + properties.put(key, entry.getValue().toString()); + } + + } + }); + return properties; + } + + public static T getProperty(@NonNull final Configuration conf, @NotEmpty final String key, + @NonNull final T defaultValue) { + if (defaultValue instanceof Integer) { + return (T) new Integer(conf.getIntProperty(key, ((Integer) defaultValue).intValue())); + } else if (defaultValue instanceof Long) { + return (T) new Long(conf.getLongProperty(key, ((Long) defaultValue).longValue())); + } else if (defaultValue instanceof String) { + return (T) conf.getProperty(key, (String) defaultValue); + } else if (defaultValue instanceof Double) { + return (T) new Double(conf.getDoubleProperty(key, ((Double) defaultValue).doubleValue())); + } else if (defaultValue instanceof Boolean) { + return (T) new Boolean(conf.getBooleanProperty(key, ((Boolean) defaultValue).booleanValue())); + } else { + throw new IllegalArgumentException("Not supported :" + defaultValue.getClass()); + } + } + + public int getIntProperty(final String key, final int defaultValue) { + final Optional val = getProperty(key); + return val.isPresent() ? Integer.parseInt(val.get()) : defaultValue; + } + + public long getLongProperty(final String key, final long defaultValue) { + final Optional val = getProperty(key); + return val.isPresent() ? Long.parseLong(val.get()) : defaultValue; + } + + public double getDoubleProperty(final String key, final double defaultValue) { + final Optional val = getProperty(key); + return val.isPresent() ? Double.parseDouble(val.get()) : defaultValue; + } + + public boolean getBooleanProperty(final String key, final boolean defaultValue) { + final Optional val = getProperty(key); + return val.isPresent() ? Boolean.parseBoolean(val.get()) : defaultValue; + } + + private void parseConfigJson(final JsonNode jsonNode, final String prefix) { + final Iterator fieldNamesIt = jsonNode.fieldNames(); + while (fieldNamesIt.hasNext()) { + final String fieldName = fieldNamesIt.next(); + final String newPrefix = prefix.isEmpty() ? fieldName.trim() : prefix + "." + fieldName.trim(); + final JsonNode newJsonNode = jsonNode.get(fieldName); + if (newJsonNode.isObject()) { + parseConfigJson(newJsonNode, newPrefix); + } else { + props.put(newPrefix, newJsonNode.asText().trim()); + } + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + this.props.entrySet().forEach( + entry -> { + sb.append(entry.getKey() + "<=>" + entry.getValue() + "\n"); + } + ); + return sb.toString(); + } + + public Set getKeySet() { + return this.props.keySet(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java new file mode 100644 index 0000000..3e741fc --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.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.configuration; + +import com.uber.marmaray.common.sinks.hoodie.HoodieWriteStatus; +import com.uber.marmaray.utilities.ConfigUtil; +import com.uber.marmaray.utilities.ErrorTableUtil; +import lombok.Getter; +import lombok.NonNull; +import org.apache.hadoop.fs.Path; +import org.hibernate.validator.constraints.NotEmpty; +import scala.Serializable; + +import java.util.Arrays; +import java.util.List; + +/** + * {@link ErrorTableConfiguration} contains the configuration used to construct ErrorTable + */ +public class ErrorTableConfiguration implements Serializable { + + public static final String ERROR_TABLE_PREFIX = Configuration.MARMARAY_PREFIX + "error_table."; + /** + * Flag to control whether error table is enabled + */ + public static final String IS_ENABLED = ERROR_TABLE_PREFIX + "enabled"; + public static final boolean DEFAULT_IS_ENABLED = false; + /** + * Parallelism for writing error table parquet files. Note: Larger value can result into large number of small + * files and HDFS Namenode performance may get affected. + */ + public static final String WRITE_PARALLELISM = ERROR_TABLE_PREFIX + "parallelism"; + public static final int DEFAULT_WRITE_PARALLELISM = 1; + /** + * Destination folder where error table files will be written + */ + public static final String DESTINATION_PATH = ERROR_TABLE_PREFIX + "dest_path"; + /** + * Flag to control whether error table is written to date partition + */ + public static final String IS_DATE_PARTITIONED = ERROR_TABLE_PREFIX + "date_partitioned"; + public static final boolean DEFAULT_IS_DATE_PARTITIONED = true; + + @Getter + private final Configuration conf; + @Getter + private final boolean isEnabled; + @Getter + private Path destPath; + @Getter + private final int writeParallelism; + @Getter + private final boolean isDatePartitioned; + + public ErrorTableConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + this.isEnabled = conf.getBooleanProperty(IS_ENABLED, DEFAULT_IS_ENABLED); + if (this.isEnabled()) { + ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties()); + this.destPath = new Path(conf.getProperty(DESTINATION_PATH).get()); + } + this.isDatePartitioned = conf.getBooleanProperty(IS_DATE_PARTITIONED, DEFAULT_IS_DATE_PARTITIONED); + this.writeParallelism = conf.getIntProperty(WRITE_PARALLELISM, DEFAULT_WRITE_PARALLELISM); + } + + /** + * @return hoodie configuration. + */ + public HoodieConfiguration getHoodieConfiguration(@NonNull final Configuration conf, + @NotEmpty final String schema, + @NotEmpty final String tableKey, + @NotEmpty final String errorTableKey, + final boolean errorMetricsEnabled) { + final HoodieConfiguration hoodieConf = new HoodieConfiguration(conf, tableKey); + final String errorTableName = getErrorTableName(hoodieConf.getTableName()); + + final HoodieConfiguration.Builder builder = HoodieConfiguration.newBuilder(conf, errorTableKey) + .withSchema(schema) + .withTableName(errorTableName) + .withBasePath(this.getDestPath().toString()) + .withBulkInsertParallelism(this.getWriteParallelism()) + .enableMetrics(errorMetricsEnabled) + .withWriteStatusClass(HoodieWriteStatus.class); + // TODO T1793431 fix error metrics and enable metrics + if (errorMetricsEnabled) { + final String errorMetricsPrefix = getErrorMetricsPrefix(hoodieConf.getHoodieMetricsPrefix()); + builder.withMetricsPrefix(errorMetricsPrefix); + } + return builder.build(); + } + + /** + * @return hoodie error table name. + */ + private String getErrorTableName(@NotEmpty final String hoodieTableName) { + return hoodieTableName + ErrorTableUtil.ERROR_TABLE_SUFFIX; + } + + /** + * @return hoodie error metrics prefix. + */ + private String getErrorMetricsPrefix(@NotEmpty final String metricsPrefix) { + return metricsPrefix + ErrorTableUtil.ERROR_TABLE_SUFFIX; + } + + public static List getMandatoryProperties() { + return Arrays.asList(DESTINATION_PATH); + } +} 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 new file mode 100644 index 0000000..75c6a67 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java @@ -0,0 +1,240 @@ +/* + * 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.DispersalType; +import com.uber.marmaray.common.FileSinkType; +import com.uber.marmaray.common.PartitionType; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.utilities.ConfigUtil; +import com.uber.marmaray.utilities.SchemaUtil; +import lombok.Getter; +import lombok.NonNull; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import com.google.common.base.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.EnumUtils; +import org.apache.commons.lang3.StringUtils; + +@Slf4j +public class FileSinkConfiguration implements Serializable { + public static final String FILE_PREFIX_ONLY = "file."; + public static final String FILE_COMM_PREFIX = Configuration.MARMARAY_PREFIX + FILE_PREFIX_ONLY; + public static final String FILE_NAME_PREFIX = "marmaray"; + public static final String FS_PATH = FILE_COMM_PREFIX + "file_path"; + public static final String DEFAULT_FS_PATH = "/dispersal_output"; + 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 CSV_COLUMN_HEADER = FILE_COMM_PREFIX + "with_column_header"; + public static final Boolean DEFAULT_CSV_COLUMN_HEADER = false; + //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. + public static final long DEFAULT_FILE_SIZE = -1; + public static final String SEPARATOR = FILE_COMM_PREFIX + "separator"; + public static final char DEFAULT_SEPARATOR = ','; + //File name related setting + public static final String PARTITION_TYPE = FILE_COMM_PREFIX + "partition_type"; + public static final PartitionType DEFAULT_PARTITION_TYPE = PartitionType.NONE; + public static final String TIMESTAMP = FILE_COMM_PREFIX + SchemaUtil.DISPERSAL_TIMESTAMP; + public static final String SOURCE_TYPE = FILE_COMM_PREFIX + "source_type"; + public static final String SOURCE_NAME_PREFIX = FILE_COMM_PREFIX + "source_name_prefix"; + public static final String SOURCE_PARTITION_PATH = FILE_COMM_PREFIX + "source_partition_path"; + public static final String DISPERSAL_TYPE = FILE_COMM_PREFIX + "dispersal_type"; + public static final DispersalType DEFAULT_DISPERSAL_TYPE = DispersalType.VERSION; + //aws s3 properties names + public static final String FILE_SINK_TYPE = FILE_COMM_PREFIX + "file_sink_type"; + public static final FileSinkType DEFAULT_FILE_SINK_TYPE = FileSinkType.valueOf("HDFS"); + public static final String AWS_REGION = FILE_COMM_PREFIX + "aws_region"; + public static final String BUCKET_NAME = FILE_COMM_PREFIX + "bucket_name"; + public static final String OBJECT_KEY = FILE_COMM_PREFIX + "object_key"; + public static final String AWS_ACCESS_KEY_ID = FILE_COMM_PREFIX + "aws_access_key_id"; + public static final String AWS_SECRET_ACCESS_KEY = FILE_COMM_PREFIX + "aws_secret_access_key"; + public static final String AWS_LOCAL = FILE_COMM_PREFIX + "aws_local"; + public static final String DEFAULT_AWS_LOCAL = "/aws_local_tmp"; + public static final String AWS_JOB_PREFIX = FILE_COMM_PREFIX + "aws_job_prefix"; + + @Getter + private final char separator; + @Getter + private final String path; + @Getter + private final String pathPrefix; + @Getter + private final String fullPath; + @Getter + private final String fileType; + @Getter + private final double fileSizeMegaBytes; + @Getter + private final boolean columnHeader; + @Getter + private final FileSinkType sinkType; + + //File name related setting + @Getter + private final String sourceType; + @Getter + private final String writeTimestamp; + @Getter + private final Optional sourcePartitionPath; + @Getter + private final DispersalType dispersalType; + @Getter + private final String sourceNamePrefix; + @Getter + private final String pathHdfs; + @Getter + private final String fileNamePrefix; + @Getter + private final PartitionType partitionType; + + //aws s3 properties + @Getter + private final Optional awsRegion; + @Getter + private final Optional bucketName; + @Getter + private final Optional objectKey; + @Getter + private final Optional awsAccessKeyId; + @Getter + private final Optional awsSecretAccesskey; + @Getter + private final String awsLocal; + @Getter + private final Configuration conf; + + 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.fileSizeMegaBytes = this.conf.getDoubleProperty(FILE_SIZE_MEGABYTE, DEFAULT_FILE_SIZE); + this.columnHeader = this.conf.getBooleanProperty(CSV_COLUMN_HEADER, DEFAULT_CSV_COLUMN_HEADER); + + //File System Prefix + this.pathPrefix = this.conf.getProperty(PATH_PREFIX).get(); + + if (this.conf.getProperty(SEPARATOR).isPresent()) { + if (this.conf.getProperty(SEPARATOR).get().length() != 1) { + throw new UnsupportedOperationException("The separator should only contain one single character."); + } else { + this.separator = this.conf.getProperty(SEPARATOR).get().charAt(0); + } + } else { + this.separator = DEFAULT_SEPARATOR; + } + + //File Partition Type + if (this.conf.getProperty(PARTITION_TYPE).isPresent()) { + this.partitionType = PartitionType.valueOf(this.conf.getProperty(PARTITION_TYPE) + .get().trim().toUpperCase()); + } else { + this.partitionType = DEFAULT_PARTITION_TYPE; + } + + //Data Dispersal Type: OverWrite or Version[default] + if (this.conf.getProperty(DISPERSAL_TYPE).isPresent()) { + final String dispersalType = this.conf.getProperty(DISPERSAL_TYPE).get().trim().toUpperCase(); + final Boolean isValid = EnumUtils.isValidEnum(DispersalType.class, dispersalType); + if (isValid) { + this.dispersalType = DispersalType.valueOf(dispersalType); + } else { + final String errorMessage + = String.format("The data dispersal type: %s is not supported.", dispersalType); + throw new UnsupportedOperationException(errorMessage); + } + } else { + this.dispersalType = DEFAULT_DISPERSAL_TYPE; + } + + //File Sink Type : HDFS(default) or AWS S3 + if (this.conf.getProperty(FILE_SINK_TYPE).isPresent()) { + final String sinkName = this.conf.getProperty(FILE_SINK_TYPE).get().trim().toUpperCase(); + final Boolean isValid = EnumUtils.isValidEnum(FileSinkType.class, sinkName); + if (isValid) { + this.sinkType = FileSinkType.valueOf(sinkName); + } else { + final String errorMessage = String.format("The file sink type: %s is not supported.", sinkName); + throw new UnsupportedOperationException(errorMessage); + } + } else { + this.sinkType = DEFAULT_FILE_SINK_TYPE; + } + + //File Name and Path Configurations + this.sourceNamePrefix = this.conf.getProperty(SOURCE_NAME_PREFIX).get(); + + if (this.partitionType != PartitionType.NONE) { + if (!this.conf.getProperty(SOURCE_PARTITION_PATH).isPresent()) { + throw new MissingPropertyException( + "The source partition path is missing while partition type is not None."); + } + this.sourcePartitionPath = this.conf.getProperty(SOURCE_PARTITION_PATH); + } else { + if (this.conf.getProperty(HiveConfiguration.PARTITION_KEY_NAME).isPresent()) { + throw new UnsupportedOperationException( + "The partition type is none and there shouldn't be partition key name in Hive Configuration."); + } + this.sourcePartitionPath = Optional.absent(); + } + + this.writeTimestamp = this.conf.getProperty(TIMESTAMP).get(); + this.sourceType = this.conf.getProperty(SOURCE_TYPE).get(); + + this.fileNamePrefix = String.format("%s_%s_%s_%s", + FILE_NAME_PREFIX, this.sourceType, this.sourceNamePrefix, this.writeTimestamp); + + //Aws S3 configuration initialization + this.awsRegion = this.conf.getProperty(AWS_REGION); + this.bucketName = this.conf.getProperty(BUCKET_NAME); + this.objectKey = this.conf.getProperty(OBJECT_KEY); + this.awsAccessKeyId = this.conf.getProperty(AWS_ACCESS_KEY_ID); + this.awsSecretAccesskey = this.conf.getProperty(AWS_SECRET_ACCESS_KEY); + this.awsLocal = this.conf.getProperty(AWS_LOCAL, DEFAULT_AWS_LOCAL); + String fullPath = StringUtils.EMPTY; + if (this.sinkType == FileSinkType.HDFS) { + fullPath = String.format("%s%s", this.pathPrefix, this.path); + if (this.sourcePartitionPath.isPresent()) { + fullPath += String.format("/%s", this.sourcePartitionPath.get()); + } + this.fullPath = String.format("%s/%s", fullPath, this.fileNamePrefix); + } else { + this.fullPath = String.format("%s%s", this.pathPrefix, this.awsLocal); + } + this.pathHdfs = fullPath; + } + + private List getMandatoryProperties() { + return Collections.unmodifiableList( + Arrays.asList( + FileSinkConfiguration.SOURCE_TYPE, + FileSinkConfiguration.TIMESTAMP, + FileSinkConfiguration.PATH_PREFIX, + FileSinkConfiguration.SOURCE_NAME_PREFIX + )); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java new file mode 100644 index 0000000..44198a6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java @@ -0,0 +1,51 @@ +/* + * 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.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +/** + * {@link HadoopConfiguration} uses system specific hadoop configurations and overrides them with the hadoop specific + * configs defined in {@link Configuration} which starts with {@link #HADOOP_COMMON_PROPERTY_PREFIX}. + */ +@Slf4j +@RequiredArgsConstructor +public class HadoopConfiguration { + public static final String HADOOP_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hadoop."; + + // Hadoop properties not defined in our yaml configuration and set dynamically based on cluster topology + public static final String HADOOP_DEFAULT_FS = "fs.defaultFS"; + + @Getter + private final Configuration conf; + + /** + * Returns hadoop configuration. + */ + public org.apache.hadoop.conf.Configuration getHadoopConf() { + final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + this.conf.getPropertiesWithPrefix(HADOOP_COMMON_PROPERTY_PREFIX, true).forEach( + (key, value) -> { + log.info("hadoop-conf-update:key:[{}]:value:[{}]", key, value); + hadoopConf.set(key, value); + } + ); + return hadoopConf; + } +} 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 new file mode 100644 index 0000000..ca812aa --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java @@ -0,0 +1,71 @@ +/* + * 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.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 HiveConfiguration} contains all the generic metadata information for where Hive is either a source or + * sink for the data pipeline job. All HiveConfiguration properties starts with {@link #HIVE_PROPERTY_PREFIX}. + */ +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"; + + @Getter + private final Configuration conf; + + /** + * This is the path where the data is either dispersed to (for sink) or read from (source) depending on context + */ + @Getter + private final String dataPath; + + @Getter + private final String jobName; + + @Getter + private final String baseMetadataPath; + + @Getter + private final Optional partitionKeyName; + + public HiveConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties()); + + 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); + } + + public static List getMandatoryProperties() { + return Collections.unmodifiableList(Arrays.asList(HIVE_DATA_PATH, JOB_NAME, BASE_METADATA_PATH)); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java new file mode 100644 index 0000000..16ba5c7 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java @@ -0,0 +1,67 @@ +/* + * 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 lombok.Getter; +import lombok.NonNull; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; + +import java.io.Serializable; +import java.util.Date; + +/** + * {@link HiveSourceConfiguration} class contains all the metadata for running of data pipeline job where Hive + * is the source of data. This class extends {@link HiveConfiguration} + * + * All properties start with {@link #HIVE_SOURCE_PREFIX}. + */ +public class HiveSourceConfiguration extends HiveConfiguration implements Serializable { + + public static final String HIVE_SOURCE_PREFIX = HIVE_PROPERTY_PREFIX + "source."; + public static final String SAVE_CHECKPOINT = HIVE_SOURCE_PREFIX + "save_checkpoint"; + + /** + * Used for the very first run to determine first hive partition to disperse (if any). + */ + public static final String HIVE_START_DATE = HIVE_SOURCE_PREFIX + "start_date"; + public static final String HIVE_START_DATE_FORMAT = "yyyy-MM-dd"; + + @Getter + private final Optional startDate; + + /** + * This allows the option to reprocess an old partition without the need to write a new checkpoint if the + * partition was processed in the past. + */ + private final boolean saveCheckpoint; + + public HiveSourceConfiguration(@NonNull final Configuration conf) { + super(conf); + this.saveCheckpoint = this.getConf().getBooleanProperty(SAVE_CHECKPOINT, true); + + this.startDate = getConf().getProperty(HIVE_START_DATE).isPresent() + ? Optional.of(DateTime.parse(getConf().getProperty(HIVE_START_DATE).get(), + DateTimeFormat.forPattern(HIVE_START_DATE_FORMAT).withZoneUTC()).toDate()) + : Optional.absent(); + } + + public boolean shouldSaveCheckpoint() { + return this.saveCheckpoint; + } +} 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 new file mode 100644 index 0000000..71fa778 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java @@ -0,0 +1,524 @@ +/* + * 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.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieCleaningPolicy; +import com.uber.hoodie.common.table.HoodieTableConfig; +import com.uber.hoodie.config.HoodieCompactionConfig; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieMetricsConfig; +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 lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.io.FileUtils; +import org.hibernate.validator.constraints.NotEmpty; +import scala.Serializable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +/** + * {@link HoodieConfiguration} class holds hoodie configurations. + * + * All common properties start with {@link #HOODIE_COMMON_PROPERTY_PREFIX}. + * All table properties start with {@link #HOODIE_TABLES_PREFIX}. + */ +@Slf4j +public class HoodieConfiguration implements Serializable { + + public static final String HOODIE_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hoodie.%s."; + public static final String HOODIE_TABLES_PREFIX = "tables.%s"; + public static final String HOODIE_DEFAULT_PREFIX = "default"; + /** + * Hoodie table name + */ + public static final String HOODIE_TABLE_NAME = HOODIE_COMMON_PROPERTY_PREFIX + "table_name"; + /** + * Hoodie base path + */ + public static final String HOODIE_BASE_PATH = HOODIE_COMMON_PROPERTY_PREFIX + "base_path"; + /** + * Schema for Hoodie dataset + */ + public static final String HOODIE_AVRO_SCHEMA = HOODIE_COMMON_PROPERTY_PREFIX + "schema"; + /** + * Flag to control whether it should combine before insert + */ + public static final String HOODIE_COMBINE_BEFORE_INSERT = HOODIE_COMMON_PROPERTY_PREFIX + "combine_before_insert"; + public static final boolean DEFAULT_HOODIE_COMBINE_BEFORE_INSERT = false; + /** + * Flag to control whether it should combine before upsert + */ + public static final String HOODIE_COMBINE_BEFORE_UPSERT = HOODIE_COMMON_PROPERTY_PREFIX + "combine_before_upsert"; + public static final boolean DEFAULT_HOODIE_COMBINE_BEFORE_UPSERT = false; + /** + * Hoodie bulk_insert, insert & upsert parallelism + * The default value is the same as HoodieWriteConfig's default (a private variable) + */ + public static final String HOODIE_BULKINSERT_PARALLELISM + = HOODIE_COMMON_PROPERTY_PREFIX + "bulkinsert_parallelism"; + public static final String HOODIE_INSERT_PARALLELISM = HOODIE_COMMON_PROPERTY_PREFIX + "insert_parallelism"; + public static final String HOODIE_UPSERT_PARALLELISM = HOODIE_COMMON_PROPERTY_PREFIX + "upsert_parallelism"; + public static final int DEFAULT_HOODIE_PARALLELISM = 200; + /** + * Auto tune insert parallelism for bulk insert + */ + public static final String HOODIE_AUTO_TUNE_PARALLELISM = + HOODIE_COMMON_PROPERTY_PREFIX + "auto_tune_parallelism"; + public static final boolean DEFAULT_AUTO_TUNE_PARALLELISM = true; + /** + * Target file size if auto tuning is enabled for insert parallelism. + */ + public static final String HOODIE_TARGET_FILE_SIZE = + HOODIE_COMMON_PROPERTY_PREFIX + "auto_target_file_size"; + // default is set to 1GB which is between HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT and + // HOODIE_PARQUET_MAX_FILE_SIZE. + public static final long DEFAULT_HOODIE_TARGET_FILE_SIZE = FileUtils.ONE_GB; + /** + * Write buffer limit in bytes to be used for bulk insert + */ + public static final String HOODIE_INSERT_BUFFER_MEMORY_BYTES = + HOODIE_COMMON_PROPERTY_PREFIX + "insert_buffer_memory_bytes"; + public static final int DEFAULT_HOODIE_INSERT_BUFFER_MEMORY_BYTES = (int) (32 * FileUtils.ONE_MB); + + // Hoodie Compaction parameters + /** + * Hoodie enable auto clean + */ + public static final String HOODIE_ENABLE_AUTO_CLEAN = HOODIE_COMMON_PROPERTY_PREFIX + "enable_auto_clean"; + public static final boolean DEFAULT_HOODIE_ENABLE_AUTO_CLEAN = true; + /** + * Hoodie cleaner policy + */ + public static final String HOODIE_CLEANER_POLICY = HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_policy"; + public static final String DEFAULT_HOODIE_CLEANER_POLICY = "KEEP_LATEST_COMMITS"; + /** + * Hoodie cleaner commits retained + */ + public static final String HOODIE_CLEANER_COMMITS_RETAINED = + HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_commits_retained"; + public static final int DEFAULT_HOODIE_CLEANER_COMMITS_RETAINED = 10; + /** + * Hoodie cleaner versions retained + */ + 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 compaction small file size + */ + public static final String HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT = + HOODIE_COMMON_PROPERTY_PREFIX + "compaction_small_file_size_limit"; + public static final long DEFAULT_HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT = 80 * FileUtils.ONE_MB; + /** + * Hoodie Storage file size. + */ + /** + * 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 + * {@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; + /** + * Hoodie insert split size + */ + public static final String HOODIE_INSERT_SPLIT_SIZE = HOODIE_COMMON_PROPERTY_PREFIX + "insert_split_size"; + + // Hoodie bloom index properties + /** + * Hoodie bloom index filter FPP + */ + public static final String HOODIE_BLOOM_FILTER_FPP = HOODIE_COMMON_PROPERTY_PREFIX + "bloom_filter_fpp"; + public static final double DEFAULT_HOODIE_BLOOM_FILTER_FPP = + Double.parseDouble(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP); + /** + * Hoodie bloom filter num entries + */ + public static final String HOODIE_BLOOM_FILTER_NUM_ENTRIES = + HOODIE_COMMON_PROPERTY_PREFIX + "bloom_filter_num_entries"; + public static final int DEFAULT_HOODIE_BLOOM_FILTER_NUM_ENTRIES = + Integer.parseInt(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES); + /** + * Hoodie bloom index parallelism + */ + public static final String HOODIE_BLOOM_INDEX_PARALLELISM = + HOODIE_COMMON_PROPERTY_PREFIX + "bloom_index_parallelism"; + public static final int DEFAULT_HOODIE_BLOOM_INDEX_PARALLELISM = 1024; + + // Hoodie Write Status config. + /** + * Hoodie Write status class + */ + public static final String HOODIE_WRITE_STATUS_CLASS = HOODIE_COMMON_PROPERTY_PREFIX + "write_status_class"; + public static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getCanonicalName(); + // Hoodie metrics config. + /** + * Hoodie metrics prefix + */ + public static final String HOODIE_METRICS_PREFIX = HOODIE_COMMON_PROPERTY_PREFIX + "metrics_prefix"; + /** + * Hoodie enable metrics + */ + public static final String HOODIE_ENABLE_METRICS = HOODIE_COMMON_PROPERTY_PREFIX + "enable_metrics"; + public static final boolean DEFAULT_HOODIE_ENABLE_METRICS = true; + + /** + * Rollback inflight commits. + */ + public static final String HOODIE_ROLLBACK_INFLIGHT_COMMITS = + HOODIE_COMMON_PROPERTY_PREFIX + "rollback_inflight_commits"; + public static final boolean DEFAULT_HOODIE_ROLLBACK_INFLIGHT_COMMITS = true; + + @Getter + private final Configuration conf; + @Getter + private final String tableKey; + @Getter + private final Optional version; + + public HoodieConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey, + @NonNull final Optional version) { + this.conf = conf; + this.tableKey = tableKey; + this.version = version; + ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties()); + } + + public HoodieConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey) { + this(conf, tableKey, Optional.absent()); + } + + /** + * @return List of mandatory properties. + */ + public List getMandatoryProperties() { + return Collections.unmodifiableList(Arrays.asList(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey), + getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey))); + } + + /** + * @return hoodie base path directory + */ + public String getBasePath() { + // HOODIE_BASE_PATH is a mandatory property. Please check {#getMandatoryProperties()}. + return this.getConf().getProperty(getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey)).get(); + } + + /** + * @return hoodie table name. + */ + public String getTableName() { + return this.getConf().getProperty(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey)).get(); + } + + /** + * @return hoodie metrics prefix. + * */ + public String getHoodieMetricsPrefix() { + return this.getConf().getProperty(getTablePropertyKey(HOODIE_METRICS_PREFIX, this.tableKey)).get(); + } + + /** + * @return true if {@link com.uber.hoodie.HoodieWriteClient} should rollback inflight commits from previous write + * call. + */ + public boolean shouldRollbackInFlight() { + return getProperty(HOODIE_ROLLBACK_INFLIGHT_COMMITS, DEFAULT_HOODIE_ROLLBACK_INFLIGHT_COMMITS); + } + + /** + * @return true if auto-clean is enabled. + */ + public boolean shouldAutoClean() { + return getProperty(HOODIE_ENABLE_AUTO_CLEAN, DEFAULT_HOODIE_ENABLE_AUTO_CLEAN); + } + + /** + * @return true if insert parallelism needs to be auto tuned. + */ + public boolean shouldAutoTuneParallelism() { + return getProperty(HOODIE_AUTO_TUNE_PARALLELISM, DEFAULT_AUTO_TUNE_PARALLELISM); + } + + /** + * @return expected target file size. Needs to be set if {@link #HOODIE_AUTO_TUNE_PARALLELISM} is enabled. + */ + public long getTargetFileSize() { + return getProperty(HOODIE_TARGET_FILE_SIZE, DEFAULT_HOODIE_TARGET_FILE_SIZE); + } + + /** + * Used for updating table property + */ + public void setTableProperty(@NotEmpty final String tablePropertyKey, @NotEmpty final String value) { + this.conf.setProperty(getTablePropertyKey(tablePropertyKey, this.tableKey), value); + } + + /** + * @return returns hoodie properties + */ + public Properties getHoodieInitProperties() { + final Properties props = new Properties(); + props.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, this.getTableName()); + return props; + } + + /** + * @return hoodie bulk insert parallelism + */ + public int getBulkInsertParallelism() { + return getProperty(HOODIE_BULKINSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM); + } + + /** + * @return hoodie insert parallelism + */ + public int getInsertParallelism() { + return getProperty(HOODIE_INSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM); + } + + /** + * @return hoodie upsert parallelism + */ + public int getUpsertParallelism() { + return getProperty(HOODIE_UPSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM); + } + + /** + * @return {@link HoodieWriteConfig}. It uses {@link #conf} to create {@link HoodieWriteConfig}. If any property is + * missing then it will throw {@link MissingPropertyException}. + */ + public HoodieWriteConfig getHoodieWriteConfig() { + final HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder(); + try { + builder.forTable(getTableName()); + builder.withPath(getBasePath()); + final boolean combineBeforeInsert = + getProperty(HOODIE_COMBINE_BEFORE_INSERT, DEFAULT_HOODIE_COMBINE_BEFORE_INSERT); + final boolean combineBeforeUpsert = + getProperty(HOODIE_COMBINE_BEFORE_UPSERT, DEFAULT_HOODIE_COMBINE_BEFORE_UPSERT); + builder.combineInput(combineBeforeInsert, combineBeforeUpsert); + final String schemaPropertyKey = getTablePropertyKey(HOODIE_AVRO_SCHEMA, this.tableKey); + final Optional schema = this.conf.getProperty(schemaPropertyKey); + if (!schema.isPresent()) { + throw new MissingPropertyException(schemaPropertyKey); + } + builder.withSchema(schema.get()); + builder.withParallelism(this.getInsertParallelism(), this.getUpsertParallelism()) + .withBulkInsertParallelism(this.getBulkInsertParallelism()); + builder.withAutoCommit(false); + + // Date partitioning. + builder.withAssumeDatePartitioning(true); + + // Hoodie compaction config. + final HoodieCompactionConfig.Builder compactionConfigBuilder = HoodieCompactionConfig.newBuilder(); + compactionConfigBuilder.withCleanerPolicy(HoodieCleaningPolicy + .valueOf(getProperty(HOODIE_CLEANER_POLICY, DEFAULT_HOODIE_CLEANER_POLICY))); + compactionConfigBuilder.retainCommits( + getProperty(HOODIE_CLEANER_COMMITS_RETAINED, DEFAULT_HOODIE_CLEANER_COMMITS_RETAINED)); + compactionConfigBuilder.retainFileVersions( + getProperty(HOODIE_CLEANER_VERSIONS_RETAINED, DEFAULT_HOODIE_CLEANER_VERSIONS_RETAINED)); + final Integer insertSplitSize = getProperty(HOODIE_INSERT_SPLIT_SIZE, -1); + if (insertSplitSize > 0) { + compactionConfigBuilder.autoTuneInsertSplits(false); + compactionConfigBuilder.insertSplitSize(insertSplitSize); + } else { + compactionConfigBuilder.autoTuneInsertSplits(true); + } + compactionConfigBuilder.compactionSmallFileSize( + getProperty(HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT, + DEFAULT_HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT)); + compactionConfigBuilder.withAutoClean(shouldAutoClean()); + builder.withCompactionConfig(compactionConfigBuilder.build()); + + // Hoodie index config + builder.withIndexConfig(new HoodieIndexConfiguration(getConf(), getTableKey()).configureHoodieIndex()); + + // Hoodie metrics config + final boolean enableMetrics = getProperty(HOODIE_ENABLE_METRICS, DEFAULT_HOODIE_ENABLE_METRICS); + if (enableMetrics) { + final HoodieMetricsConfig.Builder hoodieMetricsConfigBuilder = HoodieMetricsConfig.newBuilder(); + final String hoodieMetricsPropertyKey = getTablePropertyKey(HOODIE_METRICS_PREFIX, this.tableKey); + final Optional hoodieMetricsProperty = this.conf.getProperty(hoodieMetricsPropertyKey); + if (!hoodieMetricsProperty.isPresent()) { + throw new MissingPropertyException(hoodieMetricsPropertyKey); + } + hoodieMetricsConfigBuilder.usePrefix(hoodieMetricsProperty.get()); + hoodieMetricsConfigBuilder.on(getProperty(HOODIE_ENABLE_METRICS, DEFAULT_HOODIE_ENABLE_METRICS)); + builder.withMetricsConfig(hoodieMetricsConfigBuilder.build()); + } + // Write status StorageLevel. + builder.withWriteStatusStorageLevel("DISK_ONLY"); + final String writeStatusClassName = + getProperty(HOODIE_WRITE_STATUS_CLASS, DEFAULT_HOODIE_WRITE_STATUS_CLASS); + try { + builder.withWriteStatusClass( + (Class) Class.forName(writeStatusClassName)); + } catch (ClassNotFoundException e) { + final String errorStr = + String.format("error loading hoodie write status class :{}", writeStatusClassName); + log.error(errorStr); + throw new JobRuntimeException(errorStr, e); + } + + // enable tmp directory writes for hoodie. + builder.withUseTempFolderCopyOnWriteForCreate(true); + + return builder.build(); + } catch (IllegalArgumentException e) { + throw new MissingPropertyException(e.getMessage(), e); + } + } + + /** + * It will read property value from table and default namespace. Value will be returned in following order. + * For example for propertyKey ("common.hoodie.%s.insert_split_size") + * 1) table specific value ("common.hoodie.tables.table1.insert_split_size" defined in {@link Configuration}) + * 2) default hoodie property value ("common.hoodie.default.insert_split_size" defined in {@link Configuration}) + * 3) default value specified. (passed in as an argument). + * + * @param propertyKey hoodie property key + * @param defaultValue default value of the property + * @param DataType of the property + */ + public T getProperty(@NotEmpty final String propertyKey, + @NonNull final T defaultValue) { + final String defaultKey = getDefaultPropertyKey(propertyKey); + final String tableKey = getTablePropertyKey(propertyKey, this.tableKey); + final T retValue = Configuration.getProperty(this.conf, defaultKey, defaultValue); + return Configuration.getProperty(this.conf, tableKey, retValue); + } + + public static String getTablePropertyKey(@NotEmpty final String propertyKey, @NotEmpty final String tableKey) { + return String.format(propertyKey, String.format(HOODIE_TABLES_PREFIX, tableKey)); + } + + public static String getDefaultPropertyKey(@NotEmpty final String propertyKey) { + return String.format(propertyKey, HOODIE_DEFAULT_PREFIX); + } + + public static Builder newBuilder(@NotEmpty final String tableKey) { + return newBuilder(new Configuration(), tableKey); + } + + public static Builder newBuilder(@NonNull final Configuration conf, @NotEmpty final String tableKey) { + return new Builder(conf, tableKey); + } + + /** + * Builder class to build {@link HoodieConfiguration}. + */ + public static final class Builder { + + private final Configuration conf; + private final String tableKey; + private Optional version = Optional.absent(); + + private Builder(@NonNull final Configuration conf, @NotEmpty final String tableKey) { + this.conf = conf; + this.tableKey = tableKey; + } + + public Builder withTableName(@NotEmpty final String tableName) { + this.conf.setProperty(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey), tableName); + return this; + } + + public Builder withBasePath(@NotEmpty final String basePath) { + this.conf.setProperty(getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey), basePath); + return this; + } + + public Builder withSchema(@NotEmpty final String schema) { + this.conf.setProperty(getTablePropertyKey(HOODIE_AVRO_SCHEMA, this.tableKey), schema); + return this; + } + + public Builder withBulkInsertParallelism(final int parallelism) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_BULKINSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism)); + return this; + } + + public Builder withInsertParallelism(final int parallelism) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_INSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism)); + return this; + } + + public Builder withUpsertParallelism(final int parallelism) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_UPSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism)); + return this; + } + + public Builder withMetricsPrefix(@NotEmpty final String metricsPrefix) { + this.conf.setProperty(getTablePropertyKey(HOODIE_METRICS_PREFIX, tableKey), metricsPrefix); + return this; + } + + public Builder withCombineBeforeInsert(final boolean combineBeforeInsert) { + this.conf.setProperty(getTablePropertyKey(HOODIE_COMBINE_BEFORE_INSERT, tableKey), + Boolean.toString(combineBeforeInsert)); + return this; + } + + public Builder withCombineBeforeUpsert(final boolean combineBeforeUpsert) { + this.conf.setProperty(getTablePropertyKey(HOODIE_COMBINE_BEFORE_UPSERT, tableKey), + Boolean.toString(combineBeforeUpsert)); + return this; + } + + public Builder enableMetrics(final boolean enableMetrics) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_ENABLE_METRICS, tableKey), Boolean.toString(enableMetrics)); + return this; + } + + public Builder autoTuneParallelism(final boolean autoTuneParallelism) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_AUTO_TUNE_PARALLELISM, tableKey), Boolean.toString(autoTuneParallelism)); + return this; + } + + public Builder withWriteStatusClass(@NotEmpty final Class writeStatusClass) { + this.conf.setProperty( + getTablePropertyKey(HOODIE_WRITE_STATUS_CLASS, tableKey), writeStatusClass.getCanonicalName()); + return this; + } + + public Builder withVersion(@NotEmpty final String version) { + this.version = Optional.of(version); + return this; + } + + public HoodieConfiguration build() { + return new HoodieConfiguration(this.conf, this.tableKey, this.version); + } + } +} 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 new file mode 100644 index 0000000..d73c8dd --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java @@ -0,0 +1,207 @@ +/* + * 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.Preconditions; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.index.HoodieIndex; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.StringTypes; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.util.Bytes; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; + +@Slf4j +public class HoodieIndexConfiguration extends HoodieConfiguration { + + // Hoodie Index config + public static final String HOODIE_INDEX_PROPERTY_PREFIX = + HoodieConfiguration.HOODIE_COMMON_PROPERTY_PREFIX + "index."; + /** + * Hoodie index types + */ + public static final String HOODIE_BLOOM_INDEX = "bloom"; + public static final String HOODIE_HBASE_INDEX = "hbase"; + 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."; + public static final String DEFAULT_HOODIE_INDEX_TYPE = HOODIE_BLOOM_INDEX; + /** + * Hoodie index zookeeper + */ + public static final String HOODIE_INDEX_ZOOKEEPER_QUORUM = + HOODIE_INDEX_PROPERTY_PREFIX + "zookeeper_quorum"; + public static final String HOODIE_INDEX_ZOKEEPER_PORT = HOODIE_INDEX_PROPERTY_PREFIX + "zookeeper_port"; + public static final String HOODIE_INDEX_HBASE_ZK_ZNODEPARENT = + HOODIE_INDEX_PROPERTY_PREFIX + HOODIE_HBASE_INDEX_PREFIX + HOODIE_INDEX_ZKNODE + "path"; + /** + * Hoodie index get batch size + */ + public static final String HOODIE_INDEX_GET_BATCH_SIZE = + HOODIE_INDEX_PROPERTY_PREFIX + "get_batch_size"; + public static final int DEFAULT_HOODIE_INDEX_GET_BATCH_SIZE = 1000; + /** + * 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; + /** + * 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 String DEFAULT_VERSION = ""; + + /** + * Hoodie HBase index table name. Required if the index type is hbase. + */ + public static final String HOODIE_HBASE_INDEX_TABLE_NAME = + HOODIE_INDEX_PROPERTY_PREFIX + "hbase_index_table"; + + @Getter + private final Configuration conf; + @Getter + private final String tableKey; + + public HoodieIndexConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey) { + super(conf, tableKey); + this.conf = conf; + this.tableKey = tableKey; + } + + public HoodieIndex.IndexType getHoodieIndexType() { + final String indexName = getProperty(HOODIE_INDEX_TYPE, DEFAULT_HOODIE_INDEX_TYPE); + if (HOODIE_BLOOM_INDEX.equals(indexName.toLowerCase())) { + return HoodieIndex.IndexType.BLOOM; + } else if (HOODIE_HBASE_INDEX.equals(indexName.toLowerCase())) { + return HoodieIndex.IndexType.HBASE; + } else { + throw new IllegalStateException("Unsupported index type " + indexName); + } + } + + public String getHoodieIndexZookeeperQuorum() { + final String value = getProperty(HOODIE_INDEX_ZOOKEEPER_QUORUM, StringTypes.EMPTY); + Preconditions.checkState(!value.isEmpty(), "%s must not be empty", HOODIE_INDEX_ZOOKEEPER_QUORUM); + return value; + } + + public String getHoodieHbaseIndexTableName() { + final String value = getProperty(HOODIE_HBASE_INDEX_TABLE_NAME, StringTypes.EMPTY); + Preconditions.checkState(!value.isEmpty(), "%s must not be empty", HOODIE_HBASE_INDEX_TABLE_NAME); + return value; + } + + public int getHoodieIndexZookeeperPort() { + final int value = getProperty(HOODIE_INDEX_ZOKEEPER_PORT, 0); + Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_ZOKEEPER_PORT); + return value; + } + + public String getZkZnodeParent() { + final String value = getProperty(HOODIE_INDEX_HBASE_ZK_ZNODEPARENT, StringTypes.EMPTY); + Preconditions.checkState(!value.isEmpty(), "%s must always be set", HOODIE_INDEX_HBASE_ZK_ZNODEPARENT); + return value; + } + + public int getHoodieIndexMaxQpsPerRegionServer() { + final int value = getProperty(HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER, + DEFAULT_HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER); + Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER); + return value; + } + + public double getHoodieIndexQPSFraction() { + final double value = getProperty(HOODIE_INDEX_QPS_FRACTION, DEFAULT_HOODIE_INDEX_QPS_FRACTION); + Preconditions.checkState(value > 0 && value <= 1, "%s must be between 0 and 1", HOODIE_INDEX_QPS_FRACTION); + return value; + } + + public int getHoodieIndexGetBatchSize() { + final int value = getProperty(HOODIE_INDEX_GET_BATCH_SIZE, DEFAULT_HOODIE_INDEX_GET_BATCH_SIZE); + Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_GET_BATCH_SIZE); + return value; + } + + /** + * Configure the Hoodie HBase index. + */ + public HoodieIndexConfig configureHoodieIndex() { + final String version; + if (getVersion().isPresent()) { + version = getVersion().get(); + } else { + version = DEFAULT_VERSION; + } + final String topicName = getTableName(); + final HoodieIndexConfig.Builder builder = HoodieIndexConfig.newBuilder() + .withIndexType(getHoodieIndexType()); + + if (HoodieIndex.IndexType.HBASE.equals(getHoodieIndexType())) { + final String quorum = getHoodieIndexZookeeperQuorum(); + final Integer port = getHoodieIndexZookeeperPort(); + final String zkZnodeParent = getZkZnodeParent(); + createHbaseIndexTableIfNotExists(topicName, quorum, port.toString(), zkZnodeParent, + version); + } + + return builder.build(); + } + + public void createHbaseIndexTableIfNotExists(@NotEmpty final String dataFeed, @NotEmpty final String zkQuorum, + @NotEmpty final String zkPort, @NotEmpty final String zkZnodeParent, @NotEmpty final String version) { + final String tableName = getHoodieHbaseIndexTableName(); + final String family = "_s"; + final org.apache.hadoop.conf.Configuration hbaseConfig = new org.apache.hadoop.conf.Configuration(); + hbaseConfig.set("hbase.zookeeper.quorum", zkQuorum); + hbaseConfig.set("hbase.zookeeper.property.clientPort", zkPort); + hbaseConfig.set("zookeeper.znode.parent", zkZnodeParent); + try { + try (final Connection connection = ConnectionFactory.createConnection(hbaseConfig)) { + if (!connection.getAdmin().tableExists(TableName.valueOf(tableName))) { + final HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName)); + final HColumnDescriptor familyDesc = new HColumnDescriptor(Bytes.toBytes(family)); + familyDesc.setBloomFilterType(BloomType.ROW); + familyDesc.setCompressionType(Compression.Algorithm.SNAPPY); + tableDesc.addFamily(familyDesc); + connection.getAdmin().createTable(tableDesc); + log.info("Created HBase table {} with family {}", tableName, family); + } else { + log.debug("HBase table {} exists", tableName); + } + } + } catch (IOException e) { + //todo: better handle try catch + log.error("Error creating HBase table {} ", tableName, e); + throw new JobRuntimeException(e); + } + + } +} 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 new file mode 100644 index 0000000..bd64947 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java @@ -0,0 +1,77 @@ +/* + * 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.utilities.ConfigUtil; +import lombok.Getter; +import lombok.NonNull; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import scala.Serializable; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link KafkaConfiguration} holds all the common kafka configurations. + * + * All common properties start with {@link #KAFKA_COMMON_PROPERTY_PREFIX}. + */ +public class KafkaConfiguration implements Serializable { + + public static final String KAFKA_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "kafka."; + public static final String KAFKA_CONNECTION_PREFIX = KAFKA_COMMON_PROPERTY_PREFIX + "conn."; + public static final String KAFKA_BROKER_LIST = KAFKA_CONNECTION_PREFIX + "bootstrap.servers"; + public static final String KAFKA_GROUP_ID = KAFKA_CONNECTION_PREFIX + "group.id"; + public static final String KEY_DESERIALIZER = "key.deserializer"; + public static final String ENABLE_AUTO_COMMIT = "enable.auto.commit"; + public static final String VALUE_DESERIALIZER = "value.deserializer"; + public static final String GROUP_ID = "group.id"; + public static final String DEFAULT_GROUP_ID = "marmaray_group"; + 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"; + + @Getter + private final Configuration conf; + + /** + * It holds the connection related parameters required for connecting to kafka broker. + */ + @Getter + private final Map kafkaParams; + + public KafkaConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties()); + this.kafkaParams = new HashMap<>(); + this.kafkaParams.put(GROUP_ID, DEFAULT_GROUP_ID); + this.kafkaParams.putAll(getConf().getPropertiesWithPrefix(KAFKA_CONNECTION_PREFIX, true)); + this.kafkaParams.put(KEY_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName()); + 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); + } + } + + public List getMandatoryProperties() { + return Arrays.asList(KAFKA_BROKER_LIST); + } +} 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 new file mode 100644 index 0000000..b5525f1 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java @@ -0,0 +1,83 @@ +/* + * 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.utilities.NumberConstants; +import lombok.Getter; +import lombok.NonNull; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; + +import java.util.LinkedList; +import java.util.List; + +/** + * {@link KafkaSourceConfiguration} defines configurations for Kafka source and extends {@link KafkaConfiguration}. + * + * All properties start with {@link #KAFKA_COMMON_PROPERTY_PREFIX}. + */ +public class KafkaSourceConfiguration extends KafkaConfiguration { + + public static final String KAFKA_PROPERTY_PREFIX = KAFKA_COMMON_PROPERTY_PREFIX + "source."; + public static final String KAFKA_TOPIC_NAME = KAFKA_PROPERTY_PREFIX + "topic_name"; + public static final String KAFKA_CLUSTER_NAME = KAFKA_PROPERTY_PREFIX + "cluster_name"; + 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"; + /** + * 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"; + + @Getter + private final String topicName; + @Getter + private final String clusterName; + /** + * start time in millis. (inclusive). + */ + @Getter + private final long startTime; + + 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() + ).toDate().getTime(); + } + + public List getMandatoryProperties() { + final List ret = new LinkedList<>(); + ret.addAll(super.getMandatoryProperties()); + ret.add(KAFKA_TOPIC_NAME); + ret.add(KAFKA_CLUSTER_NAME); + ret.add(KAFKA_START_DATE); + return ret; + } + + public int getReadParallelism() { + return Math.max(1, getConf().getIntProperty(KAFKA_READ_PARALLELISM, 1)); + } + + 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/LockManagerConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java new file mode 100644 index 0000000..a64ff12 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java @@ -0,0 +1,96 @@ +/* + * 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.utilities.ConfigUtil; + +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +/** + * {@link LockManagerConfiguration} defines configurations taking locks on jobs via ZooKeeper + * + * All properties start with {@link #LOCK_MANAGER_PREFIX}. + */ +public class LockManagerConfiguration { + + public static final String LOCK_MANAGER_PREFIX = Configuration.MARMARAY_PREFIX + "lock_manager."; + + public static final String IS_ENABLED = LOCK_MANAGER_PREFIX + "is_enabled"; + public static final boolean DEFAULT_IS_ENABLED = true; + + public static final String ZK_BASE_PATH = LOCK_MANAGER_PREFIX + "zk_base_path"; + + public static final String ZK_SESSION_TIMEOUT_MS = LOCK_MANAGER_PREFIX + "zk_session_timeout_ms"; + public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000; + + public static final String ZK_CONNECTION_TIMEOUT_MS = LOCK_MANAGER_PREFIX + "zk_connection_timeout_ms"; + public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000; + + public static final String ACQUIRE_LOCK_TIME_MS = LOCK_MANAGER_PREFIX + "acquire_lock_time_ms"; + public static final int DEFAULT_ACQUIRE_LOCK_TIME_MS = 60 * 1000; + + @Getter + private final Configuration conf; + + @Getter + private final boolean isEnabled; + + @Getter + private final String zkBasePath; + + @Getter + private final int zkSessionTimeoutMs; + + @Getter + private final int zkConnectionTimeoutMs; + + @Getter + private final int acquireLockTimeMs; + + public LockManagerConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + this.isEnabled = this.getConf().getBooleanProperty(IS_ENABLED, DEFAULT_IS_ENABLED); + if (this.isEnabled()) { + ConfigUtil.checkMandatoryProperties(conf, getMandatoryProperties()); + this.zkBasePath = cleanZkBasePath(this.getConf().getProperty(ZK_BASE_PATH).get()); + } else { + this.zkBasePath = null; + } + this.zkSessionTimeoutMs = this.getConf().getIntProperty(ZK_SESSION_TIMEOUT_MS, DEFAULT_ZK_SESSION_TIMEOUT_MS); + this.zkConnectionTimeoutMs = this.getConf().getIntProperty(ZK_CONNECTION_TIMEOUT_MS, + DEFAULT_ZK_CONNECTION_TIMEOUT_MS); + this.acquireLockTimeMs = this.getConf().getIntProperty(ACQUIRE_LOCK_TIME_MS, DEFAULT_ACQUIRE_LOCK_TIME_MS); + } + + private String cleanZkBasePath(@NotEmpty final String orig) { + final String cleaned = orig.replaceAll("//*", "/").replaceAll("^/", "").replaceAll("/$", ""); + return cleaned; + } + + private static List getMandatoryProperties() { + return Collections.unmodifiableList(Arrays.asList(ZK_BASE_PATH)); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java new file mode 100644 index 0000000..1467ebe --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java @@ -0,0 +1,54 @@ +/* + * 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.exceptions.JobRuntimeException; +import com.uber.marmaray.common.retry.IRetryStrategy; +import com.uber.marmaray.common.retry.SimpleRetryStrategy; + +import java.io.Serializable; + +import lombok.NonNull; + +/** + * {@link RetryStrategyConfiguration} defines configurations related to the retry strategy. + * + * All properties start with {@link #RETRY_STRATEGY_PREFIX}. + */ +public class RetryStrategyConfiguration implements Serializable { + public static final String RETRY_STRATEGY_PREFIX = Configuration.MARMARAY_PREFIX + "retry_strategy."; + public static final String DEFAULT_STRATEGY = RETRY_STRATEGY_PREFIX + "default_strategy"; + + public static final String SIMPLE_RETRY_STRATEGY = "SimpleRetryStrategy"; + + private final Configuration conf; + + public RetryStrategyConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + } + + public IRetryStrategy getRetryStrategy() { + final String strategy = conf.getProperty(DEFAULT_STRATEGY, SIMPLE_RETRY_STRATEGY); + + switch (strategy) { + case SIMPLE_RETRY_STRATEGY: + return new SimpleRetryStrategy(new SimpleRetryStrategyConfiguration(conf)); + default: + throw new JobRuntimeException(String.format("Retry strategy %s is not supported.", strategy)); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java new file mode 100644 index 0000000..0f0b00a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.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.configuration; + +import com.uber.marmaray.common.retry.SimpleRetryStrategy; + +import java.io.Serializable; + +import lombok.NonNull; + +/** + * {@link SimpleRetryStrategyConfiguration} defines configurations related to a retry strategy based on + * total number of retries and time to wait in between retries. + * + * All properties start with {@link #SIMPLE_RETRY_PREFIX}. + */ +public class SimpleRetryStrategyConfiguration implements Serializable { + private static final String SIMPLE_RETRY_PREFIX = RetryStrategyConfiguration.RETRY_STRATEGY_PREFIX + "simple."; + private static final String NUM_OF_RETRIES = SIMPLE_RETRY_PREFIX + "num_of_retries"; + private static final String WAIT_TIME_IN_MS = SIMPLE_RETRY_PREFIX + "wait_time_in_ms"; + + private final Configuration conf; + + public SimpleRetryStrategyConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + } + + public int getNumRetries() { + return conf.getIntProperty(NUM_OF_RETRIES, SimpleRetryStrategy.DEFAULT_NUMBER_OF_RETRIES); + } + + public long getWaitTimeInMs() { + return conf.getLongProperty(WAIT_TIME_IN_MS, SimpleRetryStrategy.DEFAULT_WAIT_TIME_IN_MS); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java new file mode 100644 index 0000000..a7c6cbc --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.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.configuration; + +import com.uber.marmaray.utilities.ConfigUtil; +import lombok.Getter; +import lombok.NonNull; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * {@link ZookeeperConfiguration} defines zookeeper related configuration + * + * All properties start with {@link #ZK_PROPERTY_PREFIX}. + */ +public class ZookeeperConfiguration { + public static final String ZK_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "zookeeper."; + public static final String ZK_QUORUM = ZK_PROPERTY_PREFIX + "quorum"; + public static final String ZK_PORT = ZK_PROPERTY_PREFIX + "port"; + + @Getter + private final Configuration conf; + @Getter + private final String zkQuorum; + @Getter + private final String zkPort; + + public ZookeeperConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + ConfigUtil.checkMandatoryProperties(this.getConf(), getMandatoryProperties()); + + this.zkQuorum = this.getConf().getProperty(ZK_QUORUM).get(); + this.zkPort = this.getConf().getProperty(ZK_PORT).get(); + } + + public static List getMandatoryProperties() { + return Collections.unmodifiableList(Arrays.asList(ZK_QUORUM, ZK_PORT)); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java new file mode 100644 index 0000000..d9c355c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java @@ -0,0 +1,77 @@ +/* + * 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.converterresult; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RawData; +import com.uber.marmaray.common.data.ValidData; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.Serializable; + +/** + * {@link ConverterResult} associates data of an input type with the converted data with an output type + * + * User has option to implement a conversion data of type OD to type ID in a one-way transformation as needed + * + * This class is used to maintain parity between the ID data that was converted to OD data as needed for + * any reporting and error handling purposes. + * + * If ID is empty we assume that the OD data was valid and schema-conforming and the original input data is + * no longer needed. If ID is non-empty there was an issue with the converted data and it was non-schema conforming. + * + * @param Data Type to Convert to + * @param Original Data Type to Convert from + */ +@Slf4j +public class ConverterResult implements Serializable { + + @NonNull + @Getter + protected Optional> successData; + + @NonNull + @Getter + protected Optional> errorData; + + /** + * Constructor for case that OD is schema conforming + * @param successData + */ + public ConverterResult(@NonNull final OD successData) { + this.successData = Optional.of(new ValidData<>(successData)); + this.errorData = Optional.absent(); + } + + public ConverterResult(@NonNull final ID errorData, @NotEmpty final String errorMessage) { + this.successData = Optional.absent(); + this.errorData = Optional.of(new ErrorData<>(errorMessage, new RawData<>(errorData))); + + } + + public ConverterResult(@NonNull final ID errorData, + @NotEmpty final String errorMessage, + @NonNull final OD successData) { + this.successData = Optional.of(new ValidData<>(successData)); + this.errorData = Optional.of(new ErrorData<>(errorMessage, new RawData<>(errorData))); + } +} 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 new file mode 100644 index 0000000..42c808d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java @@ -0,0 +1,160 @@ +/* + * 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.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.ForkData; +import com.uber.marmaray.common.data.IData; +import com.uber.marmaray.common.data.RDDWrapper; +import com.uber.marmaray.common.data.RawData; +import com.uber.marmaray.common.data.RawDataHelper; +import com.uber.marmaray.common.data.ValidData; +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.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.ErrorTableUtil; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.VoidFunction; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Implementations of the {@link AbstractDataConverter} interface will convert data records from one schema type + * to another + * @param inputSchema + * @param outputSchema + * @param inputDataType + * @param outputDataType + */ +@Slf4j +public abstract class AbstractDataConverter implements Serializable { + 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"; + + @Getter @NonNull + protected Configuration conf; + /** + * If defined then {@link #failureRecordHandler} will be invoked with input record in case of any exception from + * {@link #convert(Object)}. + */ + @NonNull + @Setter + protected Optional> failureRecordHandler = Optional.absent(); + /** + * If defined then {@link #successRecordHandler} will be invoked with output record if record conversion succeeds + * with no exceptions from {@link #convert(Object)}. + */ + @NonNull + @Setter + protected Optional> successRecordHandler = Optional.absent(); + + @NonNull + protected ErrorExtractor errorExtractor; + + public AbstractDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + this.conf = conf; + this.errorExtractor = errorExtractor; + } + + 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(); + + // 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)), + errorExtractor); + + return new RDDWrapper<>(converter.getRDD(VALID_RECORD).map(r -> ((ValidData) r).getData()), + converter.getCount(VALID_RECORD)); + } + + protected abstract List> convert(@NonNull ID data) throws Exception; + + public class DataConversionFunction extends ForkFunction { + + public DataConversionFunction() { + registerKeys(Arrays.asList(VALID_RECORD, ERROR_RECORD)); + } + + @Override + protected List> process(final IData record) { + if (!(record instanceof RawData)) { + throw new JobRuntimeException("Illegal data type :" + record.getClass()); + } + + final RawData rawData = (RawData) record; + + List> results; + + try { + results = convert(rawData.getData()); + } catch (RuntimeException re) { + throw new JobRuntimeException(re); + } catch (Exception e) { + results = Collections.singletonList(new ConverterResult(rawData.getData(), e.getMessage())); + } + + final List> forkData = new ArrayList<>(); + + results.stream().forEach(t -> { + if (t.getErrorData().isPresent()) { + processRecordHandler(AbstractDataConverter.this.failureRecordHandler, + t.getErrorData().get().getRawData().getData(), FAILURE); + + forkData.add(new ForkData<>(Arrays.asList(ERROR_RECORD), t.getErrorData().get())); + } + + if (t.getSuccessData().isPresent()) { + processRecordHandler(AbstractDataConverter.this.successRecordHandler, + t.getSuccessData().get().getData(), SUCCESS); + forkData.add(new ForkData<>(Arrays.asList(VALID_RECORD), t.getSuccessData().get())); + } + }); + return forkData; + } + + private void processRecordHandler(@NonNull final Optional> recordHandler, + @NonNull final T data, @NotEmpty final String recordHandlerType) { + if (recordHandler.isPresent()) { + try { + recordHandler.get().call(data); + } catch (Exception fe) { + log.error("exception in :" + recordHandlerType , fe); + } + } + } + } +} 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 new file mode 100644 index 0000000..7abf969 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java @@ -0,0 +1,200 @@ +/* + * 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.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; +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.converterresult.ConverterResult; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +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.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; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link CassandraSinkCQLDataConverter} converts data from intermediate Avro payload to the Cassandra statement which + * can be executed by Cassandra client. This class is only to be used where the sink of the data migration is + * Cassandra. The main convert method of this class will return a RDD of Cassandra statement to the caller. + */ +public class CassandraSinkCQLDataConverter extends SinkDataConverter { + + private static final long serialVersionUID = 1L; + + private final String inputSchemaJson; + + @Setter + private String keyspaceName; + + @Setter + private String tableName; + + /* + * 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; + + /** + * This constructor gives the option to only convert certain fields from the schema + * @param inputSchema + * @param conf + * @param fieldsToConvert + * @param requiredFields + */ + public CassandraSinkCQLDataConverter(@NonNull final Schema inputSchema, + @NonNull final Configuration conf, + @NonNull final Optional> fieldsToConvert, + @NonNull final List requiredFields, + @NonNull final TimestampInfo timestampInfo, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + if (fieldsToConvert.isPresent()) { + validate(fieldsToConvert.get(), requiredFields); + } + this.inputSchemaJson = inputSchema.toString(); + this.requiredFields = Collections.unmodifiableList(requiredFields); + this.fieldsToConvert = fieldsToConvert; + this.timestampInfo = timestampInfo; + this.keyspaceName = conf.getProperty(CassandraSinkConfiguration.KEYSPACE, ""); + this.tableName = conf.getProperty(CassandraSinkConfiguration.TABLE_NAME, ""); + } + + @Override + public List> convert(final AvroPayload avroPayload) throws Exception { + final Insert insertStatement = QueryBuilder.insertInto(keyspaceName, tableName); + final Set requiredKeysToFind = new HashSet<>(this.requiredFields); + if (!this.inputSchema.isPresent()) { + this.inputSchema = Optional.of(new Schema.Parser().parse(inputSchemaJson)); + } + + for (int i = 0; i < this.inputSchema.get().getFields().size(); i++) { + final Schema.Field field = this.inputSchema.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()); + + 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; + 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"); + } + } else { + if (requiredKeysToFind.contains(field.name())) { + throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name()); + } + } + requiredKeysToFind.remove(field.name()); + } + } + + if (this.timestampInfo.hasTimestamp()) { + 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); + } + + if (!requiredKeysToFind.isEmpty()) { + final Joiner joiner = Joiner.on(","); + throw new MissingPropertyException(joiner.join(requiredKeysToFind)); + } + + return Collections.singletonList(new ConverterResult<>(insertStatement)); + } + + /** + * Not all the fields in the Avro Schema will be converted to Cassandra fields. + * + * All required fields listed, however, must exist in the fields to convert + * @param allFieldsToConvert + * @param requiredFields + */ + private void validate(final Set allFieldsToConvert, 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(","); + 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 new file mode 100644 index 0000000..5fe15f0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java @@ -0,0 +1,201 @@ +/* + * 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.Joiner; +import com.google.common.base.Optional; +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.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.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.NonNull; +import org.apache.avro.Schema; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link CassandraSinkDataConverter} extends {@link SinkDataConverter} + * This class converts data from intermediate Avro schema to the Cassandra schema. This + * class is only to be used where the sink of the data migration is Cassandra. The main convert method of this + * 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. + */ +public class CassandraSinkDataConverter extends SinkDataConverter { + + private static final long serialVersionUID = 1L; + + 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; + + /** + * This constructor gives the option to only convert certain fields from the schema + * @param inputSchema + * @param conf + * @param fieldsToConvert + * @param requiredFields + */ + public CassandraSinkDataConverter(@NonNull final Schema inputSchema, + @NonNull final Configuration conf, + @NonNull final Optional> fieldsToConvert, + @NonNull final List requiredFields, + @NonNull final TimestampInfo timestampInfo, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + if (fieldsToConvert.isPresent()) { + validate(fieldsToConvert.get(), requiredFields); + } + this.inputSchemaJson = inputSchema.toString(); + this.requiredFields = Collections.unmodifiableList(requiredFields); + this.fieldsToConvert = fieldsToConvert; + this.timestampInfo = timestampInfo; + } + + @Override + public List> convert(final AvroPayload avroPayload) + throws Exception { + 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)); + } + + for (int i = 0; i < this.inputSchema.get().getFields().size(); i++) { + final Schema.Field field = this.inputSchema.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 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"); + } + } else { + if (requiredKeysToFind.contains(field.name())) { + throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name()); + } + bb = null; + } + requiredKeysToFind.remove(field.name()); + row.addField(new CassandraDataField(ByteBufferUtil.wrap(field.name()), bb)); + } + } + + if (this.timestampInfo.hasTimestamp()) { + final ByteBuffer bb = this.timestampInfo.isSaveAsLongType() + ? 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)); + } + + if (!requiredKeysToFind.isEmpty()) { + final Joiner joiner = Joiner.on(","); + throw new MissingPropertyException(joiner.join(requiredKeysToFind)); + } + + return Collections.singletonList(new ConverterResult<>(row)); + } + + /** + * Not all the fields in the Avro Schema will be converted to Cassandra fields. + * + * All required fields listed, however, must exist in the fields to convert + * @param allFieldsToConvert + * @param requiredFields + */ + private void validate(final Set allFieldsToConvert, 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(","); + 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/DummyHoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java new file mode 100644 index 0000000..54224e5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java @@ -0,0 +1,44 @@ +/* + * 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.utilities.ErrorExtractor; + +import lombok.NonNull; + +/** + * {@link DummyHoodieSinkDataConverter} is necessary where you do not want payloads to be + * processed by the data converter. + */ +public class DummyHoodieSinkDataConverter extends HoodieSinkDataConverter { + public DummyHoodieSinkDataConverter() { + super(new Configuration(), new ErrorExtractor()); + } + + @Override + protected String getRecordKey(@NonNull final AvroPayload payload) throws Exception { + return null; + } + + @Override + protected String getPartitionPath(@NonNull final AvroPayload payload) throws Exception { + return null; + } +} 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 new file mode 100644 index 0000000..7736235 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java @@ -0,0 +1,109 @@ +/* + * 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.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 lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; + +import java.io.StringWriter; +import java.util.List; +import java.util.Collections; + +/** + * {@link FileSinkDataConverter} extends {@link SinkDataConverter} + * 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 FileSinkDataConverter extends SinkDataConverter { + public static final String CSV = "csv"; + public final String fileType; + public final char separator; + + 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; + } + + @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(); + } else { + //Todo:Add more file type options. + final String errorMessage = "Format " + this.fileType + " not supported yet."; + 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. + */ + 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/HoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java new file mode 100644 index 0000000..8487b6d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java @@ -0,0 +1,86 @@ +/* + * 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.hoodie.common.model.HoodieAvroPayload; +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +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.sinks.hoodie.HoodieSink; +import com.uber.marmaray.utilities.ErrorExtractor; + +import lombok.NonNull; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; + +import java.util.Collections; +import java.util.List; + +/** + * {@link HoodieSinkDataConverter} extends {@link SinkDataConverter} + * This class is used by {@link HoodieSink} to generate {@link com.uber.hoodie.common.model.HoodieRecord} from + * {@link com.uber.marmaray.common.AvroPayload}. + */ +public abstract class HoodieSinkDataConverter extends SinkDataConverter> { + + // store the schema as a string since Schema doesn't serialize. Used in extended classes. + protected String schema; + private final ErrorExtractor errorExtractor; + + public HoodieSinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + this.errorExtractor = errorExtractor; + } + + public HoodieSinkDataConverter(@NonNull final Configuration conf, final String schema, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + this.schema = schema; + this.errorExtractor = errorExtractor; + } + + @Override + protected final List>> convert( + @NonNull final AvroPayload payload) throws Exception { + final HoodieKey hoodieKey = new HoodieKey(getRecordKey(payload), getPartitionPath(payload)); + final HoodieRecordPayload hoodiePayload = getPayload(payload); + return Collections.singletonList(new ConverterResult<>((new HoodieRecord<>(hoodieKey, hoodiePayload)))); + } + + /** + * The implementation of it should use fields from {@link GenericRecord} to generate record key which is needed for + * {@link HoodieKey}. + * + * @param payload {@link AvroPayload}. + */ + protected abstract String getRecordKey(@NonNull final AvroPayload payload) throws Exception; + + /** + * The implementation of it should use fields from {@link AvroPayload} to generate partition path which is needed + * for {@link HoodieKey}. + * + * @param payload {@link AvroPayload}. + */ + protected abstract String getPartitionPath(@NonNull final AvroPayload payload) throws Exception; + + protected HoodieRecordPayload getPayload(@NonNull final AvroPayload payload) { + return new HoodieAvroPayload(java.util.Optional.of(payload.getData())); + } +} 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 new file mode 100644 index 0000000..11ef4fd --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.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.common.converters.data; + +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.schema.ISchemaService.ISchemaServiceReader; +import com.uber.marmaray.utilities.ErrorExtractor; +import lombok.Getter; +import lombok.NonNull; +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; + +/** + * {@link KafkaSourceDataConverter} extends {@link SourceDataConverter} + * + * This class converts kafka messages to {@link AvroPayload}. It uses {@link ISchemaServiceReader} for decoding kafka + * message bytes. + */ +public class KafkaSourceDataConverter extends SourceDataConverter { + @NonNull + @Getter + private final ISchemaServiceReader schemaServiceReader; + @NonNull + @Getter + private final List fieldsToCache; + + /** + * List of {@Link Function} to apply to the record between reading from kafka and + * transferring to the ISource + */ + @NonNull + @Getter + private final List> updateFunctions; + + public KafkaSourceDataConverter(@NonNull final ISchemaServiceReader schemaServiceReader, + @NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + this(schemaServiceReader, conf, Collections.emptyList(), Collections.emptyList(), errorExtractor); + } + + public KafkaSourceDataConverter(@NonNull final ISchemaServiceReader schemaServiceReader, + @NonNull final Configuration conf, + @NonNull final List> updateFunctions, + @NonNull final List fieldsToCache, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + this.schemaServiceReader = schemaServiceReader; + this.updateFunctions = updateFunctions; + this.fieldsToCache = fieldsToCache; + } + + @Override + public List> convert(@NonNull final byte[] data) throws Exception { + GenericRecord genericRecord = this.schemaServiceReader.read(data); + for (Function func : this.updateFunctions) { + genericRecord = func.call(genericRecord); + } + return Collections.singletonList(new ConverterResult(new AvroPayload(genericRecord, this.fieldsToCache))); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java new file mode 100644 index 0000000..178f1e5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java @@ -0,0 +1,37 @@ +/* + * 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.utilities.ErrorExtractor; +import lombok.NonNull; +import org.apache.avro.Schema; + +/** + * {@link SinkDataConverter} extends {@link AbstractDataConverter} + * This class converts records from ({@link Schema}, {@link AvroPayload}) to (OS, OD). + * @param output schema type + * @param output data type + */ +public abstract class SinkDataConverter extends AbstractDataConverter { + private static final long serialVersionUID = 1L; + + public SinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java new file mode 100644 index 0000000..55dda51 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java @@ -0,0 +1,36 @@ +/* + * 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.utilities.ErrorExtractor; +import lombok.NonNull; +import org.apache.avro.Schema; + +/** + * {@link SourceDataConverter} extends {@link AbstractDataConverter} + * + * This class converts records from (IS, ID) to ({@link Schema}, {@link AvroPayload}). + * @param input schema type + * @param input data type + */ +public abstract class SourceDataConverter extends AbstractDataConverter { + public SourceDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + } +} 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 new file mode 100644 index 0000000..657f402 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java @@ -0,0 +1,109 @@ +/* + * 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.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.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.SparkUtil; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +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.StructField; +import org.apache.spark.sql.types.StructType; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link SparkSourceDataConverter} extends {@link SourceDataConverter} + * + * This converts data from Spark format to the common intermediate {@link AvroPayload} format + */ +@Slf4j +public class SparkSourceDataConverter extends SourceDataConverter { + private static final Set supportedDataTypes = SparkUtil.getSupportedDataTypes(); + + private final String jsonOutputSchema; + private final StructField[] fields; + private final Set requiredKeys; + private Optional outputSchema = Optional.absent(); + + public SparkSourceDataConverter(@NonNull final StructType inputSchema, + @NonNull final Schema outputSchema, + @NonNull final Configuration conf, + @NonNull final Set requiredKeys, + @NonNull final ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + this.fields = Arrays.stream(inputSchema.fields()) + .filter(f -> !f.name().startsWith("_")).toArray(StructField[]::new); + this.jsonOutputSchema = outputSchema.toString(); + this.requiredKeys = requiredKeys; + } + + @Override + public List> convert(@NonNull final Row row) throws Exception { + if (!this.outputSchema.isPresent()) { + this.outputSchema = Optional.of((new Schema.Parser().parse(this.jsonOutputSchema))); + } + + Preconditions.checkNotNull(row.schema()); + + final GenericRecord gr = new GenericData.Record(this.outputSchema.get()); + + final Set required = this.requiredKeys.stream().map(String::new).collect(Collectors.toSet()); + + // 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 (supportedDataTypes.contains(dt)) { + gr.put(this.fields[i].name(), data); + } else { + throw new JobRuntimeException(dt.toString() + " field type is not supported at this time"); + } + } catch (final IllegalArgumentException e) { + // the fieldname did not exist in the row which is ok, skip it + continue; + } + } + + 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); + } + + return Collections.singletonList(new ConverterResult<>(new AvroPayload(gr))); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java new file mode 100644 index 0000000..44bea41 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.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.common.converters.data; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.InvalidDataException; +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.marmaray.utilities.HoodieSinkConverterErrorExtractor; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.concurrent.TimeUnit; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT; + +/** + * {@link TSBasedHoodieSinkDataConverter} extends {@link HoodieSinkDataConverter} + * + * This class generates {@link HoodieKey} from given {@link AvroPayload}. The passed in {@link AvroPayload} requires + * {@link #partitionFieldName} with timestamp in {@link #timeUnit}. + * + * {@link AvroPayload} also requires a {@link #recordKeyFieldName} which should be the primary key for the record. + */ +@Slf4j +public class TSBasedHoodieSinkDataConverter extends HoodieSinkDataConverter { + + public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat(DATE_PARTITION_FORMAT); + @NotEmpty + private final String recordKeyFieldName; + @NotEmpty + private final String partitionFieldName; + @NonNull + private final TimeUnit timeUnit; + + public TSBasedHoodieSinkDataConverter(@NonNull final Configuration conf, + @NotEmpty final String recordKeyFieldName, @NotEmpty final String partitionFieldName, + @NonNull final TimeUnit timeUnit) { + super(conf, new HoodieSinkConverterErrorExtractor()); + this.recordKeyFieldName = recordKeyFieldName; + this.partitionFieldName = partitionFieldName; + this.timeUnit = timeUnit; + } + + @Override + protected String getRecordKey(@NonNull final AvroPayload payload) throws Exception { + final Object recordKeyFieldVal = payload.getField(recordKeyFieldName); + if (recordKeyFieldVal == null) { + throw new InvalidDataException("required field is missing:" + recordKeyFieldName); + } + return recordKeyFieldVal.toString(); + } + + @Override + protected String getPartitionPath(final AvroPayload payload) throws Exception { + final Object partitionFieldVal = payload.getField(partitionFieldName); + if (partitionFieldVal == null) { + throw new InvalidDataException("required field is missing:" + partitionFieldName); + } + final Date date = new Date(this.timeUnit.toMillis((long) Double.parseDouble(partitionFieldVal.toString()))); + return PARTITION_FORMATTER.format(date); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java new file mode 100644 index 0000000..1584a3f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java @@ -0,0 +1,30 @@ +/* + * 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.schema; + +import org.apache.avro.Schema; + +/** + * {@link AbstractSchemaConverter} is responsible for 2 way conversion to convert an external schema to a common schema + * extending {@link Schema} as well as vice versa + * @param external schema + * @param common schema + */ +public abstract class AbstractSchemaConverter { + public abstract ES convertToExternalSchema(CS commonSchema); + public abstract CS convertToCommonSchema(ES externalSchema); +} 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 new file mode 100644 index 0000000..2b8ee14 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java @@ -0,0 +1,114 @@ +/* + * 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.schema; + +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; +import lombok.NonNull; +import org.apache.avro.Schema; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Set; + +/** + * {@link CassandraSchemaConverter} extends {@AbstractSchemaConverter} and converts {@link CassandraSchema} + * to {@link Schema} and vice versa + * + * In Cassandra, all keys and values are generic ByteBuffers. + * + * The actual cassandra schema would just be a list of ByteBuffer that describes each of the field names + * In order to encode the values of each row properly as ByteBuffers properly, however, we need to know + * the type of field from the common schema + */ +public class CassandraSchemaConverter extends AbstractSchemaConverter { + @Getter + final String keySpace; + + @Getter + final String tableName; + + final TimestampInfo tsInfo; + + /** + * This optional field is only populated when a job is configured to only handle a subset of fields + * and not all the available fields from the source data. + */ + final Optional> filteredFields; + + public CassandraSchemaConverter(@NotEmpty final String keySpace, + @NotEmpty final String tableName, + @NonNull final TimestampInfo timestampInfo, + @NonNull final Optional> filteredFields) { + this.keySpace = keySpace; + this.tableName = tableName; + this.tsInfo = timestampInfo; + this.filteredFields = filteredFields; + } + + public CassandraSchemaConverter(@NotEmpty final String keySpace, + @NotEmpty final String tableName, + @NonNull final Optional> filteredFields) { + this(keySpace, tableName, TimestampInfo.generateEmptyTimestampInfo(), filteredFields); + } + + @Override + public CassandraSchema convertToExternalSchema(final Schema commonSchema) { + // todo T936057 - Need to handle more complex schemas (i.e Record inside record) + final CassandraSchema cs = new CassandraSchema(this.keySpace, this.tableName); + + for (final Schema.Field field : commonSchema.getFields()) { + // Cassandra does not support field names starting with _ + if (this.shouldIncludeField(field.name())) { + final String cassFieldType = CassandraSchemaField.convertFromAvroType(field.schema()); + cs.addField(new CassandraSchemaField(field.name(), cassFieldType)); + } + } + + if (this.tsInfo.hasTimestamp()) { + if (this.tsInfo.isSaveAsLongType()) { + cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.LONG_TYPE)); + } else { + cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.STRING_TYPE)); + } + } + + return cs; + } + + @Override + public Schema convertToCommonSchema(final CassandraSchema schema) { + throw new UnsupportedOperationException(); + } + private boolean shouldIncludeField(@NotEmpty final String fieldName) { + boolean shouldInclude = false; + if (!fieldName.startsWith(StringTypes.UNDERSCORE)) { + if (this.filteredFields.isPresent()) { + if (this.filteredFields.get().contains(fieldName.toLowerCase())) { + shouldInclude = true; + } + } else { + shouldInclude = true; + } + } + return shouldInclude; + } +} 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 new file mode 100644 index 0000000..a918cd0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java @@ -0,0 +1,105 @@ +/* + * 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.schema; + +import com.google.common.base.Preconditions; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +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; + +/** + * {@link DataFrameSchemaConverter} extends {@AbstractSchemaConverter} and converts {@link StructType} + * to {@link Schema} + * + * Conversion from {@link Schema} to {@link StructType} is currently not supported. + * + */ +public class DataFrameSchemaConverter extends AbstractSchemaConverter { + + @Override + public Schema convertToCommonSchema(final StructType dataFrameSchema) { + Preconditions.checkNotNull(dataFrameSchema); + final SchemaBuilder.FieldAssembler fieldAssembler = SchemaBuilder.record("commonSchema").fields(); + for (final StructField sf : dataFrameSchema.fields()) { + addField(fieldAssembler, sf); + } + + return fieldAssembler.endRecord(); + } + + @Override + public StructType convertToExternalSchema(final Schema commonSchema) { + // We don't anticipate needing this currently but if needed we can implement + throw new UnsupportedOperationException(); + } + + private void addField(final SchemaBuilder.FieldAssembler fieldAssembler, final StructField structField) { + final SchemaBuilder.FieldBuilder fieldBuilder = fieldAssembler.name(structField.name()); + + 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(); + } else { + throw new RuntimeException("The field type " + dt + " is not supported"); + } + /** + * Todo: Handle following types + * CalendarIntervalType + * StructType + * MapType + * ArrayType + * NullType + * + */ + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java new file mode 100644 index 0000000..a20b50a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java @@ -0,0 +1,34 @@ +/* + * 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.data; + +import javax.xml.bind.DatatypeConverter; + +/** + * Convenience class for wrapping byte[]. + */ +public class BinaryRawData extends RawData { + + public BinaryRawData(final byte[] data) { + super(data); + } + + @Override + public String toString() { + return DatatypeConverter.printHexBinary(getData()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java new file mode 100644 index 0000000..3247420 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java @@ -0,0 +1,37 @@ +/* + * 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.data; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** + * It holds the input {@link RawData} record and an error message describing which error check condition failed. This + * will be useful information for debugging error records. + */ +@AllArgsConstructor +public class ErrorData implements IData { + + /** + * Error message. + */ + @Getter + private final String errMessage; + + @Getter + private final RawData rawData; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java new file mode 100644 index 0000000..5d852f6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java @@ -0,0 +1,40 @@ +/* + * 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.data; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; + +/** + * It is used to hold forked data. It wraps individual record entry with forked pipeline keys. + */ +@AllArgsConstructor +public class ForkData implements Serializable { + @Getter + private final List keys = new ArrayList<>(); + @Getter + private final DI record; + + public ForkData(@NonNull final List keys, @NonNull final DI record) { + this.keys.addAll(keys); + this.record = record; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java new file mode 100644 index 0000000..d5b6e12 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java @@ -0,0 +1,22 @@ +/* + * 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.data; + +import java.io.Serializable; + +public interface IData extends Serializable { +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java new file mode 100644 index 0000000..6a7b5b5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java @@ -0,0 +1,54 @@ +/* + * 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.data; + +import lombok.Getter; +import lombok.NonNull; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.Optional; + +/** + * Convenience class to wrap RDD of records with its count to avoid multiple calls for count operation. If we need to + * use count operation for a given RDD more than once in the form of isEmpty or actual count; then this wrapper will be + * useful. + * T dataType of RDD records. {@link #data} for more details. + */ +public class RDDWrapper { + + @Getter + @NonNull + final JavaRDD data; + + Optional count; + + public RDDWrapper(@NonNull final JavaRDD data) { + this.data = data; + this.count = Optional.absent(); + } + + public RDDWrapper(@NonNull final JavaRDD data, final long count) { + this.data = data; + this.count = Optional.of(count); + } + + public long getCount() { + if (!count.isPresent()) { + this.count = Optional.of(this.data.count()); + } + return count.get(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java new file mode 100644 index 0000000..1ffc7b1 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java @@ -0,0 +1,30 @@ +/* + * 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.data; + +import java.io.Serializable; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +@AllArgsConstructor +@ToString +public class RawData implements IData, Serializable { + + @Getter + private final T data; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java new file mode 100644 index 0000000..4c00f8b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java @@ -0,0 +1,32 @@ +/* + * 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.data; + +import lombok.NonNull; + +/** + * Helper class to return appropriate RawData subclass. + */ +public class RawDataHelper { + + public static RawData getRawData(@NonNull final T data) { + if (data instanceof byte[]) { + return new BinaryRawData((byte[]) data); + } + return new RawData(data); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java new file mode 100644 index 0000000..8998a3b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java @@ -0,0 +1,26 @@ +/* + * 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.data; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +@AllArgsConstructor +public class ValidData implements IData { + @Getter + private final T data; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java new file mode 100644 index 0000000..53afc2b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java @@ -0,0 +1,51 @@ +/* + * 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.dataset; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Simple Java Bean used to construct {@link UtilTable} of {@ErrorRecord} + */ +@Data +@EqualsAndHashCode(callSuper = true) +public class ErrorRecord extends UtilRecord { + private String table_name; + private String row_key; + private String column_name; + private String content; + private String error_type; + + public ErrorRecord(@NotEmpty final String applicationId, + @NotEmpty final String jobName, + final long jobStartTimestamp, + final long timestamp, + @NotEmpty final String tableName, + @NotEmpty final String rowKey, + @NotEmpty final String columnName, + @NotEmpty final String content, + @NotEmpty final String errorType) { + super(applicationId, jobName, jobStartTimestamp, timestamp); + this.table_name = tableName; + this.row_key = rowKey; + this.column_name = columnName; + this.content = content; + this.error_type = errorType; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java new file mode 100644 index 0000000..c9d6a47 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java @@ -0,0 +1,48 @@ +/* + * 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.dataset; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Simple Java Bean used to construct {@link UtilTable} of {@ExceptionRecord} + */ +@Data +@EqualsAndHashCode(callSuper = true) +public class ExceptionRecord extends UtilRecord { + private String exception; + private String exception_message; + private String stacktrace; + private boolean detected_on_driver; + + public ExceptionRecord(@NotEmpty final String applicationId, + @NotEmpty final String jobName, + final long jobStartTimestamp, + final long timestamp, + @NotEmpty final String exception, + @NotEmpty final String exceptionMessage, + @NotEmpty final String stacktrace, + final boolean isDriver) { + super(applicationId, jobName, jobStartTimestamp, timestamp); + this.exception = exception; + this.exception_message = exceptionMessage; + this.stacktrace = stacktrace; + this.detected_on_driver = isDriver; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java new file mode 100644 index 0000000..b27f3f0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java @@ -0,0 +1,45 @@ +/* + * 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.dataset; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Simple Java Bean used to construct {@link UtilTable} of {@MetricRecord} + */ +@Data +@EqualsAndHashCode(callSuper = true) +public class MetricRecord extends UtilRecord { + private String metric_name; + private Long metric_value; + private String tags; + + public MetricRecord(@NotEmpty final String applicationId, + @NotEmpty final String jobName, + final long jobStartTimestamp, + final long timestamp, + @NotEmpty final String metricName, + final long metricValue, + @NotEmpty final String tags) { + super(applicationId, jobName, jobStartTimestamp, timestamp); + this.metric_name = metricName; + this.metric_value = metricValue; + this.tags = tags; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java new file mode 100644 index 0000000..13f8510 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java @@ -0,0 +1,41 @@ +/* + * 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.dataset; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; + +/** + * {@link UtilRecord} is the member type of {@link UtilTable} collections. + * Subclasses of {@link UtilRecord} must conform to the requirements of a + * simple Java Bean so they can be converted to {@link org.apache.spark.sql.Dataset}, + * which are: + * 1) Have primitive field types + * 2) Have default values for instance fields + * 3) Have getter and setters for all fields + * 4) Have a constructor with no arguments + */ +@AllArgsConstructor +@Data +public abstract class UtilRecord implements Serializable { + private String application_id; + private String job_name; + private long job_start_timestamp; + private long timestamp; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java new file mode 100644 index 0000000..838228f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java @@ -0,0 +1,96 @@ +/* + * 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.dataset; + +import com.google.common.annotations.VisibleForTesting; +import com.uber.marmaray.utilities.SparkUtil; +import lombok.AllArgsConstructor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; + +import java.io.IOException; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; + +import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT; + +/** + * {@link UtilTable} is represented as a {@link Dataset} of {@link UtilRecord}, + * which is converted from a {@link JavaRDD} of {@link UtilRecord}. We can extend + * the functionality of {@link UtilTable} same as that of {@link Dataset} + */ +@Slf4j +@AllArgsConstructor +public class UtilTable { + private final SparkSession spark; + private final Dataset dataset; + private final Path destPath; + private final boolean isDatePartitioned; + + public UtilTable(@NonNull final Class type, + @NonNull final JavaRDD javaRDD, + @NonNull final Path destPath, + final boolean isDatePartitioned) { + this(type, javaRDD, destPath, isDatePartitioned, SparkUtil.getOrCreateSparkSession()); + } + + public UtilTable(@NonNull final Class type, + @NonNull final JavaRDD javaRDD, + @NonNull final Path destPath, + final boolean isDatePartitioned, + @NonNull final SparkSession sparkSession) { + this.spark = sparkSession; + final RDD rdd = javaRDD.rdd(); + final Encoder bean = Encoders.bean(type); + this.dataset = this.spark.createDataset(rdd, bean); + this.destPath = destPath; + this.isDatePartitioned = isDatePartitioned; + } + + public void writeParquet() throws IOException { + // TODO: Consider having a configuration to limit number records written out + this.dataset.write().mode(SaveMode.Append).parquet(getDestWritePath().toString()); + } + + public Long size() { + return this.dataset.count(); + } + + @VisibleForTesting + public void show() { + this.dataset.show(); + } + + public Path getDestWritePath() { + return this.isDatePartitioned ? getDestDatePartitionedPath() : this.destPath; + } + + private Path getDestDatePartitionedPath() { + final ZonedDateTime date = ZonedDateTime.now(ZoneOffset.UTC); + final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(DATE_PARTITION_FORMAT); + return new Path(this.destPath, date.format(formatter)); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java new file mode 100644 index 0000000..f85e953 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java @@ -0,0 +1,31 @@ +/* + * 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.exceptions; + +/** + * Thrown when ForkOperation fails to succeed. + */ +public class ForkOperationException extends JobRuntimeException { + + public ForkOperationException(final String message) { + super(message); + } + + public ForkOperationException(final String message, final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java new file mode 100644 index 0000000..6c2c441 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.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.exceptions; + +import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter; +import lombok.NonNull; +import org.apache.avro.generic.GenericRecord; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * It is a checked exception and should be thrown when there is either missing or invalid user defined field in + * data. Check {@link HoodieSinkDataConverter#getKey(GenericRecord)} + * for an example. + */ +public class InvalidDataException extends Exception { + + public InvalidDataException(@NotEmpty final String message) { + super(message); + } + + public InvalidDataException(@NotEmpty final String message, @NonNull final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java new file mode 100644 index 0000000..aa29687 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.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.exceptions; + +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * This is the parent runtime exception thrown whenever job encounters unrecoverable exception. + */ + +public class JobRuntimeException extends RuntimeException { + public JobRuntimeException(@NotEmpty final String message) { + super(message); + } + + public JobRuntimeException(@NonNull final Throwable t) { + super(t); + } + + public JobRuntimeException(@NotEmpty final String message, @NonNull final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java new file mode 100644 index 0000000..16aaefe --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java @@ -0,0 +1,30 @@ +/* + * 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.exceptions; + +/** + * This exception is thrown anytime we encounter an exception with reading or writing metadata + */ +public class MetadataException extends JobRuntimeException { + public MetadataException(final String message) { + super(message); + } + + public MetadataException(final String message, final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java new file mode 100644 index 0000000..3fe938a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java @@ -0,0 +1,31 @@ +/* + * 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.exceptions; + +/** + * Thrown when any required property for a module is not specified. + */ + +public class MissingPropertyException extends JobRuntimeException { + public MissingPropertyException(final String propertyName) { + super("property:" + propertyName); + } + + public MissingPropertyException(final String message, final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java new file mode 100644 index 0000000..ecaea6b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java @@ -0,0 +1,30 @@ +/* + * 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.exceptions; + +/** + * This exception is thrown when the task runs out of retries. + */ +public class RetryException extends JobRuntimeException { + public RetryException(final String message) { + super(message); + } + + public RetryException(final String message, final Throwable t) { + super(message, t); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java new file mode 100644 index 0000000..fa90ad5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java @@ -0,0 +1,47 @@ +/* + * 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.forkoperator; + +import com.uber.marmaray.common.data.ForkData; +import lombok.AllArgsConstructor; +import org.apache.spark.api.java.function.Function; + +/** + * This is used for filtering result RDD. The passed in filterkey should be same as the one used + * in ForkFunction. + * @param + */ +@AllArgsConstructor +public class FilterFunction implements Function, Boolean> { + + private final Integer filterKey; + + @Override + public final Boolean call(final ForkData forkData) { + return execute(forkData); + } + + /** + * It is used for filtering out tupleEntries. If it returns true then tupleEntry will be + * filtered out. It will have same set of keys as defined by corresponding ForkFunction. + * + * @param forkData : forkData to be filtered out or retained. + */ + protected Boolean execute(final ForkData forkData) { + return forkData.getKeys().contains(this.filterKey); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java new file mode 100644 index 0000000..11630c2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java @@ -0,0 +1,94 @@ +/* + * 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.forkoperator; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.data.ForkData; +import com.uber.marmaray.common.exceptions.ForkOperationException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.util.LongAccumulator; + +/** + * It associates every record with a set of fork-keys. This is an abstract class and subclass of + * it should implement {@link ForkFunction#process(Object)} method. All the keys used by + * {@link ForkFunction} should be registered by calling {@link ForkFunction#registerKeys(List)}. If + * any non registered key is returned by {@link ForkFunction#process(Object)} then it will + * raise {@link ForkOperationException}. + * @param + */ +@Slf4j +public abstract class ForkFunction implements FlatMapFunction> { + + private final Map> registeredKeys = new HashMap<>(); + + public void registerKeys(final List keys) { + keys.stream().forEach(key -> { + if (this.registeredKeys.containsKey(key)) { + throw new ForkOperationException("Duplicate key found :" + key); + } + this.registeredKeys.put(key, Optional.absent()); + }); + } + + public long getRecordCount(final int key) { + final Optional keyCount = this.registeredKeys.get(key); + final long ret = keyCount.isPresent() ? keyCount.get().count() : 0; + log.info("{} : key :{}: count :{}", this.getClass().getName(), key, ret); + return ret; + } + + public void registerAccumulators(@NonNull final SparkContext sparkContext) { + this.registeredKeys.entrySet().forEach( + entry -> { + entry.setValue(Optional.of(sparkContext.longAccumulator())); + } + ); + } + + @Override + public final Iterator> call(final DI di) { + final List> forkData = process(di); + forkData.stream().forEach(fd -> verifyKeys(fd.getKeys(), di)); + return forkData.iterator(); + } + + private void verifyKeys(final List keys, final DI di) { + keys.stream().forEach(key -> { + if (!this.registeredKeys.containsKey(key)) { + log.error("Invalid key:{}: in keys:{}:for record:{}", key, keys, di); + throw new ForkOperationException("Using unregistered key :" + key); + } + this.registeredKeys.get(key).get().add(1); + }); + } + + /** + * This method should be implemented by subclass. This method is suppose to associate every + * record with set of registered keys. + * @return {@link ForkData} with set of fork keys which should be associated with the record. + * @param record element from {@link ForkOperator#inputRDD}. + * @throws Exception + */ + protected abstract List> process(final DI record); +} 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 new file mode 100644 index 0000000..b5f4689 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java @@ -0,0 +1,113 @@ +/* + * 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.forkoperator; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.data.ForkData; +import com.uber.marmaray.common.exceptions.ForkOperationException; +import com.uber.marmaray.utilities.SparkUtil; +import com.uber.marmaray.utilities.listener.SparkJobTracker; +import lombok.Getter; +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.storage.RDDInfo; +import org.apache.spark.storage.StorageLevel; + +import java.io.Serializable; +import java.util.List; + +/** + * {@link ForkOperator} is to be used when single RDD needs to be sent to multiple forked + * pipelines. Each forked pipeline is associated with unique integer key and check + * {@link ForkFunction#registerKeys(List)} more details. It calls {@link ForkFunction} for every + * record and expects it to associate it with set of forked pipeline keys. Once entire + * {@link ForkOperator#inputRDD}; its result is persisted using persistence level provided. If + * nothing is specified then it will use {@link ForkOperator#DEFAULT_PERSIST_LEVEL}. Make sure to + * call {@link ForkOperator#execute()} before calling {@link ForkOperator#getRDD(int)} or {@link #getCount(int)}. + * Once all the data is retrieved call {@link ForkOperator#close()} to unpersist the result RDD. + * @param + */ +@Slf4j +public class ForkOperator implements Serializable { + + public static final String PERSIST_LEVEL = Configuration.MARMARAY_PREFIX + "fork.persist_level"; + public static final String DEFAULT_PERSIST_LEVEL = "DISK_ONLY"; + + private final JavaRDD inputRDD; + private final ForkFunction forkFunction; + private Optional>> groupRDD = Optional.absent(); + @Getter + private final StorageLevel persistLevel; + + public ForkOperator(@NonNull final JavaRDD inputRDD, @NonNull final ForkFunction forkFunction, + @NonNull final Configuration conf) { + this.inputRDD = inputRDD; + this.forkFunction = forkFunction; + this.persistLevel = StorageLevel + .fromString(conf.getProperty(PERSIST_LEVEL, DEFAULT_PERSIST_LEVEL)); + } + + public final void execute() { + this.forkFunction.registerAccumulators(this.inputRDD.rdd().sparkContext()); + // Converts JavaRDD -> JavaRDD, T> + JavaRDD> forkedData = this.inputRDD.flatMap(this.forkFunction) + .persist(this.persistLevel); + final String jobName = SparkJobTracker.getJobName(this.inputRDD.rdd().sparkContext()); + forkedData.setName(String.format("%s-%s", jobName, forkedData.id())); + // deliberately calling count so that DAG gets executed. + final long processedRecords = forkedData.count(); + final Optional rddInfo = SparkUtil.getRddInfo(forkedData.context(), forkedData.id()); + log.info("#processed records :{} name:{}", processedRecords, forkedData.name()); + if (rddInfo.isPresent()) { + final long size = rddInfo.get().diskSize() + rddInfo.get().memSize(); + log.info("rddInfo -> name:{} partitions:{} size:{}", forkedData.name(), rddInfo.get().numPartitions(), + size); + } + this.groupRDD = Optional.of(forkedData); + } + + public long getCount(final int filterKey) { + return this.forkFunction.getRecordCount(filterKey); + } + + public JavaRDD getRDD(final int filterKey) { + final long count = getCount(filterKey); + log.info("#records for :{} = {}", filterKey, count); + if (count > 0) { + return getRDD(new FilterFunction<>(filterKey)); + } else { + return (new JavaSparkContext(inputRDD.rdd().sparkContext())).emptyRDD(); + } + } + + public JavaRDD getRDD(final FilterFunction filterFunction) { + if (!this.groupRDD.isPresent()) { + throw new ForkOperationException("No RDD is found"); + } + return this.groupRDD.get().filter(filterFunction).map(record -> record.getRecord()); + } + + public void close() { + if (this.groupRDD.isPresent()) { + this.groupRDD.get().unpersist(); + this.groupRDD = Optional.absent(); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java new file mode 100644 index 0000000..e21d2f9 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java @@ -0,0 +1,35 @@ +/* + * 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.IPayload; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import org.apache.spark.api.java.JavaRDD; + +/** + * Helper class to pass Payload to child dag. + */ +@AllArgsConstructor +public class DagPayload implements IPayload> { + + @NonNull + @Getter + private final JavaRDD data; +} 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 new file mode 100644 index 0000000..504d299 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java @@ -0,0 +1,88 @@ +/* + * 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.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metadata.JobManagerMetadataTracker; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import scala.Tuple2; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.TimeUnit; + +@Slf4j +/** + * {@link ExecutionTimeJobExecutionStrategy} is a {@link IJobExecutionStrategy} that sorts the {@link JobDag}s by + * execution time and latest completion, so DAGs that have not finished will run first, followed by jobs sorted by + * execution time descending. + * This order should help prevent individual long-running dags from keeping the execution from completing on time. + */ +public class ExecutionTimeJobExecutionStrategy implements IJobExecutionStrategy { + + public static final int DEFAULT_LAST_EXECUTION_TIME_THRESHOLD_HOURS = 6; + + private final int lastExecutionTimeThresholdHours; + private final long currentTime; + + @NonNull + private final JobManagerMetadataTracker tracker; + + public ExecutionTimeJobExecutionStrategy(@NonNull final JobManagerMetadataTracker tracker) { + this(tracker, DEFAULT_LAST_EXECUTION_TIME_THRESHOLD_HOURS); + } + + public ExecutionTimeJobExecutionStrategy(@NonNull final JobManagerMetadataTracker tracker, + final int lastExecutionTimeThresholdHours) { + this.lastExecutionTimeThresholdHours = lastExecutionTimeThresholdHours; + this.tracker = tracker; + this.currentTime = System.currentTimeMillis(); + } + + @Override + public List sort(@NonNull final Queue inputJobDags) { + final List result = new ArrayList<>(inputJobDags.size()); + final long lastExecutionTimeThresholdMillis = TimeUnit.HOURS.toMillis(this.lastExecutionTimeThresholdHours); + inputJobDags.stream().map(dag -> { + try { + final Optional> contents = this.tracker.get(dag.getDataFeedName()); + if (contents.isPresent() && contents.get().containsKey(JobDag.LAST_RUNTIME_METADATA_KEY)) { + long lastExecutionTime = contents.get().containsKey(JobDag.LAST_EXECUTION_METADATA_KEY) + ? Long.parseLong(contents.get().get(JobDag.LAST_EXECUTION_METADATA_KEY)) + : Long.MIN_VALUE; + if (this.currentTime - lastExecutionTime > lastExecutionTimeThresholdMillis) { + return new Tuple2<>(dag, Long.MAX_VALUE); + } + return new Tuple2<>(dag, Long.valueOf(contents.get().get(JobDag.LAST_RUNTIME_METADATA_KEY))); + } else { + return new Tuple2<>(dag, Long.MAX_VALUE); + } + } catch (IOException e) { + throw new JobRuntimeException(String.format( + "Unable to get metadata for dag %s : ", dag.getDataFeedName()), e); + } + }).sorted((o1, o2) -> o2._2().compareTo(o1._2())) + .forEach(tuple -> 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 new file mode 100644 index 0000000..6f82fd5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java @@ -0,0 +1,33 @@ +/* + * 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 lombok.NonNull; + +import java.util.List; +import java.util.Queue; + +/** + * {@link IJobExecutionStrategy} determines the order for {@link JobDag} execution rather than just relying on + * submission order. + */ +public interface IJobExecutionStrategy { + + List sort(@NonNull final Queue inputJobDags); + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java b/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java new file mode 100644 index 0000000..78fc5e6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java @@ -0,0 +1,46 @@ +/* + * 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.configuration.Configuration; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.AllArgsConstructor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@AllArgsConstructor +public class Job { + + @NonNull + private final JobDag jobDag; + @NonNull + private final Configuration conf; + + public void run() { + final AtomicBoolean isSuccess = new AtomicBoolean(true); + try { + ThreadPoolService.init(this.conf); + jobDag.execute(); + } catch (final Throwable t) { + isSuccess.set(false); + throw t; + } finally { + ThreadPoolService.shutdown(!isSuccess.get()); + } + } +} 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 new file mode 100644 index 0000000..7f5f1ce --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java @@ -0,0 +1,189 @@ +/* + * 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.common.AvroPayload; +import com.uber.marmaray.common.actions.IJobDagAction; +import com.uber.marmaray.common.actions.JobDagActions; +import com.uber.marmaray.common.actions.ReporterAction; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metadata.AbstractValue; +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.JobMetrics; +import com.uber.marmaray.common.metrics.LongMetric; +import com.uber.marmaray.common.metrics.TimerMetric; +import com.uber.marmaray.common.reporters.Reporters; +import com.uber.marmaray.common.sinks.ISink; +import com.uber.marmaray.common.sources.IRunState; +import com.uber.marmaray.common.sources.ISource; +import com.uber.marmaray.common.sources.IWorkUnitCalculator; +import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +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; + +import java.io.IOException; +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> { + + public static final String LAST_RUNTIME_METADATA_KEY = "runtime"; + public static final String LAST_EXECUTION_METADATA_KEY = "last_execution"; + @NonNull + private final ISource source; + @NonNull + private final JobSubDag sinkDag; + @NonNull + 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; + + @NonNull + private final JobMetrics jobMetrics; + @Getter + private final DataFeedMetrics dataFeedMetrics; + + 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 Reporters reporters) { + this(source, new SingleSinkSubDag(sink), metadataManager, + workUnitCalculator, jobName, dataFeedName, jobMetrics, reporters); + } + + public JobDag(@NonNull final ISource source, + @NonNull final JobSubDag sinkDag, + @NonNull final IMetadataManager metadataManager, + @NonNull final IWorkUnitCalculator workUnitCalculator, + @NotEmpty final String jobName, + @NotEmpty final String dataFeedName, + @NonNull final JobMetrics jobMetrics, + @NonNull final Reporters reporters) { + 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)); + } + + /** + * Add an action on success + * @param action to execute after job is successful + */ + public void addAction(final IJobDagAction action) { + this.postJobDagActions.addAction(action); + } + + public void execute() { + log.info("Starting job dag for {}", this.jobName); + 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()); + try { + // set up metrics for downstreams + Arrays.asList(this.workUnitCalculator, this.sinkDag, this.source).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(); + log.info("Work unit calculator result :{}", workUnitCalculatorResult); + // save run state for next processing + this.workUnitCalculator.saveNextRunState(this.metadataManager, workUnitCalculatorResult.getNextRunState()); + if (workUnitCalculatorResult.hasWorkUnits()) { + // read source rdd. + final JavaRDD sourceRDD = this.source.getData(workUnitCalculatorResult); + + // execute sink dag. + this.sinkDag.execute(Optional.of(new DagPayload(sourceRDD))); + + // commit sink dag + this.sinkDag.commit(); + } + + try { + this.metadataManager.saveChanges(); + } catch (IOException e) { + final String msg = "Failed to save metadata changes" + e.getMessage(); + log.error(msg, e); + throw new JobRuntimeException(msg, e); + } + } catch (Exception e) { + // set failure status if there was an error + successful.set(false); + throw e; + } finally { + // execute all actions at the last minute + timerMetric.stop(); + reportStatus(successful.get()); + if (getJobManagerMetadata() != null && successful.get()) { + this.getJobManagerMetadata().put(LAST_RUNTIME_METADATA_KEY, timerMetric.getMetricValue().toString()); + this.getJobManagerMetadata().put(LAST_EXECUTION_METADATA_KEY, + String.valueOf(TimeUnit.SECONDS.toMillis(timerMetric.getStartTime().getEpochSecond()))); + } + this.dataFeedMetrics.createLongMetric(DataFeedMetricNames.RESULT, + successful.get() ? DataFeedMetricNames.RESULT_SUCCESS : DataFeedMetricNames.RESULT_FAILURE, + Collections.emptyMap()); + this.postJobDagActions.execute(successful.get()); + } + } + + private void reportStatus(final boolean successful) { + final long statusValue = + successful ? DataFeedMetricNames.RESULT_SUCCESS : DataFeedMetricNames.RESULT_FAILURE; + final LongMetric successMetric = new LongMetric( + DataFeedMetricNames.RESULT, statusValue); + successMetric.addTags(this.dataFeedMetrics.getBaseTags()); + this.reporters.report(successMetric); + } +} 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 new file mode 100644 index 0000000..dc46233 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java @@ -0,0 +1,368 @@ +/* + * 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.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; +import com.uber.marmaray.common.metrics.JobMetricNames; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.metrics.TimerMetric; +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.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.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; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * JobManager + * + * This class is responsible for running the entire spark application that will be created. It schedules an runs + * the {@link JobDag}s that are registered in the application, as well as any {@link IJobDagAction}s that are + * registered. + * + * This class is also responsible for the {@link JavaSparkContext}. + */ +@Slf4j +public final class JobManager { + + public static final boolean DEFAULT_LOCK_FREQUENCY = true; + + private static JobManager instance; + private static final Object lock = new Object(); + + @NonNull + 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 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 @Setter + private JobManagerMetadataTracker tracker; + + @Getter @Setter + private boolean jobManagerMetadataEnabled; + + @Setter + private Optional jobExecutionStrategy = Optional.absent(); + + private JobManager(@NonNull final Configuration conf, @NotEmpty final String appName, + @NotEmpty final String frequency, final boolean shouldLockFrequency) { + this.conf = conf; + this.appName = appName; + this.jobMetrics = new JobMetrics(appName); + this.reporters = new Reporters(); + this.reporters.addReporter(new ConsoleReporter()); + this.jobLockManager = new JobLockManager(conf, frequency, shouldLockFrequency); + this.postJobManagerActions = new JobDagActions(this.reporters, "jobManager"); + } + + /** + * 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 lockFrequency whether the frequency should be locked + */ + public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName, + @NotEmpty final String frequency, final boolean lockFrequency) { + synchronized (lock) { + Preconditions.checkState(instance == null, + "JobManager was already created"); + instance = new JobManager(conf, appName, frequency, lockFrequency); + } + return instance; + } + + /** + * 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 + */ + public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName, + @NotEmpty final String frequency) { + return createJobManager(conf, appName, frequency, DEFAULT_LOCK_FREQUENCY); + } + + /** + * 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 + */ + 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(); + } + + /** + * Execute all registered {@link JobDag}, then perform all registered {@link IJobDagAction} + */ + public void run() { + 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 boolean hasMultipleDags = this.jobDags.size() > 1; + final Queue runtimeJobDagOrder; + if (hasMultipleDags && this.jobExecutionStrategy.isPresent()) { + runtimeJobDagOrder = new ConcurrentLinkedDeque<>(this.jobExecutionStrategy.get().sort(this.jobDags)); + } else { + runtimeJobDagOrder = this.jobDags; + } + try { + ThreadPoolService.init(this.conf); + runtimeJobDagOrder.forEach(jobDag -> + futures.add(ThreadPoolService.submit( + () -> { + SparkJobTracker.setJobName(sc.sc(), jobDag.getDataFeedName()); + if (hasMultipleDags) { + setSparkStageName(sc, jobDag.getDataFeedName()); + } + jobDag.execute(); + return null; + }, ThreadPoolServiceTier.JOB_DAG_TIER))); + TimeoutManager.getInstance().startMonitorThread(); + futures.forEach(future -> { + try { + future.get(); + } catch (Exception e) { + log.error("Error running job", e); + isSuccess.set(false); + } + } + ); + + if (!isSuccess.get()) { + throw new JobRuntimeException("Error while running job. Look at previous log entries for detail"); + } + } catch (final Throwable t) { + isSuccess.set(false); + 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(); + this.reporters.getReporters().forEach(IReporter::finish); + } + } + + /** + * Add {@link JobDag} to be executed on {@link #run()} + * @param jobDag JobDag to be added + */ + public void addJobDag(@NonNull final JobDag jobDag) { + if (jobLockManager.lockDag(jobDag.getJobName(), jobDag.getDataFeedName())) { + this.jobDags.add(jobDag); + } else { + log.warn("Failed to obtain lock for JobDag {} - {}", jobDag.getJobName(), jobDag.getDataFeedName()); + } + } + + /** + * Add collection of {@link JobDag} to be executed on {@link #run()} + * @param jobDags collection of JobDags to be added + */ + public void addJobDags(@NonNull final Collection jobDags) { + jobDags.forEach(this::addJobDag); + } + + /** + * Add {@link IJobDagAction} to be executed after all {@link JobDag} have completed + * @param action action to add + */ + public void addPostJobManagerAction(@NonNull final IJobDagAction action) { + this.postJobManagerActions.addAction(action); + } + + /** + * Add collection of {@link IJobDagAction} to be executed after all {@link JobDag} have completed + * @param actions action to add + */ + public void addPostJobManagerActions(@NonNull final Collection actions) { + actions.forEach(this::addPostJobManagerAction); + } + + /** + * Add schema for registration into {@link JavaSparkContext}. Must not be called after the JavaSparkContext has been + * created. + * @param schema schema to register into spark context + */ + public void addSchema(@NonNull final Schema schema) { + Preconditions.checkState(!this.sparkContext.isPresent(), + "Schemas must be added before sparkContext is instantiated"); + this.schemas.add(schema); + } + + /** + * Add serialization classes for registration into {@link JavaSparkContext}. Must not be called after the + * {@link JavaSparkContext} have been created. + * @param serializationClasses serialization classes + */ + public void addSerializationClasses(@NonNull final List serializationClasses) { + Preconditions.checkState(!this.sparkContext.isPresent(), + "Serialization classes must be added before sparkContext is instantiated"); + this.serializationClasses.addAll(serializationClasses); + } + + /** + * 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 static void setSparkStageName(@NonNull final JavaSparkContext jsc, @NotEmpty final String dataFeedName) { + // For now we will only set stageName as "dataFeedName" but long term we would want to also include spark's + // action name in it; which will need support from spark. + jsc.setCallSite(dataFeedName); + } + + private final class JobLockManager { + private static final String MANAGER_LOCK_KEY = "JOBMANAGER"; + private static final String DAG_LOCK_KEY = "JOBDAGS"; + + private static final String JOB_FREQUENCY_TAG = "job_frequency"; + private static final String JOB_NAME_TAG = "job_name"; + private static final String DATA_FEED_TAG = "data_feed_name"; + + @NonNull + private final LockManager lockManager; + @NonNull + private final String jobFrequency; + + @NonNull + private final TimerMetric managerTimerMetric; + @NonNull + private final HashMap dagTimerMetricMap; + + private JobLockManager(@NonNull final Configuration conf, @NotEmpty final String frequency, + final boolean shouldLockFrequency) { + this.lockManager = new LockManager(conf); + this.jobFrequency = frequency; + + final String key = LockManager.getLockKey(MANAGER_LOCK_KEY, jobFrequency); + this.managerTimerMetric = new TimerMetric(JobMetricNames.JOB_MANAGER_LOCK_TIME_MS, + ImmutableMap.of(JOB_FREQUENCY_TAG, jobFrequency, + JOB_NAME_TAG, appName)); + + if (shouldLockFrequency) { + final boolean success = lockManager.lock(key, + String.format("JobManager %s AppId %s", jobFrequency, appId)); + this.managerTimerMetric.stop(); + if (!success) { + lockManager.close(); + throw new IllegalStateException("Failed to obtain lock for JobManager " + jobFrequency); + } + } else { + managerTimerMetric.stop(); + log.info("Frequency lock disabled"); + } + this.dagTimerMetricMap = new HashMap<>(); + } + + private boolean lockDag(@NotEmpty final String jobName, @NotEmpty final String dagName) { + final String key = LockManager.getLockKey(DAG_LOCK_KEY, dagName); + 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)); + final boolean success = lockManager.lock(key, + String.format("JobDag %s AppId %s", dagName, appId)); + timerMetric.stop(); + dagTimerMetricMap.put(dagName, timerMetric); + return success; + } + + private void stop() { + log.info("Closing the LockManager in the JobManager."); + this.lockManager.close(); + reporters.report(managerTimerMetric); + dagTimerMetricMap.forEach((dagName, timerMetric) -> reporters.report(timerMetric)); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java new file mode 100644 index 0000000..edf6ec3 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java @@ -0,0 +1,236 @@ +/* + * 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.google.common.base.Preconditions; +import com.uber.marmaray.common.IPayload; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.metrics.JobMetrics; +import lombok.Data; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * {@link JobSubDag} is useful when we need to run custom job dag which can be represented in the form of an n-way + * tree. User should implement {@link #executeNode(Optional)}, {@link #commitNode()} and + * {@link #getDataForChild(JobSubDag)} for every node in the jobDag. The execution sequence is as follows. + * Consider an example Dag. + * # Note :- Numbers in the bracket denote their priority. Priority is only compared between children of same parent + * # node. + * # N1 + * # / \ + * # / \ + * # (0)N2 (1)N3 + * # / + * # / + * # (0)N4 + * Caller should call below methods for complete execution of the above dag. [N1.execute() and N1.commit()]. + * This is how the job dag will get executed. + * 1) N1.execute() <- user calls this. + * 1-a) N1.executeNode() [Step1] + * 2-a) N2.execute() [Step2] + * 2-a-a) N2.executeNode() [Step3] + * 2-a-b) N4.execute() [Step4] + * 2-a-b-a) N4.executeNode() [Step5] + * 2-a) N3.execute() [Step2] + * 2-a-a) N3.executeNode() [Step3] + * // It will wait for all tasks to finish. + * 2) N1.commit() <- user calls this. + * 2-a) N2.commit() [Step1] + * 2-a-a) N4.commit() [Step2] + * 2-a-b) N4.commitNode() [Step3] + * 2-a-c) N2.commitNode() [Step4] + * 2-b) N3.commit() [Step5] + * 2-b-a) N3.commitNode() [Step6] + * 2-c) N1.commitNode() [Step7] + */ +@Slf4j +public abstract class JobSubDag implements IMetricable { + + @NotEmpty + @Getter + // Logical name of the job node. + private final String name; + + @Getter + private Optional jobMetrics = Optional.absent(); + + @Getter + private Optional dataFeedMetrics = Optional.absent(); + + /** + * Holds child nodes with their priority. ChildNodes are committed from lower to higher priority. All nodes at the + * same priority are committed in parallel (where ordering is not guaranteed). However + * {@link #executeNode(Optional)} method for all child nodes is called in parallel. + */ + private final TreeMap> childNodes = new TreeMap<>(); + + protected JobSubDag(@NotEmpty final String name) { + this.name = name; + } + + public void addSubDag(final int priority, @NonNull final JobSubDag subDag) { + if (!this.childNodes.containsKey(priority)) { + this.childNodes.put(priority, new LinkedList<>()); + } + this.childNodes.get(priority).add(subDag); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + this.jobMetrics = Optional.of(jobMetrics); + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + /** + * Should pre process input and create appropriate {@link IPayload} for child nodes. Later when childNode's + * {@link #execute(Optional)} method is called it will call {@link #getDataForChild(JobSubDag)} to retrieve payload + * for it. + * + * @param data input payload + */ + protected abstract void executeNode(@NonNull final Optional data); + + /** + * An optional method which user can implement. Ideally this should be used to call atomic operations. + */ + protected void commitNode() { + } + + /** + * Called to retrieve payload for child dag. + * + * @param childSubDag childSubDag for which {@link IPayload} data needs to be retrieved + * @return payload for child dag + */ + protected Optional getDataForChild(@NonNull final JobSubDag childSubDag) { + return Optional.absent(); + } + + /** + * Will execute current node's {@link #executeNode(Optional)} followed by parallel execution of {@link + * #childNodes}'s {@link #execute(Optional)} method. + */ + public final void execute(@NonNull final Optional data) { + Preconditions.checkState(this.dataFeedMetrics.isPresent() && this.jobMetrics.isPresent(), + "Missing dataFeed or job metrics"); + log.info("running : executeNode {}", this.name); + // first call Current node's executeNode(). + executeNode(data); + + // setup metrics for child nodes. + setupChildMetrics(); + + final Queue statuses = new LinkedList<>(); + this.childNodes.entrySet().stream().forEach( + childNodesAtSamePriority -> { + childNodesAtSamePriority.getValue().stream().forEach( + childNode -> statuses.add( + new SubDagExecutionStatus(childNode, + ThreadPoolService.submit( + () -> { + childNode.execute(getDataForChild(childNode)); + return 0; + }, ThreadPoolServiceTier.JOB_DAG_TIER + ))) + ); + } + ); + waitForTasksToFinish(statuses); + } + + private void setupChildMetrics() { + + this.childNodes.entrySet().stream().forEach( + entry -> entry.getValue().stream().forEach( + jobSubDag -> { + jobSubDag.setJobMetrics(this.jobMetrics.get()); + jobSubDag.setDataFeedMetrics(this.dataFeedMetrics.get()); + } + )); + } + + // Helper method to wait for parallel tasks to finish execution. + private void waitForTasksToFinish(@NonNull final Queue statuses) { + while (!statuses.isEmpty()) { + final SubDagExecutionStatus status = statuses.poll(); + while (!status.getStatus().isDone()) { + try { + status.getStatus().get(); + } catch (InterruptedException e) { + log.error("interrupted {} {}", status.getSubDag().getName(), e); + throw new JobRuntimeException("dag execution interrupted", e); + } catch (ExecutionException e) { + log.error("failed to execute subdag {} {}", status.getSubDag().getName(), e.getCause()); + throw new JobRuntimeException("failed to execute subDag", e.getCause()); + } + } + } + } + + /** + * Will execute childNode's {@link #commitNode()} method from lower priority to higher priority. + * {@link #commitNode()} method of the nodes at same priority level will get executed in parallel. + */ + public final void commit() { + log.info("calling {}'s childNodes' commit", this.name); + this.childNodes.entrySet().stream().forEach( + childNodesAtSamePriority -> { + final Queue statuses = new LinkedList<>(); + childNodesAtSamePriority.getValue().stream().forEach( + childNode -> statuses.add( + new SubDagExecutionStatus(childNode, + ThreadPoolService.submit( + () -> { + childNode.commit(); + return 0; + }, ThreadPoolServiceTier.JOB_DAG_TIER + ))) + ); + waitForTasksToFinish(statuses); + } + ); + log.info("calling {}'s commitNode", this.name); + commitNode(); + } + + /** + * Helper class to wrap {@link JobSubDag} with it's run status. + */ + @Data + class SubDagExecutionStatus { + + private final JobSubDag subDag; + private final Future status; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java new file mode 100644 index 0000000..c8482f2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java @@ -0,0 +1,58 @@ +/* + * 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.google.common.base.Preconditions; +import com.uber.marmaray.common.IPayload; +import com.uber.marmaray.common.sinks.ISink; +import javax.annotation.Nonnull; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +/** + * {@link SingleSinkSubDag} is used when {@link JobDag} is configured to run for single + * {@link com.uber.marmaray.common.sinks.ISink}. + */ +@Slf4j +public class SingleSinkSubDag extends JobSubDag { + + @NonNull + @Getter + private final ISink sink; + + public SingleSinkSubDag(@Nonnull final ISink sink) { + super(String.format("%s:%s", SingleSinkSubDag.class.getName(), sink.getClass().getName())); + this.sink = sink; + } + + @Override + protected void executeNode(@NonNull final Optional data) { + Preconditions.checkState(data.isPresent() && (data.get() instanceof DagPayload), + "Invalid payload :" + (data.isPresent() ? data.get().getClass() : null)); + + // setup job and topic metrics. + setupMetrics(); + this.sink.write(((DagPayload) data.get()).getData()); + } + + private void setupMetrics() { + this.sink.setJobMetrics(getJobMetrics().get()); + this.sink.setDataFeedMetrics(getDataFeedMetrics().get()); + } +} 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 new file mode 100644 index 0000000..8821a28 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java @@ -0,0 +1,424 @@ +/* + * 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.google.common.base.Preconditions; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.util.Queue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Helper class to manage thread pool service. This is a singleton service and number of threads are controlled by + * {@link #NUM_THREADS} configuration. + */ +@Slf4j +public final class ThreadPoolService { + + public static final String THREAD_POOL_SERVICE_PREFIX = Configuration.MARMARAY_PREFIX + "thread_pool."; + public static final String NUM_THREADS = THREAD_POOL_SERVICE_PREFIX + "num_threads"; + public static final String JOB_DAG_THREADS = THREAD_POOL_SERVICE_PREFIX + "job_dag_threads"; + public static final String ACTIONS_THREADS = THREAD_POOL_SERVICE_PREFIX + "actions_threads"; + + public static final int DEFAULT_NUM_THREADS = 8; + public static final int AWAIT_TERMINATION_ATTEMPTS = 30; + public static final int AWAIT_TERMINATION_CHECK_INTERVAL_MS = 1000; + public static final int DEFAULT_JOB_DAG_THREADS = 6; + public static final int DEFAULT_ACTIONS_THREADS = 2; + public static final long NO_TIMEOUT = -1; + // Singleton service. + private static Optional service = Optional.absent(); + + @NonNull + private final ExecutorService threadPool; + + private final int reservedJobDagThreads; + private final int reservedActionsThreads; + private final int numThreads; + + private boolean isShutdown = false; + + private final AtomicInteger currentThreads = new AtomicInteger(); + private final AtomicInteger currentJobDagThreads = new AtomicInteger(); + private final AtomicInteger currentActionsThreads = new AtomicInteger(); + + private final Queue jobDagQueue; + private final Queue actionsQueue; + + 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 >= reservedJobDagThreads + reservedActionsThreads, + String.format( + "Total threads must be at least equal to reserved threads: total: %d, jobDag: %d, actions: %d ", + numThreads, reservedJobDagThreads, reservedActionsThreads)); + this.reservedActionsThreads = reservedActionsThreads; + this.reservedJobDagThreads = reservedJobDagThreads; + this.threadPool = Executors.newFixedThreadPool(numThreads); + this.actionsQueue = new ConcurrentLinkedDeque<>(); + this.jobDagQueue = new ConcurrentLinkedDeque<>(); + this.numThreads = numThreads; + } + + /** + * Initialize the ThreadPoolService + * + * @param conf Configuration from which to load the properties. + */ + public static synchronized void init(@NonNull final Configuration conf) { + final int numThreads = conf.getIntProperty(NUM_THREADS, DEFAULT_NUM_THREADS); + final int reservedJobDagThreads = conf.getIntProperty(JOB_DAG_THREADS, + DEFAULT_JOB_DAG_THREADS); + final int reservedActionsThreads = conf.getIntProperty(ACTIONS_THREADS, + DEFAULT_ACTIONS_THREADS); + if (service.isPresent()) { + throw new JobRuntimeException("Re-initializing thread pool service."); + } + service = Optional.of(new ThreadPoolService(numThreads, reservedJobDagThreads, reservedActionsThreads)); + } + + /** + * Submit a callable to the pool, in the correct tier. + * + * @param callable callable to submit + * @param tier tier to submit to + * @return Future tied to the callable's execution + */ + public static Future submit(@NonNull final Callable callable, @NonNull final ThreadPoolServiceTier tier) { + return submit(callable, tier, NO_TIMEOUT); + } + + /** + * Submit a callable to the pool, in the correct tier. + * Callable has a timeout of timeoutSeconds, which starts once the callable has been added to the pool. + * + * @param callable callable to submit + * @param tier tier to submit to + * @param timeoutSeconds timeout in seconds of the callable + * @return Future tied to the callable's execution + */ + public static synchronized Future submit(@NonNull final Callable callable, + @NonNull final ThreadPoolServiceTier tier, final long timeoutSeconds) { + Preconditions.checkState(service.isPresent(), "No thread pool service is running"); + Preconditions.checkState(!service.get().isShutdown, "Service is shutting down"); + final ThreadPoolService service = ThreadPoolService.service.get(); + final ThreadPoolServiceCallable threadPoolServiceCallable = service + .new ThreadPoolServiceCallable(callable, tier); + final ThreadPoolServiceFuture future = service + .new ThreadPoolServiceFuture(threadPoolServiceCallable, timeoutSeconds, + TimeUnit.SECONDS); + threadPoolServiceCallable.setFuture(future); + synchronized (service) { + if (tier.equals(ThreadPoolServiceTier.JOB_DAG_TIER)) { + if (service.canScheduleJobDag()) { + future.addWrappedFuture(service.scheduleJobDag(threadPoolServiceCallable)); + } else { + service.queueJobDag(future); + } + } else if (tier.equals(ThreadPoolServiceTier.ACTIONS_TIER)) { + if (service.canScheduleAction()) { + future.addWrappedFuture(service.scheduleAction(threadPoolServiceCallable)); + } else { + service.queueAction(future); + } + } else { + throw new JobRuntimeException("Trying to submit to illegal tier " + ThreadPoolServiceTier.JOB_DAG_TIER); + } + } + return future; + } + + /** + * Check if the service is already initialized + * @return true if the service is ready to submit + */ + public static boolean isInitialized() { + return service.isPresent(); + } + + private void queueAction(final ThreadPoolServiceFuture future) { + this.actionsQueue.add(future); + } + + private Future scheduleAction(final Callable callable) { + this.currentActionsThreads.incrementAndGet(); + this.currentThreads.incrementAndGet(); + return this.threadPool.submit(callable); + } + + private boolean canScheduleAction() { + return (this.currentThreads.get() < this.numThreads + // if we have fewer than reserved job dag threads, we can schedule + && (this.currentActionsThreads.get() < this.reservedActionsThreads + // if we have room to schedule a job dag without reaching into the actions reserve, we can schedule + || this.currentActionsThreads.get() < this.numThreads - this.reservedJobDagThreads)); + } + + private void queueJobDag(@NonNull final ThreadPoolServiceFuture future) { + this.jobDagQueue.add(future); + } + + private Future scheduleJobDag(final Callable callable) { + this.currentThreads.incrementAndGet(); + this.currentJobDagThreads.incrementAndGet(); + return this.threadPool.submit(callable); + } + + private boolean canScheduleJobDag() { + return (this.currentThreads.get() < this.numThreads + // if we have fewer than reserved job dag threads, we can schedule + && (this.currentJobDagThreads.get() < this.reservedJobDagThreads + // if we have room to schedule a job dag without reaching into the actions reserve, we can schedule + || this.currentJobDagThreads.get() < this.numThreads - this.reservedActionsThreads)); + } + + /** + * Will wait for all threads to finish their task unless forceShutdown is set to true in which case service will + * be forcefully shutdown. + */ + public static void shutdown(final boolean forceShutdown) { + ThreadPoolService currentService = null; + synchronized (ThreadPoolService.class) { + if (!service.isPresent()) { + return; + } + currentService = service.get(); + if (currentService.isShutdown) { + return; + } + log.info("Shutting down thread pool service"); + currentService.isShutdown = true; + service = Optional.absent(); + } + if (forceShutdown) { + log.error("forcefully shutting down waiting threads"); + currentService.threadPool.shutdownNow(); + } else { + currentService.shutdown(); + } + } + + private synchronized boolean removeFromQueue(@NonNull final ThreadPoolServiceFuture future) { + return this.actionsQueue.remove(future) || this.jobDagQueue.remove(future); + } + + private void shutdown() { + int terminationChecks = 0; + while (terminationChecks < AWAIT_TERMINATION_ATTEMPTS && (!this.jobDagQueue.isEmpty() + || !this.actionsQueue.isEmpty())) { + log.info("waiting for tasks to clear out of queue."); + try { + Thread.sleep(AWAIT_TERMINATION_CHECK_INTERVAL_MS); + terminationChecks += 1; + } catch (InterruptedException e) { + // pass + } + } + this.threadPool.shutdown(); + while (!this.threadPool.isTerminated()) { + try { + log.info("waiting for tasks to stop."); + if (!this.threadPool.awaitTermination( + (AWAIT_TERMINATION_ATTEMPTS - terminationChecks) * AWAIT_TERMINATION_CHECK_INTERVAL_MS, + TimeUnit.MILLISECONDS)) { + this.threadPool.shutdownNow(); + } + } catch (InterruptedException e) { + throw new JobRuntimeException("Error waiting for threadpool to stop", e); + } + } + } + + private synchronized void finishExecution(@NonNull final ThreadPoolServiceTier tier) { + if (tier.equals(ThreadPoolServiceTier.ACTIONS_TIER)) { + if (this.actionsQueue.isEmpty()) { + if (this.jobDagQueue.isEmpty() || !canScheduleJobDag()) { + // nothing else to run, just reduce number of threads + this.currentThreads.decrementAndGet(); + this.currentActionsThreads.decrementAndGet(); + } else { + // can now move a jobDag thread to an actions thread + submitFuture(this.jobDagQueue.poll()); + this.currentActionsThreads.decrementAndGet(); + this.currentJobDagThreads.incrementAndGet(); + } + } else { + submitFuture(this.actionsQueue.poll()); + } + } else if (tier.equals(ThreadPoolServiceTier.JOB_DAG_TIER)) { + if (this.jobDagQueue.isEmpty()) { + if (this.actionsQueue.isEmpty() || !canScheduleAction()) { + // nothing else to run, reduce count of running threads + this.currentThreads.decrementAndGet(); + this.currentJobDagThreads.decrementAndGet(); + } else { + // can now move an actions thread to a jobDag thread + submitFuture(this.actionsQueue.poll()); + this.currentJobDagThreads.decrementAndGet(); + this.currentActionsThreads.incrementAndGet(); + } + } else { + submitFuture(this.jobDagQueue.poll()); + } + } else { + throw new JobRuntimeException(String.format("Attempting to finish illegal tier %s", tier.toString())); + } + } + + private void submitFuture(@NonNull final ThreadPoolServiceFuture future) { + future.addWrappedFuture(this.threadPool.submit(future.getWrappedCallable())); + } + + private final class ThreadPoolServiceFuture implements Future { + + @Getter + private final Callable wrappedCallable; + private Future wrappedFuture = null; + private LinkedBlockingQueue wrappedFutureWaitQ = new LinkedBlockingQueue<>(); + private final long timeout; + private final TimeUnit timeUnit; + private boolean cancelled = false; + private boolean done = false; + + private ThreadPoolServiceFuture(@NonNull final Callable wrappedCallable, final long timeout, + @NonNull final TimeUnit timeUnit) { + this.wrappedCallable = wrappedCallable; + this.timeout = timeout; + this.timeUnit = timeUnit; + } + + public void addWrappedFuture(@NonNull final Future wrappedFuture) { + this.wrappedFutureWaitQ.offer(wrappedFuture); + } + + private void waitForWrappedFuture(final long timeout, + @NonNull final TimeUnit timeUnit) throws TimeoutException { + if (this.wrappedFuture == null) { + try { + this.wrappedFuture = this.wrappedFutureWaitQ.poll(timeout, timeUnit); + if (this.wrappedFuture == null) { + throw new TimeoutException("no wrapped future received"); + } + } catch (InterruptedException e) { + throw new JobRuntimeException(e); + } + } + } + + private void waitForWrappedFuture() { + if (this.wrappedFuture == null) { + try { + this.wrappedFuture = this.wrappedFutureWaitQ.take(); + } catch (InterruptedException e) { + throw new JobRuntimeException(e); + } + } + } + + @Override + public boolean cancel(final boolean mayInterruptIfRunning) { + this.cancelled = removeFromQueue(this); + if (!this.wrappedFutureWaitQ.isEmpty()) { + waitForWrappedFuture(); + } + if (!this.cancelled && this.wrappedFuture != null) { + this.cancelled = this.wrappedFuture.cancel(mayInterruptIfRunning); + } + return this.cancelled; + } + + @Override + public boolean isCancelled() { + return this.cancelled; + } + + /** + * @return true if the operation finished (with or without error) or it was cancelled else false. + */ + @Override + public boolean isDone() { + return this.done || this.cancelled; + } + + @Override + public Object get() throws InterruptedException, ExecutionException { + waitForWrappedFuture(); + if (this.timeout != ThreadPoolService.NO_TIMEOUT) { + try { + return this.wrappedFuture.get(this.timeout, this.timeUnit); + } catch (TimeoutException e) { + throw new JobRuntimeException(e); + } + } else { + return this.wrappedFuture.get(); + } + } + + @Override + public Object get(final long timeout, @NonNull final TimeUnit timeUnit) + throws InterruptedException, ExecutionException, TimeoutException { + final long endTimeMs = System.currentTimeMillis() + timeUnit.toMillis(timeout); + waitForWrappedFuture(timeout, timeUnit); + return this.wrappedFuture.get(Math.max(1, endTimeMs - System.currentTimeMillis()), TimeUnit.MILLISECONDS); + } + } + + private final class ThreadPoolServiceCallable implements Callable { + + private final Callable wrappedCallable; + private final ThreadPoolServiceTier tier; + private ThreadPoolServiceFuture future; + + private ThreadPoolServiceCallable(@NonNull final Callable wrappedCallable, + @NonNull final ThreadPoolServiceTier tier) { + this.wrappedCallable = wrappedCallable; + this.tier = tier; + } + + @Override + public Object call() throws Exception { + try { + final Object result = this.wrappedCallable.call(); + return result; + } finally { + finishExecution(this.tier); + this.future.done = true; + } + } + + private void setFuture(@NonNull final ThreadPoolServiceFuture future) { + this.future = future; + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java new file mode 100644 index 0000000..50e03bb --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java @@ -0,0 +1,22 @@ +/* + * 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; + +public enum ThreadPoolServiceTier { + JOB_DAG_TIER, + ACTIONS_TIER +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java new file mode 100644 index 0000000..3d6a9c0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.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.metadata; + +import java.io.Serializable; + +/** + * {@link AbstractValue} wraps a generic data value type that is required to be serializable. The data represents + * some metadata that will be stored for a job. + * @param + */ +public abstract class AbstractValue implements Serializable { + abstract D getValue(); +} 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 new file mode 100644 index 0000000..18b5c24 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java @@ -0,0 +1,137 @@ +/* + * 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.base.Optional; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.DateUtil; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.FileSystem; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.ZoneId; +import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; + +/** + * {@link HDFSDatePartitionManager} is an extension of {@link HDFSPartitionManager} + * and provides functionality to specifically handle date partitions. + * All date times should be in UTC time and in the format YYYY-MM-DD + */ +@Slf4j +public class HDFSDatePartitionManager extends HDFSPartitionManager { + + private static final String DEFAULT_START_DATE = "1970-01-01"; + private final String partitionKeyName; + private final boolean hasPartitionKeyInHDFSPartitionPath; + 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); + this.partitionKeyName = partitionKeyName + StringTypes.EQUAL; + this.hasPartitionKeyInHDFSPartitionPath = hasPartitionKeyNameInPartition(); + this.startDate = startDate; + log.info("HDFSDatePartitionManager has partitionKey in HDFS path: {}", this.hasPartitionKeyInHDFSPartitionPath); + } + + @Override + public Optional getNextPartition() throws IOException { + + if (this.isSinglePartition()) { + log.info("Next partition: {}", this.rawDataRootPath); + return Optional.of(this.rawDataRootPath); + } else { + final Optional lastCheckPoint = calculateLastCheckpoint(); + + if (lastCheckPoint.isPresent()) { + log.info("Last checkpoint: {}", lastCheckPoint.get().getValue()); + } else { + log.info("No last checkpoint found"); + } + + final LocalDate startDate = getDefaultStartDate(); + + final Optional dt = lastCheckPoint.isPresent() + ? Optional.of(DateUtil.convertToUTCDate( + lastCheckPoint.get().getValue().replace(this.partitionKeyName, StringTypes.EMPTY))) + : Optional.absent(); + + final LocalDate compareDate = !dt.isPresent() || startDate.isAfter(dt.get()) ? startDate : dt.get(); + + final List existingPartitions = listSortedPartitionsAfterDate(compareDate); + if (!existingPartitions.isEmpty()) { + // get first partition after the checkpoint + final String nextPartition = this.hasPartitionKeyInHDFSPartitionPath + ? this.partitionKeyName + existingPartitions.get(0).toString() + : existingPartitions.get(0).toString(); + + log.info("Next partition to process: {}", nextPartition); + return Optional.of(nextPartition); + } else { + log.info("No partitions found to be processed"); + return Optional.absent(); + } + } + } + + private LocalDate getDefaultStartDate() { + final ZoneId UTC = ZoneId.of("Z"); + final LocalDate ld = this.startDate.isPresent() ? this.startDate.get().toInstant().atZone(UTC).toLocalDate() + : LocalDate.parse(DEFAULT_START_DATE); + log.info("Default start date: {}", ld.toString()); + return ld; + } + /** + * Returns the partitions in sorted ascending order which are after the date value + * @param localDate + * @return + * @throws IOException + */ + private List listSortedPartitionsAfterDate(final LocalDate localDate) throws IOException { + final LocalDate startDate = localDate.plusDays(1); + + final List partitions = getExistingPartitions() + .stream() + .map(dt -> DateUtil.convertToUTCDate(dt.replace(this.partitionKeyName, StringTypes.EMPTY))) + .filter(dt -> dt.compareTo(startDate) >= 0) + .collect(Collectors.toList()); + + return partitions; + } + + private boolean hasPartitionKeyNameInPartition() { + try { + final boolean hasPartitionKey = getExistingPartitions() + .stream() + .anyMatch(partition -> partition.startsWith(this.partitionKeyName)); + return hasPartitionKey; + } catch (IOException e) { + throw new JobRuntimeException(String.format("Unable to read existing partitions in the HDFS Path {}", + this.rawDataRootPath)); + } + } +} 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 new file mode 100644 index 0000000..65f8ae0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java @@ -0,0 +1,278 @@ +/* + * 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.github.rholder.retry.RetryException; +import com.github.rholder.retry.Retryer; +import com.github.rholder.retry.RetryerBuilder; +import com.github.rholder.retry.StopStrategies; +import com.github.rholder.retry.WaitStrategies; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.exceptions.MetadataException; +import com.uber.marmaray.utilities.FSUtils; +import lombok.Getter; +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.parquet.Strings; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +@Slf4j +/** + * {@link HDFSMetadataManager} implements the {@link IMetadataManager} interface, adding the capability + * to put and retrieve generic metadata from HDFS. All metadata will be stored under a single filename + * with the name = System.currentTimeMillis() when {@link IMetadataManager#saveChanges()} is invoked + */ +public class HDFSMetadataManager implements IMetadataManager { + public static final int DEFAULT_NUM_METADATA_FILES_TO_RETAIN = 5; + private static final int SERIALIZATION_VERSION = 1; + private static final Comparator byDateAsc = + Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName())); + + // 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; + + @NonNull + private final FileSystem fileSystem; + + @NotEmpty @Getter + private final String baseMetadataPath; + + /* + * 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. + */ + @Getter + private final AtomicBoolean shouldSaveChanges; + + public HDFSMetadataManager(@NonNull final FileSystem fs, @NotEmpty final String baseMetadataPath, + @NonNull final AtomicBoolean shouldSaveChanges) + throws IOException { + this.fileSystem = fs; + this.baseMetadataPath = baseMetadataPath; + this.shouldSaveChanges = shouldSaveChanges; + this.metadataMap = loadMetadata(); + } + + @Override + public void set(@NotEmpty final String key, @NonNull final StringValue value) throws MetadataException { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key)); + this.metadataMap.put(key, value); + } + + @Override + public Optional remove(@NotEmpty final String key) { + return Optional.fromNullable(this.metadataMap.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(); + } + + @Override + public Set getAllKeys() { + return this.metadataMap.keySet(); + } + + /** + * This method will first attempt to save the metadata file to a temp file. Once that succeeds it will + * copy the file to it's permanent location without the temp extension. + * + * @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; + } + + final Callable callable = () -> { + writeMetadataFile(); + pruneMetadataFiles(); + return null; + }; + + final Retryer retryer = RetryerBuilder.newBuilder() + .retryIfExceptionOfType(Exception.class) + .retryIfRuntimeException() + .withWaitStrategy(WaitStrategies.exponentialWait(5, 20, TimeUnit.SECONDS)) + .withStopStrategy(StopStrategies.stopAfterAttempt(3)) + .build(); + + try { + retryer.call(callable); + } catch (RetryException | ExecutionException e) { + throw new JobRuntimeException("Could not save metadata file. " + e.getMessage(), e); + } + } + + private void pruneMetadataFiles() { + try { + final Path metadataPath = new Path(this.baseMetadataPath); + if (this.fileSystem.exists(metadataPath)) { + final FileStatus[] fileStatuses = fileSystem.listStatus(metadataPath); + if (fileStatuses.length > 0) { + FSUtils.deleteHDFSMetadataFiles(fileStatuses, + this.fileSystem, DEFAULT_NUM_METADATA_FILES_TO_RETAIN, false); + } + } + } catch (final IOException e) { + final String errMsg = + String.format("IOException occurred while pruning metadata files. Message: %s", e.getMessage()); + log.warn(errMsg); + } + } + + private void writeMetadataFile() { + final Long currentTime = System.currentTimeMillis(); + + final String fileLocation = new Path(this.baseMetadataPath, currentTime.toString()).toString(); + final String tmpFileLocation = fileLocation.toString() + MetadataConstants.TEMP_FILE_EXTENSION; + + try (final OutputStream os = new BufferedOutputStream( + this.fileSystem.create( + new Path(tmpFileLocation)))) { + try (final ObjectOutputStream oos = new ObjectOutputStream(os)) { + serialize(oos); + } + + log.info("Saving metadata to: {}", fileLocation); + this.fileSystem.rename(new Path(tmpFileLocation), new Path(fileLocation)); + } catch (final IOException e) { + final String errMsg = + String.format("IOException occurred while saving changes. Message: %s", e.getMessage()); + throw new MetadataException(errMsg, e); + } + } + + public Optional getLatestMetadataFile() throws IOException { + // find the file name by timestamp and get the checkpoint metadata inside + final Path parentFolder = new Path(this.baseMetadataPath); + + if (this.fileSystem.exists(parentFolder)) { + // Get the latest metadata file written + final java.util.Optional fs = Arrays.stream(this.fileSystem.globStatus( + new Path(parentFolder, "*"))) + .filter(f -> !f.getPath().getName().endsWith(MetadataConstants.TEMP_FILE_EXTENSION)) + .sorted(byDateAsc.reversed()).findFirst(); + + // Deserialize the map and load the checkpoint data + return fs.isPresent() ? Optional.of(fs.get()) : Optional.absent(); + } + return Optional.absent(); + } + + /** + * This method will load the latest metadata file within the base metadata path for the + * stated metadataKey. + * + * @return Map of Metadata keys to values + * @throws IOException + */ + public Map loadMetadata() throws IOException { + log.info("Attempting to load metadata"); + final Optional fs = getLatestMetadataFile(); + if (fs.isPresent()) { + log.info("Loading metadata from: {}", fs.get().getPath()); + return loadMetadata(fs.get().getPath()); + } else { + log.info("No metadata file found"); + } + return new HashMap(); + } + + @VisibleForTesting + /** + * This method assumes that the path points explicitly to a metadata file and is not a directory + * @param path + * @return Map + * @throws IOException + */ + public Map loadMetadata(final Path path) throws IOException { + try (final InputStream is = new BufferedInputStream(this.fileSystem.open(path))) { + try (final ObjectInputStream input = new ObjectInputStream(is)) { + return deserialize(input); + } + } + } + + 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()) { + log.info("Serializing key: {} and value: {}", entry.getKey(), entry.getValue().getValue()); + out.writeUTF(entry.getKey()); + out.writeUTF(entry.getValue().getValue()); + } + + } + + public static Map deserialize(final ObjectInputStream ois) throws IOException { + final int version = ois.readInt(); + + if (version == SERIALIZATION_VERSION) { + final Map map = new HashMap<>(); + final int numEntries = ois.readInt(); + + for (int i = 0; i < numEntries; i++) { + final String key = ois.readUTF(); + final StringValue value = new StringValue(ois.readUTF()); + log.info("Deserializing key: {} and value: {}", key, value.getValue()); + map.put(key, value); + } + + if (ois.available() > 0) { + throw new MetadataException("Deserialization error, not all bytes were read off the stream"); + } + + return map; + } else { + throw new MetadataException("Version: " + version + " is not supported"); + } + } + +} 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 new file mode 100644 index 0000000..542b65f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSPartitionManager.java @@ -0,0 +1,154 @@ +/* + * 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.base.Optional; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.StringTypes; +import lombok.Getter; +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.hibernate.validator.constraints.NotEmpty; + +/** + * {@link HDFSPartitionManager} wraps {@link HDFSMetadataManager} to add specific capabilities to + * add, read, and retrieve partition information that is stored in HDFS. This is built on top of the + * generic read/write capabilities of {@link HDFSMetadataManager} + */ +@Slf4j +public class HDFSPartitionManager extends HDFSMetadataManager { + @NotEmpty @Getter + protected final String rawDataRootPath; + + @NonNull + protected final FileSystem fileSystem; + + @NotEmpty + protected final String metadataKey; + + @Getter + private final boolean isSinglePartition; + + @Getter + private Optional latestCheckpoint; + + public HDFSPartitionManager(@NotEmpty final String metadataKey, + @NotEmpty final String genericBaseMetadataPath, + @NotEmpty final String baseDataPath, + @NonNull final FileSystem fileSystem) throws IOException { + super(fileSystem, new Path(genericBaseMetadataPath, metadataKey).toString(), new AtomicBoolean(true)); + this.metadataKey = metadataKey; + this.rawDataRootPath = new Path(baseDataPath).toString(); + log.info(this.toString()); + this.fileSystem = fileSystem; + try { + final FileStatus[] fileStatuses = this.fileSystem.listStatus(new Path(baseDataPath)); + this.isSinglePartition = !Arrays.stream(fileStatuses).anyMatch(fs -> fs.isDirectory()); + this.latestCheckpoint = calculateLastCheckpoint(); + } catch (final IOException e) { + throw new JobRuntimeException("IOException encountered. Path:" + baseDataPath, e); + } + } + + /** + * Our explicit assumption is that all partitions will be nested 1 level deep + * under the file path under the metadata key. There's no need at this time to have + * more directories but that is subject to change in the future based on evolving requirements. + * @return + * @throws IOException + */ + public Optional getNextPartition() throws IOException { + if (this.isSinglePartition) { + log.info("Next partition: {}", this.rawDataRootPath); + return Optional.of(this.rawDataRootPath); + } else { + this.latestCheckpoint = calculateLastCheckpoint(); + if (this.latestCheckpoint.isPresent()) { + log.info("Last checkpoint: {}", this.latestCheckpoint.get()); + } else { + log.info("No last checkpoint found"); + } + + final java.util.Optional nextPartition = listPartitionsAfterCheckpoint(this.latestCheckpoint) + .stream() + .sorted() + .findFirst(); + + if (nextPartition.isPresent()) { + log.info("Next partition: {}", nextPartition.get()); + return Optional.of((String) nextPartition.get()); + } else { + log.info("No partitions were found to process"); + return Optional.absent(); + } + } + } + + public List getExistingPartitions() throws IOException { + final String partitionFolderRegex = this.rawDataRootPath + File.separator + "*"; + log.info("Searching for partitions in path: {}", partitionFolderRegex); + final FileStatus[] fileStatuses = this.fileSystem.globStatus(new Path(partitionFolderRegex)); + final List partitions = Arrays.asList(fileStatuses).stream() + .map(fileStatus -> fileStatus.getPath().getName()) + // filter out hidden files/directories + .filter(path -> !path.startsWith(StringTypes.DOT)) + .collect(Collectors.toList()); + return partitions; + } + + @Override + public String toString() { + final String info = String.format("HDFSPartitionManager Metadata Path: %s, Data Path: %s, Metadata Key %s", + this.getBaseMetadataPath(), this.rawDataRootPath, this.metadataKey); + return info; + } + + protected Optional calculateLastCheckpoint() throws IOException { + final Map metadataMap = this.loadMetadata(); + + return metadataMap.containsKey(MetadataConstants.CHECKPOINT_KEY) + ? Optional.of(metadataMap.get(MetadataConstants.CHECKPOINT_KEY)) + : Optional.absent(); + } + + private List listPartitionsAfterCheckpoint(final Optional checkpoint) throws IOException { + final List partitions = getExistingPartitions(); + + if (checkpoint.isPresent()) { + return partitions.stream() + .filter(partition -> partition.compareTo(checkpoint.get().getValue()) > 0) + .collect(Collectors.toList()); + } else { + return partitions; + } + } + + private String convertPartitionToPath(final String partition) { + return new Path(this.rawDataRootPath, partition).toString(); + } +} 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 new file mode 100644 index 0000000..c0633a2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HoodieBasedMetadataManager.java @@ -0,0 +1,195 @@ +/* + * 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.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; +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +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(). + */ +@Slf4j +public class HoodieBasedMetadataManager implements IMetadataManager { + + public static final String HOODIE_METADATA_KEY = "HoodieMetadataKey"; + + @Getter + private final HoodieConfiguration hoodieConf; + private final AtomicBoolean saveChanges; + private transient Optional jsc = Optional.absent(); + private final Map metadataMap; + + /** + * @param shouldSaveChanges {@link AtomicBoolean} which {@link #saveChanges} will use to determine if + * it should create new commit and save changes or not. It will save changes into new commit only if CAS operation + * succeeds in setting it to true (from false). + * @param hoodieConf {@link HoodieConfiguration} + */ + public HoodieBasedMetadataManager(@NonNull final HoodieConfiguration hoodieConf, + @NonNull final AtomicBoolean shouldSaveChanges, @NonNull final JavaSparkContext jsc) throws IOException { + this.hoodieConf = hoodieConf; + this.saveChanges = shouldSaveChanges; + this.jsc = Optional.of(jsc); + this.metadataMap = readMetadataInfo(this.hoodieConf); + } + + /** + * Updates in-memory metadata map with given key-value pair. + */ + public void set(@NotEmpty final String key, @NonNull final StringValue value) { + if (!this.saveChanges.get()) { + throw new JobRuntimeException( + String.format("Metadata manager changes are already saved.key:%s:value%s", key, value)); + } + this.metadataMap.put(key, value.getValue()); + } + + /** + * Remove the specified key from the metadata + * + * @param key the key to remove + * @return Optional of value if it was present, Optional.absent() if not + */ + @Override + public Optional remove(@NotEmpty final String key) { + final String val = this.metadataMap.remove(key); + return val == null ? Optional.absent() : Optional.of(new StringValue(val)); + } + + /** + * Returns given metadata key. + */ + public Optional get(@NotEmpty final String key) { + final String val = this.metadataMap.get(key); + return val == null ? Optional.absent() : Optional.of(new StringValue(val)); + } + + /** + * Returns all metadata manager keys. + */ + @Override + public Set getAllKeys() { + return this.metadataMap.keySet(); + } + + /** + * Returns immutable map of metadata key-value pairs. + */ + public Map getAll() { + return Collections.unmodifiableMap(this.metadataMap); + } + + public AtomicBoolean shouldSaveChanges() { + return this.saveChanges; + } + + /** + * If it is able to update {@link #saveChanges} from true to false; then only it will create new hoodie + * commit and will save metadata information in it. + */ + public void saveChanges() { + if (!this.saveChanges.compareAndSet(true, false)) { + log.info("Metadata info is already saved. Not saving it again."); + return; + } + final HoodieWriteClient writeClient = + new HoodieWriteClient<>(jsc.get(), this.hoodieConf.getHoodieWriteConfig(), true); + final String commitTime = writeClient.startCommit(); + log.info("Saving metadata info using hoodie-commit: {}", commitTime); + final List dummyWrites = new ArrayList<>(); + final boolean ret = + writeClient + .commit(commitTime, jsc.get().parallelize(dummyWrites), java.util.Optional.of(getMetadataInfo())); + if (!ret) { + throw new JobRuntimeException("Failed to save metadata information."); + } + } + + /** + * This method will also be used by HoodieSink to retrieve and store metadata information. + * It returns {@link HashMap} with hoodie metadata information to be saved into commit file. + * It returns {@link HashMap} instead of {@link Map} because hoodie needs it that way. Checkout + * {@link HoodieWriteClient#commit(String, JavaRDD, java.util.Optional)} for more info. + */ + public HashMap getMetadataInfo() { + final HashMap map = new HashMap<>(); + map.put(HOODIE_METADATA_KEY, MapUtil.serializeMap(this.metadataMap)); + return map; + } + + /* + * It reads metadata from latest hoodie commit file. Hoodie metadata info is stored in commit file using + * {@link #HOODIE_METADATA_KEY} key. + */ + private static Map readMetadataInfo( + @NonNull final HoodieConfiguration hoodieConf) { + try { + final FileSystem fs = FSUtils.getFs(hoodieConf.getConf()); + HoodieUtil.initHoodieDataset(fs, hoodieConf); + final HoodieTableMetaClient hoodieTableMetaClient = + new HoodieTableMetaClient(new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), + hoodieConf.getBasePath(), true); + final HoodieActiveTimeline hoodieActiveTimeline = hoodieTableMetaClient.getActiveTimeline(); + final java.util.Optional lastInstant = hoodieActiveTimeline.getCommitTimeline() + .filterCompletedInstants().lastInstant(); + if (lastInstant.isPresent()) { + log.info("using hoodie instant for reading checkpoint info :{}", lastInstant.get().getTimestamp()); + final HoodieCommitMetadata commitMetadata = + HoodieCommitMetadata.fromBytes(hoodieActiveTimeline.getInstantDetails(lastInstant.get()).get()); + final String serCommitInfo = commitMetadata.getMetadata(HOODIE_METADATA_KEY); + if (!Strings.isNullOrEmpty(serCommitInfo)) { + return MapUtil.deserializeMap(serCommitInfo); + } + } + return new HashMap<>(); + } catch (IOException e) { + log.error("failed to read metadata info", e); + throw new JobRuntimeException("failed to read metadata information", e); + } + } +} 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 new file mode 100644 index 0000000..f442f3c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/IMetadataManager.java @@ -0,0 +1,41 @@ +/* + * 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.base.Optional; +import lombok.NonNull; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Set; + +/** + * {@link IMetadataManager} describes the generic contract that any implementing metadata manager must adhere to + * and provides basic get/set operations. + * + * The {@link IMetadataManager#saveChanges()} method may be a no-op depending on the implementation of the metadata + * manager. Some implementations will keep all changes in memory until a job completes and changes will be persisted + * 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 { + 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(); +} 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 new file mode 100644 index 0000000..70e79a6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/JobManagerMetadataTracker.java @@ -0,0 +1,125 @@ +/* + * 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.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.MetadataException; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.utilities.FSUtils; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * {@link JobManagerMetadataTracker} offers functionality to store JobDag's metadata information at + * the Job Manager's level + * The previous job's metadata is consumed at the start of each run and serialized after current execution + * Uses {@link HDFSMetadataManager} internally to interact with the backend if sourceType is set to HDFS + * + */ + +@Slf4j +public class JobManagerMetadataTracker { + + @NonNull + private IMetadataManager metadataManager; + @Getter + private AtomicBoolean shouldSaveChanges; + + private final ObjectMapper mapper = new ObjectMapper(); + private final TypeReference> typeRef = new TypeReference>() { }; + + public JobManagerMetadataTracker(@NonNull final Configuration config) throws IOException { + final Optional sourceType = config.getProperty(MetadataConstants.JOBMANAGER_METADATA_STORAGE); + if (!sourceType.isPresent()) { + throw new MissingPropertyException("Source information for the JobManager Metadata Tracker is missing."); + } + if (sourceType.get().toUpperCase().equals(MetadataConstants.JOBMANAGER_METADATA_SOURCE_HDFS)) { + final Optional basePath = config.getProperty(MetadataConstants.JOBMANAGER_METADATA_HDFS_BASEPATH); + if (!basePath.isPresent()) { + throw new MissingPropertyException("Base Path for HDFS JobManager Metadata Tracker is missing."); + } + this.metadataManager = + new HDFSMetadataManager(FSUtils.getFs(config), basePath.get(), new AtomicBoolean(true)); + this.shouldSaveChanges = new AtomicBoolean(true); + } + } + + /** + * Set the metadata for this DAG, if not empty + * @param key + * @param value + */ + public void set(@NotEmpty final String key, @NonNull final Map value) { + try { + if (!value.isEmpty()) { + this.metadataManager.set(key, new StringValue(mapper.writeValueAsString(value))); + } + } catch (JsonProcessingException e) { + throw new MetadataException("Unable to set the JobManager metadata for key :" + key); + } + } + + /*** + * Checks if metadata for a given DAG already exists + * @param key + * @return + */ + public boolean contains(@NotEmpty final String key) { + return this.metadataManager.get(key).isPresent() ? true : false; + } + + /*** + * Returns the metadata for the given DAG + * @param key + * @return + * @throws IOException + */ + public Optional> get(@NotEmpty final String key) throws IOException { + final Optional metadataValues = this.metadataManager.get(key); + if (metadataValues.isPresent()) { + return Optional.of(mapper.readValue(metadataValues.get().getValue(), typeRef)); + } + return Optional.absent(); + } + + /*** + * Invokes the {@link IMetadataManager} to store the metadata information + * + */ + public void writeJobManagerMetadata() { + if (!this.shouldSaveChanges.get()) { + throw new MetadataException("JobManager metadata can only be saved once."); + } else { + try { + this.metadataManager.saveChanges(); + this.shouldSaveChanges.compareAndSet(true, false); + } catch (IOException e) { + throw new MetadataException("Unable to save JobManager Metadata", e); + } + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/MetadataConstants.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/MetadataConstants.java new file mode 100644 index 0000000..2606f5a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/MetadataConstants.java @@ -0,0 +1,36 @@ +/* + * 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.configuration.Configuration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +public final class MetadataConstants { + public static final String TEMP_FILE_EXTENSION = ".tmp"; + public static final String CHECKPOINT_KEY = "checkpoint"; + public static final String JOBMANAGER_PREFIX = Configuration.MARMARAY_PREFIX + "jobmanager"; + public static final String JOBMANAGER_METADATA_PREFIX = JOBMANAGER_PREFIX + ".metadata"; + public static final String JOBMANAGER_METADATA_ENABLED = JOBMANAGER_METADATA_PREFIX + ".enabled"; + public static final String JOBMANAGER_METADATA_HDFS_PREFIX = JOBMANAGER_METADATA_PREFIX + ".hdfs"; + public static final String JOBMANAGER_METADATA_HDFS_BASEPATH = JOBMANAGER_METADATA_HDFS_PREFIX + ".basePath"; + public static final String JOBMANAGER_METADATA_STORAGE = JOBMANAGER_METADATA_PREFIX + ".sourceType"; + public static final String JOBMANAGER_METADATA_SOURCE_HDFS = "HDFS"; + + private MetadataConstants() { + throw new JobRuntimeException("This class should never be instantiated"); + } +} 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 new file mode 100644 index 0000000..98e57c2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/NoOpMetadataManager.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.metadata; + +import com.google.common.base.Optional; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +public class NoOpMetadataManager implements IMetadataManager { + @Override + public void set(@NotEmpty final String key, @NonNull final StringValue value) { + // no-op + } + + @Override + public Optional remove(@NotEmpty final String key) { + return Optional.absent(); + } + + @Override + public Optional get(@NotEmpty final String key) { + return Optional.absent(); + } + + @Override + public void saveChanges() throws IOException { + //no-op + } + + @Override + public Set getAllKeys() { + return Collections.emptySet(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/StringValue.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/StringValue.java new file mode 100644 index 0000000..930d483 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/StringValue.java @@ -0,0 +1,36 @@ +/* + * 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 lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; + +@AllArgsConstructor +@EqualsAndHashCode(callSuper = false) +/** + * {@link StringValue} extends {@link AbstractValue} and wraps a String that represents the job metadata + * that will be stored in HDFS + */ +public class StringValue extends AbstractValue { + + private final String value; + + @Override + public String getValue() { + return this.value; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraPayloadRDDSizeEstimator.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraPayloadRDDSizeEstimator.java new file mode 100644 index 0000000..3fc4d3b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/CassandraPayloadRDDSizeEstimator.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.metrics; + +import com.uber.marmaray.common.data.RDDWrapper; +import com.uber.marmaray.common.schema.cassandra.CassandraPayload; + +import java.util.List; + +public class CassandraPayloadRDDSizeEstimator { + private final int NO_OF_SAMPLE_ROWS = 1000; + + public long estimateTotalSize(final RDDWrapper rdd) { + final long totalRows = rdd.getCount(); + + final List sampleRows = rdd.getData().takeSample(true, NO_OF_SAMPLE_ROWS); + + final long byteSize = sampleRows + .stream() + .map(element -> element.estimateRowSize()) + .reduce((size, accumulator) -> size + accumulator) + .orElse(0); + + final long totalSize = (long) (byteSize * (((totalRows) * 1.0) / (NO_OF_SAMPLE_ROWS))); + + return totalSize; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/ChargebackMetricType.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ChargebackMetricType.java new file mode 100644 index 0000000..3856a52 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/ChargebackMetricType.java @@ -0,0 +1,26 @@ +/* + * 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; + +public enum ChargebackMetricType { + // count the number of rows + ROW_COUNT, + // count the runtime of the execution + RUN_TIME, + // count the number of executors used + EXECUTORS +} 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 new file mode 100644 index 0000000..6599c3c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetricNames.java @@ -0,0 +1,65 @@ +/* + * 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; + +/** + * Known {@link DataFeedMetrics} names. + */ +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"; + + public static final String AVAILABLE_ROWCOUNT = "available_rowcount"; + public static final String INPUT_ROWCOUNT = "input_rowcount"; + public static final String OUTPUT_ROWCOUNT = "output_rowcount"; + public static final String OUTPUT_BYTE_SIZE = "output_byte_size"; + public static final String ERROR_ROWCOUNT = "error_rowcount"; + public static final String DUPLICATE_ROWCOUNT = "duplicate_rowcount"; + public static final String BULK_INSERT_COUNT = "insert_count"; + public static final String UPSERT_COUNT = "upsert_count"; + + public static final String CHANGED_ROWCOUNT = "changed_rowcount"; + public static final String NON_CONFORMING_ROWCOUNT = "non_conforming_rowcount"; + + // Used to indicate if job succeeded or not. + public static final String RESULT = "result"; + public static final int RESULT_SUCCESS = 1; + public static final int RESULT_FAILURE = -1; + + public static final String DISPERSAL_CONFIGURATION_INIT_ERRORS = "dispersal_config_error_count"; + + // metrics in context of data flow + public static final String FRESHNESS = "freshness"; + public static final String INTERVAL_INPUT_ROWCOUNT = "interval_input_rowcount"; + public static final String INTERVAL_OUTPUT_ROWCOUNT = "interval_output_rowcount"; + public static final String ROWCOUNT_BEHIND = "rowcount_behind"; + + // Timer related metrics + public static final String INIT_CONFIG_LATENCY_MS = "init_config_latency_ms"; + public static final String INIT_METADATAMANAGER_LATENCY_MS = "init_metadatamanager_latency_ms"; + public static final String CONVERT_SCHEMA_LATENCY_MS = "convert_schema_latency_ms"; + public static final String TOTAL_LATENCY_MS = "total_latency_ms"; + + private DataFeedMetricNames() { + throw new JobRuntimeException("This class should never be instantiated"); + } +} + 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 new file mode 100644 index 0000000..54bc3e1 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DataFeedMetrics.java @@ -0,0 +1,86 @@ +/* + * 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.google.common.base.Optional; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.uber.marmaray.common.reporters.IReporter; +import com.uber.marmaray.common.reporters.Reportable; +import lombok.Getter; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +import java.time.Instant; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +/** + * Topic level metric should be created via {@link DataFeedMetrics} using {@link DataFeedMetrics#createLongMetric} + * and {@link DataFeedMetrics#createTimerMetric}, and reported using {@link DataFeedMetrics#gaugeAll} + */ +@Getter +public class DataFeedMetrics implements Reportable { + public static final String DATA_FEED_NAME = "datafeed"; + private static final String JOB_TAG = "job"; + @Getter + private final Set metricSet = Sets.newHashSet(); + private final Map baseTags; + @NotEmpty + private final String jobName; + + public DataFeedMetrics(final String jobName, final Map tags) { + this.jobName = jobName; + final Map tagsMap = Maps.newHashMap(tags); + tagsMap.put(JOB_TAG, jobName); + this.baseTags = Collections.unmodifiableMap(tagsMap); + } + + public LongMetric createLongMetric(@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.metricSet.add(metric); + return metric; + } + + public TimerMetric createTimerMetric(@NonNull final String metricName) { + final TimerMetric metric = new TimerMetric(metricName); + metric.addTags(getBaseTags()); + metricSet.add(metric); + return metric; + } + + public TimerMetric createTimerMetric(@NonNull final String metricName, + @NonNull final Map additionalTags, + @NonNull final Optional startTime) { + final TimerMetric metric = startTime.isPresent() + ? new TimerMetric(metricName, additionalTags, startTime.get()) + : new TimerMetric(metricName); + metric.addTags(getBaseTags()); + metric.addTags(additionalTags); + metricSet.add(metric); + return metric; + } + + public void gaugeAll(final IReporter reporter) { + this.metricSet.forEach(reporter::gauge); + } +} 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 new file mode 100644 index 0000000..4e589f4 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/DoubleMetric.java @@ -0,0 +1,44 @@ +/* + * 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 lombok.Getter; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * {@link Double} implementation of {@link Metric} + */ +@Getter +public class DoubleMetric extends Metric { + public DoubleMetric(final String metricName) { + super(metricName); + } + + public DoubleMetric(@NotEmpty final String metricName, final double metricValue) { + super(metricName, metricValue); + this.addTag("metric-type", "double"); + } + + public void setMetricValue(final double metricValue) { + this.metricValue = metricValue; + } + + @Override + public String toString() { + return super.toString(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/IChargebackCalculator.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/IChargebackCalculator.java new file mode 100644 index 0000000..a163bb7 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/IChargebackCalculator.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.metrics; + +import java.util.Map; + +/** + * Calculator to determine cost of a particular chargeback instance. + * + * Computation can be over a number of fields. + */ +public interface IChargebackCalculator { + + /** + * Compute the cost based on the input fields. + * @return the final cost value + */ + Map computeCost(); + + /** + * Add a cost for a particular datafeed + * @param datafeedName the name of the datafeed to add cost to + * @param metricType the type of cost to add + * @param value the cost value to add + */ + void addCost(String datafeedName, ChargebackMetricType metricType, Long value); + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/IMetricable.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/IMetricable.java new file mode 100644 index 0000000..3e7ac55 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/IMetricable.java @@ -0,0 +1,32 @@ +/* + * 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 lombok.NonNull; + +public interface IMetricable { + + /* + Take a DataFeedMetrics to report metrics to, if present + */ + void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics); + + /* + Take a JobMetrics to report metrics to, if present + */ + void setJobMetrics(@NonNull final JobMetrics jobMetrics); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricNames.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricNames.java new file mode 100644 index 0000000..7feafe2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricNames.java @@ -0,0 +1,33 @@ +/* + * 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 JobMetricNames { + public static final String RUN_JOB_DAG_LATENCY_MS = "run_job_dag_latency_ms"; + public static final String RUN_JOB_ERROR_COUNT = "run_job_error_count"; + public static final String JOB_SETUP_LATENCY_MS = "job_setup_latency_ms"; + + // JobLockManager-related metrics + public static final String JOB_MANAGER_LOCK_TIME_MS = "job_manager_lock_time_ms"; + public static final String JOB_DAG_LOCK_TIME_MS = "job_dag_lock_time_ms"; + + private JobMetricNames() { + throw new JobRuntimeException("Class should never be instantiated"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricType.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricType.java new file mode 100644 index 0000000..8cf70e5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetricType.java @@ -0,0 +1,35 @@ +/* + * 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; + +/** + * Known {@link JobMetrics} names. + */ +public enum JobMetricType { + // metrics in context of the job + RUNTIME, + STAGE_RUNTIME, + SIZE, + + // resource usage metrics + DRIVER_MEMORY, + EXECUTOR_MEMORY, + NUM_EXECUTORS, + + // error count metrics + RUN_ERROR_COUNT +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetrics.java b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetrics.java new file mode 100644 index 0000000..c63af0b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/JobMetrics.java @@ -0,0 +1,75 @@ +/* + * 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.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import com.uber.marmaray.common.reporters.Reportable; +import com.uber.marmaray.common.reporters.IReporter; +import lombok.Getter; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Map; +import java.util.Set; + +/** + * Job level metric should be created via {@link JobMetrics} using {@link JobMetrics#createJobLongMetric} + * and {@link JobMetrics#createJobTimerMetric}, and reported using {@link JobMetrics#gaugeAll} + */ +@Getter +public class JobMetrics implements Reportable { + private static final String METRIC_PREFIX = "job-"; + private static final String JOB_TAG = "job"; + private final Set metricSet = Sets.newHashSet(); + private final Map baseTags; + @NotEmpty + private final String jobName; + + public JobMetrics(final String jobName) { + this.jobName = jobName; + this.baseTags = ImmutableMap.of(JOB_TAG, jobName); + } + + public LongMetric createJobLongMetric(@NonNull final JobMetricType metricType, + final long metricValue, + @NonNull final Map additionalTags) { + final LongMetric metric = new LongMetric(getMetricName(metricType), metricValue); + metric.addTags(getBaseTags()); + metric.addTags(additionalTags); + this.metricSet.add(metric); + return metric; + } + + public TimerMetric createJobTimerMetric(@NonNull final JobMetricType metricType, + @NonNull final Map additionalTags) { + final TimerMetric metric = new TimerMetric(getMetricName(metricType)); + metric.addTags(getBaseTags()); + metric.addTags(additionalTags); + metricSet.add(metric); + return metric; + } + + private static String getMetricName(@NonNull final JobMetricType metricType) { + return METRIC_PREFIX + metricType.toString().toLowerCase(); + } + + @Override + public void gaugeAll(@NonNull final IReporter reporter) { + metricSet.forEach(reporter::gauge); + } +} 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 new file mode 100644 index 0000000..1a9ee9b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/LongMetric.java @@ -0,0 +1,44 @@ +/* + * 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 lombok.Getter; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * {@link Long} implementation of {@link Metric} + */ +@Getter +public class LongMetric extends Metric { + public LongMetric(final String metricName) { + super(metricName); + } + + public LongMetric(@NotEmpty final String metricName, final long metricValue) { + super(metricName, metricValue); + this.addTag("metric-type", "long"); + } + + public void setMetricValue(final long metricValue) { + this.metricValue = metricValue; + } + + @Override + public String toString() { + return super.toString(); + } +} 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 new file mode 100644 index 0000000..4ed8df3 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/Metric.java @@ -0,0 +1,76 @@ +/* + * 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.google.common.collect.Maps; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.reporters.IReporter; +import lombok.Getter; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.Map; + +import static java.util.Objects.isNull; + +/** + * {@link Metric} encapsulates a unit of statistics either to be reported + * by {@link IReporter} or written + * to a UtilTable (todo) + */ +@Getter +public abstract class Metric { + @NotEmpty + protected String metricName; + protected T metricValue; + protected Map tags = Maps.newHashMap(); + + protected Metric(@NotEmpty final String metricName) { + this.metricName = metricName; + } + + protected Metric(@NotEmpty final String metricName, @NonNull final T metricValue) { + this(metricName); + this.metricValue = metricValue; + } + + public void addTag(@NotEmpty final String key, @NotEmpty final String value) { + this.tags.put(key, value); + } + + /** + * Additional tag values with existing keys will replace old ones + * @param additionalTags + * @return + */ + public void addTags(@NonNull final Map additionalTags) { + this.tags.putAll(additionalTags); + } + + public T getMetricValue() { + if (isNull(this.metricValue)) { + throw new JobRuntimeException(String.format("Metric:%s", this)); + } + return this.metricValue; + } + + @Override + public String toString() { + return String.format("metricName[%s]:metricValue[%s]:tags[%s]", + this.metricName, this.metricValue, this.tags); + } +} 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 new file mode 100644 index 0000000..1161236 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/metrics/TimerMetric.java @@ -0,0 +1,82 @@ +/* + * 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.google.common.base.Preconditions; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.time.Duration; +import java.time.Instant; +import java.util.Map; + +/** + * {@link TimerMetric} Calculates the duration between two times in milliseconds + * + * {@link TimerMetric} get its value by surrounding the test subject by calling + * {@link TimerMetric#stop()} + */ +@Slf4j +@Getter +public class TimerMetric extends LongMetric { + private final Instant startTime; + private Instant endTime; + private boolean stopped; + + public TimerMetric(@NotEmpty final String metricName, + @NonNull final Map tags, + @NonNull final Instant startTime) { + super(metricName); + this.addTag("metric-type", "timer"); + this.addTags(tags); + this.startTime = startTime; + } + + public TimerMetric(@NotEmpty final String metricName, @NonNull final Map tags) { + this(metricName.toString()); + this.addTags(tags); + } + + public TimerMetric(@NotEmpty final String metricName) { + super(metricName); + this.addTag("metric-type", "timer"); + this.startTime = Instant.now(); + } + + public void stop() { + if (!this.stopped) { + this.endTime = Instant.now(); + calculateElapsed(); + this.stopped = true; + } + } + + @Override + public java.lang.String toString() { + return java.lang.String.format("%s:startTime[%s]:endTime[%s]", + super.toString(), this.startTime, this.endTime); + } + + private void calculateElapsed() { + Preconditions.checkState(this.startTime.compareTo(this.endTime) <= 0); + final Duration elapsed = Duration.between(startTime, endTime); + log.info("{} elapsed {} milliseconds", this.metricName, elapsed.toMillis()); + super.setMetricValue(elapsed.toMillis()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/reporters/ConsoleReporter.java b/marmaray/src/main/java/com/uber/marmaray/common/reporters/ConsoleReporter.java new file mode 100644 index 0000000..a589797 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/ConsoleReporter.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.reporters; + +import com.uber.marmaray.common.metrics.Metric; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +/** + * Console implementation of {@link IReporter} + */ +@Slf4j +public class ConsoleReporter implements IReporter { + public void gauge(@NonNull final Metric m) { + final String metricName = m.getMetricName(); + final String metricValue = m.getMetricValue().toString(); + final String tags = m.getTags().toString(); + + log.info("{}={}, Tags: {}", metricName, metricValue, tags); + } + + public void finish() { + // do nothing + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/reporters/IReporter.java b/marmaray/src/main/java/com/uber/marmaray/common/reporters/IReporter.java new file mode 100644 index 0000000..d9d2168 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/IReporter.java @@ -0,0 +1,29 @@ +/* + * 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.reporters; + +import com.uber.marmaray.common.metrics.Metric; +import lombok.NonNull; + +/** + * {@link IReporter} gauges {@link Metric} to a sink + */ +public interface IReporter { + void gauge(@NonNull final T m); + + void finish(); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reportable.java b/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reportable.java new file mode 100644 index 0000000..f25069c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reportable.java @@ -0,0 +1,26 @@ +/* + * 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.reporters; + +import lombok.NonNull; + +/** + * {@link Reportable} that can be gauged by {@link IReporter} + */ +public interface Reportable { + void gaugeAll(@NonNull final IReporter reporter); +} 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 new file mode 100644 index 0000000..0c23b88 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/reporters/Reporters.java @@ -0,0 +1,59 @@ +/* + * 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.reporters; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.Metric; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedDeque; + +@Slf4j +public final class Reporters { + @Getter + private final Queue reporters; + private boolean isFinished = false; + + public Reporters() { + this.reporters = new ConcurrentLinkedDeque<>(); + } + + public void addReporter(final IReporter reporter) { + this.reporters.add(reporter); + } + + public void report(final Metric m) { + + // Todo T1137075 - Figure out how to report metrics from spark executor side + if (this.reporters.isEmpty()) { + throw new JobRuntimeException("No reporters registered"); + } + + this.reporters.forEach(r -> r.gauge(m)); + } + + public void finish() { + if (!isFinished) { + this.reporters.forEach(r -> r.finish()); + isFinished = true; + } else { + log.warn("Reporters were already flushed and closed. Please check that why this is being done again"); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/retry/IFunctionThrowsException.java b/marmaray/src/main/java/com/uber/marmaray/common/retry/IFunctionThrowsException.java new file mode 100644 index 0000000..a4c85e8 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/retry/IFunctionThrowsException.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.retry; + +import lombok.NonNull; + +/** + * {@link IFunctionThrowsException} is the interface for an function throwing exceptions. + */ + +@FunctionalInterface +public interface IFunctionThrowsException { + R apply(@NonNull final T t) throws Exception; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/retry/IRetryStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/retry/IRetryStrategy.java new file mode 100644 index 0000000..f98adcc --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/retry/IRetryStrategy.java @@ -0,0 +1,29 @@ +/* + * 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.retry; + +import com.uber.marmaray.common.exceptions.RetryException; + +/** + * {@link IRetryStrategy} determines if a function should be retried or not. retryMessage returns + * the description of the current attempt. + */ + +public interface IRetryStrategy { + boolean shouldRetry() throws RetryException; + String retryMessage(); +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/retry/RetryableFunction.java b/marmaray/src/main/java/com/uber/marmaray/common/retry/RetryableFunction.java new file mode 100644 index 0000000..46e3507 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/retry/RetryableFunction.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.common.retry; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.RetryStrategyConfiguration; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +/** + * {@link RetryableFunction#apply} will execute a given function with a given retry strategy. + * @param is the type of the input of the function; + * @param is the type of the result of the function. + */ +@Slf4j +public class RetryableFunction { + @NonNull protected final IRetryStrategy retryStrategy; + @NonNull protected final IFunctionThrowsException func; + + public RetryableFunction(@NonNull final IFunctionThrowsException func, + @NonNull final IRetryStrategy retryStrategy) { + this.func = func; + this.retryStrategy = retryStrategy; + } + + protected IFunctionThrowsException getUserFunction() { + return func; + } + + public R apply(final T t) throws Exception { + while (true) { + try { + return getUserFunction().apply(t); + } catch (Exception e) { + if (retryStrategy.shouldRetry()) { + log.info(retryStrategy.retryMessage()); + } else { + log.info(retryStrategy.retryMessage()); + throw e; + } + } + } + } + + public static final class Builder { + private Configuration conf = new Configuration(); + @NonNull + private final IFunctionThrowsException func; + + public Builder(@NonNull final IFunctionThrowsException func) { + this.func = func; + } + + public Builder withConfiguration(@NonNull final Configuration conf) { + this.conf = conf; + return this; + } + + public RetryableFunction build() { + final IRetryStrategy retryStrategy = new RetryStrategyConfiguration(this.conf).getRetryStrategy(); + return new RetryableFunction(this.func, retryStrategy); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/retry/SimpleRetryStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/retry/SimpleRetryStrategy.java new file mode 100644 index 0000000..a218f82 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/retry/SimpleRetryStrategy.java @@ -0,0 +1,75 @@ +/* + * 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.retry; + +import com.uber.marmaray.common.configuration.SimpleRetryStrategyConfiguration; +import com.uber.marmaray.common.exceptions.RetryException; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class SimpleRetryStrategy implements IRetryStrategy { + public static final int DEFAULT_NUMBER_OF_RETRIES = 3; + public static final long DEFAULT_WAIT_TIME_IN_MS = 1000; + + private final int numRetries; + private final long waitTimeInMs; + private int retriesLeft; + private String message; + + public SimpleRetryStrategy() { + this(DEFAULT_NUMBER_OF_RETRIES, DEFAULT_WAIT_TIME_IN_MS); + } + + public SimpleRetryStrategy(@NonNull final SimpleRetryStrategyConfiguration conf) { + this(conf.getNumRetries(), conf.getWaitTimeInMs()); + } + + public SimpleRetryStrategy(final int numRetries, final long waitTimeInMs) { + this.numRetries = numRetries; + this.waitTimeInMs = waitTimeInMs; + this.retriesLeft = numRetries; + this.message = ""; + } + + @Override + public boolean shouldRetry() throws RetryException { + this.retriesLeft--; + this.message = ""; + if (this.retriesLeft < 0) { + this.message = String.format("SimpleRetryStrategy failed for %d times with %d ms wait time.", + numRetries, waitTimeInMs); + return false; + } + try { + Thread.sleep(waitTimeInMs); + } catch (InterruptedException e) { + this.message = String.format("The %d-st retry failed during the %d ms wait time.", + numRetries - retriesLeft, waitTimeInMs); + throw new RetryException(this.message, e); + } + this.message = String.format("SimpleRetryStrategy is retrying for the %d-st time, after %d ms wait time", + numRetries - retriesLeft, waitTimeInMs); + return true; + } + + @Override + public String retryMessage() { + return this.message; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/ISchemaService.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/ISchemaService.java new file mode 100644 index 0000000..555d2a1 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/ISchemaService.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.schema; + +import com.uber.marmaray.common.exceptions.InvalidDataException; +import com.uber.marmaray.common.schema.ISchemaService.ISchemaServiceReader; +import com.uber.marmaray.common.schema.ISchemaService.ISchemaServiceWriter; +import lombok.NonNull; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * The implementing class of {@link ISchemaService} provides support to fetch avro schema and serialize/deserialize + * {@link GenericRecord}. Optionally implementations of it can encode raw bytes. + * @param Implementing class of {@link ISchemaServiceReader} provides implementation to deserialize + * raw bytes into {@link GenericRecord}. + * @param Implementing class of {@link ISchemaServiceWriter} provides implementation to serialize + * {@link GenericRecord} into raw bytes. + */ +public interface ISchemaService { + + /** + * It fetches latest version of the schema and wraps it with custom wrapper fields. + * @param schemaName Fully qualified schema name + * @return Avro schema + */ + Schema getWrappedSchema(@NotEmpty final String schemaName); + + /** + * It fetches latest version of the schema. + * @param schemaName Fully qualified schema name + * @return Avro schema + */ + Schema getSchema(@NotEmpty final String schemaName); + + /** + * @param schemaName Fully qualified schema name + * @param schemaVersion schema version + * @return An instance of {@link ISchemaServiceWriter} + */ + W getWriter(@NotEmpty final String schemaName, final int schemaVersion); + + /** + * @param schemaName Fully qualified schema name + * @param schemaVersion schema version + * @return An instance of {@link ISchemaServiceReader} + */ + R getReader(@NotEmpty final String schemaName, final int schemaVersion); + + /** + * Implementing class of {@link ISchemaServiceReader} provides support to deserialize raw bytes into + * {@link GenericRecord}. Optionally it can decode bytes if corresponding {@link ISchemaServiceWriter} used + * encoded bytes during serialization. + */ + interface ISchemaServiceReader { + GenericRecord read(@NonNull final byte[] buffer) throws InvalidDataException; + } + + /** + * Implementing class of {@link ISchemaServiceWriter} provides support to serialize {@link GenericRecord} into + * raw bytes. Optionally it can encode bytes. + */ + interface ISchemaServiceWriter { + byte[] write(@NonNull final GenericRecord record) throws InvalidDataException; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/ISinkSchemaManager.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/ISinkSchemaManager.java new file mode 100644 index 0000000..6e68dc5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/ISinkSchemaManager.java @@ -0,0 +1,20 @@ +/* + * 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; + +public interface ISinkSchemaManager { +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraDataField.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraDataField.java new file mode 100644 index 0000000..41e637f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraDataField.java @@ -0,0 +1,33 @@ +/* + * 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 lombok.AllArgsConstructor; +import lombok.Getter; + +import java.io.Serializable; +import java.nio.ByteBuffer; + +@AllArgsConstructor +public class CassandraDataField implements Serializable { + + @Getter + private final ByteBuffer columnKey; + + @Getter + private final ByteBuffer value; +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraPayload.java new file mode 100644 index 0000000..c5e9285 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraPayload.java @@ -0,0 +1,59 @@ +/* + * 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.uber.marmaray.common.IPayload; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +public class CassandraPayload implements IPayload>, Serializable { + + /** + * No need to model keyspace and table name here, we already have it in the schema metadata + */ + private final List fields = new ArrayList<>(); + + @Override + public List getData() { + return this.fields; + } + + public void addField(final CassandraDataField field) { + this.fields.add(field); + } + + public List convertData() { + return fields.stream().map(field -> field.getValue()).collect(Collectors.toList()); + } + + /** + * estimate the size of the underlying data payload + */ + public int estimateRowSize() { + return convertData() + .stream() + .filter((row) -> row != null) + .map((row) -> row.capacity()) + .reduce((size, accumulator) -> size + accumulator) + .orElse(0); + } + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchema.java b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchema.java new file mode 100644 index 0000000..9d54519 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchema.java @@ -0,0 +1,50 @@ +/* + * 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 java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; + +public class CassandraSchema implements Serializable { + + @Getter + private final String keySpace; + + @Getter + private final String tableName; + + @Getter + private final List fields; + + public CassandraSchema(final String keySpace, final String tableName) { + this.keySpace = keySpace; + this.tableName = tableName; + this.fields = new ArrayList<>(); + } + + public CassandraSchema(final String keySpace, final String tableName, final List fields) { + this.keySpace = keySpace; + this.tableName = tableName; + this.fields = fields; + } + + public void addField(final CassandraSchemaField field) { + fields.add(field); + } +} 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 new file mode 100644 index 0000000..3161f2d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSchemaField.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.common.schema.cassandra; + +import com.google.common.base.Joiner; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.GenericRecordUtil; +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; + +@AllArgsConstructor +public final class CassandraSchemaField implements Serializable { + public static final String STRING_TYPE = "text"; + public static final String INT_TYPE = "int"; + public static final String LONG_TYPE = "bigint"; + public static final String FLOAT_TYPE = "float"; + public static final String BOOLEAN_TYPE = "boolean"; + public static final String DOUBLE_TYPE = "double"; + + private static final Joiner joiner = Joiner.on(StringTypes.COMMA); + + @Getter + @NotEmpty + private final String fieldName; + + @Getter + @NotEmpty + private final String type; + + public String toString() { + return this.fieldName + StringTypes.SPACE + type; + } + + public static String convertFromAvroType(final Schema schema) { + // If schema is a UNION type we need to take the non-nullable schema for schema backwards compatibility + final Schema nonNullSchema = GenericRecordUtil.isOptional(schema) ? GenericRecordUtil.getNonNull(schema) + : schema; + final Schema.Type type = nonNullSchema.getType(); + + switch (type) { + case STRING: + return STRING_TYPE; + case INT: + return INT_TYPE; + case LONG: + return LONG_TYPE; + case FLOAT: + return FLOAT_TYPE; + case BOOLEAN: + return BOOLEAN_TYPE; + case DOUBLE: + return DOUBLE_TYPE; + default: + // todo T935985: Support more complex types from Schema.Type + throw new JobRuntimeException("Type " + type + " is not supported for conversion. Field(s): " + + joiner.join(schema.getFields())); + } + } +} 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 new file mode 100644 index 0000000..f7f8e1f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/CassandraSinkSchemaManager.java @@ -0,0 +1,254 @@ +/* + * 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.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.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 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(); + } + + 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"); + } + } +} 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 new file mode 100644 index 0000000..0c208f8 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/schema/cassandra/ClusterKey.java @@ -0,0 +1,65 @@ +/* + * 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.Preconditions; +import com.google.common.base.Splitter; +import com.uber.marmaray.utilities.StringTypes; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.Serializable; +import java.util.List; + +/** + * {@link ClusterKey} defines a clustering key in Cassandra as well as the ordering of the key. + * We assume by default that the order is ascending. + */ +@AllArgsConstructor +@EqualsAndHashCode +public class ClusterKey implements Serializable { + private static final Splitter splitter = Splitter.on(StringTypes.COLON); + + @Getter @NotEmpty final String name; + @Getter final Order order; + + public enum Order { + ASC, + DESC + } + + public String toString() { + return name + StringTypes.SPACE + order; + } + + /* + * Parse a string for the name and the order delimited by the delimiter + */ + public static ClusterKey parse(@NotEmpty final String value) { + final List tokens = splitter.splitToList(value); + Preconditions.checkState(tokens.size() <= 2); + + if (tokens.size() == 1) { + return new ClusterKey(tokens.get(0), Order.ASC); + } else { + return new ClusterKey(tokens.get(0), Order.valueOf(tokens.get(1))); + } + } +} + diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/ISink.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/ISink.java new file mode 100644 index 0000000..aeee202 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/ISink.java @@ -0,0 +1,36 @@ +/* + * 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; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.metrics.IMetricable; +import org.apache.spark.api.java.JavaRDD; + +/** + * Implementations of {@link ISink} should implement {@link #write(JavaRDD)}. + * TODO: ISink#write() should return Stats object. + */ +public interface ISink extends IMetricable { + + /** + * It writes data to sink. If there is any exception while writing then it throws + * {@link com.uber.marmaray.common.exceptions.JobRuntimeException}. All invalid / error records will be + * written to ErrorTable. + * @param data data to write to sink + */ + void write(JavaRDD data); +} 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 new file mode 100644 index 0000000..8125bc6 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/SinkStatManager.java @@ -0,0 +1,133 @@ +/* + * 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; + +import com.google.common.base.Optional; +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; +import lombok.ToString; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Convenience class for managing {@link SinkStat} for {@link #MAX_HISTORY_SIZE} runs. + */ +@Slf4j +@AllArgsConstructor +@ToString +public class SinkStatManager { + + public static final int MAX_HISTORY_SIZE = 256; + public static final String METAKEY = "sink-stat-%s"; + + @Getter + @NonNull + private final String tableName; + + @NonNull + private final IMetadataManager metadataManager; + + @Getter + @NonNull + private final SinkStat currentStat = new SinkStat(); + + private final Queue sinkStatQ = new LinkedList<>(); + + public String getMetakey() { + return String.format(METAKEY, tableName); + } + + public void init() { + final Optional serialisedStats = this.metadataManager.get(getMetakey()); + if (serialisedStats.isPresent()) { + final Map statHistory = MapUtil.deserializeMap(serialisedStats.get().getValue()); + for (int i = 0; i < statHistory.size(); i++) { + this.sinkStatQ.add(SinkStat.deserialize(statHistory.get(Integer.toString(i)))); + } + } + } + + public void persist() { + final Map stats = new HashMap<>(); + if (!this.currentStat.isEmpty()) { + this.sinkStatQ.add(this.currentStat); + } + while (this.sinkStatQ.size() > MAX_HISTORY_SIZE) { + this.sinkStatQ.poll(); + } + for (int i = 0; !this.sinkStatQ.isEmpty(); i++) { + stats.put(Integer.toString(i), SinkStat.serialize(this.sinkStatQ.poll())); + } + this.metadataManager.set(getMetakey(), new StringValue(MapUtil.serializeMap(stats))); + } + + public boolean isStatHistoryAvailable() { + return !sinkStatQ.isEmpty(); + } + + public long getAvgRecordSize() { + long avgRecordSize = 0; + long numEntries = 0; + for (final SinkStat stat : this.sinkStatQ) { + final Optional avgRecordSizeStat = stat.get(SinkStat.AVG_RECORD_SIZE); + if (avgRecordSizeStat.isPresent()) { + avgRecordSize += Long.parseLong(avgRecordSizeStat.get()); + numEntries += 1; + } + } + return avgRecordSize / Math.max(numEntries, 1); + } + + @Slf4j + @ToString + public static class SinkStat { + public static final String AVG_RECORD_SIZE = "AVG_RECORD_SIZE"; + + private final Map stats = new HashMap<>(); + + public static SinkStat deserialize(@NonNull final String serializedStat) { + final SinkStat sinkStat = new SinkStat(); + sinkStat.stats.putAll(MapUtil.deserializeMap(serializedStat)); + return sinkStat; + } + + public static String serialize(@NonNull final SinkStat sinkStat) { + return MapUtil.serializeMap(sinkStat.stats); + } + + public Optional get(@NotEmpty final String statKey) { + final String statVal = this.stats.get(statKey); + return statVal == null ? Optional.absent() : Optional.of(statVal); + } + + public void put(@NotEmpty final String statKey, @NotEmpty final String statVal) { + this.stats.put(statKey, statVal); + } + + public boolean isEmpty() { + return this.stats.isEmpty(); + } + } +} 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 new file mode 100644 index 0000000..1db8edf --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraClientSink.java @@ -0,0 +1,128 @@ +/* + * 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.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.converters.data.CassandraSinkCQLDataConverter; +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.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. + * + * Valid records will then be written to the Cassandra backend + */ +@Slf4j +public class CassandraClientSink extends CassandraSink { + + private final CassandraSinkCQLDataConverter converter; + + public CassandraClientSink(@NonNull final CassandraSinkCQLDataConverter converter, + @NonNull final CassandraSinkSchemaManager schemaManager, + @NonNull final CassandraSinkConfiguration conf) { + super(schemaManager, conf); + this.converter = converter; + } + + @Override + public void write(@NonNull final JavaRDD data) { + + final Configuration hadoopConf = this.conf.getHadoopConf(); + + log.info("Setting up Cassandra Table"); + this.setupCassandraTable(hadoopConf); + + log.info("Converting data to cassandra payload"); + converter.setKeyspaceName(conf.getKeyspace()); + converter.setTableName(conf.getTableName()); + final RDDWrapper payloadWrapper = this.converter.map(data); + + if (payloadWrapper.getCount() == 0) { + /* + * As a safeguard and precaution, we fail the job if no records are dispersed. The root cause can be + * something innocuous like an accidental empty dataset. But this is to explicitly protect us against + * any data converter bugs that can take valid data and not convert it correctly resulting in an + * incorrect error record. + * + * For dispersal this is ok since we assume the data that existed in Hive has been ingested and conforms + * to a schema. The main trade-off here is if there really is an empty dataset we cannot proceed until + * the customer has deleted that Hive partition. We believe that is much more preferable than a silent + * 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 " + , "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); + + try (final Cluster cluster = builder.build(); + final Session session = cluster.connect(keyspaceName)) { + while (iter.hasNext()) { + Statement statement = iter.next(); + try { + session.execute(statement); + } catch (Exception e) { + log.error("Exception: {}", e); + throw new JobRuntimeException(e); + } + } + } + return iter; + } + ); + writtenRdd.collect(); + + log.info("Finished write process"); + } +} 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 new file mode 100644 index 0000000..901b0cc --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSSTableSink.java @@ -0,0 +1,166 @@ +/* + * 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.cassandra; + +import com.google.common.base.Optional; +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.RDDWrapper; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.CassandraPayloadRDDSizeEstimator; +import com.uber.marmaray.common.schema.cassandra.CassandraPayload; +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.cassandra.config.Config; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import scala.Tuple2; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@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 +public class CassandraSSTableSink extends CassandraSink { + + static { + Config.setClientMode(true); + } + + private static final String TMP_FILE_PATH = new Path(File.separator + "tmp", + "dispersal" + System.nanoTime()).toString(); + private static final Class keyClass = ByteBuffer.class; + private static final Class outputFormatClass = CqlBulkOutputFormat.class; + private final CassandraSinkDataConverter converter; + + public CassandraSSTableSink(@NonNull final CassandraSinkDataConverter converter, + @NonNull final CassandraSinkSchemaManager schemaManager, + @NonNull final CassandraSinkConfiguration conf) { + super(schemaManager, conf); + this.converter = converter; + } + + @Override + public void write(@NonNull final JavaRDD data) { + final Configuration hadoopConf = this.conf.getHadoopConf(); + + log.info("Setting up Cassandra Table"); + this.setupCassandraTable(hadoopConf); + + log.info("Converting data to cassandra payload"); + final RDDWrapper payloadWrapper = this.converter.map(data); + + if (payloadWrapper.getCount() == 0) { + /* + * As a safeguard and precaution, we fail the job if no records are dispersed. The root cause can be + * something innocuous like an accidental empty dataset. But this is to explicitly protect us against + * any data converter bugs that can take valid data and not convert it correctly resulting in an + * incorrect error record. + * + * For dispersal this is ok since we assume the data that existed in Hive has been ingested and conforms + * to a schema. The main trade-off here is if there really is an empty dataset we cannot proceed until + * the customer has deleted that Hive partition. We believe that is much more preferable than a silent + * 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 " + , "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); + } + + 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); + } + + final JavaRDD cassandraRecords = payloadWrapper.getData(); + + log.info("Initializing system properties"); + this.initSystemProperties(cassandraRecords); + + log.info("Mapping cassandra payloads to ByteBuffer RDD values"); + /* + * Need to transform the Cassandra payload here to a kv format of bytebuffer -> List mapping here + * so it can be persisted to Cassandra. The key here doesn't actually matter so a dummy value is used + * because although the Hadoop API uses key & value the Cassandra bulk writer only needs the value. The + * key for the Cassandra row is implicitly derived from the defined primary key in the schema. + */ + final JavaPairRDD byteBufferRDDs = + 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); + + log.info("Finished write process"); + } + + 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 -> { + if (storagePort.isPresent()) { + System.setProperty("cassandra.storage_port", storagePort.get()); + } + + if (sslStoragePort.isPresent()) { + System.setProperty("cassandra.ssl_storage_port", sslStoragePort.get()); + } + }); + } + } + + /** + * This returns a dummy value to conform with Hadoop API but isn't used for Cassandra writing + */ + private ByteBuffer generateDummyValue() { + return Int32Type.instance.decompose(1); + } +} 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 new file mode 100644 index 0000000..82ad7cd --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/cassandra/CassandraSink.java @@ -0,0 +1,149 @@ +/* + * 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.cassandra; + +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 com.google.common.base.Optional; +import com.google.common.base.Strings; +import com.uber.marmaray.common.configuration.CassandraSinkConfiguration; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.schema.cassandra.CassandraSinkSchemaManager; +import com.uber.marmaray.common.sinks.ISink; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.cassandra.hadoop.ConfigHelper; +import org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat; +import org.apache.hadoop.conf.Configuration; + +import java.io.Serializable; +import java.util.List; +import java.util.stream.Collectors; + +/** + * {@link CassandraSink} implements the {@link ISink} interface 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 +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(); + + public CassandraSink(@NonNull final CassandraSinkSchemaManager schemaManager, + @NonNull final CassandraSinkConfiguration conf) { + this.schemaManager = schemaManager; + this.conf = conf; + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.tableMetrics = Optional.of(dataFeedMetrics); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + /** + * This method prepares the Cassandra table for the bulk load + * + * @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 String keySpace = this.conf.getKeyspace(); + + log.info("Connecting cluster with keyspace : {}", keySpace); + + try (final Cluster cluster = builder.build(); + final Session session = cluster.connect(keySpace)) { + ResultSet results; + + log.info("Attempting to getting column names"); + try { + final String columnNameQuery = schemaManager.getColumnNamesFromTableQuery(); + log.info("Getting column names with table query: {}", columnNameQuery); + results = session.execute(columnNameQuery); + } catch (InvalidQueryException e) { + final String columnNameFromCFQuery = schemaManager.getColumnNamesFromColumnFamilyQuery(); + log.error("Saw an InvalidQueryException. Getting column names using column families: {}", + columnNameFromCFQuery); + results = session.execute(columnNameFromCFQuery); + } + + final List columnNames = results.all() + .stream() + .map(r -> r.getString("column_name")) + .collect(Collectors.toList()); + + if (columnNames.isEmpty()) { + log.info("No existing columns found. Executing create table statement: {}", + this.schemaManager.generateCreateTableStmt()); + session.execute(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 -> { + log.info("Executing statement: {}", stmt); + session.execute(stmt); + }); + } + } + + log.info("Using keyspace {}", this.conf.getKeyspace()); + log.info("Using table name: {}", this.conf.getTableName()); + ConfigHelper.setOutputColumnFamily(hadoopConf, + this.conf.getKeyspace(), + this.conf.getTableName()); + + log.info("Using table schema: {}", schemaManager.generateCreateTableStmt()); + CqlBulkOutputFormat.setTableSchema(hadoopConf, + this.conf.getTableName(), + schemaManager.generateCreateTableStmt()); + + log.info("Using insert statement: {}", schemaManager.generateInsertStmt()); + CqlBulkOutputFormat.setTableInsertStatement(hadoopConf, + this.conf.getTableName(), + schemaManager.generateInsertStmt()); + } +} 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 new file mode 100644 index 0000000..4957521 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/AwsFileSink.java @@ -0,0 +1,224 @@ +/* + * 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.file; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest.KeyVersion; +import com.amazonaws.services.s3.model.DeleteObjectsResult; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.amazonaws.util.IOUtils; +import com.uber.marmaray.common.AvroPayload; +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 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.spark_project.guava.annotations.VisibleForTesting; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; + +/** + * {@link AwsFileSink} implements {@link FileSink} interface to build a FileSink + * that first convert data to String with csv format + * and then save to Aws bucket with config defined in {@link AwsConfiguration} + */ +@Slf4j +public class AwsFileSink extends FileSink { + private static final String SUCCESS = "_SUCCESS"; + private static final String CRC = ".crc"; + protected final AmazonS3 s3Client; + private final AwsConfiguration awsConf; + + public AwsFileSink(@NonNull final FileSinkConfiguration conf, + @NonNull final FileSinkDataConverter converter) { + super(conf, converter); + this.awsConf = new AwsConfiguration(conf); + this.s3Client = getS3Connection(); + } + + /** + * This method is used to initialize {@link AwsFileSink#s3Client} + * with aws configurations in {@link AwsConfiguration} + * + * @return new AmazonS3 client + */ + protected AmazonS3 getS3Connection() { + final AWSStaticCredentialsProvider awsCredentialProvider + = new AWSStaticCredentialsProvider( + new BasicAWSCredentials(this.awsConf.getAwsAccessKeyId(), + this.awsConf.getAwsSecretAccessKey())); + return AmazonS3ClientBuilder.standard().withRegion(this.awsConf.getRegion()) + .withCredentials(awsCredentialProvider).build(); + } + + /** + * This method is used to upload single file to aws s3. + * + * @param fileSystem file system of intermediate path + * @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) { + 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) { + log.error("Failed while reading bytes from source path with message %s", e.getMessage()); + throw new JobRuntimeException(e); + } + final ObjectMetadata metadata = new ObjectMetadata(); + metadata.setContentLength(contentBytes.length); + 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(), + objKey, inputStream, metadata); + this.s3Client.putObject(request); + } catch (AmazonServiceException e) { + log.error("Failed while putObject to bucket %s with message %s" + , this.awsConf.getBucketName(), e.getErrorMessage()); + throw new JobRuntimeException(e); + } catch (IOException e) { + log.error("Failed while open source path with %s", e.getMessage()); + throw new JobRuntimeException(e); + } + } + + /** + * This method overrides {@link FileSink#write(JavaRDD)} + * 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)} + * + * @param data data to upload to aws s3 + */ + @Override + public void write(@NonNull final JavaRDD data) { + //Delete temporary path for aws s3 if it exists. + try { + final Path destPath = new Path(this.awsConf.getSourcePath()); + final FileSystem fs = + destPath.getFileSystem(new HadoopConfiguration(this.conf.getConf()).getHadoopConf()); + if (fs.exists(destPath)) { + fs.delete(destPath, true); + } + } catch (IOException e) { + log.error("Job failure while deleting temporary path {} before s3 sink write" + , this.awsConf.getSourcePath()); + throw new JobRuntimeException(e); + } + //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 { + final FileSystem fileSystem = + destPath.getFileSystem(new HadoopConfiguration(this.conf.getConf()).getHadoopConf()); + + //OVERWRITE Mode deletes all existing files in bucket with prefix: objectKey/partitionPath + if (this.conf.getDispersalType() == DispersalType.OVERWRITE) { + log.info("Start to overwrite files."); + final ListObjectsRequest listObjectsRequest + = new ListObjectsRequest().withBucketName(this.awsConf.getBucketName()) + .withPrefix(this.awsConf.getPathKey()); + final ArrayList keysToDelete = new ArrayList<>(); + ObjectListing objects = this.s3Client.listObjects(listObjectsRequest); + int needDeleteNum = 0; + do { + for (S3ObjectSummary objectSummary : objects.getObjectSummaries()) { + keysToDelete.add(new KeyVersion(objectSummary.getKey())); + needDeleteNum++; + } + objects = this.s3Client.listNextBatchOfObjects(objects); + } while (objects.isTruncated()); + //Delete existing objects + if (!keysToDelete.isEmpty()) { + final DeleteObjectsRequest multiObjectDeleteRequest = + new DeleteObjectsRequest(this.awsConf.getBucketName()) + .withKeys(keysToDelete) + .withQuiet(false); + final DeleteObjectsResult delObjRes = this.s3Client.deleteObjects(multiObjectDeleteRequest); + final int successfulDeletes = delObjRes.getDeletedObjects().size(); + // Verify that objects were deleted successfully. + log.info("Total number of overwritten files: {}", successfulDeletes); + if (successfulDeletes != needDeleteNum) { + final String errorMessage = + String.format("aws s3 client failed to delete objects, " + + "expected num: %s, actual num: %s", + needDeleteNum, successfulDeletes); + throw new JobRuntimeException(errorMessage); + } + } else { + log.info("No files to overwrite in aws s3 bucket {} with prefix {}", + this.awsConf.getBucketName(), this.awsConf.getPathKey()); + } + } + //Both OVERWRITE Mode and VERSION Mode upload new file to S3 bucket + int partitionId = 0; + log.info("Start to collect file list."); + final Path temporaryOutput = new Path(this.awsConf.getSourcePath()); + final FileStatus[] status = fileSystem.listStatus(temporaryOutput); + log.info("Start to upload to S3 bucket."); + for (final FileStatus s : status) { + if (s.isFile()) { + final Path path = s.getPath(); + final String fileName = path.getName(); + if (!fileName.equals(SUCCESS) && !fileName.endsWith(CRC)) { + this.uploadFileToS3(fileSystem, path, partitionId); + partitionId += 1; + } + } + } + log.info("Finished uploading to S3 bucket."); + fileSystem.delete(temporaryOutput, true); + log.info("Finished deleting temporary output path: {}", this.awsConf.getSourcePath()); + } catch (IOException e) { + log.error("Failed Job while writing " + + "to s3 bucket %s with error message: %s", this.awsConf.getBucketName(), e.getMessage()); + throw new JobRuntimeException(e); + } + } + + @VisibleForTesting + protected AmazonS3 getS3Client() { + return this.s3Client; + } +} 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 new file mode 100644 index 0000000..1f1c0f5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/FileSink.java @@ -0,0 +1,203 @@ +/* + * 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.file; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.AvroPayload; +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.JobMetrics; +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.spark.api.java.JavaRDD; + +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link FileSink} implements the {@link ISink} interface for a File sink. + * {@link FileSinkDataConverter} converts data from AvroPayload type to String with csv format + * The transformed data is repartitioned as desired and dispersal to required destination path + */ +@Slf4j +public abstract class FileSink implements ISink, Serializable { + private static final long ROW_SAMPLING_THRESHOLD = 100; + private static final int OUTPUT_ONE_FILE_CONFIG = -1; + private static final int DEFAULT_DIGIT_NUM = 5; + private static final int MIN_PARTITION_NUM = 1; + private static final String SINK_INFO_TAG = "file_sink"; + protected final FileSinkConfiguration conf; + protected final FileSinkDataConverter converter; + protected Optional dataFeedMetrics = Optional.absent(); + protected int digitNum; + + public FileSink(@NonNull final FileSinkConfiguration conf, @NonNull final FileSinkDataConverter converter) { + this.conf = conf; + this.converter = converter; + } + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored for now + } + + /** + * This method overrides write method in {@link ISink} and write data to FileSink + * Data first converted by {@link FileSinkDataConverter} + * then repartition by {@link FileSink#getRepartitionNum(JavaRDD)} + * Finally saved in destination {@link FileSinkConfiguration#fullPath} + * + * @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); + 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); + } 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()); + dataToWrite.saveAsTextFile(this.conf.getFullPath()); + log.info("Finished save to path: {}.", this.conf.getFullPath()); + } + + /** + * This method add column header to the first line of each partition in @param data. + * @param header column header + * @param data data with partitions to add column header + * @return new JavaRDD with column header + */ + protected JavaRDD addColumnHeader(@NonNull final String header, @NonNull final JavaRDD data) { + final JavaRDD result = data.mapPartitions((lines) -> { + final List partitionList = IteratorUtils.toList(lines); + partitionList.add(0, header); + return partitionList.iterator(); + }); + return result; + } + + /** + * This method repartition data based on {@link FileSinkConfiguration#fileSizeMegaBytes} + * 1) {@link FileSinkConfiguration#fileSizeMegaBytes} = OUTPUT_ONE_FILE_CONFIG + * => repartition number is 1 + * 2) {@link FileSinkConfiguration#fileSizeMegaBytes} != OUTPUT_ONE_FILE_CONFIG + * => each output file with size in megabytes around {@link FileSinkConfiguration#fileSizeMegaBytes} + * => repartition number = total data size in megabytes {@link FileSink#getRddSizeInMegaByte(JavaRDD)} / required megabytes per file + * + * @param data converted data to calculate repartition number + * @return repartition number of data to save + */ + protected int getRepartitionNum(@NonNull final JavaRDD data) { + final int parNum; + if (this.conf.getFileSizeMegaBytes() == OUTPUT_ONE_FILE_CONFIG) { + parNum = MIN_PARTITION_NUM; + } else { + final double rddSize = getRddSizeInMegaByte(data); + log.info("Write data with megabytes: {}", rddSize); + final int suggestedNum = (int) Math.round(rddSize / this.conf.getFileSizeMegaBytes()); + parNum = suggestedNum < MIN_PARTITION_NUM ? MIN_PARTITION_NUM : suggestedNum; + } + log.info("Write data with repartition number: {} ", parNum); + return parNum; + } + + /** + * This method calculate data size in megabytes + * If total row number > ROW_SAMPLE_THRESHOLD + * => It samples data to row number = ROW_SAMPLE_THRESHOLD + * => Calculate sample data size by {@link FileSink#getSampleSizeInBytes(JavaRDD)} + * => Calculate total data sizes by fraction and change to megabyte + * Else calculate total data size by {@link FileSink#getSampleSizeInBytes(JavaRDD)} + * + * @param data data to calculate size in megabytes + * @return estimated data size in megabytes + */ + protected double getRddSizeInMegaByte(@NonNull final JavaRDD data) { + final RDDWrapper dataWrapper = new RDDWrapper<>(data); + final long totalRows = dataWrapper.getCount(); + final double totalSize; + if (totalRows > ROW_SAMPLING_THRESHOLD) { + log.debug("Start sampling on Write Data."); + final double fraction = (double) ROW_SAMPLING_THRESHOLD / (double) totalRows; + log.debug("Sample fraction: {}", fraction); + final JavaRDD sampleRdd = data.sample(false, fraction); + final long sampleSizeInBytes = getSampleSizeInBytes(sampleRdd); + final double sampleSizeInMB = (double) sampleSizeInBytes / FileUtils.ONE_MB; + totalSize = sampleSizeInMB / fraction; + } else { + totalSize = (double) getSampleSizeInBytes(data) / FileUtils.ONE_MB; + } + return totalSize; + } + + /** + * This method calculates size[bytes] of Strings in data + * + * @param data data to calculate size in bytes + * @return sample size in bytes + */ + protected long getSampleSizeInBytes(@NonNull final JavaRDD data) { + long size = 0; + try { + for (final String row: data.collect()) { + log.debug("Line to calculate: {}", row); + final byte[] utf8Bytes; + utf8Bytes = row.getBytes("UTF-8"); + log.debug("Row Size: {}", utf8Bytes.length); + //Add 1 byte size for end of line '/n' for each sample row + size += utf8Bytes.length + 1; + } + log.debug("Sample size in bytes: {}", size); + } catch (UnsupportedEncodingException e) { + log.error("Failed while calculating sample size: {}", e.getMessage()); + throw new JobRuntimeException(e); + } + return size; + } +} 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 new file mode 100644 index 0000000..c80f413 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/file/HdfsFileSink.java @@ -0,0 +1,101 @@ +/* + * 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.file; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.DispersalType; +import com.uber.marmaray.common.configuration.HadoopConfiguration; +import com.uber.marmaray.common.converters.data.FileSinkDataConverter; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +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 java.io.IOException; + +/** + * {@link HdfsFileSink} implements {@link FileSink} interface to build a FileSink + * that first convert data to String with csv format + * and then save to Hdfs path defined in {@link FileSinkConfiguration#fullPath} + */ +@Slf4j +public class HdfsFileSink extends FileSink { + private static final String SUCCESS = "_SUCCESS"; + private static final String CRC = ".crc"; + public HdfsFileSink(@NonNull final FileSinkConfiguration conf, @NonNull final FileSinkDataConverter converter) { + super(conf, converter); + } + + /** + * This method overrides write method in {@link FileSink} + * If the {@link FileSinkConfiguration#dispersalType} is OVERWRITE, + * it will delete previous files in the destination path. + * Then it will write new files to hdfs path: {@link FileSinkConfiguration #fullpath} + * @param data data to write to hdfs file sink + */ + @Override + public void write(@NonNull final JavaRDD data) { + if (this.conf.getDispersalType() == DispersalType.OVERWRITE) { + log.info("Start to delete previous files."); + log.debug("Full path: {}", this.conf.getPathHdfs()); + final Path dataFolder = new Path(this.conf.getPathHdfs()); + try { + final FileSystem fileSystem = + dataFolder.getFileSystem(new HadoopConfiguration(this.conf.getConf()).getHadoopConf()); + if (fileSystem.exists(dataFolder)) { + fileSystem.delete(dataFolder, true); + } + } catch (IOException e) { + log.error("Exception: {}", e.getMessage()); + throw new JobRuntimeException(e); + } + } + super.write(data); + log.info("Start to rename hdfs files with prefix {}", this.conf.getFileNamePrefix()); + final Path destPath = new Path(this.conf.getPathPrefix()); + try { + final FileSystem fileSystem = + destPath.getFileSystem(new HadoopConfiguration(this.conf.getConf()).getHadoopConf()); + final FileStatus[] status = fileSystem.listStatus(new Path(this.conf.getFullPath())); + int partitionId = 0; + for (final FileStatus s : status) { + if (s.isFile()) { + final Path path = s.getPath(); + final String fileName = path.getName(); + if (!fileName.equals(SUCCESS) && !fileName.endsWith(CRC)) { + final String pathUrl + = String.format("%s_%0" + this.digitNum + "d", this.conf.getFullPath(), partitionId); + final Path newPath = new Path(pathUrl); + fileSystem.rename(path, newPath); + partitionId += 1; + } + } + } + fileSystem.delete(new Path(this.conf.getFullPath()), true); + log.info("Finished write files to hdfs path: {}", this.conf.getFullPath()); + } catch (IOException e) { + 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 new file mode 100644 index 0000000..6b79342 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieErrorSink.java @@ -0,0 +1,87 @@ +/* + * 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; + +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.exception.HoodieInsertException; +import com.uber.hoodie.exception.HoodieUpsertException; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metadata.IMetadataManager; + +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; + +/** + * This class should be used when we need to write error data to Hoodie storage. Error writes need a different + * implementation than {@link HoodieSink} to avoid a loop while writing Errors. If writing errors to Hoodie fails, + * then we throw a {@link JobRuntimeException}. For more details {@see HoodieSink} + */ +@Slf4j +public class HoodieErrorSink extends HoodieSink { + + public HoodieErrorSink(@NonNull final HoodieConfiguration hoodieConf, + @NonNull final HoodieSinkDataConverter hoodieSinkDataConverter, + @NonNull final JavaSparkContext jsc, + @NonNull final HoodieSinkOp op, + @NonNull final IMetadataManager metadataMgr, + final boolean shouldSaveChangesInFuture) { + super(hoodieConf, hoodieSinkDataConverter, jsc, op, metadataMgr, shouldSaveChangesInFuture); + } + + public void writeRecordsAndErrors(@NonNull final HoodieWriteResult result) { + try { + if (result.getException().isPresent()) { + throw result.getException().get(); + } + if (result.getWriteStatuses().isPresent()) { + final JavaRDD> errorRDD = result.getWriteStatuses().get().flatMap( + (FlatMapFunction>) + writeStatus -> writeStatus + .getErrors() + .entrySet().iterator()); + long errorCount = errorRDD.count(); + if (errorCount > 0) { + final Map.Entry firstRecord = errorRDD.first(); + final HoodieKey hoodieKey = firstRecord.getKey(); + final Throwable t = firstRecord.getValue(); + final String errorMsg = String.format("There are errors when writing to error table. " + + "First error record -> HoodieKey : %s, Exception : %s", + hoodieKey.toString(), t.getMessage()); + log.error(errorMsg); + throw new JobRuntimeException("Failed to write error hoodie records. HoodieWriteResult Exception", + result.getException().get()); + } + } + } catch (HoodieInsertException | HoodieUpsertException e) { + log.error("Error writing to hoodie", e); + String errorMsg = "hoodie write failed for errors :" + + (result.getWriteStatuses().isPresent() ? result.getWriteStatuses().get().count() : -1); + throw new JobRuntimeException(errorMsg, e); + } catch (Exception e) { + throw new JobRuntimeException("Error writing to hoodie", e); + } + } +} 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 new file mode 100644 index 0000000..2a91e7c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSink.java @@ -0,0 +1,528 @@ +/* + * 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; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.exception.HoodieInsertException; +import com.uber.hoodie.exception.HoodieUpsertException; +import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RDDWrapper; +import com.uber.marmaray.common.data.RawDataHelper; +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.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +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.utilities.ErrorTableUtil; +import com.uber.marmaray.utilities.FSUtils; +import com.uber.marmaray.utilities.HoodieSinkErrorExtractor; +import com.uber.marmaray.utilities.HoodieUtil; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.util.LongAccumulator; +import org.hibernate.validator.constraints.NotEmpty; +import scala.Option; +import scala.Tuple2; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** + * This class should be used when we need to write data to Hoodie storage. Check {@link HoodieSinkOp} for supported + * operations. {@link HoodieSinkDataConverter} is used to generate {@link HoodieRecord} for given data type. Error records + * generated as a part of either {@link HoodieRecord} generation or while writing to Hoodie storage will be written to + * ErrorTable. If {@link HoodieBasedMetadataManager} is passed in then metadata from it will be saved into newly + * generated hoodie commit file and it will also reset {@link HoodieBasedMetadataManager#shouldSaveChanges()} flag. + */ +@Slf4j +public class HoodieSink implements ISink, scala.Serializable { + + private final HoodieConfiguration hoodieConf; + // It is used for generating HoodieKey from AvroPayload. + private final HoodieSinkDataConverter hoodieSinkDataConverter; + private final transient JavaSparkContext jsc; + private final HoodieSinkOp op; + private Optional dataFeedMetrics = Optional.absent(); + private final Map dataFeedMetricsTags = new HashMap<>(); + /** + * If set to {@link HoodieBasedMetadataManager} then metadata will be retrieved from it and saved into hoodie + * commit file. + */ + private final IMetadataManager metadataMgr; + /** + * Used for managing sink stats. + */ + private final SinkStatManager sinkStatMgr; + private final boolean shouldSaveChangesInFuture; + + @Setter + @NonNull + /** + * If set then it is used for sorting records during {@link HoodieWriteClientWrapper#bulkInsert(JavaRDD, String)} + */ + private Optional bulkInsertPartitioner = Optional.absent(); + + @Setter + @NonNull + @Getter + // If set then it is used for creating new hoodie commit. + private Optional commitTime = Optional.absent(); + + @Setter + @NonNull + @Getter + private HoodieSinkOperations hoodieSinkOperations = new HoodieSinkOperations(); + + 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); + } + + public HoodieSink(@NonNull final HoodieConfiguration hoodieConf, + @NonNull final HoodieSinkDataConverter hoodieSinkDataConverter, + @NonNull final JavaSparkContext jsc, + @NonNull final HoodieSinkOp op, + @NonNull final IMetadataManager metadataMgr, + final boolean shouldSaveChangesInFuture) { + this.hoodieConf = hoodieConf; + this.hoodieSinkDataConverter = hoodieSinkDataConverter; + this.jsc = jsc; + this.op = op; + this.metadataMgr = metadataMgr; + this.sinkStatMgr = new SinkStatManager(this.hoodieConf.getTableName(), this.metadataMgr); + this.sinkStatMgr.init(); + this.shouldSaveChangesInFuture = shouldSaveChangesInFuture; + } + + public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) { + this.dataFeedMetrics = Optional.of(dataFeedMetrics); + } + + public void addDataFeedMetricsTags(@NonNull final Map addtionalTags) { + this.dataFeedMetricsTags.putAll(addtionalTags); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + @Override + public void write(@NonNull final JavaRDD data) { + // Generate HoodieRecord from AvroPayload. It may generate error records in the process and should be + // written to error tables. + final RDDWrapper> hoodieRecords = this.hoodieSinkDataConverter.map(data); + write(hoodieRecords); + } + + public void write(@NonNull final RDDWrapper> hoodieRecords) { + /** + * 1) create new commit -> getOrCreate() + * 2) insert records -> bulkInsert() / insert() / upsert() + * 3) commit() -> commit() + */ + this.initDataset(); + if (this.hoodieConf.shouldAutoTuneParallelism()) { + calculateAndUpdateParallelism(hoodieRecords.getCount()); + } + + final HoodieWriteConfig hoodieWriteConfig = this.hoodieConf.getHoodieWriteConfig(); + try (final HoodieWriteClientWrapper hoodieWriteClient = getHoodieWriteClient(hoodieWriteConfig)) { + final String commitTime = + this.commitTime.isPresent() ? this.commitTime.get() : hoodieWriteClient.startCommit(); + + // Handle writes to hoodie. It can be an insert or upsert. + final HoodieWriteResult result = handleWrite(hoodieWriteClient, hoodieRecords.getData(), commitTime, op); + writeRecordsAndErrors(result, true); + + commit(hoodieWriteClient, commitTime, result.getWriteStatuses()); + } + } + + /** + * Ensure that hoodie dataset is present. + */ + protected void initDataset() { + try { + HoodieUtil.initHoodieDataset(FSUtils.getFs(this.hoodieConf.getConf()), this.hoodieConf); + } catch (IOException e) { + log.error("Error initializing hoodie dataset.", e); + throw new JobRuntimeException("Could not initialize hoodie dataset", e); + } + } + + /** + * If {@link HoodieConfiguration#HOODIE_AUTO_TUNE_PARALLELISM} is enabled then it will use + * {@link HoodieConfiguration#HOODIE_TARGET_FILE_SIZE} and {@link SinkStatManager#getAvgRecordSize()} to figure + * out what should be the optimal insert parallelism. + * @param numRecords + */ + public boolean updateInsertParallelism(final long numRecords) { + if (this.hoodieConf.shouldAutoTuneParallelism()) { + final int newParallelism = calculateNewBulkInsertParallelism(numRecords); + if (0 < newParallelism) { + this.hoodieConf.setTableProperty(HoodieConfiguration.HOODIE_INSERT_PARALLELISM, + Integer.toString(newParallelism)); + log.info("new hoodie insert parallelism is set to :{}", newParallelism); + return true; + } + } + return false; + } + + /** + * If {@link HoodieConfiguration#HOODIE_AUTO_TUNE_PARALLELISM} is enabled then it will use + * {@link HoodieConfiguration#HOODIE_TARGET_FILE_SIZE} and {@link SinkStatManager#getAvgRecordSize()} to figure + * out what should be the optimal bulk insert parallelism. + * @param numRecords + */ + public boolean updateBulkInsertParallelism(final long numRecords) { + if (this.hoodieConf.shouldAutoTuneParallelism()) { + final int newParallelism = calculateNewBulkInsertParallelism(numRecords); + if (0 < newParallelism) { + this.hoodieConf.setTableProperty(HoodieConfiguration.HOODIE_BULKINSERT_PARALLELISM, + Integer.toString(newParallelism)); + log.info("new hoodie bulk insert parallelism is set to :{}", newParallelism); + return true; + } + } + return false; + } + + @VisibleForTesting + protected int calculateNewBulkInsertParallelism(final long numRecords) { + final long avgRecordSize = this.sinkStatMgr.getAvgRecordSize(); + final long targetFileSize = this.hoodieConf.getTargetFileSize(); + final int newParallelism = (int) Math.ceil((numRecords * avgRecordSize * 1.0) / Math.max(1, targetFileSize)); + final int currentParallelism = this.hoodieConf.getBulkInsertParallelism(); + log.info( + "StatsManager:targetFileSize:{}:avgRecordSize:{}:numRecords:{}:" + + "newBulkInsertParallelism:{}:currentBulkInsertParallelism:{}", + targetFileSize, avgRecordSize, numRecords, newParallelism, currentParallelism); + return newParallelism; + } + + @VisibleForTesting + protected HoodieWriteClientWrapper getHoodieWriteClient(@NonNull final HoodieWriteConfig hoodieWriteConfig) { + final HoodieWriteClient hoodieWriteClient = + new HoodieWriteClient(this.jsc, hoodieWriteConfig, + this.hoodieConf.shouldRollbackInFlight()); + return new HoodieWriteClientWrapper(hoodieWriteClient, this.bulkInsertPartitioner); + } + + /** + * If {@link #metadataMgr} is defined then it will retrieve metadata info from metadata manager and reset + * {@link HoodieBasedMetadataManager#shouldSaveChanges()} flag. + */ + public void commit(@NonNull final HoodieWriteClientWrapper hoodieWriteClient, + @NotEmpty final String commitTime, + @NonNull final Optional> writesStatuses) { + this.commit(hoodieWriteClient, commitTime, writesStatuses, this.shouldSaveChangesInFuture); + } + + public void commit(@NonNull final HoodieWriteClientWrapper hoodieWriteClient, + @NotEmpty final String commitTime, + @NonNull final Optional> writesStatuses, + final boolean shouldSaveChangesInFuture) { + updateSinkStat(writesStatuses); + logWriteMetrics(writesStatuses); + + java.util.Optional> hoodieExtraMetadata = java.util.Optional.empty(); + if (this.metadataMgr instanceof HoodieBasedMetadataManager) { + // Retrieve metadata from metadata manager and update metadata manager to avoid it creating extra + // hoodie commit. + final HoodieBasedMetadataManager hoodieBasedMetadataManager = (HoodieBasedMetadataManager) this.metadataMgr; + hoodieExtraMetadata = java.util.Optional.of(hoodieBasedMetadataManager.getMetadataInfo()); + if (!shouldSaveChangesInFuture) { + hoodieBasedMetadataManager.shouldSaveChanges().set(false); + } + } + hoodieSinkOperations.preCommitOperations(this.hoodieConf, commitTime); + if (writesStatuses.isPresent() || hoodieExtraMetadata.isPresent()) { + if (writesStatuses.isPresent()) { + hoodieWriteClient.commit(commitTime, writesStatuses.get(), hoodieExtraMetadata); + } else { + hoodieWriteClient.commit(commitTime, this.jsc.emptyRDD(), hoodieExtraMetadata); + } + } + } + + private void calculateAndUpdateParallelism(final long numRecords) { + switch (this.op) { + case BULK_INSERT: + case DEDUP_BULK_INSERT: + final int newBulkInsertParallelism = calculateNewBulkInsertParallelism(numRecords); + updateBulkInsertParallelism(newBulkInsertParallelism); + break; + case INSERT: + case DEDUP_INSERT: + // no-op + log.warn("Hoodie insert parallelism is not updated."); + break; + case UPSERT: + // no-op + log.warn("Hoodie upsert parallelism is not updated."); + break; + default: + throw new JobRuntimeException("Cannot update the parallelism for HoodieOP " + this.op); + } + } + + private void updateBulkInsertParallelism(final int newParallelism) { + if (newParallelism > 0) { + this.hoodieConf.setTableProperty(HoodieConfiguration.HOODIE_BULKINSERT_PARALLELISM, + Integer.toString(newParallelism)); + log.info("new hoodie bulk insert parallelism is set to :{}", newParallelism); + } + } + + private void logWriteMetrics(final Optional> writesStatuses) { + if (writesStatuses.isPresent() && this.dataFeedMetrics.isPresent()) { + final LongAccumulator totalCount = writesStatuses.get().rdd().sparkContext().longAccumulator(); + final LongAccumulator errorCount = writesStatuses.get().rdd().sparkContext().longAccumulator(); + writesStatuses.get().foreach(writeStatus -> { + errorCount.add(writeStatus.getFailedRecords().size()); + totalCount.add(writeStatus.getTotalRecords()); + }); + this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.ERROR_ROWCOUNT, errorCount.value(), + this.dataFeedMetricsTags); + this.dataFeedMetrics.get().createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, + totalCount.value() - errorCount.value(), this.dataFeedMetricsTags); + } + } + + /** + * {@link #updateSinkStat(Optional)} will compute {@link SinkStat} and persist changes into {@link IMetadataManager}. + * As a part of {@link SinkStat} computation; it will compute avg record size for current run. + * @param writesStatuses + */ + private void updateSinkStat(final Optional> writesStatuses) { + if (writesStatuses.isPresent()) { + final LongAccumulator avgRecordSizeCounter = writesStatuses.get().rdd().sparkContext().longAccumulator(); + writesStatuses.get().foreach( + writeStatus -> { + final long writeBytes = writeStatus.getStat().getTotalWriteBytes(); + final long numInserts = writeStatus.getStat().getNumWrites() + - writeStatus.getStat().getNumUpdateWrites(); + if (writeBytes > 0 && numInserts > 0) { + avgRecordSizeCounter.add(writeBytes / numInserts); + } + } + ); + final long avgRecordSize = (int) avgRecordSizeCounter.avg(); + if (avgRecordSize > 0) { + log.info("Updating Sink Stat manager : avgRecordSize : {}", avgRecordSize); + this.sinkStatMgr.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Long.toString(avgRecordSize)); + } + } + this.sinkStatMgr.persist(); + } + + public void writeRecordsAndErrors(@NonNull final HoodieWriteResult result, + final boolean isErrorTableEnabled) { + try { + if (result.getException().isPresent()) { + throw result.getException().get(); + } + if (result.getWriteStatuses().isPresent()) { + if (isErrorTableEnabled) { + // TODO: Can we make this more readable, please? + final JavaRDD> hoodieRecordAndErrorTupleRDD + = result.getWriteStatuses().get() + .flatMap(ws -> ws.getFailedRecords().stream().map(fr -> + new Tuple2<>(fr, ws.getErrors().get(fr.getKey()).getMessage())).iterator()); + + final JavaRDD errorRDD = hoodieRecordAndErrorTupleRDD + .map(r -> new ErrorData(r._2, RawDataHelper.getRawData(r._1))); + + ErrorTableUtil.writeErrorRecordsToErrorTable(this.jsc.sc(), + this.hoodieConf.getConf(), Optional.of(this.hoodieConf.getTableName()), + new RDDWrapper<>(errorRDD), new HoodieSinkErrorExtractor()); + } + } + } catch (HoodieInsertException | HoodieUpsertException e) { + log.error("Error writing to hoodie", e); + throw new JobRuntimeException("hoodie write failed :" + + (result.getWriteStatuses().isPresent() ? result.getWriteStatuses().get().count() : -1), e); + } catch (Exception e) { + throw new JobRuntimeException("Error writing to hoodie", e); + } + } + + public HoodieWriteResult handleWrite( + @NonNull final HoodieWriteClientWrapper writeClient, + @NonNull final JavaRDD> hoodieRecords, + @NotEmpty final String commitTime, + @NonNull final HoodieSinkOp op) { + + Optional> writeStatuses = Optional.absent(); + Optional exception = Optional.absent(); + final JavaRDD> filteredRecords; + try { + switch (op) { + case INSERT: + writeStatuses = Optional.of(writeClient.insert(hoodieRecords, commitTime)); + break; + case BULK_INSERT: + writeStatuses = Optional.of(writeClient.bulkInsert(hoodieRecords, commitTime)); + break; + case DEDUP_INSERT: + filteredRecords = dedupRecords(writeClient, hoodieRecords); + writeStatuses = Optional.of(writeClient.insert(filteredRecords, commitTime)); + break; + case DEDUP_BULK_INSERT: + filteredRecords = dedupRecords(writeClient, hoodieRecords); + writeStatuses = Optional.of(writeClient.bulkInsert(filteredRecords, commitTime)); + break; + case UPSERT: + writeStatuses = Optional.of(writeClient.upsert(hoodieRecords, commitTime)); + break; + default: + exception = Optional.of(new JobRuntimeException("Unsupported hoodie sink operation:" + op)); + } + } catch (Exception e) { + exception = Optional.of(e); + } + return new HoodieWriteResult(writeStatuses, exception); + } + + private JavaRDD> dedupRecords(@NonNull final HoodieWriteClientWrapper writeClient, + @NonNull final JavaRDD> hoodieRecords) { + return writeClient.filterExists(hoodieRecords).persist(StorageLevel.DISK_ONLY()); + } + + /** + * This class is a wrapper around hoodie write client to enable testing. + */ + @VisibleForTesting + @AllArgsConstructor + public static class HoodieWriteClientWrapper implements Serializable, Closeable { + + @VisibleForTesting + @Getter + private final HoodieWriteClient hoodieWriteClient; + private final Optional bulkInsertPartitioner; + + public String startCommit() { + return this.hoodieWriteClient.startCommit(); + } + + public void startCommitWithTime(@NotEmpty final String commitTime) { + this.hoodieWriteClient.startCommitWithTime(commitTime); + } + + public boolean commit(@NotEmpty final String commitTime, @NonNull final JavaRDD writeStatuses, + final java.util.Optional> extraMetadata) { + return this.hoodieWriteClient.commit(commitTime, writeStatuses, extraMetadata); + } + + public JavaRDD insert(@NonNull final JavaRDD> records, + @NotEmpty final String commitTime) { + return this.hoodieWriteClient.insert(records, commitTime); + } + + public 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); + } + } + + public JavaRDD upsert(@NonNull final JavaRDD> records, + @NotEmpty final String commitTime) { + return this.hoodieWriteClient.upsert(records, commitTime); + } + + public JavaRDD upsertPreppedRecords( + @NonNull final JavaRDD> records, + @NotEmpty final String commitTime) { + return this.hoodieWriteClient.upsertPreppedRecords(records, commitTime); + } + + public JavaRDD bulkInsertPreppedRecords( + @NonNull final JavaRDD> records, + @NotEmpty final String commitTime, + @NonNull final Option partitioner) { + return this.hoodieWriteClient.bulkInsertPreppedRecords(records, commitTime, partitioner); + } + + public void close() { + this.hoodieWriteClient.close(); + } + + public JavaRDD> filterExists( + final JavaRDD> hoodieRecords) { + return this.hoodieWriteClient.filterExists(hoodieRecords); + } + } + + /** + * Supported hoodie write operations. + */ + public enum HoodieSinkOp { + /** {@link HoodieWriteClient#insert(JavaRDD, String)}*/ + INSERT, + /** {@link HoodieWriteClient#bulkInsert(JavaRDD, String)}*/ + BULK_INSERT, + /** {@link HoodieWriteClient#insert(JavaRDD, String)} {@link HoodieWriteClient#filterExists(JavaRDD)}*/ + DEDUP_INSERT, + /** {@link HoodieWriteClient#bulkInsert(JavaRDD, String)} {@link HoodieWriteClient#filterExists(JavaRDD)}*/ + DEDUP_BULK_INSERT, + /** {@link com.uber.hoodie.HoodieWriteClient#upsert(org.apache.spark.api.java.JavaRDD, java.lang.String)}*/ + UPSERT, + /** No operation */ + NO_OP + } + + @Getter + @AllArgsConstructor + public class HoodieWriteResult { + @NonNull + private final Optional> writeStatuses; + @NonNull + private final Optional exception; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSinkOperations.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSinkOperations.java new file mode 100644 index 0000000..f93d543 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieSinkOperations.java @@ -0,0 +1,37 @@ +/* + * 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; + +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import lombok.NonNull; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Helper class which invokes various operations before / after certain {@link HoodieSink} actions. See individual + * operations for more details. + */ +public class HoodieSinkOperations { + + /** + * Gets executed before calling {@link HoodieSink}'s underlying commit action. All the parquet write operations are + * guaranteed to finish before this. Only thing left is the final commit file creation. + */ + public void preCommitOperations(@NonNull final HoodieConfiguration hoodieConfiguration, + @NotEmpty final String commitTime) { + // do nothing. + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieWriteStatus.java b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieWriteStatus.java new file mode 100644 index 0000000..eb54222 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sinks/hoodie/HoodieWriteStatus.java @@ -0,0 +1,44 @@ +/* + * 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; + +import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.model.HoodieRecord; +import java.util.Map; +import java.util.Optional; + +/** + * Helper class to change default behavior for {@link WriteStatus} + */ +public class HoodieWriteStatus extends WriteStatus { + + private long totalRecords; + + /** + * Overriding {@link #markSuccess(HoodieRecord, Optional)} to avoid caching + * {@link com.uber.hoodie.common.model.HoodieKey} for successfully written hoodie records. + */ + @Override + public void markSuccess(final HoodieRecord record, final Optional> optionalRecordMetadata) { + this.totalRecords++; + } + + @Override + public long getTotalRecords() { + return super.getTotalRecords() + this.totalRecords; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/IRunState.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/IRunState.java new file mode 100644 index 0000000..4315460 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/IRunState.java @@ -0,0 +1,24 @@ +/* + * 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.sources; + +/** + * It is an abstraction to hold job run state. + */ +public interface IRunState { + +} 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 new file mode 100644 index 0000000..efb4f0d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/ISource.java @@ -0,0 +1,34 @@ +/* + * 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.sources; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +import org.apache.spark.api.java.JavaRDD; + +/** + * @param {@link IWorkUnitCalculator} which calculates work units for the source. + */ +public interface ISource extends IMetricable { + + /** + * It reads and returns the data. + */ + JavaRDD getData(K k); +} 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 new file mode 100644 index 0000000..f054933 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/IWorkUnitCalculator.java @@ -0,0 +1,87 @@ +/* + * 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.sources; + +import com.uber.marmaray.common.metadata.AbstractValue; +import com.uber.marmaray.common.metadata.IMetadataManager; +import com.uber.marmaray.common.metrics.IChargebackCalculator; +import com.uber.marmaray.common.metrics.IMetricable; +import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult; +import java.util.List; + +/** + * It uses previous {@link IRunState} information to compute WorkUnits for next run. It will also compute + * {@link IRunState} for the next run. + * + * @param WorkUnit + * @param {@link IRunState} which holds run state information for the run. + */ +public interface IWorkUnitCalculator, + K extends IWorkUnitCalculatorResult, V extends AbstractValue> extends IMetricable { + + /** + * Initializes previous run state using the state stored in {@link IMetadataManager} + * @param metadataManager + */ + void initPreviousRunState(IMetadataManager metadataManager); + + /** + * Saves next run state in {@link IMetadataManager} + * @param metadataManager + * @param nextRunState + */ + void saveNextRunState(IMetadataManager metadataManager, S nextRunState); + + /** + * It computes {@link IWorkUnitCalculatorResult} which holds information about work units for current run + * and also about the run state for next run if the current run succeeds. Avoid saving next run state until + * current job has succeeded. + */ + K computeWorkUnits(); + + /** + * Compute the cost of the execution of this work unit. + */ + void setChargebackCalculator(IChargebackCalculator calculator); + + /** + * It is returned from {@link #computeWorkUnits()} when computation is successful. Constructor for it should be + * kept private to ensure that it is only created from within {@link IWorkUnitCalculator}. + * + * @param WorkUnit + * @param {@link IRunState} which holds run state information for the run. + */ + interface IWorkUnitCalculatorResult { + + /** + * Returns new set of work units for the run. + */ + List getWorkUnits(); + + /** + * Returns true if there are work units available to read. It should throw + * {@link com.uber.marmaray.common.exceptions.JobRuntimeException} if it is called before workunits are + * computed for the current job run. + */ + boolean hasWorkUnits(); + + /** + * Returns {@link IRunState} for next run. + */ + S getNextRunState(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveRunState.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveRunState.java new file mode 100644 index 0000000..010c376 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveRunState.java @@ -0,0 +1,30 @@ +/* + * 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.sources.hive; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.sources.IRunState; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; + +@EqualsAndHashCode +@AllArgsConstructor +public class HiveRunState implements IRunState { + @Getter + private Optional partition; +} 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 new file mode 100644 index 0000000..976cecb --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/HiveSource.java @@ -0,0 +1,100 @@ +/* + * 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.sources.hive; + +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.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.sources.ISource; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import parquet.Preconditions; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +@Slf4j +@AllArgsConstructor +public class HiveSource implements ISource, Serializable { + + @Getter + final HiveSourceConfiguration hiveConf; + + private final SQLContext sqlContext; + + private final SparkSourceDataConverter converter; + + public void setDataFeedMetrics(final DataFeedMetrics dataFeedMetrics) { + // ignored + } + + public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + + @Override + public JavaRDD getData(@NonNull final ParquetWorkUnitCalculatorResult workUnitCalcResult) { + Preconditions.checkState(workUnitCalcResult.hasWorkUnits(), + "No work to process for: " + hiveConf.getDataPath()); + + /** + * Current implementation of HiveSource assumes that only a single work unit exists which + * corresponds to the single partition that is processed per job. + */ + final List workUnits = workUnitCalcResult.getWorkUnits(); + + 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 int numPartitions = calculateHiveNumPartitions(data); + + log.info("Using {} partitions", numPartitions); + + final JavaRDD hiveRawData = 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; + } + + private int calculateHiveNumPartitions(@NonNull final Dataset data) { + /* + * For now we just return the number of partitions in the underlying RDD, but in the future we can define + * the type of strategy in the configuration and heuristically calculate the number of partitions. + * + * todo: T923425 to actually do the heuristic calculation to optimize num partitions + */ + return data.rdd().getNumPartitions(); + } +} 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 new file mode 100644 index 0000000..163cb71 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculator.java @@ -0,0 +1,141 @@ +/* + * 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.sources.hive; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metadata.HDFSDatePartitionManager; +import com.uber.marmaray.common.metadata.HDFSPartitionManager; +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.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IChargebackCalculator; +import com.uber.marmaray.common.metrics.JobMetrics; +import com.uber.marmaray.common.sources.IWorkUnitCalculator; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +@Slf4j +public class ParquetWorkUnitCalculator implements + IWorkUnitCalculator { + + @Getter + private Optional nextPartition = Optional.absent(); + + @Override + 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 + } + + @Override public void setJobMetrics(final JobMetrics jobMetrics) { + // ignored + } + + @Override public void setChargebackCalculator(final IChargebackCalculator calculator) { + // ignored + } + + @Override + public void initPreviousRunState(@NonNull final IMetadataManager metadataManager) { + try { + if (metadataManager instanceof HDFSPartitionManager) { + this.nextPartition = ((HDFSPartitionManager) metadataManager).getNextPartition(); + } else if (metadataManager instanceof HDFSDatePartitionManager) { + this.nextPartition = ((HDFSDatePartitionManager) metadataManager).getNextPartition(); + } else { + throw new JobRuntimeException("The only supported metadata managers for ParquetWorkUnitCalculator" + + " are HDFSPartitionManager or HDFSDataPartitionManager"); + } + } catch (final IOException e) { + throw new JobRuntimeException("Unable to get the next partition. Error message: " + this.nextPartition, e); + } + } + + @Override + public void saveNextRunState(@NonNull final IMetadataManager metadataManager, + @NonNull final HiveRunState nextRunState) { + /* + * For the Hive/Parquet use case we don't need the nextRunState information. + * The current checkpoint saved is used to determine the next partition to process + * when the next run is processed, especially since we don't know when another hive partition + * will be added. There is therefore no need to calculate the next run state during execution of job. + * + * Until we add Cassandra metadata information, we assume explicitly this is a HDFSPartitionManager. + * Todo: T898695 - Implement metadata manager using Cassandra backend + */ + Preconditions.checkState(metadataManager instanceof HDFSPartitionManager); + final HDFSPartitionManager partitionManager = (HDFSPartitionManager) metadataManager; + + if (!this.nextPartition.isPresent()) { + log.warn("No partition was found to process. Reusing latest checkpoint if exists as checkpoint key"); + if (partitionManager.getLatestCheckpoint().isPresent()) { + metadataManager.set(MetadataConstants.CHECKPOINT_KEY, partitionManager.getLatestCheckpoint().get()); + } + return; + } + + if (partitionManager.isSinglePartition()) { + metadataManager.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(this.nextPartition.get())); + } else { + /* + * We explicitly always save the latest checkpoint in the metadata file. Even in cases where + * we explicitly reprocess a older single existing partition of a Hive table, we write out the latest + * checkpoint that we have processed so on the next run we can continue processing at the latest point. + */ + if (checkpointGreaterThanNextPartition(partitionManager.getLatestCheckpoint())) { + metadataManager.set(MetadataConstants.CHECKPOINT_KEY, partitionManager.getLatestCheckpoint().get()); + } else { + metadataManager.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(this.nextPartition.get())); + } + } + } + + @Override + public ParquetWorkUnitCalculatorResult computeWorkUnits() { + /** + * The logic for computing work units is pretty straightforward here. + * + * We are making the explicit assumption & trade-off for now that each job run + * only processes data for one partition in Hive. + * + * The partition manager will have enough context to automatically determine the + * next partition to process and this partition will also be saved as the next checkpoint + * which is why it is returned as the entry in the next run state. Only if the job succeeds will the + * 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; + return new ParquetWorkUnitCalculatorResult(workUnits, nextRunState); + } + + private boolean checkpointGreaterThanNextPartition(@NonNull final Optional checkPoint) { + if (checkPoint.isPresent() + && checkPoint.get().getValue().compareTo(this.nextPartition.get()) > 0) { + return true; + } + return false; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculatorResult.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculatorResult.java new file mode 100644 index 0000000..02c54bf --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/hive/ParquetWorkUnitCalculatorResult.java @@ -0,0 +1,47 @@ +/* + * 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.sources.hive; + +import com.uber.marmaray.common.sources.IWorkUnitCalculator; +import lombok.Getter; +import lombok.NonNull; + +import java.util.List; + +final class ParquetWorkUnitCalculatorResult + implements IWorkUnitCalculator.IWorkUnitCalculatorResult { + + @Getter + private final HiveRunState nextRunState; + + /** + * A workunit in this context is the name of a Hive partition + */ + @Getter + private final List workUnits; + + public ParquetWorkUnitCalculatorResult(@NonNull final List workUnits, + @NonNull final HiveRunState nextRunState) { + this.workUnits = workUnits; + this.nextRunState = nextRunState; + } + + @Override + public boolean hasWorkUnits() { + return !this.workUnits.isEmpty(); + } +} 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 new file mode 100644 index 0000000..614cc7a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaBootstrapOffsetSelector.java @@ -0,0 +1,46 @@ +/* + * 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.sources.kafka; + +import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.common.TopicPartition; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Helper class used to select partition offsets when there are no previous checkpoints available to start + * reading data from kafka topic. See {@link KafkaWorkUnitCalculator} for how it is used. + */ +@Slf4j +public class KafkaBootstrapOffsetSelector { + + /** + * @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) { + 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/KafkaRunState.java b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaRunState.java new file mode 100644 index 0000000..e5941a8 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaRunState.java @@ -0,0 +1,50 @@ +/* + * 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.sources.kafka; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.sources.IRunState; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** + * It holds the run state of the kafka run. This state needs to be persisted into checkpoint manager. + */ +@AllArgsConstructor +public class KafkaRunState implements IRunState { + @Getter + private Map partitionOffsets; + + /** + * It sets partition offset. + */ + public void setPartitionOffset(final int partition, final long offset) { + this.partitionOffsets.put(partition, offset); + } + + /** + * Get offset for a partition. + */ + public Optional getPartitionOffset(final int partition) { + if (this.partitionOffsets.containsKey(partition)) { + return Optional.of(this.partitionOffsets.get(partition)); + } else { + return Optional.absent(); + } + } +} 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 new file mode 100644 index 0000000..e9128b4 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaSource.java @@ -0,0 +1,214 @@ +/* + * 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.sources.kafka; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; +import com.uber.marmaray.common.converters.data.KafkaSourceDataConverter; +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 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.Partitioner; +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.streaming.kafka010.OffsetRange; +import org.hibernate.validator.constraints.NotEmpty; +import scala.Serializable; +import scala.Tuple2; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * It gets work units from {@link KafkaWorkUnitCalculatorResult} as a list of {@link OffsetRange}, reads messages from + * kafka and returns {@link JavaRDD}. + */ + +@Slf4j +@AllArgsConstructor +public class KafkaSource implements ISource, Serializable { + + @Getter + private final KafkaSourceConfiguration conf; + @Getter + private transient Optional jsc = Optional.absent(); + @Getter + private final KafkaSourceDataConverter dataConverter; + @Getter + private final Optional> startDateFilterFunction; + @Setter + @NonNull + private final Optional> filterRecordHandler; + + public void setDataFeedMetrics(@NonNull final DataFeedMetrics topicMetrics) { + // ignored + } + + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + @Override + public JavaRDD getData(@NonNull final KafkaWorkUnitCalculatorResult workUnitCalc) { + Preconditions.checkState(workUnitCalc.hasWorkUnits(), "no work to do :" + this.conf.getTopicName()); + final List workUnits = workUnitCalc.getWorkUnits(); + /** + * Since we are not opening more than one connection per "topic+partition"; so single spark partition may end up + * reading data more than spark's 2G partition limit. In order to avoid this and also to take advantage of all + * executors we will be repartitioning kafkaData. + */ + final int readParallelism; + if (workUnits.size() < this.conf.getReadParallelism()) { + readParallelism = this.conf.getReadParallelism(); + } else { + readParallelism = workUnits.size(); + } + 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()) + .mapToPair( + new PairFunction, Integer, byte[]>() { + int lastSparkPartition = -1; + + @Override + public Tuple2 call(final ConsumerRecord v) throws Exception { + final int sparkPartition = + kafkaPartitionOffsetToSparkPartitionMap.get(v.partition()) + .floorEntry(v.offset()).getValue(); + if (lastSparkPartition != sparkPartition) { + lastSparkPartition = sparkPartition; + log.info("starting new spark partition == kafkaPartition:{} offset:{} sparkPartition:{}", + v.partition(), v.offset(), sparkPartition); + } + return new Tuple2<>(sparkPartition, v.value()); + } + } + ).partitionBy( + new Partitioner() { + @Override + public int numPartitions() { + return readParallelism; + } + + @Override + public int getPartition(final Object key) { + return (Integer) key; + } + }) + .values(); + + final JavaRDD inputRDD = this.dataConverter.map(kafkaData).getData(); + + 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; + } + + /** + * It maps kafka partition's offset ranges to output spark partitions such that each every output spark partition + * gets equal number of messages. + */ + @VisibleForTesting + public static Map> getKafkaPartitionOffsetToOutputSparkPartitionMap( + @NotEmpty final String topicName, @NonNull final List offsetRanges, 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; + + while (true) { + long currentOffsetRangeMsgCnt = currentOffsetRange.untilOffset() - currentOffsetRange.fromOffset(); + if (currentOffsetRangeMsgCnt == 0) { + if (offsetRangesI.hasNext()) { + currentOffsetRange = offsetRangesI.next(); + continue; + } else { + break; + } + } + if (!retMap.containsKey(currentOffsetRange.partition())) { + retMap.put(currentOffsetRange.partition(), new TreeMap<>()); + } + if (remainingSparkPartitionMessageCapacity == 0) { + remainingSparkPartitionMessageCapacity = messagesPerPartition; + outputSparkPartition++; + } + retMap.get(currentOffsetRange.partition()).put(currentOffsetRange.fromOffset(), outputSparkPartition); + + if (currentOffsetRangeMsgCnt < remainingSparkPartitionMessageCapacity) { + remainingSparkPartitionMessageCapacity -= currentOffsetRangeMsgCnt; + currentOffsetRange = OffsetRange.create(currentOffsetRange.topic(), + currentOffsetRange.partition(), currentOffsetRange.fromOffset() + currentOffsetRangeMsgCnt, + currentOffsetRange.untilOffset()); + } else { + currentOffsetRange = OffsetRange.create(currentOffsetRange.topic(), + currentOffsetRange.partition(), + currentOffsetRange.fromOffset() + remainingSparkPartitionMessageCapacity, + currentOffsetRange.untilOffset()); + 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 new file mode 100644 index 0000000..9dfb65e --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/sources/kafka/KafkaWorkUnitCalculator.java @@ -0,0 +1,433 @@ +/* + * 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.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; +import com.uber.marmaray.common.metadata.IMetadataManager; +import com.uber.marmaray.common.metadata.StringValue; +import com.uber.marmaray.common.metrics.ChargebackMetricType; +import com.uber.marmaray.common.metrics.DataFeedMetricNames; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.IChargebackCalculator; +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.utilities.StringTypes; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.streaming.kafka010.OffsetRange; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.ArrayList; +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; + +import static com.uber.marmaray.utilities.KafkaUtil.getEarliestLeaderOffsets; +import static com.uber.marmaray.utilities.KafkaUtil.getKafkaConsumer; +import static com.uber.marmaray.utilities.KafkaUtil.getLatestLeaderOffsets; +import static com.uber.marmaray.utilities.KafkaUtil.getTopicPartitions; + +/** + * {@link KafkaWorkUnitCalculator} uses previous job run state to compute work units for the current run and also + * creates run state for next run. Previous {@link KafkaRunState} should be set using + * {@link #initPreviousRunState(IMetadataManager)} before computing work units. {@link #computeWorkUnits()} will + * compute work units and will create {@link KafkaWorkUnitCalculatorResult} to hold next run state and work units for . + * Take a look at {@link KafkaSourceConfiguration} for supported configuration settings. + */ +@Slf4j +@RequiredArgsConstructor +public class KafkaWorkUnitCalculator implements IWorkUnitCalculator { + + public static final String KAFKA_METADATA_PREFIX = "kafka_metadata"; + public static final String KAFKA_METADATA_WITH_SEPARATOR = KAFKA_METADATA_PREFIX + StringTypes.COLON; + public static final String PARTITION_TAG = "partition"; + public static final String TOTAL_PARTITION = "total"; + + @Getter + private final KafkaSourceConfiguration conf; + @Getter + @Setter + private KafkaBootstrapOffsetSelector offsetSelector = new KafkaBootstrapOffsetSelector(); + @Getter + private Optional previousRunState = Optional.absent(); + + private Optional topicMetrics = Optional.absent(); + + private Optional chargebackCalculator = Optional.absent(); + + @Override + public void setDataFeedMetrics(@NonNull final DataFeedMetrics topicMetrics) { + this.topicMetrics = Optional.of(topicMetrics); + } + + @Override + public void setJobMetrics(@NonNull final JobMetrics jobMetrics) { + // ignored + } + + @Override + public void setChargebackCalculator(@NonNull final IChargebackCalculator chargebackCalculator) { + this.chargebackCalculator = Optional.of(chargebackCalculator); + } + + @Override + public void initPreviousRunState(@NonNull final IMetadataManager metadataManager) { + final String topicName = this.conf.getTopicName(); + final Map metadata = new HashMap<>(); + final String topicSpecificName = getTopicSpecificMetadataKey(topicName); + final List toDelete = new LinkedList<>(); + metadataManager.getAllKeys().forEach(key -> { + if (key.startsWith(topicSpecificName)) { + // this is my specific topic + metadata.put(Integer.parseInt(key.substring(topicSpecificName.length())), + Long.parseLong(metadataManager.get(key).get().getValue())); + } else if (key.startsWith(KAFKA_METADATA_WITH_SEPARATOR)) { + // this is a specific topic, but not mine. ignore. + assert true; + } else if (key.startsWith(KAFKA_METADATA_PREFIX)) { + // this is unspecified topic + metadata.put(Integer.parseInt(key.substring(KAFKA_METADATA_PREFIX.length())), + Long.parseLong(metadataManager.get(key).get().getValue())); + // delete the old, unspecified metadata + toDelete.add(key); + } + }); + toDelete.forEach(metadataManager::remove); + this.previousRunState = Optional.of(new KafkaRunState(metadata)); + } + + @VisibleForTesting + String getTopicSpecificMetadataKey(@NotEmpty final String topicName) { + return String.format("%s%s:", KAFKA_METADATA_WITH_SEPARATOR, topicName); + } + + @Override + public void saveNextRunState(@NonNull final IMetadataManager metadataManager, + final KafkaRunState nextRunState) { + final String topicName = this.conf.getTopicName(); + final String topicSpecificName = getTopicSpecificMetadataKey(topicName); + nextRunState.getPartitionOffsets().entrySet().forEach( + entry -> { + metadataManager.set(topicSpecificName + entry.getKey(), new StringValue(entry.getValue().toString())); + }); + } + + @Override + public KafkaWorkUnitCalculatorResult computeWorkUnits() { + final String topicName = this.conf.getTopicName(); + final KafkaConsumer kafkaConsumer = getKafkaConsumer(this.conf.getKafkaParams()); + + final Set topicPartitions; + final Map earliestLeaderOffsets; + final Map latestLeaderOffsets; + try { + // Retrieve topic partitions and leader offsets. + topicPartitions = getTopicPartitions(kafkaConsumer, topicName); + earliestLeaderOffsets = getEarliestLeaderOffsets(kafkaConsumer, topicName, topicPartitions); + latestLeaderOffsets = getLatestLeaderOffsets(kafkaConsumer, topicName, topicPartitions); + } finally { + kafkaConsumer.close(); + } + + // Read checkpointed topic partition offsets and update it with newly added partitions. + final Map oldPartitionOffsets = readExistingPartitionOffsets(); + 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))); + } + 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())) { + 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()); + log.debug("topicPartition:{}:messages:{}:latestOffset:{}", entry.getKey(), messages, entry.getValue()); + if (messages == 0) { + continue; + } + totalNewMessages += messages; + partitionMessages.add(new PartitionMessages(entry.getKey(), messages)); + } + if (partitionMessages.isEmpty()) { + // No messges to read. + log.info("No new offsets are found. :{}", topicName); + return new KafkaWorkUnitCalculatorResult(Collections.emptyList(), new KafkaRunState(oldPartitionOffsets)); + } + final List workUnits = + calculatePartitionOffsetRangesToRead(partitionMessages, oldPartitionOffsets, + totalNewMessages); + // compute run state for the next run. + final KafkaRunState nextRunState = createNextRunState(workUnits); + final KafkaWorkUnitCalculatorResult kafkaWorkUnitCalculatorResult = + new KafkaWorkUnitCalculatorResult(workUnits, nextRunState); + + computeRunMetrics(latestLeaderOffsets, nextRunState, workUnits); + log.info("workunits: {}", kafkaWorkUnitCalculatorResult); + return kafkaWorkUnitCalculatorResult; + } + + private List calculatePartitionOffsetRangesToRead( + @NonNull final List partitionMessages, + @NonNull final Map partitionStartOffsets, final long numMessages) { + // This will make sure that we can read more messages from partition with more than average messages per + // partition at the same time we will read all the messages from partition with less than avg messags. + Collections.sort(partitionMessages); + final long maxMessagesToRead = this.conf.getMaxMessagesToRead(); + log.info("topicName:{}:newMessages:{}:maxMessagesToRead:{}", this.conf.getTopicName(), numMessages, + maxMessagesToRead); + final boolean hasExtraMessages = numMessages > maxMessagesToRead; + final long numMessagesToRead = Math.min(numMessages, maxMessagesToRead); + + final List offsetRanges = new ArrayList<>(partitionMessages.size()); + long pendingMessages = numMessagesToRead; + int pendingPartitions = partitionMessages.size(); + for (final PartitionMessages m : partitionMessages) { + final long numMsgsToBeSelected; + if (!hasExtraMessages) { + numMsgsToBeSelected = m.getMessages(); + } else { + numMsgsToBeSelected = Math.min(pendingMessages / pendingPartitions, m.getMessages()); + pendingMessages -= numMsgsToBeSelected; + pendingPartitions--; + } + if (numMsgsToBeSelected > 0) { + offsetRanges.add(OffsetRange.create(m.getTopicPartition(), + partitionStartOffsets.get(m.getTopicPartition().partition()), + partitionStartOffsets.get(m.getTopicPartition().partition()) + numMsgsToBeSelected)); + } + } + return offsetRanges; + } + + // Helper method to read existing KafkaRunState. + private Map readExistingPartitionOffsets() { + if (!this.previousRunState.isPresent()) { + throw new JobRuntimeException("Previous run state is not set."); + } + final Map ret = this.previousRunState.get().getPartitionOffsets(); + log.info("existing partition offset :{}", ret); + return ret; + } + + private KafkaRunState createNextRunState(@NonNull final List workUnits) { + final Map partitionOffsets = new HashMap<>(); + workUnits.forEach( + offsetRange -> { + final int partition = offsetRange.partition(); + if (partitionOffsets.containsKey(partition)) { + partitionOffsets + .put(partition, Math.max(partitionOffsets.get(partition), offsetRange.untilOffset())); + } else { + partitionOffsets.put(partition, offsetRange.untilOffset()); + } + } + ); + return new KafkaRunState(partitionOffsets); + } + + /* + 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) { + 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(), + 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); + } + } + ); + } + } + + /* + Creates metrics for the current execution based on the source. + */ + private void computeRunMetrics(@NonNull final Map latestLeaderOffsets, + @NonNull final KafkaRunState nextRunState, + @NonNull final List offsetRanges) { + if (!this.topicMetrics.isPresent()) { + log.error("No topicMetrics, unable to produce metrics"); + return; + } + final DataFeedMetrics topicMetrics = this.topicMetrics.get(); + final Map offsetMap = new HashMap<>(); + final Map totalTags = new HashMap<>(); + totalTags.put(PARTITION_TAG, TOTAL_PARTITION); + final MessageCounters counter = new MessageCounters(); + offsetRanges.forEach(offsetRange -> { + final Long oldCount = offsetMap.getOrDefault(offsetRange.topicPartition(), 0L); + offsetMap.put(offsetRange.topicPartition(), oldCount + offsetRange.count()); + }); + latestLeaderOffsets.forEach( + (topicPartition, leaderOffset) -> + computePartitionMetrics( + topicPartition, leaderOffset, nextRunState, + topicMetrics, offsetMap.getOrDefault(topicPartition, 0L), counter) + ); + + topicMetrics.createLongMetric(DataFeedMetricNames.ROWCOUNT_BEHIND, + counter.getTotalAvailable() - counter.getTotalCurrent(), totalTags); + topicMetrics.createLongMetric(DataFeedMetricNames.INPUT_ROWCOUNT, counter.getTotalInput(), totalTags); + if (this.chargebackCalculator.isPresent()) { + this.chargebackCalculator.get().addCost( + this.topicMetrics.get().getBaseTags().get(DataFeedMetrics.DATA_FEED_NAME), + ChargebackMetricType.ROW_COUNT, counter.getTotalInput()); + } + } + + /* + Creates metrics per partition based on current progress in data processing + */ + private void computePartitionMetrics(@NonNull final TopicPartition topicPartition, + final long leaderOffset, + @NonNull final KafkaRunState nextRunState, + @NonNull final DataFeedMetrics topicMetrics, + @NonNull final Long inputCount, + @NonNull final MessageCounters counter) { + final Optional nextPartitionState = nextRunState.getPartitionOffset(topicPartition.partition()); + final Map tags = new HashMap<>(); + tags.put(PARTITION_TAG, Integer.toString(topicPartition.partition())); + if (nextPartitionState.isPresent()) { + final long current = nextPartitionState.get(); + counter.add(leaderOffset, current, inputCount); + + topicMetrics.createLongMetric(DataFeedMetricNames.AVAILABLE_ROWCOUNT, leaderOffset, tags); + topicMetrics.createLongMetric(DataFeedMetricNames.CURRENT_STATUS, current, tags); + topicMetrics.createLongMetric(DataFeedMetricNames.INPUT_ROWCOUNT, inputCount, tags); + } else { + topicMetrics.createLongMetric(DataFeedMetricNames.AVAILABLE_ROWCOUNT, leaderOffset, tags); + topicMetrics.createLongMetric(DataFeedMetricNames.CURRENT_STATUS, leaderOffset, tags); + topicMetrics.createLongMetric(DataFeedMetricNames.INPUT_ROWCOUNT, 0, tags); + } + } + + /** + * It holds current set of work units and also {@link KafkaRunState} for the next run. + */ + public final class KafkaWorkUnitCalculatorResult implements + IWorkUnitCalculator.IWorkUnitCalculatorResult { + + @Getter + private final KafkaRunState nextRunState; + @Getter + private final List workUnits; + + /* + * We need constructor to be private so that it is created only from KafkaWorkUnitCalculator. + */ + private KafkaWorkUnitCalculatorResult(@NonNull final List workUnits, + @NonNull final KafkaRunState nextRunState) { + this.nextRunState = nextRunState; + this.workUnits = workUnits; + } + + /** + * @return true if there are work units available for current run. + */ + @Override + public boolean hasWorkUnits() { + return !this.workUnits.isEmpty(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append("offsetRanges="); + this.workUnits.forEach( + workUnit -> sb.append( + workUnit.partition()).append(":").append(workUnit.fromOffset()).append("->") + .append(workUnit.untilOffset()).append(";")); + return sb.toString(); + } + } + + // Helper class used for counting and sorting number of messages per partition. + @AllArgsConstructor + @EqualsAndHashCode + private static class PartitionMessages implements Comparable { + + @Getter + private final TopicPartition topicPartition; + @Getter + private final Long messages; + + @Override + public int compareTo(final PartitionMessages m) { + return messages.compareTo(m.getMessages()); + } + } + + // Helper class to track all of the message stats to accumulate + private static class MessageCounters { + + @Getter + private long totalAvailable = 0; + + @Getter + private long totalCurrent = 0; + + @Getter + private long totalInput = 0; + + private void add(final long available, final long current, final long input) { + this.totalAvailable += available; + this.totalCurrent += current; + this.totalInput += input; + } + } + +} 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 new file mode 100644 index 0000000..97a8597 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkArgs.java @@ -0,0 +1,58 @@ +/* + * 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.google.common.base.Optional; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import lombok.ToString; +import org.apache.avro.Schema; + +@ToString +@AllArgsConstructor +public class SparkArgs { + + /** + * Avro schemas to be added to spark context for serialization + */ + @Getter + @NonNull + private final Optional> avroSchemas; + /** + * User serialization classes to be added for kryo serialization + */ + @Getter + @NonNull + private final List userSerializationClasses; + /** + * Other spark properties provided to override defaults + */ + @Getter + @NonNull + private final Map overrideSparkProperties; + + /** + * Hadoop Configuration to be added as a resource to SparkContext + */ + @Getter + @NonNull + private final org.apache.hadoop.conf.Configuration hadoopConfiguration; +} 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 new file mode 100644 index 0000000..65d6d86 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/common/spark/SparkFactory.java @@ -0,0 +1,203 @@ +/* + * 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.google.common.base.Optional; +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.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 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.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.util.Utf8; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.scheduler.SparkListener; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.SparkSession.Builder; +import scala.collection.JavaConverters; +import scala.collection.mutable.WrappedArray.ofRef; + +/** + * {@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 +public class SparkFactory { + + private Optional sparkSession = 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()); + } + + /** + * 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(); + } + final Builder sparkSessionBuilder = SparkSession.builder(); + if (enableHiveSupport) { + sparkSessionBuilder.enableHiveSupport(); + } + this.sparkSession = Optional.of(sparkSessionBuilder + .config(createSparkConf(sparkArgs)).getOrCreate()); + log.info("Created new SparkSession using {}", sparkArgs); + updateSparkContext(sparkArgs, this.sparkSession.get().sparkContext()); + return this.sparkSession.get(); + } + + /** + * Creates {@link SparkConf} with {@link org.apache.spark.serializer.KryoSerializer} along with + * 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) { + /** + * 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"); + final List serializableClasses = getDefaultSerializableClasses(); + serializableClasses.addAll(sparkArgs.getUserSerializationClasses()); + sparkConf.registerKryoClasses(serializableClasses.toArray(new Class[0])); + + if (sparkArgs.getAvroSchemas().isPresent()) { + sparkConf.registerAvroSchemas( + JavaConverters + .iterableAsScalaIterableConverter(sparkArgs.getAvroSchemas().get()) + .asScala() + .toSeq()); + } + + // override spark properties + final Map sparkProps = sparkArgs.getOverrideSparkProperties(); + for (Entry entry : sparkProps.entrySet()) { + log.info("Setting spark key:val {} : {}", entry.getKey(), entry.getValue()); + sparkConf.set(entry.getKey(), entry.getValue()); + } + return sparkConf; + } + + /** + * Hook for plugging in custom SparkListers + */ + protected List getSparkEventListeners() { + return Arrays.asList(new SparkEventListener()); + } + + private void updateSparkContext(@NonNull final SparkArgs sparkArgs, + @NonNull final SparkContext sc) { + for (SparkListener sparkListener : getSparkEventListeners()) { + sc.addSparkListener(sparkListener); + } + sc.hadoopConfiguration().addResource(sparkArgs.getHadoopConfiguration()); + } + + private List getDefaultSerializableClasses() { + 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, + ofRef.class, + Object[].class, + TimestampInfo.class, + UtilRecord.class, + ValidData.class, + HashMap.class, + Optional.absent().getClass(), + Utf8.class, + Class.class)); + + SparkUtil.addClassesIfFound(serializableClasses, + Arrays.asList( + "com.google.common.base.Present", + "scala.reflect.ClassTag$$anon$1")); + return serializableClasses; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ByteBufferUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ByteBufferUtil.java new file mode 100644 index 0000000..baa4c95 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ByteBufferUtil.java @@ -0,0 +1,48 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; + +/** + * {@link ByteBufferUtil }Provides standard ByteBuffer functionality to convert types to and from ByteBuffers + */ +public final class ByteBufferUtil { + private ByteBufferUtil() { + throw new JobRuntimeException("This is a utility class that should not be instantiated"); + } + + public static String convertToString(final ByteBuffer bb) { + return new String(bb.array(), Charset.forName(StandardCharsets.UTF_8.toString())); + } + + public static ByteBuffer wrap(final String value) { + try { + return ByteBuffer.wrap(value.getBytes(StandardCharsets.UTF_8.toString())); + } catch (final UnsupportedEncodingException e) { + // should never see this + throw new JobRuntimeException( + String.format("Unsupported encoding exception on string: %s. Error Message: %s", + value, e.getMessage())); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java new file mode 100644 index 0000000..9723f10 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/CommandLineUtil.java @@ -0,0 +1,92 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.PrintWriter; +import lombok.NonNull; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.io.IoBuilder; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * {@link CommandLineUtil} provides utility methods to interact with the command line + */ +public class CommandLineUtil { + public static String executeCommand(@NotEmpty final String cmd) { + final StringBuffer outputBuffer = new StringBuffer(); + + try { + final Process process = Runtime.getRuntime().exec(cmd); + process.waitFor(); + + String line; + try (final BufferedReader br = new BufferedReader(new InputStreamReader(process.getInputStream()))) { + while ((line = br.readLine()) != null) { + outputBuffer.append(line + "\n"); + } + } + } catch (IOException | InterruptedException e) { + throw new JobRuntimeException("Exception occurred while executing command: " + cmd + + " Error Message: " + e.getMessage(), e); + } + + return outputBuffer.toString(); + } + + public static Option generateOption(@NotEmpty final String opt, + @NotEmpty final String longOpt, + final boolean hasArg, + @NotEmpty final String description, + final boolean required) { + final Option option = new Option(opt, longOpt, hasArg, description); + option.setRequired(required); + return option; + } + + public static CommandLine parseOptions(@NonNull final Options options, + @NonNull final String[] args) throws ParseException { + + final CommandLine cmd; + final CommandLineParser parser = new GnuParser(); + try { + cmd = parser.parse(options, args); + } catch (final ParseException e) { + throw e; + } + return cmd; + } + + public static void printHelp(@NonNull final Options options, + @NotEmpty final String cmdLineSyntax, + @NotEmpty final String header, + @NotEmpty final String footer) { + final PrintWriter pw = IoBuilder.forLogger().setLevel(Level.ERROR).buildPrintWriter(); + final HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.printHelp(pw, 120, cmdLineSyntax, header, options, 0, 0, footer, true); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java new file mode 100644 index 0000000..5549476 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ConfigUtil.java @@ -0,0 +1,54 @@ +/* + * 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; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.util.List; + +@Slf4j +/** + * {@link ConfigUtil} provides utility methods for job configurations + */ +public final class ConfigUtil { + + private ConfigUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + /** + * Checks if all mandatory properties are present or not. If not Present then it will throw + * {@link MissingPropertyException} + */ + public static void checkMandatoryProperties(@NonNull final Configuration conf, + @NonNull final List mandatoryProps) { + if (mandatoryProps.isEmpty()) { + log.warn("mandatory properties are empty"); + } + mandatoryProps.stream().forEach( + prop -> { + if (!conf.getProperty(prop).isPresent()) { + log.error("Missing property:{}", prop); + throw new MissingPropertyException("property:" + prop); + } + }); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ConverterUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ConverterUtil.java new file mode 100644 index 0000000..695216b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ConverterUtil.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.utilities; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.io.JsonEncoder; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * {@link ConverterUtil} is a utility class to convert different types of payloads + */ +public final class ConverterUtil { + + private ConverterUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + public static byte[] convertData(@NonNull final AvroPayload avroPayload) throws IOException { + // We use JsonEncoder explicitly to guarantee if ever needed we can convert back to GenericRecord + // There are cases where calling toString() on GenericRecord doesn't allow conversion back + try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) { + final GenericRecord record = avroPayload.getData(); + final JsonEncoder encoder = EncoderFactory.get().jsonEncoder(record.getSchema(), out, false); + final DatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + + writer.write(record, encoder); + encoder.flush(); + out.flush(); + return out.toByteArray(); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/DateUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/DateUtil.java new file mode 100644 index 0000000..365d657 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/DateUtil.java @@ -0,0 +1,40 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; + +/** + * {@link DateUtil} is a utility class providing helpful Date functions + */ +public final class DateUtil { + + public static final String DATE_PARTITION_FORMAT = "yyyy/MM/dd"; + private static final DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + private DateUtil() { + throw new JobRuntimeException("DateUtil is a utility class and should never be instantiated"); + } + + public static LocalDate convertToUTCDate(final String dateStr) { + return LocalDate.parse(dateStr, formatter.withZone(ZoneOffset.UTC)); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorExtractor.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorExtractor.java new file mode 100644 index 0000000..9cba4c5 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorExtractor.java @@ -0,0 +1,46 @@ +/* + * 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; + +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RawData; + +import lombok.NonNull; + +import java.io.Serializable; + +public class ErrorExtractor implements Serializable { + protected static final String DEFAULT_ROW_KEY = "ROW_KEY_NOT_FOUND"; + protected static final String DEFAULT_CHANGELOG_COLUMNS = "CHANGELOG_COLUMNS_NOT_FOUND"; + protected static final String DEFAULT_ERROR_SOURCE_DATA = "ERROR_SOURCE_DATA_NOT_FOUND"; + + public String getRowKey(@NonNull final RawData rawdata) { + return DEFAULT_ROW_KEY; + } + + public String getChangeLogColumns(@NonNull final RawData rawdata) { + return DEFAULT_CHANGELOG_COLUMNS; + } + + public String getErrorSourceData(@NonNull final ErrorData errorData) { + return DEFAULT_ERROR_SOURCE_DATA; + } + + public String getErrorException(@NonNull final ErrorData errorData) { + return errorData.getErrMessage(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java new file mode 100644 index 0000000..7fffd2a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ErrorTableUtil.java @@ -0,0 +1,173 @@ +/* + * 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; + +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.marmaray.common.HoodieErrorPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.ErrorTableConfiguration; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.converters.data.DummyHoodieSinkDataConverter; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RDDWrapper; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.metadata.HoodieBasedMetadataManager; +import com.uber.marmaray.common.sinks.hoodie.HoodieErrorSink; +import com.uber.marmaray.common.sinks.hoodie.HoodieSink; +import com.google.common.base.Optional; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.hibernate.validator.constraints.NotEmpty; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT; + +@Slf4j +/** + * {@link ErrorTableUtil} defines utility methods to interact with the error tables + */ +public final class ErrorTableUtil { + + public static final String HADOOP_ROW_KEY = "Hadoop_Row_Key"; + public static final String HADOOP_ERROR_SOURCE_DATA = "hadoop_error_source_data"; + 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 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"; + + /** + * Default flag to control whether error table metrics is enabled + */ + public static final boolean ERROR_METRICS_IS_ENABLED = false; + + private ErrorTableUtil() { + throw new JobRuntimeException("This is a utility class that should never be instantiated"); + } + + /** + * Helper method to write to error table. + * + * @param sc {@link SparkContext} + * @param conf {@link Configuration} + * @param tableName Name of the table for which the current ingestion job ran. It will help in grouping error + * records for a particular table. + * @param errorData RDDWrapper of ErrorData(RDD) and count. It should have error message populated per record. + */ + public static void writeErrorRecordsToErrorTable(@NonNull final SparkContext sc, + @NonNull final Configuration conf, + @NonNull final Optional tableName, + @NonNull final RDDWrapper errorData, + @NonNull final ErrorExtractor errorExtractor) { + + JavaSparkContext jsc = JavaSparkContext.fromSparkContext(sc); + final ErrorTableConfiguration errorTableConf = new ErrorTableConfiguration(conf); + if (!errorTableConf.isEnabled()) { + return; + } + final long numErrors = errorData.getCount(); + log.info("number of Errors : {}", numErrors); + if (numErrors == 0) { + return; + } + String partitionPath = getPartitionPath(); + final String applicationId = jsc.getConf().getAppId(); + String errorTableSchema = sc.getConf().get(ERROR_SCHEMA_IDENTIFIER); + String targeTable = sc.getConf().get(TABLE_KEY); + String errorTable = sc.getConf().get(ERROR_TABLE_KEY); + + final AtomicBoolean shouldSaveChanges = new AtomicBoolean(true); + HoodieConfiguration hoodieConf = errorTableConf.getHoodieConfiguration(conf, errorTableSchema, targeTable, + errorTable, ERROR_METRICS_IS_ENABLED); + + try { + final HoodieBasedMetadataManager metadataManager = + new HoodieBasedMetadataManager(hoodieConf, shouldSaveChanges, jsc); + final HoodieSink hoodieSink = new HoodieErrorSink(hoodieConf, new DummyHoodieSinkDataConverter(), jsc, + HoodieSink.HoodieSinkOp.BULK_INSERT, metadataManager, + false); + + 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); + } + ); + + RDDWrapper> hoodieErrorRecords = new RDDWrapper(hoodieRecords, numErrors); + hoodieSink.write(hoodieErrorRecords); + } catch (IOException ioe) { + final String errMessage = String.format("Failed to write error records for table:%s in application:%s", + tableName, applicationId); + log.error(errMessage, ioe); + throw new JobRuntimeException(errMessage, ioe); + } + } + + 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) + .withBasePath(errorTableConf.getDestPath().toString()) + .withTableName(errorTableName) + .enableMetrics(false) + .build(); + HoodieUtil.initHoodieDataset(FSUtils.getFs(conf), hoodieConf); + } + + private static GenericRecord generateGenericErrorRecord(@NonNull final ErrorExtractor errorExtractor, + @NotEmpty final String schema, + @NonNull final ErrorData error, + @NotEmpty final String applicationId) { + Schema errorSchema = new Schema.Parser().parse(schema); + GenericRecord newRecord = new GenericData.Record(errorSchema); + newRecord.put(HADOOP_ROW_KEY.toLowerCase(), errorExtractor.getRowKey(error.getRawData())); + newRecord.put(HADOOP_ERROR_SOURCE_DATA, errorExtractor.getErrorSourceData(error)); + newRecord.put(HADOOP_ERROR_EXCEPTION, errorExtractor.getErrorException(error)); + newRecord.put(HADOOP_CHANGELOG_COLUMNS.toLowerCase(), errorExtractor.getChangeLogColumns(error.getRawData())); + newRecord.put(HADOOP_APPLICATION_ID, applicationId); + return newRecord; + } + + private static String getPartitionPath() { + final ZonedDateTime date = ZonedDateTime.now(ZoneOffset.UTC); + final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(DATE_PARTITION_FORMAT); + return date.format(formatter); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java b/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java new file mode 100644 index 0000000..74ae263 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/FSUtils.java @@ -0,0 +1,161 @@ +/* + * 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; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.HadoopConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Deque; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Stack; +import java.util.stream.IntStream; +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; + +@Slf4j +/** + * {@link FSUtils} defines utility methods with interacting with a filesystem + */ +public class FSUtils { + + // Metadata file names in HDFS = nanoseconds since epoch so we can sort by name + private static final Comparator byTimestampedNameAsc = + Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName())); + + /** + * It returns FileSystem based on fs.defaultFS property defined in conf. + */ + public static FileSystem getFs(final Configuration conf) throws IOException { + return FileSystem.get((new HadoopConfiguration(conf)).getHadoopConf()); + } + + public static void deleteHDFSMetadataFiles(@NonNull final FileStatus[] fileStatuses, + @NonNull final FileSystem fs, + final int numFilesToRetain, + final boolean fakeDelete) throws IOException { + if (fileStatuses.length > numFilesToRetain) { + Arrays.sort(fileStatuses, byTimestampedNameAsc); + final int numToRemove = fileStatuses.length - numFilesToRetain; + IntStream.range(0, numToRemove) + .forEach(i -> { + if (fileStatuses[i].isDirectory()) { + throw new RuntimeException("An unexpected directory was encountered. " + + fileStatuses[i].getPath()); + } + + try { + if (fakeDelete) { + log.info("{} would have been deleted", fileStatuses[i].getPath()); + } else { + log.info("Deleting {}", fileStatuses[i].getPath()); + fs.delete(fileStatuses[i].getPath(), false); + } + } catch (final IOException e) { + throw new RuntimeException("Unable to delete file: " + fileStatuses[i].getPath(), e); + } + }); + } + } + + /** + * Helper method to list files under "basePath". This needs to be used when you only need files which start after + * "toProcessAfter" but ends before "toProcessBefore". Note that both the paths are exclusive. + */ + public static Iterator listFiles(@NonNull final FileSystem fs, @NonNull final Path basePath, + @NonNull final Path toProcessAfter, @NonNull final Path toProcessBefore) throws IOException { + return new Iterator() { + private final Stack pathsToProcess = new Stack<>(); + private final Deque newFilesQ = new LinkedList(); + private String lastProcessedFile = toProcessAfter.toUri().getRawPath(); + private final String pathToStopProcessingAfter = toProcessBefore.toUri().getRawPath(); + { + Path currDir = toProcessAfter; + if (!fs.isDirectory(currDir)) { + currDir = currDir.getParent(); + } + final Stack tmpFolders = new Stack<>(); + while (currDir != null && basePath.toUri().getRawPath().compareTo(currDir.toUri().getRawPath()) <= 0) { + tmpFolders.push(currDir.toUri().getRawPath()); + currDir = currDir.getParent(); + } + while (!tmpFolders.isEmpty()) { + this.pathsToProcess.push(tmpFolders.pop()); + } + } + + // Helper method to recursively traverse folders. It aborts and returns as soon as newFilesQ becomes non + // empty or we run out of folders to process + private void computeNext() { + while (!this.pathsToProcess.isEmpty() && this.newFilesQ.isEmpty()) { + final Path currentPath = new Path(this.pathsToProcess.pop()); + try { + if (fs.isFile(currentPath)) { + this.lastProcessedFile = currentPath.toUri().getRawPath(); + this.newFilesQ.add(fs.getFileStatus(currentPath)); + break; + } + final FileStatus[] dirFiles = fs.listStatus(currentPath); + // Uses FileStatus's sorting based on name. + Arrays.sort(dirFiles); + final Stack levelDirStack = new Stack<>(); + for (final FileStatus nextFile : dirFiles) { + final String nextRawFilePath = nextFile.getPath().toUri().getRawPath(); + if (nextRawFilePath.compareTo(this.lastProcessedFile) <= 0 + || nextRawFilePath.compareTo(this.pathToStopProcessingAfter) >= 0) { + continue; + } + if (nextFile.isDirectory() || !levelDirStack.isEmpty()) { + levelDirStack.push(nextRawFilePath); + continue; + } + this.lastProcessedFile = nextRawFilePath; + this.newFilesQ.add(nextFile); + } + while (!levelDirStack.isEmpty()) { + this.pathsToProcess.push(levelDirStack.pop()); + } + } catch (IOException e) { + throw new JobRuntimeException("error listing newFilesQ", e); + } + } + } + + @Override + public boolean hasNext() { + computeNext(); + return !this.newFilesQ.isEmpty(); + } + + @Override + public FileStatus next() { + computeNext(); + if (this.newFilesQ.isEmpty()) { + return null; + } else { + return this.newFilesQ.pollFirst(); + } + } + }; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/GenericRecordUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/GenericRecordUtil.java new file mode 100644 index 0000000..3969e53 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/GenericRecordUtil.java @@ -0,0 +1,278 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericRecord; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Slf4j +/** + * {@link GenericRecordUtil} defines utility methods for working with Generic Records + */ +public final class GenericRecordUtil { + + private GenericRecordUtil() { + throw new JobRuntimeException("Utility class should not be instantiated"); + } + + /** + * @return true if given schema is a union schema and is nullable + */ + public static boolean isOptional(@NonNull final Schema schema) { + return schema.getType().equals(Schema.Type.UNION) + && (schema.getTypes().size() == 2) + && (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) + || schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); + } + + /** + * @return non null schema. Make sure that {@link #isOptional(Schema)} returns true for the given + * schema. + */ + public static Schema getNonNull(@NonNull final Schema schema) { + final List types = schema.getTypes(); + Preconditions.checkState(types.size() == 2, "not an optional schema"); + return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0); + } + + /** + * @return true if both records match + */ + public static boolean compareRecords(final GenericRecord record1, final GenericRecord record2) { + if (record1 == null || record2 == null) { + return record1 == record2; + } else { + for (final Schema.Field f : record1.getSchema().getFields()) { + final String name = f.name(); + final Object value = record1.get(name); + final Object newValue = record2.get(name); + if (value == null || newValue == null) { + if (value != newValue) { + return false; + } + continue; + } + if (!compareFields(value, newValue, f.schema())) { + return false; + } + } + return true; + } + } + + private static boolean compareFields(final Object field1, final Object field2, @NonNull final Schema schema) { + final Schema nonNullSchema; + if (isOptional(schema)) { + nonNullSchema = getNonNull(schema); + } else { + nonNullSchema = schema; + } + if (field1 == null || field2 == null) { + return field1 == field2; + } + switch (nonNullSchema.getType()) { + case BOOLEAN: + case INT: + case DOUBLE: + case FLOAT: + case LONG: + case STRING: + return field1.equals(field2); + case RECORD: + return compareRecords((GenericRecord) field1, (GenericRecord) field2); + case ENUM: + if (!field1.toString().equals(field2.toString())) { + return false; + } + return true; + case ARRAY: + final Schema elementSchema = nonNullSchema.getElementType(); + final List list1 = (List) field1; + final List list2 = (List) field2; + if (list1.size() != list2.size()) { + return false; + } + for (int idx = 0; idx < list1.size(); idx++) { + if (!compareFields(list1.get(idx), list2.get(idx), elementSchema)) { + return false; + } + } + return true; + case MAP: + final Schema valueSchema = nonNullSchema.getValueType(); + final Map map1 = (Map) field1; + final Map map2 = (Map) field2; + if (map1.size() != map2.size()) { + return false; + } + for (String key : map1.keySet()) { + if (!map2.containsKey(key)) { + return false; + } + if (!compareFields(map1.get(key), map2.get(key), valueSchema)) { + return false; + } + } + return true; + default: + throw new JobRuntimeException("Invalid data type"); + } + } + + /** + * Helper method to fix {@link org.apache.avro.Schema.Type#ENUM} datatypes in {@link GenericRecord}. + */ + public static GenericRecord fixEnumRecord(@NonNull final GenericRecord record, @NonNull final Schema schema) + throws IOException { + final GenericRecord retRecord = new Record(schema); + for (final Schema.Field f : schema.getFields()) { + final String name = f.name(); + retRecord.put(f.name(), fixEnumRecordField(record.get(name), f.name(), f.schema())); + } + return retRecord; + } + + private static Object fixEnumRecordField(final Object value, @NotEmpty final String name, + @NonNull final Schema schema) throws IOException { + log.debug("name:{}:value:{}:schema:{}", name, value, schema); + final Schema nonOptionalSchema; + if (isOptional(schema)) { + if (value == null) { + return null; + } else { + nonOptionalSchema = getNonNull(schema); + } + } else if (value == null) { + // Always fail on null for non-nullable schemas + throw new JobRuntimeException(String.format("value cannot be null :name {} : schema : {}", + name, schema)); + } else { + nonOptionalSchema = schema; + } + switch (nonOptionalSchema.getType()) { + case BOOLEAN: + if (value instanceof Boolean) { + return ((Boolean) value).booleanValue(); + } + throw new JobRuntimeException( + "data type is not boolean :" + name + " : type :" + value.getClass().getCanonicalName() + + ": value :" + value.toString()); + case DOUBLE: + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + throw new JobRuntimeException( + "data type is not double :" + name + " : type :" + value.getClass().getCanonicalName() + ": value :" + + value.toString()); + case FLOAT: + if (value instanceof Number) { + return ((Number) value).floatValue(); + } + throw new JobRuntimeException( + "data type is not float :" + name + " : type :" + value.getClass().getCanonicalName() + ": value :" + + value.toString()); + case INT: + if (value instanceof Number) { + return ((Number) value).intValue(); + } + throw new JobRuntimeException( + "data type is not int :" + name + " : type :" + value.getClass().getCanonicalName() + ": value :" + + value.toString()); + case LONG: + if (value instanceof Number) { + return ((Number) value).longValue(); + } + throw new JobRuntimeException( + "data type is not long :" + name + " : type :" + value.getClass().getCanonicalName() + ": value :" + + value.toString()); + case STRING: + return value.toString(); + case BYTES: + return value; + case RECORD: + return fixEnumRecord((GenericRecord) value, nonOptionalSchema); + case ENUM: + if (nonOptionalSchema.getEnumSymbols().contains(value.toString())) { + return new GenericData.EnumSymbol(nonOptionalSchema, value.toString()); + } + throw new JobRuntimeException(String.format("Invalid symbol - symbol :{} , schema :{}", + value.toString(), nonOptionalSchema.toString())); + case ARRAY: + final Schema elementSchema = nonOptionalSchema.getElementType(); + final List listRes = new ArrayList(); + final List inputList = (List) value; + if (inputList.contains(null)) { + return listRes; + } + for (final Object v : inputList) { + try { + switch (elementSchema.getType()) { + case BOOLEAN: + case DOUBLE: + case FLOAT: + case INT: + case LONG: + case STRING: + case ENUM: + case BYTES: + case RECORD: + log.debug("name:{}:v:{}:v.class:{}:elementSchema:{}", name, v, + v.getClass().getCanonicalName(), elementSchema); + if (v instanceof Record && ((Record) v).get("array") != null) { + listRes.add(fixEnumRecordField(((Record) v).get("array"), name, elementSchema)); + } else { + listRes.add(fixEnumRecordField(v, name, elementSchema)); + } + break; + default: + throw new JobRuntimeException("invalid array element's data type :" + + v.getClass().getCanonicalName()); + } + } catch (Exception e) { + log.error("elementSchema:{}:value:{}:valueClass:{}:v:{}:v.getClass{}", elementSchema, value, + value.getClass().getCanonicalName(), v, v.getClass().getCanonicalName()); + throw e; + } + } + return listRes; + case MAP: + final Schema valueSchema = nonOptionalSchema.getValueType(); + Map ret = new HashMap<>(); + for (final Map.Entry v : ((Map) value).entrySet()) { + ret.put(v.getKey(), fixEnumRecordField(v.getValue(), name, valueSchema)); + } + return ret; + default: + log.error("name:{}:value:{}:valueClass:{}:schema:{}", name, value, + value == null ? "" : value.getClass(), schema); + throw new JobRuntimeException("unsupported data type :" + nonOptionalSchema.getType()); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkConverterErrorExtractor.java b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkConverterErrorExtractor.java new file mode 100644 index 0000000..088a28d --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkConverterErrorExtractor.java @@ -0,0 +1,59 @@ +/* + * 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; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RawData; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class HoodieSinkConverterErrorExtractor extends ErrorExtractor { + + @Override + public String getRowKey(@NonNull final RawData rawdata) { + try { + return (String) ((AvroPayload) rawdata.getData()).getField(ErrorTableUtil.HADOOP_ROW_KEY); + } catch (Exception e) { + log.debug("Not able to extract Hadoop_row_key from RawData"); + return DEFAULT_ROW_KEY; + } + } + + @Override + public String getChangeLogColumns(@NonNull final RawData rawdata) { + return DEFAULT_CHANGELOG_COLUMNS; + } + + @Override + public String getErrorSourceData(@NonNull final ErrorData errorData) { + try { + AvroPayload payload = (AvroPayload) errorData.getRawData().getData(); + return payload.getData().toString(); + } catch (Exception e) { + log.debug("Not able to extract Error source data from ErrorData"); + return DEFAULT_ERROR_SOURCE_DATA; + } + } + + @Override + public String getErrorException(@NonNull final ErrorData errorData) { + return errorData.getErrMessage(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkErrorExtractor.java b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkErrorExtractor.java new file mode 100644 index 0000000..9ee508c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieSinkErrorExtractor.java @@ -0,0 +1,62 @@ +/* + * 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; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RawData; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class HoodieSinkErrorExtractor extends ErrorExtractor { + + @Override + public String getRowKey(@NonNull final RawData rawdata) { + try { + return ((HoodieRecord) rawdata.getData()).getKey().getRecordKey(); + } catch (Exception e) { + log.debug("Not able to extract Hadoop_row_key from RawData"); + return DEFAULT_ROW_KEY; + } + } + + @Override + public String getChangeLogColumns(@NonNull final RawData rawdata) { + return DEFAULT_CHANGELOG_COLUMNS; + } + + @Override + public String getErrorSourceData(@NonNull final ErrorData errorData) { + try { + HoodieRecord payload = (HoodieRecord) errorData.getRawData().getData(); + String data = String.format("%s. %s", payload.getKey().toString(), + String.format("HoodieRecordPayload %s", payload.getData().toString())); + return data; + } catch (Exception e) { + log.debug("Not able to extract Error source data from ErrorData"); + return DEFAULT_ERROR_SOURCE_DATA; + } + } + + @Override + public String getErrorException(@NonNull final ErrorData errorData) { + return errorData.getErrMessage(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.java new file mode 100644 index 0000000..c312a7c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/HoodieUtil.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.utilities; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableConfig; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import scala.Tuple2; + +import java.io.IOException; + +/** + * {@link HoodieUtil} defines utility methods for interacting with Hoodie + */ +public final class HoodieUtil { + + private HoodieUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + /** + * It initializes hoodie dataset + * @param fs {@link FileSystem} + * @param hoodieConf {@link HoodieConfiguration} + * @throws IOException + */ + public static void initHoodieDataset(@NonNull final FileSystem fs, + @NonNull final HoodieConfiguration hoodieConf) throws IOException { + final Path hoodieMetaFolder = new Path(hoodieConf.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME); + final Path hoodiePropertiesFile = new Path(hoodieMetaFolder.toString(), + HoodieTableConfig.HOODIE_PROPERTIES_FILE); + if (!fs.exists(hoodiePropertiesFile)) { + HoodieTableMetaClient + .initializePathAsHoodieDataset(FSUtils.getFs(hoodieConf.getConf()), + hoodieConf.getBasePath(), hoodieConf.getHoodieInitProperties()); + } + } + + public static String getRowKeyFromCellRecord(@NonNull final HoodieRecord cellRecord) { + final String cellRecordKey = cellRecord.getRecordKey(); + return cellRecordKey.substring(0, cellRecordKey.indexOf(StringTypes.HASHTAG)); + } + + public static JavaRDD> combineRecords( + final JavaRDD> records, final Function, Object> recordKeyFunc, + final int parallelism) { + return records + .mapToPair(record -> new Tuple2<>(recordKeyFunc.call(record), record)) + .reduceByKey((rec1, rec2) -> { + @SuppressWarnings("unchecked") + T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + return new HoodieRecord(rec1.getKey(), reducedData); + }, parallelism) + .map(recordTuple -> recordTuple._2()); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java new file mode 100644 index 0000000..7f8ef99 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/JobUtil.java @@ -0,0 +1,48 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Scanner; + +/** + * {@link JobUtil} provides various utility functions during the execution of a job + */ +@Slf4j +public final class JobUtil { + private JobUtil() { + throw new JobRuntimeException("This utility class should not be instantiated"); + } + + public static String getDataCenterForJob(@NotEmpty final String dcPath) throws IOException { + log.info("Looking up datacenter information in: {}", dcPath); + final File dcFile = new File(dcPath); + try (final FileInputStream fis = new FileInputStream(dcFile); + final InputStream is = new BufferedInputStream(fis)) { + final Scanner scanner = new Scanner(is); + return scanner.next(); + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java new file mode 100644 index 0000000..1e9941c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaSourceConverterErrorExtractor.java @@ -0,0 +1,54 @@ +/* + * 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; + +import com.uber.marmaray.common.data.ErrorData; +import com.uber.marmaray.common.data.RawData; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import javax.xml.bind.DatatypeConverter; + +@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; + } + + @Override + public String getErrorSourceData(@NonNull final ErrorData errorData) { + try { + return DatatypeConverter.printHexBinary((byte []) errorData.getRawData().getData()); + } catch (Exception e) { + log.debug("Not able to retrieve Error source data from ErrorData"); + return DEFAULT_ERROR_SOURCE_DATA; + } + } + + @Override + public String getErrorException(@NonNull final ErrorData errorData) { + return errorData.getErrMessage(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java new file mode 100644 index 0000000..0a23195 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/KafkaUtil.java @@ -0,0 +1,228 @@ +/* + * 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; + +import com.github.rholder.retry.AttemptTimeLimiters; +import com.github.rholder.retry.Retryer; +import com.github.rholder.retry.RetryerBuilder; +import com.github.rholder.retry.StopStrategies; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.configuration.KafkaConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +@Slf4j +/** + * {@link KafkaUtil} provides utility methods for interacting with Kafka + */ +public final class KafkaUtil { + + public static final int FETCH_OFFSET_TIMEOUT_SEC = 60; + public static final int FETCH_OFFSET_RETRY_CNT = 3; + // Local topic partition cache to avoid additional topic partition lookups. + public static Map> topicPartitions; + + private KafkaUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + @VisibleForTesting + public static int getFetchOffsetTimeoutSec() { + return FETCH_OFFSET_TIMEOUT_SEC; + } + + @VisibleForTesting + public static int getFetchOffsetRetryCnt() { + return FETCH_OFFSET_RETRY_CNT; + } + + @VisibleForTesting + public static Map getTopicPartitionOffsets(@NonNull final KafkaConsumer kafkaConsumer, + @NotEmpty final String topicName, @NonNull final Set topicPartitions) { + final Map partitionOffsets = new ConcurrentHashMap<>(); + try { + log.info("requesting topicPartitions for {} - start", topicName); + 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()); + topicPartitions.stream().forEach( + tp -> { + try { + if (!partitionOffsets.containsKey(tp)) { + partitionOffsets.put(tp, kafkaConsumer.position(tp)); + } + } catch (Exception e) { + log.error("ERROR requesting topicPartitions for {} - % success {}/{}", + topicName, partitionOffsets.size(), topicPartitions.size(), e); + kafkaConsumer.wakeup(); + throw e; + } + } + ); + return null; + }; + // As the kafka fetch operations can hang we would like to add timeout with retry logic while fetching + // offsets from broker. + final Retryer retryer = RetryerBuilder.newBuilder() + .withAttemptTimeLimiter(AttemptTimeLimiters.fixedTimeLimit(getFetchOffsetTimeoutSec(), + TimeUnit.SECONDS)) + .retryIfExceptionOfType(Exception.class) + .withStopStrategy(StopStrategies.stopAfterAttempt(getFetchOffsetRetryCnt())) + .build(); + retryer.call(fetchOffsetTask); + log.info("requesting topicPartitions for {} - % success {}/{} - end", topicName, + partitionOffsets.size(), topicPartitions.size()); + Preconditions.checkState(topicPartitions.size() == partitionOffsets.size(), + "could not retrieve offsets for few partitions"); + return partitionOffsets; + } catch (Exception e) { + log.error("retrieving topic partition offsets timed out for {} - % success {}/{}", topicName, + partitionOffsets.size(), topicPartitions.size()); + throw new JobRuntimeException("failed to fetch offsets - Timeout", e); + } + } + + /** + * 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<>(); + topicPartitions.stream().forEach( + tp -> { + partitions.add(tp.partition()); + } + ); + getTopicPartitions(kafkaConsumer, topicName).stream().forEach(p -> partitions.remove(p.partition())); + if (!partitions.isEmpty()) { + throw new JobRuntimeException(String.format("invalid partitions :{} : topic : {}", + partitions.toString(), topicName)); + } + } + + /** + * It fetches earliest offset ranges available for given topic-partitions. + */ + public static Map getEarliestLeaderOffsets(@NonNull final KafkaConsumer kafkaConsumer, + @NotEmpty final String topicName, @NonNull final Set topicPartitions) { + kafkaConsumer.assign(topicPartitions); + verifyTopicPartitions(kafkaConsumer, topicName, topicPartitions); + final Map earliestLeaderOffsets = + kafkaConsumer.beginningOffsets(topicPartitions); + log.info("topic-partition earliest offsets :{}", earliestLeaderOffsets); + return earliestLeaderOffsets; + } + + /** + * It fetches latest offset ranges available for given topic-partitions. + */ + public static Map getLatestLeaderOffsets(@NonNull final KafkaConsumer kafkaConsumer, + @NotEmpty final String topicName, @NonNull final Set topicPartitions) { + kafkaConsumer.assign(topicPartitions); + verifyTopicPartitions(kafkaConsumer, topicName, topicPartitions); + final Map latestLeaderOffsets = + kafkaConsumer.endOffsets(topicPartitions); + log.info("topic-partition latest offsets :{}", latestLeaderOffsets); + return latestLeaderOffsets; + } + + @VisibleForTesting + public static synchronized void resetTopicPartitionCache() { + KafkaUtil.topicPartitions = null; + } + + /** + * It returns available {@link TopicPartition}s for given topic. + */ + public static Set getTopicPartitions(@NonNull final KafkaConsumer kafkaConsumer, + @NotEmpty final String topicName) { + try { + Map> topicPartitions = KafkaUtil.topicPartitions; + if (topicPartitions == null) { + synchronized (KafkaUtil.class) { + if (topicPartitions == null) { + topicPartitions = kafkaConsumer.listTopics(); + KafkaUtil.topicPartitions = new ConcurrentHashMap<>(topicPartitions); + } + } + } + if (!topicPartitions.containsKey(topicName)) { + throw new JobRuntimeException("topic is not found :" + topicName); + } + final List partitions = topicPartitions.get(topicName); + final Set topicPartitionSet = new HashSet<>(); + partitions.forEach( + p -> { + topicPartitionSet.add(new TopicPartition(p.topic(), p.partition())); + }); + log.info("topic-partitions:{}", partitions); + return topicPartitionSet; + } catch (KafkaException e) { + log.error("error retrieving topic partitions:", e); + throw new JobRuntimeException(e); + } + } + + /** + * Helper method to get {@link KafkaConsumer}. + */ + public static KafkaConsumer getKafkaConsumer(final Map kafkaPrams) { + return new KafkaConsumer(kafkaPrams); + } + + public static Map getKafkaParams(@NonNull final KafkaConfiguration kafkaConf) { + final Map newKafkaParams = new HashMap<>(); + kafkaConf.getKafkaParams().entrySet().stream().forEach( + entry -> { + final String val = entry.getValue(); + try { + final long longVal = Long.parseLong(val); + if (longVal >= Integer.MAX_VALUE || longVal <= Integer.MIN_VALUE) { + newKafkaParams.put(entry.getKey(), longVal); + } else { + newKafkaParams.put(entry.getKey(), (int) longVal); + } + return; + } catch (NumberFormatException e) { + // ignore it. + } + // Add all remaining (key,value) pairs as Strings. + newKafkaParams.put(entry.getKey(), entry.getValue()); + } + ); + return newKafkaParams; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/LockManager.java b/marmaray/src/main/java/com/uber/marmaray/utilities/LockManager.java new file mode 100644 index 0000000..f0b4029 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/LockManager.java @@ -0,0 +1,214 @@ +/* + * 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; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.LockManagerConfiguration; +import com.uber.marmaray.common.configuration.ZookeeperConfiguration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +import org.hibernate.validator.constraints.NotEmpty; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.google.common.base.Optional; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +/** + * {@link LockManager} manages locks via zookeeper. Internally, a map between names and locks are + * maintained. This class implements {@link AutoCloseable} interfaces, so it will be automatically + * closed if used as a resource in try blocks. + * + * {@link #getLockKey(String...)} generates one single lock key based on the list of strings. {@link + * #lock(String, String)} and {@link #unlock(String)} accquires and releases the lock with the given + * name. {@link #batchLock(List, String)} acquire locks a list of lock name strings. It only + * succeeds and keeps all the locks if all the acquiring were successful. + */ + +@Slf4j +public class LockManager implements AutoCloseable { + @NonNull + private final ConcurrentHashMap lockMap; + @NonNull + private Optional client; + @NonNull + private final LockManagerConfiguration lockConf; + + private final boolean isEnabled; + + public LockManager(@NonNull final Configuration conf) { + this.lockConf = new LockManagerConfiguration(conf); + this.isEnabled = lockConf.isEnabled(); + this.lockMap = new ConcurrentHashMap(); + if (this.isEnabled) { + final ZookeeperConfiguration zkConf = new ZookeeperConfiguration(conf); + this.client = Optional.of(CuratorFrameworkFactory.builder() + .connectString(zkConf.getZkQuorum()) + .retryPolicy(new BoundedExponentialBackoffRetry(1000, 5000, 3)) + .namespace(lockConf.getZkBasePath()) + .sessionTimeoutMs(lockConf.getZkSessionTimeoutMs()) + .connectionTimeoutMs(lockConf.getZkConnectionTimeoutMs()) + .build()); + this.client.get().start(); + } else { + this.client = Optional.absent(); + } + } + + public static String getLockKey(@NotEmpty final String... keys) { + return "/" + String.join("/", keys); + } + + public boolean lock(@NotEmpty final String lockKey, final String lockInfo) throws JobRuntimeException { + if (!this.isEnabled) { + log.info("The LockManager is not enabled. Consider the key {} locked.", lockKey); + return true; + } + if (this.lockMap.containsKey(lockKey)) { + log.info("The existing lock {} has info {}", lockKey, + new String(this.lockMap.get(lockKey).getLockNodeBytes())); + return this.lockMap.get(lockKey).isAcquiredInThisProcess(); + } else { + log.info("Acquiring a new lock for {}", lockKey); + final CustomizedInterProcessMutex newLockValue = new CustomizedInterProcessMutex( + client.get(), lockKey, lockInfo); + try { + if (acquireLock(newLockValue)) { + log.info("Acquired a new lock for {}", lockKey); + lockMap.put(lockKey, newLockValue); + return true; + } else { + log.info("Unable to acquire a new lock for {}", lockKey); + return false; + } + } catch (Exception e) { + throw new JobRuntimeException(String.format("Failed to acquire a new lock for %s", lockKey), e); + } + } + } + + public boolean unlock(@NotEmpty final String lockKey) throws JobRuntimeException { + if (!this.isEnabled) { + log.info("The LockManager is not enabled. Consider the key {} unlocked.", lockKey); + return true; + } + if (!this.lockMap.containsKey(lockKey)) { + log.error("This lock has not been acquired yet: {}", lockKey); + throw new JobRuntimeException(String.format("Failed to unlock {}: not acquired.", lockKey)); + } + final CustomizedInterProcessMutex lockValue = lockMap.get(lockKey); + if (!lockValue.isAcquiredInThisProcess()) { + log.error("This lock was not acquired by this job: {}", lockKey); + throw new JobRuntimeException(String.format("Failed to unlock %s: not acquired by this job.", lockKey)); + } + try { + log.info("Releasing the lock for {}", lockKey); + lockValue.release(); + lockMap.remove(lockKey); + return true; + } catch (Exception e) { + throw new JobRuntimeException(String.format("Unbable to unlock %s", lockKey), e); + } + } + + public boolean batchLock(@NotEmpty final List lockKeyList, + final String lockInfo) + throws JobRuntimeException { + + final List locked = new ArrayList(); + final AtomicBoolean isSuccess = new AtomicBoolean(true); + try { + for (final String lockKey : lockKeyList) { + if (this.lock(lockKey, lockInfo)) { + locked.add(lockKey); + } else { + isSuccess.set(false); + break; + } + } + } catch (Exception e) { + isSuccess.set(false); + new JobRuntimeException("Failed to lock all keys at once", e); + } finally { + if (!isSuccess.get()) { + log.info("Failed to lock all keys at once, will release them."); + log.info("The requested lock list is " + lockKeyList); + log.info("The locked list is " + locked); + for (final String lockKey : locked) { + this.unlock(lockKey); + } + } + return isSuccess.get(); + } + } + + @Override + public void close() { + if (!this.isEnabled) { + log.info("The LockManager is not enabled. Closing it anyways."); + return; + } + lockMap.forEach((key, mutex) -> { + try { + unlock(key); + lockMap.remove(key); + } catch (Exception e) { + throw new JobRuntimeException("Failed to close the LockManager", e); + } + } + ); + if (client.isPresent()) { + client.get().close(); + client = Optional.absent(); + } + } + + private boolean acquireLock(@NonNull final InterProcessMutex lock) throws Exception { + return lock.acquire(lockConf.getAcquireLockTimeMs(), TimeUnit.MILLISECONDS); + } + + private final class CustomizedInterProcessMutex extends InterProcessMutex { + + private final byte[] lockInfo; + + private CustomizedInterProcessMutex(@NonNull final CuratorFramework client, + @NotEmpty final String path, + final String info) { + super(client, path); + if (info.equals("")) { + this.lockInfo = null; + } else { + this.lockInfo = info.getBytes(); + } + } + + @Override + protected byte[] getLockNodeBytes() { + return this.lockInfo; + } + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/MapUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/MapUtil.java new file mode 100644 index 0000000..6d0614b --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/MapUtil.java @@ -0,0 +1,96 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +@Slf4j +/** + * {@link MapUtil} defines utility methods for working with maps + */ +public class MapUtil { + + public static final String KEY_VALUE_SEPARATOR = StringTypes.COLON; + public static final String KEYS_SEPARATOR = StringTypes.SEMICOLON; + public static final String ENCODING_TYPE = StandardCharsets.UTF_8.toString(); + + /** + * It returns deserialized {@link Map}. Key and values for map are string encoded using + * {@link URLEncoder} and uses {@link #ENCODING_TYPE} for encoding. + * + * @param serializedMap Example format is k1{@link #KEY_VALUE_SEPARATOR}v1{@link #KEYS_SEPARATOR}k2{@link + * #KEY_VALUE_SEPARATOR}v2. + * @return deserialized {@link Map}. + */ + public static Map deserializeMap(@NotEmpty final String serializedMap) { + final Map ret = new HashMap<>(); + Arrays.stream(serializedMap.trim().split(KEYS_SEPARATOR)).forEach( + entry -> { + final String[] keyValue = entry.split(KEY_VALUE_SEPARATOR); + Preconditions.checkState((keyValue.length <= 2)); + if (keyValue.length == 2) { + try { + ret.put(URLDecoder.decode(keyValue[0], ENCODING_TYPE), + URLDecoder.decode(keyValue[1], ENCODING_TYPE)); + } catch (UnsupportedEncodingException e) { + // ideally this is never going to happen. + log.error("Invalid encoding type :{}", ENCODING_TYPE); + throw new JobRuntimeException("Failed to deserialize map", e); + } + } + } + ); + return ret; + } + + /** + * Helper method to serialize map. + * + * @param map Map to be serialized. + */ + public static String serializeMap(@NonNull final Map map) { + final StringBuilder sb = new StringBuilder(); + map.entrySet().stream().forEach( + entry -> { + if (sb.length() > 0) { + sb.append(KEYS_SEPARATOR); + } + try { + sb.append(URLEncoder.encode(entry.getKey(), ENCODING_TYPE)); + sb.append(KEY_VALUE_SEPARATOR); + sb.append(URLEncoder.encode(entry.getValue(), ENCODING_TYPE)); + } catch (UnsupportedEncodingException e) { + // ideally this is never going to happen. + log.error("Invalid encoding type :{}", ENCODING_TYPE); + throw new JobRuntimeException("Failed to serialize map", e); + } + } + ); + return sb.toString(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/NumberConstants.java b/marmaray/src/main/java/com/uber/marmaray/utilities/NumberConstants.java new file mode 100644 index 0000000..e0d3969 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/NumberConstants.java @@ -0,0 +1,30 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +/** + * {@link NumberConstants} defines various number constants + */ +public final class NumberConstants { + public static final int ONE_MILLION = 1000000; + + private NumberConstants() { + throw new JobRuntimeException("This constants class should never be instantiated"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/ScalaUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/ScalaUtil.java new file mode 100644 index 0000000..bf0cd35 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/ScalaUtil.java @@ -0,0 +1,58 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import scala.Predef; +import scala.Tuple2; +import scala.collection.JavaConverters; +import scala.collection.immutable.Set; +import scala.collection.mutable.Buffer; + +import java.util.List; +import java.util.Map; + +/** + * {@link ScalaUtil} defines scala utility methods + */ +public final class ScalaUtil { + + private ScalaUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + public static scala.collection.immutable.Map toScalaMap(@NonNull final Map javaMap) { + return JavaConverters.mapAsScalaMapConverter(javaMap).asScala().toMap(Predef.>conforms()); + } + + public static Map toJavaMap(@NonNull final scala.collection.Map scalaMap) { + return JavaConverters.mapAsJavaMapConverter(scalaMap).asJava(); + } + + public static Set toScalaSet(@NonNull final java.util.Set javaSet) { + return JavaConverters.asScalaSetConverter(javaSet).asScala().toSet(); + } + + public static java.util.Set toJavaSet(@NonNull final Set scalaSet) { + return JavaConverters.setAsJavaSetConverter(scalaSet).asJava(); + } + + public static List toJavaList(@NonNull final Buffer scalaBuffer) { + return JavaConverters.bufferAsJavaListConverter(scalaBuffer).asJava(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java new file mode 100644 index 0000000..ca6a7e0 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/SchemaUtil.java @@ -0,0 +1,103 @@ +/* + * 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; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.IOException; + +@Slf4j +/** + * {@link SchemaUtil} defines utility methods for working with schemas + */ +public final class SchemaUtil { + + public static final String DISPERSAL_TIMESTAMP = "dispersal_timestamp"; + + private SchemaUtil() { + throw new JobRuntimeException("This utility class should never be instantiated"); + } + + /** + * This utility method will iterate through a directory containing parquet files, find the first file, + * and only read in the Parquet metadata and convert the parquet schema to the equivalent Spark StructType. + * + * Thsis method is useful because it does not require reading in all the data into memory to determine the schema + * and only reads in the required metadata located in the footer + * @param parquetDir + * @return StructType equivalent of the parquet schema + * @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)); + + Optional parquetFilePath = Optional.absent(); + + final FileStatus lastEntry = fileStatuses[fileStatuses.length - 1]; + + if (lastEntry.isFile()) { + log.info("Reading from last FileStatus object: {}", lastEntry.getPath()); + parquetFilePath = Optional.of(lastEntry.getPath()); + } else if (lastEntry.isDirectory()) { + + final FileStatus[] directoryEntries = fs.listStatus(lastEntry.getPath()); + + final FileStatus lastFile = directoryEntries[directoryEntries.length - 1]; + + if (lastFile.isFile()) { + 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()); + } + } + + if (!parquetFilePath.isPresent()) { + throw new JobRuntimeException("No files were found in path: " + parquetDir); + } + + log.info("Reading parquet file: {} for schema", parquetFilePath.get()); + + final ParquetMetadata metadata = + ParquetFileReader.readFooter(new Configuration(), + parquetFilePath.get(), ParquetMetadataConverter.NO_FILTER); + final MessageType messageType = metadata.getFileMetaData().getSchema(); + final ParquetSchemaConverter converter = new ParquetSchemaConverter(new SQLConf()); + final StructType structType = converter.convert(messageType); + return structType; + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java new file mode 100644 index 0000000..19bb9ce --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/SparkUtil.java @@ -0,0 +1,234 @@ +/* + * 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; + +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.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.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; + +/** + * {@link SparkUtil} defines utility methods for working with Apache Spark + */ +@Slf4j +public final class SparkUtil { + + 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) { + try { + serializableClasses.add(Class.forName(className)); + } catch (ClassNotFoundException e) { + log.error(String.format("Not adding %s to kryo serialization list", className), e); + } + } + } + + public static Set getSupportedDataTypes() { + return Collections.unmodifiableSet(Sets.newHashSet(DataTypes.StringType, DataTypes.IntegerType, + DataTypes.LongType, DataTypes.BooleanType, DataTypes.DoubleType, + DataTypes.ShortType, DataTypes.FloatType)); + } + + /** + * KryoSerializer is the the default serializaer + * @return SerializerInstance + */ + public static SerializerInstance getSerializerInstance() { + if (serializerInstance.get() == null) { + serializerInstance.set(new KryoSerializer(SparkEnv.get().conf()).newInstance()); + } + return serializerInstance.get(); + } + + public static > T deserialize(@NonNull final byte[] serializedRecord, + @NonNull final K classTag) { + 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 Optional getRddInfo(@NonNull final SparkContext sc, final int rddId) { + for (RDDInfo rddInfo : sc.getRDDStorageInfo()) { + if (rddInfo.id() != rddId) { + continue; + } + return Optional.of(rddInfo); + } + return Optional.absent(); + } + + /** + * All code paths should use this central method to create a {@link SparkSession} as the getOrCreate() + * could return a previously created Spark Session with slightly different configuration (i.e hive support). + * This can cause hard to debug failures since it isn't obvious that the SparkSession + * returned wasn't the one that the builder attempted to create but a previously existing one. + * + * @return SparkSession + */ + public static SparkSession getOrCreateSparkSession() { + final String sparkWarning = String.join(StringTypes.SPACE, + "Getting/Creating a Spark Session without hive support enabled." + , "Warning: If a previous SparkSession exists it will be returned" + , "Please check that it is using the correct configuration needed (i.e hive support, etc)" + ); + log.info(sparkWarning); + return SparkSession.builder().getOrCreate(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/StringTypes.java b/marmaray/src/main/java/com/uber/marmaray/utilities/StringTypes.java new file mode 100644 index 0000000..7820f7c --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/StringTypes.java @@ -0,0 +1,41 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +/** + * {@link StringTypes} defines a set of commonly used immutable strings + */ +public final class StringTypes { + public static final String EMPTY = ""; + public static final String SPACE = " "; + public static final String DOT = "."; + public static final String COMMA = ","; + public static final String HASHTAG = "#"; + public static final String COLON = ":"; + public static final String SEMICOLON = ";"; + public static final String NOT_DEFINED = "N/D"; + public static final String UNDERSCORE = "_"; + public static final String EQUAL = "="; + public static final String FORWARD_SLASH = "/"; + public static final String DASH = "-"; + + private StringTypes() { + throw new JobRuntimeException("This class defines constants and should not be instantiated"); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/StringUtil.java b/marmaray/src/main/java/com/uber/marmaray/utilities/StringUtil.java new file mode 100644 index 0000000..7309ad2 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/StringUtil.java @@ -0,0 +1,44 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; + +/** + * Utility class for operating on Strings. + */ +public final class StringUtil { + + private StringUtil() { + throw new JobRuntimeException("StringUTil is a static utility class and should not be instantiated"); + } + + /** + * Use {@link String#intern} to save space when storing common strings. + * @param input String to input + * @return null if input is null, otherwise input.intern() + */ + public static String internString(final String input) { + if (input == null) { + return null; + } else { + return input.intern(); + } + } + +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java b/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java new file mode 100644 index 0000000..c8b66b4 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/TimestampInfo.java @@ -0,0 +1,45 @@ +/* + * 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; + +import com.google.common.base.Optional; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +import java.io.Serializable; + +@ToString +@AllArgsConstructor +/** + * {@link TimestampInfo} contains timestamp information, either as a String or a long + */ +public class TimestampInfo implements Serializable { + @Getter + final Optional timestamp; + + @Getter + final boolean saveAsLongType; + + public boolean hasTimestamp() { + return this.timestamp.isPresent(); + } + + public static TimestampInfo generateEmptyTimestampInfo() { + return new TimestampInfo(Optional.absent(), false); + } +} 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 new file mode 100644 index 0000000..b2a0b8a --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/cluster/CassandraClusterInfo.java @@ -0,0 +1,56 @@ +/* + * 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.cluster; + +import com.google.common.base.Optional; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; + +/** + * POJO object that contains all the information + * related to the cassandra cluster + */ +public class CassandraClusterInfo { + + @Getter + private final Optional sslStoragePort; + + @Getter + private final Optional rpcPort; + + @Getter + private final Optional storagePort; + + @Getter + @Setter + private Optional nativeApplicationPort; + + @Getter + @Setter + private Optional listOfNodes; + + public CassandraClusterInfo(@NonNull final Optional sslStoragePart, + @NonNull final Optional rpcPort, + @NonNull final Optional storagePort) { + this.sslStoragePort = sslStoragePart; + this.rpcPort = rpcPort; + this.storagePort = storagePort; + this.nativeApplicationPort = Optional.absent(); + this.listOfNodes = Optional.absent(); + } +} 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 new file mode 100644 index 0000000..f188779 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkEventListener.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.spark.scheduler.SparkListener; +import org.apache.spark.scheduler.SparkListenerApplicationEnd; +import org.apache.spark.scheduler.SparkListenerStageCompleted; +import org.apache.spark.scheduler.SparkListenerStageSubmitted; +import org.apache.spark.scheduler.SparkListenerTaskEnd; +import org.apache.spark.scheduler.SparkListenerTaskStart; +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) { + SparkJobTracker.recordStageInfo(stageSubmitted.stageInfo(), stageSubmitted.properties()); + } + + @Override + public void onStageCompleted(final SparkListenerStageCompleted stageCompleted) { + TimeoutManager.getInstance().setLastEventTime(stageCompleted.stageInfo().stageId()); + final StageInfo stageInfo = stageCompleted.stageInfo(); + if (stageInfo.completionTime().isDefined() && stageInfo.submissionTime().isDefined()) { + SparkJobTracker.recordStageTime(stageInfo, + (long) stageInfo.completionTime().get() - (long) stageInfo.submissionTime().get()); + } else { + log.error("Stage completed without submission or completion time. Stage {}: {}", + stageInfo.stageId(), stageInfo.name()); + } + SparkJobTracker.removeStageInfo(stageInfo); + } + + @Override + public void onTaskEnd(final SparkListenerTaskEnd taskEnd) { + final TaskInfo taskInfo = taskEnd.taskInfo(); + TimeoutManager.getInstance().setLastEventTime(taskEnd.stageId()); + SparkJobTracker.recordTaskTime(taskEnd.stageId(), taskInfo.finishTime() - taskInfo.launchTime()); + } + + @Override + public void onTaskStart(final SparkListenerTaskStart taskStart) { + TimeoutManager.getInstance().setLastEventTime(taskStart.stageId()); + } + + @Override + public void onApplicationEnd(final SparkListenerApplicationEnd applicationEnd) { + SparkJobTracker.logResult(); + } +} diff --git a/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkJobTracker.java b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkJobTracker.java new file mode 100644 index 0000000..76f223f --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/SparkJobTracker.java @@ -0,0 +1,224 @@ +/* + * 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 lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.SparkContext; +import org.apache.spark.scheduler.StageInfo; +import org.hibernate.validator.constraints.NotEmpty; + +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.Properties; +import java.util.concurrent.ConcurrentHashMap; + +/** + * {@link SparkJobTracker} takes a list of registered methods (via {@link SparkJobTracker#registerMethod}), + * and records the total running time of spark stages and the max running time of a single + * task in each method. + */ +@Slf4j +public class SparkJobTracker { + + public static final String JOB_NAME_PROP = "marmaray.job_name_prop"; + + private static final String UNKNOWN_JOB_NAME = "unknown"; + + @Getter + private static final Map jobNameMap = new HashMap<>(); + private static final Map stageIdNameMap = new ConcurrentHashMap<>(); + + // To register method, the className needs to be the full class name with package information. + // Example: SparkJobTracker.registerMethod(MyClass.class.getCanonicalName(), "myFunc"); + public static void registerMethod(@NotEmpty final String className, + @NotEmpty final String methodName) { + MethodManager.registerMethod(className, methodName); + } + + /** + * Sets job name in spark context's local properties. It is used for identifying jobName of running spark DAG. + * @param sc {@link SparkContext} + * @param jobName logical job name. for example kafka-topic-name + */ + public static void setJobName(@NonNull final SparkContext sc, @NotEmpty final String jobName) { + sc.setLocalProperty(SparkJobTracker.JOB_NAME_PROP, jobName); + } + + /** + * @param sc {@link SparkContext} + * @return job name set using {@link #setJobName(SparkContext, String)} else {@link #UNKNOWN_JOB_NAME}. + */ + public static String getJobName(@NonNull final SparkContext sc) { + return getJobName(sc.getLocalProperties()); + } + + private static String getJobName(@NonNull final Properties sparkContextProperties) { + final String jobName = sparkContextProperties.getProperty(SparkJobTracker.JOB_NAME_PROP); + return jobName == null ? UNKNOWN_JOB_NAME : jobName; + } + + public static void logResult() { + jobNameMap.forEach((name, jobRunTimeManager) -> jobRunTimeManager.logResult()); + } + + public static void logResult(@NotEmpty final String name) { + if (jobNameMap.containsKey(name)) { + jobNameMap.get(name).logResult(); + } else { + log.error("Job Name {} is not found in SparkJobTracker.", name); + } + } + + public static void recordStageInfo(@NonNull final StageInfo stageInfo, @NonNull final Properties properties) { + // record all registered methods related to the stage + MethodManager.recordStageInfo(stageInfo); + + // add the stageId-jobName mapping + final String jobName = properties.getProperty(JOB_NAME_PROP, UNKNOWN_JOB_NAME); + stageIdNameMap.put(stageInfo.stageId(), jobName); + if (jobName.equals(UNKNOWN_JOB_NAME)) { + log.warn("Stage Id #{} has an unknown Job Name.", stageInfo.stageId()); + } else { + log.debug("Stage #{} has a JobName {}.", stageInfo.stageId(), jobName); + } + } + + public static void removeStageInfo(@NonNull final StageInfo stageInfo) { + MethodManager.removeStageInfo(stageInfo); + } + + public static void recordStageTime(@NonNull final StageInfo stageInfo, final long timeInMs) { + final String jobName = getJobName(stageInfo.stageId()); + final JobRunTimeManager jobRunTimeManager = getJobRunTimeManager(jobName); + final List methods = MethodManager.getRelatedMethods(stageInfo.stageId()); + methods.forEach(method -> jobRunTimeManager.reportStageTime(method, timeInMs)); + } + + public static void recordTaskTime(final int stageId, final long timeInMs) { + final String jobName = getJobName(stageId); + final JobRunTimeManager jobRunTimeManager = getJobRunTimeManager(jobName); + final List methods = MethodManager.getRelatedMethods(stageId); + log.debug("Recording task_time={}ms for StageId={} in methods {}", + timeInMs, stageId, methods.toString()); + methods.forEach(method -> jobRunTimeManager.reportTaskTime(method, timeInMs)); + } + + private static String getJobName(final int stageId) { + final String jobName = stageIdNameMap.getOrDefault(stageId, UNKNOWN_JOB_NAME); + return jobName; + } + + private static JobRunTimeManager getJobRunTimeManager(@NonNull final String jobName) { + if (!jobNameMap.containsKey(jobName)) { + jobNameMap.putIfAbsent(jobName, new JobRunTimeManager(jobName)); + } + return jobNameMap.get(jobName); + } + + public static final class JobRunTimeManager { + private final String jobName; + @Getter + private final Map methodTime; + @Getter + private final Map maxTaskTime; + + public JobRunTimeManager(@NotEmpty final String jobName) { + this.jobName = jobName; + methodTime = new ConcurrentHashMap<>(); + maxTaskTime = new ConcurrentHashMap<>(); + } + + public void reportStageTime(@NonNull final String method, final long timeInMs) { + final long oldTotal = methodTime.getOrDefault(method, 0L); + methodTime.put(method, timeInMs + oldTotal); + } + + public void reportTaskTime(@NonNull final String method, final long timeInMs) { + final long oldMax = maxTaskTime.getOrDefault(method, 0L); + maxTaskTime.put(method, Math.max(oldMax, timeInMs)); + } + + public void logResult() { + methodTime.forEach((method, time) -> + log.info("{} - Method {} takes {} ms in total, the max task time is {}", + jobName, method, time, maxTaskTime.get(method))); + } + } + + private static class MethodManager { + private static final Map registeredMethods = new ConcurrentHashMap<>(); + + private static final Map> stageIdRelatedMethodMap = new ConcurrentHashMap<>(); + + public static void registerMethod(@NotEmpty final String className, + @NotEmpty final String methodName) { + final String name = String.format("%s.%s", className, methodName); + registeredMethods.putIfAbsent(name, ""); + log.info("Method {} registered for SparkListener.", name); + } + + public static void recordStageInfo(@NonNull final StageInfo stageInfo) { + final int stageId = stageInfo.stageId(); + if (!stageIdRelatedMethodMap.containsKey(stageId)) { + stageIdRelatedMethodMap.putIfAbsent(stageId, getRelatedMethods(stageInfo)); + } + } + + public static List getRelatedMethods(final int stageId) { + if (!stageIdRelatedMethodMap.containsKey(stageId)) { + log.warn("StageId {} not recorded.", stageId); + return new LinkedList<>(); + } else { + return stageIdRelatedMethodMap.get(stageId); + } + } + + public static void removeStageInfo(@NonNull final StageInfo stageInfo) { + final int stageId = stageInfo.stageId(); + if (!stageIdRelatedMethodMap.containsKey(stageId)) { + log.warn("Cannot find StageId {} for removeStageInfo.", stageId); + } else { + stageIdRelatedMethodMap.remove(stageId); + } + } + + private static List parse(final String details) { + final List stack = new ArrayList<>(); + for (String str: Arrays.asList(details.split("\n"))) { + stack.add(str.replaceFirst("\\([a-zA-Z0-9:.\\s]*\\)", "")); + } + return stack; + } + + private static List getRelatedMethods(@NonNull final StageInfo stageInfo) { + if (registeredMethods.isEmpty()) { + return Collections.singletonList(stageInfo.name()); + } else { + final List stack = parse(stageInfo.details()); + stack.removeIf(s -> !registeredMethods.containsKey(s)); + return stack; + } + } + } +} 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 new file mode 100644 index 0000000..f99b0b7 --- /dev/null +++ b/marmaray/src/main/java/com/uber/marmaray/utilities/listener/TimeoutManager.java @@ -0,0 +1,155 @@ +/* + * 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.exceptions.JobRuntimeException; +import lombok.NonNull; +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.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * TimeoutManager + * + * This class tracks the total running time of the Spark job and the last activity in every individual stages. + * If the job times out, TimeoutManager will kill all spark stages. + * If an individual stage is inactive for a long time, TimeoutManager kills the stage and all related jobs. + * + */ +@Slf4j +public final class TimeoutManager { + + public static final String TIMEOUT_MANAGER_PREFIX = Configuration.MARMARAY_PREFIX + "timeout_manager."; + public static final String JOB_TIMEOUT_IN_MINS = TIMEOUT_MANAGER_PREFIX + "job_timeout_in_mins"; + public static final String STAGE_STALLED_IN_MINS = TIMEOUT_MANAGER_PREFIX + "stage_stalled_in_mins"; + public static final int DEFAULT_JOB_TIMEOUT_IN_MINS = (int) TimeUnit.HOURS.toMinutes(3); + public static final int DEFAULT_STAGE_STALLED_IN_MINS = (int) TimeUnit.HOURS.toMinutes(3); + + // checking whether the job has timed out every 1 minute + public static final long FREQUENCY_IN_MINS = 1; + public static final TimeUnit FREQUENCY_UNIT = TimeUnit.MINUTES; + + private static TimeoutManager instance = null; + + private final long jobTimeoutMillis; + private final long stageTimeoutMillis; + private final SparkContext sc; + + private final long startTime; + 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(); + log.info("Initializing TimeoutManager, job_timeout = {}ms, stage_timeout = {}ms", + this.jobTimeoutMillis, this.stageTimeoutMillis); + } + + public static TimeoutManager getInstance() { + if (instance == null) { + throw new JobRuntimeException("TimeoutManager has not been initialized..."); + } else { + return instance; + } + } + + public static synchronized void init(@NonNull final Configuration conf, @NonNull final SparkContext sc) { + if (instance != null) { + log.info("TimeoutManager instance already exists, job_timeout = {}ms, stage_timeout = {}ms", + instance.jobTimeoutMillis, instance.stageTimeoutMillis); + } else { + final int jobTimeoutInMins = conf.getIntProperty(JOB_TIMEOUT_IN_MINS, DEFAULT_JOB_TIMEOUT_IN_MINS); + final int stageStalledInMins = conf.getIntProperty(STAGE_STALLED_IN_MINS, DEFAULT_STAGE_STALLED_IN_MINS); + instance = new TimeoutManager(jobTimeoutInMins, stageStalledInMins, sc); + } + } + + public void startMonitorThread() { + log.info("Start timeout monitoring..."); + final Thread monitor = new Thread(() -> monitorTimeout()); + monitor.setDaemon(true); + monitor.start(); + } + + private void monitorTimeout() { + try { + while (true) { + FREQUENCY_UNIT.sleep(FREQUENCY_IN_MINS); + log.info("Checking whether the job or any Spark stage has timed out..."); + + if (jobTimeout()) { + log.error("The spark job is taking longer than {} ms. Cancelling all jobs...", + this.jobTimeoutMillis); + 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) { + log.error("Cancelling stage {}-{} and its related jobs due to inactivity... details: {}", + stage.id(), stage.name(), stage.details()); + this.sc.cancelStage(stage.id()); + } + } + log.info("The job and all stages are running fine within the timeout limits."); + } + } catch (InterruptedException | TimeoutException e) { + log.info("Shutting down timeout monitor thread"); + throw new JobRuntimeException(e); + } + } + + public void setLastEventTime(final int id) { + lastActiveTime.put(id, getCurrentTime()); + } + + public boolean jobTimeout() { + return (getCurrentTime() - startTime > jobTimeoutMillis); + } + + public List stalledStages() { + final List stalledStages = new ArrayList<>(); + 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) { + stalledStages.add(stage); + } + } + } + return stalledStages; + } + + private static long getCurrentTime() { + return System.currentTimeMillis(); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java b/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java new file mode 100644 index 0000000..2c165e7 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/TestSparkUtil.java @@ -0,0 +1,51 @@ +/* + * 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; + +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 java.io.File; +import java.util.Arrays; +import org.apache.spark.SparkConf; +import org.junit.Assert; +import org.junit.Test; + +public class TestSparkUtil { + + @Test + 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); + Assert.assertEquals("4g", sparkConf.get("spark.executor.memory")); + Assert.assertEquals("4g", sparkConf.get("spark.driver.memory")); + Assert.assertEquals("100s", sparkConf.get("spark.network.timeout")); + } + + @Test + public void testSparkConfOverrideDoesNotFailWithoutAnySparkConfDefinitions() { + final Configuration conf = new Configuration(new File(TestConfiguration.CONFIG_YAML), + Optional.absent()); + SparkUtil + .getSparkConf("fooApp", Optional.absent(), Arrays.asList(), conf); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/actions/TestJobDagActions.java b/marmaray/src/test/java/com/uber/marmaray/common/actions/TestJobDagActions.java new file mode 100644 index 0000000..009b948 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/actions/TestJobDagActions.java @@ -0,0 +1,244 @@ +/* + * 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.actions; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.job.ThreadPoolService; +import com.uber.marmaray.common.metrics.LongMetric; +import com.uber.marmaray.common.reporters.IReporter; +import com.uber.marmaray.common.reporters.Reporters; +import com.uber.marmaray.common.util.MultiThreadTestCoordinator; +import lombok.AllArgsConstructor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_FAILURE; +import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_SUCCESS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@Slf4j +public class TestJobDagActions { + + private static final int NUM_THREADS = 4; + private static final int NUM_JOB_DAG_THREADS = 2; + + @BeforeClass + public static void setUp() { + final Configuration conf = new Configuration(); + conf.setProperty(ThreadPoolService.NUM_THREADS, Integer.toString(NUM_THREADS)); + conf.setProperty(ThreadPoolService.JOB_DAG_THREADS, Integer.toString(NUM_JOB_DAG_THREADS)); + try { + ThreadPoolService.init(conf); + } catch (JobRuntimeException e) { + // thread pool service already initialized + } + } + + @AfterClass + public static void shutDown() { + // make sure that the thread pool service has stopped + ThreadPoolService.shutdown(true); + } + + @Test + public void testExecuteHandlesException() { + final JobDagActions jobDagActions = new JobDagActions(new Reporters()); + final List output = new ArrayList<>(2); + jobDagActions.addAction(new TestSuccessfulJobDagAction(output)); + jobDagActions.addAction(new TestFailedJobDagAction()); + jobDagActions.addAction(new TestSuccessfulJobDagAction(output)); + final AtomicBoolean isSuccess = new AtomicBoolean(true); + isSuccess.set(jobDagActions.execute(true)); + + assertEquals(false, isSuccess.get()); + assertEquals(2, output.size()); + assertEquals(output.get(0), true); + assertEquals(output.get(1), true); + } + + @Test + public void testActionsRunInParallel() { + final MultiThreadTestCoordinator coordinator = new MultiThreadTestCoordinator(); + final JobDagActions jobDagActions = new JobDagActions(new Reporters()); + jobDagActions.addAction(new TestParallelJobAction2(coordinator)); + jobDagActions.addAction(new TestParallelJobAction1(coordinator)); + jobDagActions.execute(true); + } + + @Test + public void testWaitForActionsToFinish() { + final JobDagActions jobDagActions = new JobDagActions(new Reporters()); + final AtomicBoolean success = new AtomicBoolean(false); + final IJobDagAction act = new IJobDagAction() { + public boolean execute(final boolean dagSuccess) { + try { + Thread.sleep(10000); + success.set(true); + } catch (Exception e) { + throw new IllegalStateException(e); + } + return success.get(); + } + + public int getTimeoutSeconds() { + return 20; + } + }; + + jobDagActions.addAction(act); + jobDagActions.execute(true); + assertTrue(success.get()); + } + + @Test + public void testTimeoutForLongRunningActions() { + final JobDagActions jobDagActions = new JobDagActions(new Reporters()); + final MultiThreadTestCoordinator coordinator = new MultiThreadTestCoordinator(); + final AtomicBoolean success = new AtomicBoolean(true); + jobDagActions.addAction(new TestParallelJobActionFailure(coordinator)); + success.set(jobDagActions.execute(true)); + assertTrue(!success.get()); + } + + @Test + public void testJobDagMetricsReporting() { + verifyJobMetricResult(new TestSuccessfulJobDagAction(new LinkedList<>()), true, RESULT_SUCCESS, false); + verifyJobMetricResult(new TestSuccessfulJobDagAction(new LinkedList<>()), false, RESULT_FAILURE, false); + verifyJobMetricResult(new TestFailedJobDagAction(), true, RESULT_FAILURE, true); + } + + private void verifyJobMetricResult(@NonNull final IJobDagAction jobDagAction, final boolean dagSuccess, + final long expectedMetricValue, final boolean isFailureExpected) { + final Reporters reporters = new Reporters(); + final IReporter mockReporter = spy(new TestReporter()); + reporters.addReporter(mockReporter); + final JobDagActions jobDagActions = new JobDagActions(reporters); + jobDagActions.addAction(jobDagAction); + Assert.assertEquals(!isFailureExpected, jobDagActions.execute(dagSuccess)); + final ArgumentCaptor metricCapture = ArgumentCaptor.forClass(LongMetric.class); + verify(mockReporter, times(2)).gauge(metricCapture.capture()); + Assert.assertEquals(expectedMetricValue, metricCapture.getValue().getMetricValue().longValue()); + } + + private class TestReporter implements IReporter { + + @Override + public void gauge(final LongMetric m) { + } + + @Override + public void finish() { + + } + } + + @AllArgsConstructor + private class TestSuccessfulJobDagAction implements IJobDagAction { + @NonNull + private final List output; + + public boolean execute(final boolean success) { + output.add(success); + return success; + } + + @Override + public int getTimeoutSeconds() { + return 20; + } + } + + @AllArgsConstructor + private class TestFailedJobDagAction implements IJobDagAction { + public boolean execute(final boolean success) { + throw new IllegalStateException("This just fails"); + } + + @Override + public int getTimeoutSeconds() { + return 20; + } + } + + @AllArgsConstructor + private class TestParallelJobAction1 implements IJobDagAction { + @NonNull + private final MultiThreadTestCoordinator coordinator; + + public boolean execute(final boolean success) { + this.coordinator.nextStep(); + this.coordinator.waitUntilStep(2); + this.coordinator.nextStep(); + return true; + } + + @Override + public int getTimeoutSeconds() { + return 20; + } + + } + + @AllArgsConstructor + private class TestParallelJobAction2 implements IJobDagAction { + @NonNull + private final MultiThreadTestCoordinator coordinator; + + public boolean execute(final boolean success) { + this.coordinator.waitUntilStep(1); + this.coordinator.nextStep(); + this.coordinator.waitUntilStep(3); + return true; + } + + @Override + public int getTimeoutSeconds() { + return 20; + } + } + + @AllArgsConstructor + private class TestParallelJobActionFailure implements IJobDagAction { + @NonNull + private final MultiThreadTestCoordinator coordinator; + + public boolean execute(final boolean success) { + this.coordinator.waitUntilStep(1000); + return true; + } + + @Override + public int getTimeoutSeconds() { + return 20; + } + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestAwsConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestAwsConfiguration.java new file mode 100644 index 0000000..7c522ca --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestAwsConfiguration.java @@ -0,0 +1,69 @@ +/* + * 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.exceptions.MissingPropertyException; +import com.uber.marmaray.common.util.FileSinkConfigTestUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestAwsConfiguration extends FileSinkConfigTestUtil { + + @Test + public void testConfigurationWithS3() { + final Configuration c = initS3("S3"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + final AwsConfiguration awsConf = new AwsConfiguration(conf); + Assert.assertEquals(conf.getSinkType().name(), "S3" ); + Assert.assertEquals(conf.getAwsLocal(), "/aws_test"); + Assert.assertEquals(awsConf.getBucketName(), "aws-test" ); + Assert.assertEquals(awsConf.getObjectKey(), "marmaray_test/test1" ); + Assert.assertEquals(awsConf.getAwsAccessKeyId(), "username"); + Assert.assertEquals(awsConf.getAwsSecretAccessKey(), "password"); + Assert.assertEquals(awsConf.getRegion(), "us-east-1"); + Assert.assertEquals(awsConf.getSourcePath(), this.fileSystem.get().getWorkingDirectory()+"/aws_test"); + } + + @Test(expected = MissingPropertyException.class) + public void testConfigurationWithS3MissRegion() { + final Configuration c = initS3MissConfig("S3", FileSinkConfiguration.AWS_REGION); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + final AwsConfiguration awsConf = new AwsConfiguration(conf); + } + + @Test(expected = MissingPropertyException.class) + public void testConfigurationWithS3MissCredentials() { + final Configuration c = initS3MissConfig("S3", FileSinkConfiguration.AWS_ACCESS_KEY_ID); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + final AwsConfiguration awsConf = new AwsConfiguration(conf); + } + + @Test(expected = MissingPropertyException.class) + public void testConfigurationWithS3MissBucketName() { + final Configuration c = initS3MissConfig("S3", FileSinkConfiguration.BUCKET_NAME); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + final AwsConfiguration awsConf = new AwsConfiguration(conf); + } + + @Test(expected = MissingPropertyException.class) + public void testConfigurationWithS3MissObjectKey() { + final Configuration c = initS3MissConfig("S3", FileSinkConfiguration.OBJECT_KEY); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + final AwsConfiguration awsConf = new AwsConfiguration(conf); + } +} \ No newline at end of file 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 new file mode 100644 index 0000000..c201b16 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestCassandraSinkConfiguration.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.configuration; + +import com.uber.marmaray.common.PartitionType; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import com.uber.marmaray.common.schema.cassandra.ClusterKey; +import com.uber.marmaray.utilities.StringTypes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; + +public class TestCassandraSinkConfiguration { + + @Test + public void testValidConfigurationInitialized() { + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(getConfig(StringTypes.EMPTY)); + Assert.assertEquals(new HashSet<>(Arrays.asList("f1", "f2", "f3", "f4", "f5")), + (conf.getFilteredColumns().get())); + Assert.assertEquals(Arrays.asList("f1", "f2"), conf.getPartitionKeys()); + Assert.assertEquals(Arrays.asList(new ClusterKey("f3", ClusterKey.Order.ASC)), conf.getClusteringKeys()); + Assert.assertEquals(Arrays.asList("localhost1", "localhost2"), conf.getInitialHosts()); + Assert.assertEquals("testKeyspace", conf.getKeyspace()); + Assert.assertEquals("clusterFoo", conf.getClusterName()); + Assert.assertEquals("testTable", conf.getTableName()); + Assert.assertEquals(PartitionType.NORMAL, conf.getPartitionType()); + } + + @Test + public void testSetHadoopSparkProperties() { + final String EXECUTOR_MEM_PROP = "spark.executor.memory"; + final String DRIVER_MEM_PROP = "spark.driver.memory"; + final String NETWORK_TIMEOUT = "spark.network.timeout"; + final String FILE_FETCH_TIMEOUT = "spark.files.fetchTimeout"; + final String NUM_EXECUTORS_PROP = "spark.executor.instances"; + + final Configuration conf = getConfig(StringTypes.EMPTY); + conf.setProperty(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX + EXECUTOR_MEM_PROP, "6g"); + conf.setProperty(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX + DRIVER_MEM_PROP, "7g"); + conf.setProperty(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX + NETWORK_TIMEOUT, "1234s"); + conf.setProperty(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX + FILE_FETCH_TIMEOUT, "1111s"); + conf.setProperty(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX + NUM_EXECUTORS_PROP, "32"); + + final CassandraSinkConfiguration cassConf = new CassandraSinkConfiguration(conf); + org.apache.hadoop.conf.Configuration hadoopConf = cassConf.getHadoopConf(); + + Assert.assertEquals("6g", hadoopConf.get(EXECUTOR_MEM_PROP)); + Assert.assertEquals("7g", hadoopConf.get(DRIVER_MEM_PROP)); + Assert.assertEquals("1234s", hadoopConf.get(NETWORK_TIMEOUT)); + Assert.assertEquals("1111s", hadoopConf.get(FILE_FETCH_TIMEOUT)); + Assert.assertEquals("32", hadoopConf.get(NUM_EXECUTORS_PROP)); + } + + @Test(expected = MissingPropertyException.class) + public void testMissingDataCenter() { + final Configuration rawConf = getConfig(CassandraSinkConfiguration.DATACENTER); + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(rawConf); + conf.getHadoopConf(); + Assert.fail(); + } + + + @Test(expected = MissingPropertyException.class) + public void testMissingKeySpace() { + final Configuration rawConf = getConfig(CassandraSinkConfiguration.KEYSPACE); + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(rawConf); + Assert.fail(); + } + + @Test(expected = MissingPropertyException.class) + public void testMissingTableName() { + final Configuration rawConf = getConfig(CassandraSinkConfiguration.TABLE_NAME); + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(rawConf); + Assert.fail(); + } + + @Test(expected = MissingPropertyException.class) + public void testMissingClusterName() { + final Configuration rawConf = getConfig(CassandraSinkConfiguration.CLUSTER_NAME); + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(rawConf); + Assert.fail(); + } + + @Test(expected = MissingPropertyException.class) + public void testMissingPartitionKeys() { + final Configuration rawConf = getConfig(CassandraSinkConfiguration.PARTITION_KEYS); + final CassandraSinkConfiguration conf = new CassandraSinkConfiguration(rawConf); + Assert.fail(); + } + + private Configuration getConfig(String propToExclude) { + final Configuration conf = new Configuration(); + + conf.setProperty(CassandraSinkConfiguration.DISABLE_QUERY_UNS, "true"); + + // Add some spaces in between elements and values on purpose, these should be trimmed + if (!CassandraSinkConfiguration.KEYSPACE.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.KEYSPACE, "testKeyspace"); + } + + if (!CassandraSinkConfiguration.TABLE_NAME.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.TABLE_NAME, "testTable"); + } + + if (!CassandraSinkConfiguration.CLUSTER_NAME.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.CLUSTER_NAME, "clusterFoo"); + } + + if (!CassandraSinkConfiguration.INITIAL_HOSTS.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.INITIAL_HOSTS, "localhost1,localhost2"); + } + + if (!CassandraSinkConfiguration.COLUMN_LIST.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.COLUMN_LIST, "f1,f2,f3,f4,f5"); + } + + if (!CassandraSinkConfiguration.PARTITION_KEYS.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.PARTITION_KEYS, "f1,f2"); + } + + if (!CassandraSinkConfiguration.CLUSTERING_KEYS.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.CLUSTERING_KEYS, "f3"); + } + + if (!CassandraSinkConfiguration.PARTITION_TYPE.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.PARTITION_TYPE, "normal"); + } + + if (!CassandraSinkConfiguration.DATACENTER.equals(propToExclude)) { + conf.setProperty(CassandraSinkConfiguration.DATACENTER, "TEST_DC"); + } + + return conf; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfigScopeResolver.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfigScopeResolver.java new file mode 100644 index 0000000..20bf152 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfigScopeResolver.java @@ -0,0 +1,92 @@ +/* + * 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.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.uber.marmaray.common.util.AbstractSparkTest; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.io.InputStream; + +public class TestConfigScopeResolver extends AbstractSparkTest { + + private String scopeOverrideMapKey; + private String sampleYamlConfigFileName; + private ConfigScopeResolver configOverrideResolver; + + @Before + public void setup() { + scopeOverrideMapKey = Configuration.SCOPE_OVERRIDE_MAPPING_KEY; + sampleYamlConfigFileName = "/configWithScopes.yaml"; + configOverrideResolver = new ConfigScopeResolver(scopeOverrideMapKey); + } + + @Test + public void testConfigOverride() throws IOException { + final JsonNode expectedWithBootstrapScopeResolution = + getJsonNode("/expectedConfigWithBootstrapScope.yaml"); + final JsonNode actualWithBootstrapScopeResolution = configOverrideResolver + .projectOverrideScopeOverDefault(Optional.of("bootstrap"), + getJsonNode(sampleYamlConfigFileName)); + Assert + .assertEquals(expectedWithBootstrapScopeResolution, actualWithBootstrapScopeResolution); + + final JsonNode expectedWithIncrementalScopeResolution = + getJsonNode("/expectedConfigWithIncrementalScope.yaml"); + final JsonNode actualWithIncrementalScopeResolution = configOverrideResolver + .projectOverrideScopeOverDefault(Optional.of("incremental"), + getJsonNode(sampleYamlConfigFileName)); + Assert + .assertEquals(expectedWithIncrementalScopeResolution, + actualWithIncrementalScopeResolution); + } + + @Test + public void testNoConfigOverrideWhenScopeIsAbsent() throws IOException { + final JsonNode expectedWithoutAnyScopeResolution = + getJsonNode(sampleYamlConfigFileName); + final JsonNode actualWithoutAnyScopeResolution = configOverrideResolver + .projectOverrideScopeOverDefault(Optional.absent(), + getJsonNode(sampleYamlConfigFileName)); + Assert + .assertEquals(expectedWithoutAnyScopeResolution, + actualWithoutAnyScopeResolution); + } + + @Test(expected = IllegalArgumentException.class) + public void testExceptionNonExistentScopeIsProvided() throws IOException { + configOverrideResolver + .projectOverrideScopeOverDefault(Optional.of("non-existent-scope"), + getJsonNode(sampleYamlConfigFileName)); + } + + private JsonNode getJsonNode(final String resourceFileName) throws IOException { + final InputStream yamlInputStream = + TestConfigScopeResolver.class.getResourceAsStream(resourceFileName); + final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); + return yamlReader.readTree(yamlInputStream); + } +} 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 new file mode 100644 index 0000000..d95deb6 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestConfiguration.java @@ -0,0 +1,97 @@ +/* + * 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.uber.marmaray.common.util.AbstractSparkTest; +import java.io.File; +import java.io.InputStream; +import java.util.Map; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestConfiguration extends AbstractSparkTest { + + public static final String CONFIG_YAML = "src/test/resources/config.yaml"; + private static final double DELTA = 1e-10; + private InputStream scopeAwareConfigInputStream; + + @Before + public void setup() { + scopeAwareConfigInputStream = Configuration.class.getResourceAsStream("/configWithScopes.yaml"); + } + + @Test + public void loadConfigurationYaml() { + final Configuration conf = new Configuration(new File(CONFIG_YAML), Optional.absent()); + conf.setProperty("marmaray.string_configs.string3", "string3"); + + Assert.assertEquals(1, conf.getIntProperty("marmaray.scalar_configs.integer", -1)); + Assert.assertEquals(1234567890123L, conf.getLongProperty("marmaray.scalar_configs.long", -1)); + Assert.assertEquals(1.23, conf.getDoubleProperty("marmaray.scalar_configs.double", -1.0), DELTA); + Assert.assertEquals(true, conf.getBooleanProperty("marmaray.scalar_configs.boolean", false)); + Assert.assertEquals("string1", conf.getProperty("marmaray.string_configs.string1", "not string1")); + Assert.assertEquals("string2", conf.getProperty("marmaray.string_configs.stringlist.string2").get()); + + final Map configMap = conf.getPropertiesWithPrefix("marmaray.string_configs.", true); + Assert.assertEquals("string2", configMap.get("stringlist.string2")); + Assert.assertEquals("string3", configMap.get("string3")); + + Assert.assertEquals(-1, conf.getIntProperty("marmaray.scalar_configs.notinteger", -1)); + Assert.assertEquals(-1, conf.getLongProperty("marmaray.scalar_configs.notlong", -1)); + Assert.assertEquals(-1.0, conf.getDoubleProperty("marmaray.scalar_configs.notdouble", -1.0), DELTA); + Assert.assertEquals(false, conf.getBooleanProperty("marmaray.scalar_configs.notboolean", false)); + Assert.assertEquals("not string1", conf.getProperty("marmaray.string_configs.notstring1", "not string1")); + } + + // Test happy path with config scope resolving + @Test + public void testConfigurationParseWithScope() { + final Configuration conf = new Configuration(scopeAwareConfigInputStream, Optional.of("bootstrap")); + Assert.assertEquals("jdbc", conf.getProperty("database.connection.type").get()); + Assert.assertEquals("5000", conf.getProperty("database.connection.port").get()); + Assert.assertEquals("4000", conf.getProperty("hbase.connection.port").get()); + Assert.assertEquals("1", + conf.getProperty("marmaray.hoodie.tables.non_primitive_field_from_default.hello") .get()); + Assert.assertEquals("2", + conf.getProperty("marmaray.hoodie.tables.non_primitive_field_from_default.hi").get()); + Assert.assertEquals("3", + conf.getProperty("marmaray.hoodie.tables.non_primitive_field_from_bootstrap.heya") .get()); + Assert.assertEquals("1000", + conf.getProperty("marmaray.hoodie.tables.target_table.parallelism").get()); + Assert.assertEquals("false", + conf.getProperty("marmaray.hoodie.tables.target_table.combine_before_insert").get()); + Assert.assertEquals("2147483647", + conf.getProperty("marmaray.hoodie.tables.target_table.parquet_max_file_size").get()); + + } + + // Assert Preconditions failure since Configuration.SCOPE_OVERRIDE_MAPPING_KEY is not present in the config yaml + @Test(expected = IllegalStateException.class) + public void testConfigurationParseFailWithScopeButWithoutScopeOverrideKey() { + final Configuration conf = new Configuration(new File(CONFIG_YAML), + Optional.of("non-existent-scope")); + } + + // Assert Preconditions failure since an invalid scope is passed + @Test(expected = IllegalArgumentException.class) + public void testConfigurationParseWithNonExistentScope() { + new Configuration(scopeAwareConfigInputStream, Optional.of("non-existent-scope")); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestErrorTableConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestErrorTableConfiguration.java new file mode 100644 index 0000000..28cc4eb --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestErrorTableConfiguration.java @@ -0,0 +1,65 @@ +/* + * 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.utilities.ErrorTableUtil; + +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.Assert; +import org.junit.Test; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TestErrorTableConfiguration { + private static final String TARGET_TABLE = "target_table"; + private static final String ERROR_TABLE = "error_table"; + final String tableName = "table1"; + final String errorTableName = tableName + ErrorTableUtil.ERROR_TABLE_SUFFIX; + final String basePath = "/tmp/basePath"; + final String errorBasePath = "/tmp/basePath/.error_table"; + final String metricsPrefix = "test_metrics"; + final String errorMetricsPrefix = metricsPrefix + ErrorTableUtil.ERROR_TABLE_SUFFIX; + final String testSchema = "TEST_SCHEMA"; + + @Test + public void testErrorTableConfiguration() { + final String hoodieTableNameKey = HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_TABLE_NAME, TARGET_TABLE); + final String metricsPrefixKey = HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_METRICS_PREFIX, TARGET_TABLE); + final String base = HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_BASE_PATH, TARGET_TABLE); + + Configuration conf = new Configuration(); + conf.setProperty(hoodieTableNameKey, tableName); + conf.setProperty(metricsPrefixKey, metricsPrefix); + conf.setProperty(ErrorTableConfiguration.IS_ENABLED, "true"); + conf.setProperty(ErrorTableConfiguration.DESTINATION_PATH, errorBasePath); + conf.setProperty(base, basePath); + + HoodieConfiguration hoodieConfiguration = createHoodieConfiguration(conf); + + Assert.assertEquals(hoodieConfiguration.getBasePath(), errorBasePath); + Assert.assertEquals(hoodieConfiguration.getTableName(), errorTableName); + Assert.assertEquals(hoodieConfiguration.getHoodieMetricsPrefix(), errorMetricsPrefix); + } + + private HoodieConfiguration createHoodieConfiguration(@NotEmpty final Configuration conf) { + ErrorTableConfiguration errorTableConfiguration = new ErrorTableConfiguration(conf); + HoodieConfiguration hoodieConfiguration = errorTableConfiguration.getHoodieConfiguration(conf, testSchema, + TARGET_TABLE, ERROR_TABLE, true); + return hoodieConfiguration; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestFileSinkConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestFileSinkConfiguration.java new file mode 100644 index 0000000..97eb616 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestFileSinkConfiguration.java @@ -0,0 +1,124 @@ +/* + * 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.exceptions.MissingPropertyException; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.FileSinkConfigTestUtil; +import lombok.NonNull; +import org.apache.hadoop.hdfs.DFSClient; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import sun.security.krb5.Config; + +import static org.junit.Assert.*; + +public class TestFileSinkConfiguration extends FileSinkConfigTestUtil { + @Test + public void testDefaultConfigurationInitialized() { + final Configuration c = initFileNameAndPath("date", true, "version"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + Assert.assertEquals(conf.getSeparator(), conf.DEFAULT_SEPARATOR); + Assert.assertEquals(conf.getFileSizeMegaBytes(), conf.DEFAULT_FILE_SIZE, 0); + Assert.assertEquals(conf.getFileType(), conf.DEFAULT_FILE_TYPE); + } + + @Test + public void testDefinedConfigurationInitialized() { + final Configuration c = initCommon("/newpath/newtest", "json", 1500, " ", "HDFS", "version"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + Assert.assertEquals(conf.getSeparator(), ' '); + Assert.assertEquals(conf.getFileSizeMegaBytes(), 1500, 0); + Assert.assertEquals(conf.getFileType(), "json"); + Assert.assertEquals(conf.getSourcePartitionPath().get(), "2018/08/01"); + Assert.assertEquals(conf.getSourceType(), "hive"); + Assert.assertEquals(conf.getSourceNamePrefix(), "test.db_test_trip.table"); + Assert.assertEquals(conf.getPartitionType().name(), "DATE"); + Assert.assertEquals(conf.getFileNamePrefix(), "marmaray_hive_test.db_test_trip.table_201808011025"); + Assert.assertEquals(conf.getFullPath(), this.fileSystem.get().getWorkingDirectory() + "/newpath/newtest/2018/08/01/" + conf.getFileNamePrefix() ); + Assert.assertEquals(conf.getPathHdfs(), this.fileSystem.get().getWorkingDirectory() + "/newpath/newtest/2018/08/01"); + } + + @Test(expected = UnsupportedOperationException.class) + public void testWithConfSeparatorNotSupported() { + final Configuration c = initCommon("/newpath/newtest", "json", 1500, " ,", "HDFS", "version"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = MissingPropertyException.class) + public void testWithConfMissPartitionType() { + final Configuration c = initFileNameAndPathMissConfig(FileSinkConfiguration.PATH_PREFIX); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = MissingPropertyException.class) + public void testWithConfMissSourceDataPath() { + final Configuration c = initFileNameAndPathMissConfig(FileSinkConfiguration.SOURCE_NAME_PREFIX); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = MissingPropertyException.class) + public void testWithConfMissTimeStamp() { + final Configuration c = initFileNameAndPathMissConfig(FileSinkConfiguration.TIMESTAMP); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = MissingPropertyException.class) + public void testWithConfMissSourceType() { + final Configuration c = initFileNameAndPathMissConfig(FileSinkConfiguration.SOURCE_TYPE); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = MissingPropertyException.class) + public void testWithPartitionTypeButMissSourceSubPath() { + final Configuration c = initFileNameAndPath("date", false, "version"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = UnsupportedOperationException.class) + public void testWithPartitionTypeNoneButPartitionKeyStr() { + final Configuration c = initFileNameAndPath("none", false, "version"); + c.setProperty(HiveConfiguration.PARTITION_KEY_NAME, "datestr"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test + public void testConfigurationWithS3() { + final Configuration c = initS3("S3"); + //Aws Property + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + Assert.assertEquals(conf.getSinkType().name(), "S3" ); + Assert.assertEquals(conf.getAwsLocal(), "/aws_test"); + Assert.assertEquals(conf.getBucketName().get(), "aws-test" ); + Assert.assertEquals(conf.getObjectKey().get(), "marmaray_test/test1" ); + Assert.assertEquals(conf.getFullPath(), this.fileSystem.get().getWorkingDirectory()+"/aws_test"); + } + + @Test(expected = UnsupportedOperationException.class) + public void testConfWithS3SinkTypeNotSupported() { + final Configuration c = initCommon("/newpath/newtest", "csv", 1500, " ", "LOCAL", "version"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } + + @Test(expected = UnsupportedOperationException.class) + public void testConfWithS3DispersalTypeNotSupported() { + final Configuration c = initCommon("/newpath/newtest", "csv", 1500, " ", "hdfs", "unknownType"); + final FileSinkConfiguration conf = new FileSinkConfiguration(c); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHadoopConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHadoopConfiguration.java new file mode 100644 index 0000000..9884d31 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHadoopConfiguration.java @@ -0,0 +1,44 @@ +/* + * 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 org.junit.Assert; +import org.junit.Test; + +public class TestHadoopConfiguration { + + @Test + public void testHadoopConf() { + final Configuration conf = new Configuration(); + final Map hadoopProps = new HashMap<>(); + hadoopProps.put("hadoopProp1", "value1"); + hadoopProps.put("hadoopProp2", "value2"); + hadoopProps.entrySet().stream().forEach(entry -> conf + .setProperty(HadoopConfiguration.HADOOP_COMMON_PROPERTY_PREFIX + entry.getKey(), entry.getValue())); + final Map nonHadoopProps = new HashMap<>(); + nonHadoopProps.put("nonHadoopProp1", "value1"); + nonHadoopProps.put("nonHadoopProp2", "value2"); + nonHadoopProps.entrySet().stream().forEach(entry -> conf.setProperty(entry.getKey(), entry.getValue())); + final org.apache.hadoop.conf.Configuration hadoopConf = new HadoopConfiguration(conf).getHadoopConf(); + Assert.assertTrue(hadoopProps.entrySet().stream() + .allMatch(entry -> hadoopConf.get(entry.getKey()).equals(entry.getValue()))); + Assert.assertTrue(nonHadoopProps.entrySet().stream() + .noneMatch(entry -> hadoopConf.get(entry.getKey()) != null)); + } +} 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 new file mode 100644 index 0000000..aae5a66 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieConfiguration.java @@ -0,0 +1,90 @@ +/* + * 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.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.Assert; +import org.junit.Test; + +import static com.uber.marmaray.common.util.SchemaTestUtil.getSchema; + +@Slf4j +public class TestHoodieConfiguration { + + @Test + public void testMultiTableConfiguration() { + final String tableName = "table1"; + final String basePath = "/tmp/basePath"; + + Assert.assertEquals("marmaray.hoodie.tables.table1.insert_split_size", + HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE, tableName)); + Assert.assertEquals("marmaray.hoodie.default.insert_split_size", + HoodieConfiguration.getDefaultPropertyKey(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE)); + + // Integer. + verifyProperty(tableName, createHoodieConfiguration(tableName, basePath), new Integer(101), new Integer(280), + new Integer(880)); + // Long. + verifyProperty(tableName, createHoodieConfiguration(tableName, basePath), new Long(101), new Long(280), + new Long(880)); + // Double. + verifyProperty(tableName, createHoodieConfiguration(tableName, basePath), new Double(10.1), new Double(2.80), + new Double(8.80)); + // Boolean. + verifyProperty(tableName, createHoodieConfiguration(tableName, basePath), new Boolean(true), new Boolean(false), + new Boolean(false)); + // String. + verifyProperty(tableName, createHoodieConfiguration(tableName, basePath), new Integer("101"), + new Integer("280"), new Integer("880")); + } + + public static HoodieConfiguration createHoodieConfiguration(@NotEmpty final String tableName, + @NotEmpty final String basePath) { + final String schemaStr = getSchema("ts", "uuid", 4, 8).toString(); + return HoodieConfiguration + .newBuilder(tableName) + .withTableName(tableName) + .withBasePath(basePath) + .withSchema(schemaStr) + .enableMetrics(false) + .build(); + } + + private static void verifyProperty(@NotEmpty final String tableName, + @NonNull final HoodieConfiguration hoodieConf, @NonNull final T defaultValue, + @NonNull final T defaultPropertyValue, @NonNull final T tableValue) { + Object value = + hoodieConf.getProperty(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE, defaultValue); + Assert.assertTrue(value.equals(defaultValue) && value.getClass() == defaultValue.getClass()); + hoodieConf.getConf().setProperty( + HoodieConfiguration.getDefaultPropertyKey(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE), + defaultPropertyValue.toString()); + value = + hoodieConf.getProperty(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE, defaultValue); + Assert.assertTrue( + value.equals(defaultPropertyValue) && value.getClass() == defaultPropertyValue.getClass()); + hoodieConf.getConf().setProperty( + HoodieConfiguration.getTablePropertyKey(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE, + tableName), tableValue.toString()); + value = + hoodieConf.getProperty(HoodieConfiguration.HOODIE_INSERT_SPLIT_SIZE, defaultValue); + Assert.assertTrue( + value.equals(tableValue) && value.getClass() == tableValue.getClass()); + } +} 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 new file mode 100644 index 0000000..2b3b8f9 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestHoodieIndexConfiguration.java @@ -0,0 +1,91 @@ +/* + * 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.uber.hoodie.config.HoodieIndexConfig; +import lombok.NonNull; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.tools.ant.filters.StringInputStream; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Properties; + +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(ConnectionFactory.class) +public class TestHoodieIndexConfiguration { + + private static final String TABLE_KEY = "test_table"; + + @Test + public void configureHoodieIndex() throws Exception { + final Configuration conf = new Configuration(new StringInputStream("a: b"), Optional.absent()); + setHoodieTableProperty(conf, "table_name", "myTestTable"); + setHoodieTableProperty(conf, "base_path", "/path/to/my/test/table"); + + // should use a bloom index + Assert.assertEquals("BLOOM", getHoodiePropertyFromConfig(conf, "hoodie.index.type")); + + // test hbase index + mockStatic(ConnectionFactory.class); + final Connection mockConn = mock(Connection.class); + final Admin mockAdmin = mock(Admin.class); + when(ConnectionFactory.createConnection(Matchers.any())).thenReturn(mockConn); + when(mockConn.getAdmin()).thenReturn(mockAdmin); + when(mockAdmin.tableExists(Matchers.any())).thenReturn(true); + setHoodieTableProperty(conf, "index.type", "hbase"); + setHoodieTableProperty(conf, "index.zookeeper_quorum", "foo, bar"); + setHoodieTableProperty(conf, "index.zookeeper_port", String.valueOf(500)); + setHoodieTableProperty(conf, "index.hbase.zknode.path", "/path/to/zk"); + setHoodieTableProperty(conf, "index.hbase_index_table", "myHbaseTable"); + Assert.assertEquals("HBASE", getHoodiePropertyFromConfig(conf, "hoodie.index.type")); + + } + + private String getHoodiePropertyFromConfig(@NonNull final Configuration conf, @NotEmpty final String property, + @NonNull final Optional version) { + final HoodieIndexConfiguration indexConf = new HoodieIndexConfiguration(conf, TABLE_KEY); + final HoodieIndexConfig hoodieIndexConfig = indexConf.configureHoodieIndex(); + final Properties props = hoodieIndexConfig.getProps(); + return props.getProperty(property); + } + + private String getHoodiePropertyFromConfig(@NonNull final Configuration conf, @NotEmpty final String property) { + return getHoodiePropertyFromConfig(conf, property, Optional.absent()); + } + + private void setHoodieTableProperty(@NonNull final Configuration conf, @NotEmpty final String property, + @NotEmpty final String value) { + final String fullProperty = String.format("marmaray.hoodie.tables.%s.%s", TABLE_KEY, property); + conf.setProperty(fullProperty, value); + + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaConfiguration.java b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaConfiguration.java new file mode 100644 index 0000000..dece018 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/configuration/TestKafkaConfiguration.java @@ -0,0 +1,67 @@ +/* + * 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.util.KafkaTestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static com.uber.marmaray.common.configuration.KafkaConfiguration.DEFAULT_GROUP_ID; +import static com.uber.marmaray.common.configuration.KafkaConfiguration.ENABLE_AUTO_COMMIT; +import static com.uber.marmaray.common.configuration.KafkaConfiguration.ENABLE_AUTO_COMMIT_VALUE; +import static com.uber.marmaray.common.configuration.KafkaConfiguration.GROUP_ID; +import static com.uber.marmaray.common.configuration.KafkaConfiguration.KAFKA_BROKER_LIST; +import static com.uber.marmaray.common.configuration.KafkaConfiguration.KAFKA_GROUP_ID; +import static com.uber.marmaray.common.configuration.KafkaSourceConfiguration.KAFKA_PROPERTY_PREFIX; + +public class TestKafkaConfiguration { + + public static final String BROKER_LIST = "foo"; + + @Test + public void testAutoCommitNotConfigurable() { + final Configuration conf = new Configuration(); + KafkaTestHelper.setMandatoryConf(conf, + Arrays.asList(KAFKA_BROKER_LIST, KAFKA_PROPERTY_PREFIX + ENABLE_AUTO_COMMIT), + Arrays.asList(BROKER_LIST, "bar")); + final KafkaConfiguration kafkaConf = new KafkaConfiguration(conf); + Assert.assertEquals(ENABLE_AUTO_COMMIT_VALUE, kafkaConf.getKafkaParams().get(ENABLE_AUTO_COMMIT)); + } + + @Test + public void testGroupIdConfigurable() { + final String myGroupId = "mygroup"; + final Configuration conf = new Configuration(); + KafkaTestHelper.setMandatoryConf(conf, Arrays.asList(KAFKA_BROKER_LIST, KAFKA_GROUP_ID), + Arrays.asList(BROKER_LIST, myGroupId)); + final KafkaConfiguration kafkaConf = new KafkaConfiguration(conf); + Assert.assertEquals(myGroupId, kafkaConf.getKafkaParams().get(GROUP_ID)); + } + + @Test + public void testGroupIdDefault() { + final Configuration conf = new Configuration(); + KafkaTestHelper.setMandatoryConf(conf, Collections.singletonList(KAFKA_BROKER_LIST), + Collections.singletonList(BROKER_LIST)); + final KafkaConfiguration kafkaConf = new KafkaConfiguration(conf); + Assert.assertEquals(DEFAULT_GROUP_ID, kafkaConf.getKafkaParams().get(GROUP_ID)); + } + +} \ No newline at end of file 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 new file mode 100644 index 0000000..d456257 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestAbstractDataConverter.java @@ -0,0 +1,94 @@ +/* + * 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; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.converterresult.ConverterResult; +import com.uber.marmaray.common.converters.data.AbstractDataConverter; +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.util.AbstractSparkTest; +import com.uber.marmaray.utilities.ErrorExtractor; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.spark.SparkException; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +@Slf4j +public class TestAbstractDataConverter extends AbstractSparkTest { + + public static String SUCCESS = "success"; + public static String INVALID_DATA = "invalid_data"; + public static String RUNTIME_EXCEPTION = "runtime_exception"; + + @Test + public void testExceptionHandling() { + final int successRecords = 5; + final int invalidDataRecords = 7; + final int runtimeExceptionRecords = 1; + + final List inputList = new ArrayList<>(); + // Adding only success & invalid_data records. + IntStream.range(0, successRecords).forEach(i -> inputList.add(SUCCESS)); + IntStream.range(0, invalidDataRecords).forEach(i -> inputList.add(INVALID_DATA)); + + final MockAbstractDataConverter mockConverter = new MockAbstractDataConverter(new Configuration(), new ErrorExtractor()); + final RDDWrapper result = mockConverter.map(this.jsc.get().parallelize(inputList)); + Assert.assertEquals(successRecords, result.getCount()); + + // Adding runtime exception records. This should fail the spark job. + IntStream.range(0, runtimeExceptionRecords).forEach(i -> inputList.add(RUNTIME_EXCEPTION)); + try { + mockConverter.map(this.jsc.get().parallelize(inputList)); + Assert.fail("expecting error here"); + } 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(RUNTIME_EXCEPTION, e.getCause().getCause().getMessage()); + } + + } + + private static class MockAbstractDataConverter extends AbstractDataConverter { + + MockAbstractDataConverter(@NonNull final Configuration conf, ErrorExtractor errorExtractor) { + super(conf, errorExtractor); + } + + @Override + protected List> convert(@NotEmpty final String data) throws Exception { + if (SUCCESS.equals(data)) { + return Arrays.asList(new ConverterResult(data)); + } else if (INVALID_DATA.equals(data)) { + throw new InvalidDataException(data); + } else { + throw new JobRuntimeException(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 new file mode 100644 index 0000000..6413307 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraDataFrameConverter.java @@ -0,0 +1,108 @@ +/* + * 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; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.data.CassandraSinkDataConverter; +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.TimestampInfo; +import java.util.Collections; +import java.util.List; +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.api.java.JavaRDD; +import org.junit.Assert; +import org.junit.Test; + +public class TestCassandraDataFrameConverter extends AbstractSparkTest { + + private static final String INT_FIELD = "int_field"; + private static final String DOUBLE_FIELD = "double_field"; + private static final String STRING_FIELD = "string_field"; + private static final String BOOLEAN_FIELD = "boolean_field"; + + @Test + public void convertCommonSchemaDataToCassandra() { + final Schema avroSchema = SchemaBuilder.record("commonSchema") + .fields() + .name(INT_FIELD).type().intType().noDefault() + .name(DOUBLE_FIELD).type().doubleType().noDefault() + .name(STRING_FIELD).type().stringType().noDefault() + .name(BOOLEAN_FIELD).type().booleanType().noDefault() + .endRecord(); + + final GenericRecord gr = new GenericData.Record(avroSchema); + gr.put(INT_FIELD, Integer.MAX_VALUE); + gr.put(DOUBLE_FIELD, Double.MAX_VALUE); + gr.put(STRING_FIELD, "foo"); + gr.put(BOOLEAN_FIELD, true); + + final AvroPayload ap = new AvroPayload(gr); + final List records = Collections.singletonList(ap); + final JavaRDD payloadRDD = this.jsc.get().parallelize(records); + + final CassandraSinkDataConverter csdc = new CassandraSinkDataConverter(avroSchema, + new Configuration(), + Optional.absent(), + Collections.EMPTY_LIST, + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + + final JavaRDD cassRDD = csdc.map(payloadRDD).getData(); + final List payloads = cassRDD.collect(); + + Assert.assertEquals(payloads.size(), 1); + final CassandraPayload cp = payloads.get(0); + final List fields = cp.getData(); + + Assert.assertEquals(4, fields.size()); + + for (int i = 0; i <= 3; i++) { + final CassandraDataField field = fields.get(i); + + switch (i) { + case 0: + Assert.assertEquals(INT_FIELD, ByteBufferUtil.convertToString(field.getColumnKey())); + Assert.assertEquals(Integer.MAX_VALUE, field.getValue().getInt()); + break; + + case 1: + Assert.assertEquals(ByteBufferUtil.convertToString(field.getColumnKey()), DOUBLE_FIELD); + Assert.assertEquals(Double.MAX_VALUE, field.getValue().getDouble(), 0.0); + break; + + case 2: + Assert.assertEquals(ByteBufferUtil.convertToString(field.getColumnKey()), STRING_FIELD); + Assert.assertEquals("foo", ByteBufferUtil.convertToString(field.getValue())); + break; + + case 3: + Assert.assertEquals(BOOLEAN_FIELD, ByteBufferUtil.convertToString(field.getColumnKey())); + boolean value = field.getValue().get() > 0 ? true : false; + Assert.assertTrue(value); + break; + } + } + } +} 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 new file mode 100644 index 0000000..d922dd0 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestCassandraSchemaConverter.java @@ -0,0 +1,156 @@ +/* + * 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; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.converters.schema.CassandraSchemaConverter; +import com.uber.marmaray.common.schema.cassandra.CassandraSchema; +import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField; +import com.uber.marmaray.common.util.SchemaTestUtil; +import com.uber.marmaray.utilities.SchemaUtil; +import com.uber.marmaray.utilities.TimestampInfo; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; + +public class TestCassandraSchemaConverter { + @Test + public void testConvertCommonToCassandraSchemaNoTimestamp() { + final Schema record = SchemaBuilder.record("commonSchema") + .fields() + .name("field0").type().intType().noDefault() + .name("field1").type().doubleType().noDefault() + .name("field2").type().stringType().noDefault() + .name("field3").type().booleanType().noDefault() + .endRecord(); + + final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", + "testTableName", Optional.absent()); + + Assert.assertEquals("testKeyspace", converter.getKeySpace()); + Assert.assertEquals("testTableName", converter.getTableName()); + + final CassandraSchema cassSchema = converter.convertToExternalSchema(record); + + Assert.assertTrue(cassSchema.getFields().size() == 4); + + for (int i = 0; i <= 3; i++) { + final CassandraSchemaField field = cassSchema.getFields().get(i); + Assert.assertEquals(field.getFieldName(), "field" + i); + + switch (i) { + case 0: + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.INT_TYPE)), + field.getType()); + break; + case 1: + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.DOUBLE_TYPE)), + field.getType()); + break; + case 2: + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE)), + field.getType()); + break; + case 3: + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.BOOLEAN_TYPE)), + field.getType()); + break; + default: + throw new RuntimeException("Field was unexpected"); + } + } + } + + @Test + public void testConvertCommonToCassandraSchemaWithStringTimestamp() { + final Schema record = SchemaBuilder.record("commonSchema") + .fields() + .name("field0").type().intType().noDefault() + .endRecord(); + + final TimestampInfo tsInfo = new TimestampInfo(Optional.of("10000"), false); + final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", + "testTableName", tsInfo, Optional.absent()); + final CassandraSchema cassSchema = converter.convertToExternalSchema(record); + + Assert.assertTrue(cassSchema.getFields().size() == 2); + + final CassandraSchemaField intField = cassSchema.getFields().get(0); + Assert.assertEquals("field0", intField.getFieldName()); + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.INT_TYPE)), intField.getType()); + + final CassandraSchemaField timestampField = cassSchema.getFields().get(1); + Assert.assertEquals(SchemaUtil.DISPERSAL_TIMESTAMP, timestampField.getFieldName()); + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE)), + timestampField.getType()); + } + + @Test + public void testConvertCommonToCassandraSchemaRemovesFieldStartingWithUnderscore() { + final Schema record = SchemaBuilder.record("commonSchema") + .fields() + .name("_field0").type().intType().noDefault() + .name("_field1").type().doubleType().noDefault() + .name("_field2").type().stringType().noDefault() + .name("field3").type().booleanType().noDefault() + .endRecord(); + + final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", + "testTableName", Optional.absent()); + final CassandraSchema cassSchema = converter.convertToExternalSchema(record); + Assert.assertTrue(cassSchema.getFields().size() == 1); + Assert.assertEquals("field3", cassSchema.getFields().get(0).getFieldName()); + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.BOOLEAN_TYPE)), + cassSchema.getFields().get(0).getType()); + + } + + @Test + public void testConvertCommonToCassandraSchemaWithFilteredFields() { + final Schema record = SchemaBuilder.record("commonSchema") + .fields() + .name("field0").type().intType().noDefault() + .name("field1").type().doubleType().noDefault() + .name("field2").type().stringType().noDefault() + .name("field3").type().booleanType().noDefault() + .endRecord(); + + final CassandraSchemaConverter converter = new CassandraSchemaConverter("testKeyspace", + "testTableName", Optional.of(new HashSet<>(Arrays.asList("field2", "field3")))); + final CassandraSchema cassSchema = converter.convertToExternalSchema(record); + Assert.assertTrue(cassSchema.getFields().size() == 2); + Assert.assertEquals("field2", cassSchema.getFields().get(0).getFieldName()); + Assert.assertEquals("field3", cassSchema.getFields().get(1).getFieldName()); + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE)), + cassSchema.getFields().get(0).getType()); + Assert.assertEquals(CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.BOOLEAN_TYPE)), + cassSchema.getFields().get(1).getType()); + } +} 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 new file mode 100644 index 0000000..1321508 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameDataConverter.java @@ -0,0 +1,166 @@ +/* + * 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; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.converters.data.SparkSourceDataConverter; +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.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Tests for converting data from DataFrame (Spark) -> Avro schema with data + */ +public class TestDataFrameDataConverter { + + private static final String STRING_FIELD = "string_field"; + private static final String INT_FIELD = "int_field"; + private static final String BOOL_FIELD = "bool_field"; + private static final String DOUBLE_FIELD = "double_field"; + + private Optional spark = Optional.absent(); + private Optional jsc = Optional.absent(); + + @Before + public void setupTest() { + final SparkConf sparkConf = SparkTestUtil.getSparkConf(TestCassandraDataFrameConverter.class.getName()); + this.spark = Optional.of(SparkTestUtil.getSparkSession(sparkConf)); + this.jsc = Optional.of(new JavaSparkContext(spark.get().sparkContext())); + } + + @After + public void teardownTest() { + if (this.spark.isPresent()) { + this.spark.get().stop(); + this.spark = Optional.absent(); + } + + if (this.jsc.isPresent()) { + this.jsc.get().stop(); + this.jsc = Optional.absent(); + } + } + + @Test + public void convertDataFrameToCommonDataSchemaTest() { + final List dataList = new ArrayList<>(); + dataList.add(new Object[] {"data0", Integer.MIN_VALUE, true, Double.MIN_VALUE}); + dataList.add(new Object[] {"data1", Integer.MAX_VALUE, false, Double.MAX_VALUE}); + + final JavaRDD rowRDDNoSchema = this.jsc.get() + .parallelize(dataList).map((Object[] row) -> RowFactory.create(row)); + + final StructType schema = DataTypes + .createStructType(new StructField[] { + DataTypes.createStructField(STRING_FIELD, DataTypes.StringType, false), + DataTypes.createStructField(INT_FIELD, DataTypes.IntegerType, false), + DataTypes.createStructField(BOOL_FIELD, DataTypes.BooleanType, false), + DataTypes.createStructField(DOUBLE_FIELD, DataTypes.DoubleType, false)}); + + final Dataset df = this.spark.get().createDataFrame(rowRDDNoSchema, schema); + final DataFrameSchemaConverter dfsc = new DataFrameSchemaConverter(); + final Schema commonSchema = dfsc.convertToCommonSchema(df.schema()); + + final SparkSourceDataConverter converter = + new SparkSourceDataConverter(schema, commonSchema, new Configuration(), Collections.singleton(STRING_FIELD), + new ErrorExtractor()); + final JavaRDD payloadRDD = converter.map(df.javaRDD()).getData(); + + final List payloads = payloadRDD.collect(); + + Assert.assertEquals(2, payloads.size()); + + final AvroPayload ap = payloads.get(0); + final GenericRecord gr = ap.getData(); + + Assert.assertEquals("data0", gr.get(STRING_FIELD).toString()); + Assert.assertEquals(Integer.MIN_VALUE, gr.get(INT_FIELD)); + Assert.assertEquals(true, gr.get(BOOL_FIELD)); + Assert.assertEquals(Double.MIN_VALUE, gr.get(DOUBLE_FIELD)); + + final AvroPayload ap2 = payloads.get(1); + final GenericRecord gr2 = ap2.getData(); + + Assert.assertEquals("data1", gr2.get(STRING_FIELD).toString()); + Assert.assertEquals(Integer.MAX_VALUE, gr2.get(INT_FIELD)); + Assert.assertEquals(false, gr2.get(BOOL_FIELD)); + Assert.assertEquals(Double.MAX_VALUE, gr2.get(DOUBLE_FIELD)); + } + + @Test + public void convertDataFrameToCommonDataNotAllColumnsPopulatedTest() throws Exception { + final List dataList = new ArrayList<>(); + dataList.add(new Object[] {"data0", Integer.MIN_VALUE, false}); + + final JavaRDD rowRDDNoSchema = this.jsc.get() + .parallelize(dataList).map((Object[] row) -> RowFactory.create(row)); + + final StructType schemaWithMissingField = DataTypes + .createStructType(new StructField[] { + DataTypes.createStructField(STRING_FIELD, DataTypes.StringType, false), + DataTypes.createStructField(INT_FIELD, DataTypes.IntegerType, false), + DataTypes.createStructField(BOOL_FIELD, DataTypes.BooleanType, false)}); + + final StructType expectedSchema = DataTypes + .createStructType(new StructField[] { + DataTypes.createStructField(STRING_FIELD, DataTypes.StringType, false), + DataTypes.createStructField(INT_FIELD, DataTypes.IntegerType, false), + DataTypes.createStructField(BOOL_FIELD, DataTypes.BooleanType, false), + DataTypes.createStructField(DOUBLE_FIELD, DataTypes.DoubleType, true)}); + + final Dataset df = this.spark.get().createDataFrame(rowRDDNoSchema, schemaWithMissingField); + final DataFrameSchemaConverter dfsc = new DataFrameSchemaConverter(); + final Schema commonSchema = dfsc.convertToCommonSchema(expectedSchema); + + final List rows = df.javaRDD().collect(); + + final SparkSourceDataConverter converter = + new SparkSourceDataConverter(expectedSchema, commonSchema, + new Configuration(), Collections.singleton(STRING_FIELD), new ErrorExtractor()); + + // The missing field should be handled gracefully without failing since rows can have sparse data + final AvroPayload payload = converter.convert(rows.get(0)).get(0).getSuccessData().get().getData(); + final GenericRecord gr = payload.getData(); + + Assert.assertEquals("data0", gr.get(STRING_FIELD).toString()); + Assert.assertEquals(Integer.MIN_VALUE, gr.get(INT_FIELD)); + Assert.assertEquals(false, gr.get(BOOL_FIELD)); + Assert.assertNull(gr.get(DOUBLE_FIELD)); + } +} 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 new file mode 100644 index 0000000..5ca044e --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/TestDataFrameSchemaConverter.java @@ -0,0 +1,102 @@ +/* + * 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; + +import com.uber.marmaray.common.converters.schema.DataFrameSchemaConverter; +import org.apache.avro.Schema; +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; + +public class TestDataFrameSchemaConverter { + + private static final String INT_TYPE = "intType"; + private static final String STRING_TYPE = "stringType"; + private static final String BOOLEAN_TYPE = "booleanType"; + private static final String DOUBLE_TYPE = "doubleType"; + private static final String BYTE_TYPE = "byteType"; + private static final String FLOAT_TYPE = "floatType"; + private static final String BINARY_TYPE = "binaryType"; + private static final String TIMESTAMP_TYPE = "timestampType"; + private static final String SHORT_TYPE = "shortType"; + private static final String LONG_TYPE = "longType"; + private static final String DATE_TYPE = "dateType"; + + @Test + public void testConvertSparkToAvroSchema() { + // StructType is the type of schema returned from a DataFrame (aka Dataset) + final StructType schema = DataTypes.createStructType(new StructField[]{ + DataTypes.createStructField(INT_TYPE, DataTypes.IntegerType, false), + DataTypes.createStructField(STRING_TYPE, DataTypes.StringType, false), + DataTypes.createStructField(BOOLEAN_TYPE, DataTypes.BooleanType, false), + DataTypes.createStructField(DOUBLE_TYPE, DataTypes.DoubleType, false), + DataTypes.createStructField(BYTE_TYPE, DataTypes.ByteType, false), + DataTypes.createStructField(FLOAT_TYPE, DataTypes.FloatType, false), + DataTypes.createStructField(BINARY_TYPE, DataTypes.ByteType, false), + DataTypes.createStructField(TIMESTAMP_TYPE, DataTypes.TimestampType, false), + DataTypes.createStructField(SHORT_TYPE, DataTypes.IntegerType, false), + DataTypes.createStructField(LONG_TYPE, DataTypes.LongType, false), + DataTypes.createStructField(DATE_TYPE, DataTypes.DateType, false) + }); + + final DataFrameSchemaConverter converter = new DataFrameSchemaConverter(); + final Schema commonSchema = converter.convertToCommonSchema(schema); + + Assert.assertEquals(commonSchema.getType(), Schema.Type.RECORD); + Assert.assertEquals(commonSchema.getFields().size(), 11); + + for (final Schema.Field field : commonSchema.getFields()) { + switch (field.name()) { + case INT_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.INT); + break; + case STRING_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.STRING); + break; + case BOOLEAN_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.BOOLEAN); + break; + case DOUBLE_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.DOUBLE); + break; + case BYTE_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.BYTES); + break; + case FLOAT_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.FLOAT); + break; + case BINARY_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.BYTES); + break; + case TIMESTAMP_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.STRING); + break; + case SHORT_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.INT); + break; + case LONG_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.LONG); + break; + case DATE_TYPE: + Assert.assertEquals(field.schema().getType(), Schema.Type.STRING); + break; + } + } + } +} 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 new file mode 100644 index 0000000..4aa5abe --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/converters/data/TestFileSinkDataConverter.java @@ -0,0 +1,109 @@ +/* + * 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/data/TestRDDWrapper.java b/marmaray/src/test/java/com/uber/marmaray/common/data/TestRDDWrapper.java new file mode 100644 index 0000000..992d660 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/data/TestRDDWrapper.java @@ -0,0 +1,41 @@ +/* + * 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.data; + +import com.uber.marmaray.common.util.AbstractSparkTest; +import java.util.Arrays; +import java.util.List; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; +import org.junit.Test; + +public class TestRDDWrapper extends AbstractSparkTest { + + @Test + public void testCount() { + final List list1 = Arrays.asList(1,2,3,4,5); + final JavaRDD list1RDD = this.jsc.get().parallelize(list1); + + final RDDWrapper rddWrapper1 = new RDDWrapper(list1RDD); + Assert.assertEquals(5, rddWrapper1.getCount()); + Assert.assertEquals(5, rddWrapper1.getData().count()); + + final RDDWrapper rddWrapper2 = new RDDWrapper(list1RDD, 1); + Assert.assertEquals(1, rddWrapper2.getCount()); + Assert.assertEquals(5, rddWrapper2.getData().count()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/dataset/TestUtilTable.java b/marmaray/src/test/java/com/uber/marmaray/common/dataset/TestUtilTable.java new file mode 100644 index 0000000..bc4bccf --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/dataset/TestUtilTable.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.dataset; + +import com.google.common.collect.ImmutableMap; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.FileTestUtil; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.Test; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link UtilTable} creation + */ +@Slf4j +public class TestUtilTable extends AbstractSparkTest { + + private static final Long JOB_START_TIME = Instant.now().getEpochSecond(); + private static final String TABLE_NAME = "test-table"; + + @Test + public void testWriteErrorTable() throws IOException { + final String dest = "errors"; + testWriteUtilTable(ErrorRecord.class, generateTestErrorRecords(), dest, true); + testWriteUtilTable(ErrorRecord.class, generateTestErrorRecords(), dest, false); + } + + @Test + public void testWriteExceptionTable() throws IOException { + final String destFolder = "exceptions"; + testWriteUtilTable(ExceptionRecord.class, generateTestExceptionRecords(), destFolder, true); + testWriteUtilTable(ExceptionRecord.class, generateTestExceptionRecords(), destFolder, false); + } + + @Test + public void testWriteMetricTable() throws IOException { + final String destFolder = "metrics"; + testWriteUtilTable(MetricRecord.class, generateTestMetricRecords(), destFolder, true); + testWriteUtilTable(MetricRecord.class, generateTestMetricRecords(), destFolder, false); + } + + public void testWriteUtilTable(@NonNull final Class type, + @NonNull final JavaRDD utilRecords, + @NotEmpty final String destFolder, + final boolean isDatePartitioned) throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final Path destPath = new Path(basePath, destFolder); + final UtilTable utilTable = new UtilTable(type, utilRecords, destPath, isDatePartitioned, spark.get()); + final Path destWritePath = utilTable.getDestWritePath(); + + assertEquals(new Long(10), utilTable.size()); + assertFalse(this.fileSystem.get().exists(destPath)); + + utilTable.show(); + utilTable.writeParquet(); + final FileStatus[] destPathChildren = this.fileSystem.get().listStatus(destWritePath); + + log.debug("Destination folder content:"); + Stream.of(destPathChildren).forEach(f -> log.debug(f.getPath().toString())); + + assertTrue(this.fileSystem.get().exists(destWritePath)); + assertTrue(destPathChildren.length > 0); + + final Dataset outputDataset = spark.get().read().parquet(destWritePath.toString()); + log.debug("Output dataset content"); + outputDataset.show(); + + final List datasetFieldNames = Arrays.asList(outputDataset.schema().fieldNames()); + final List requiredFieldNames = Arrays.asList("application_id", "job_name", "job_start_timestamp", "timestamp"); + assertTrue(datasetFieldNames.containsAll(requiredFieldNames)); + } + + private JavaRDD generateTestErrorRecords() { + final List recordList = new ArrayList<>(); + IntStream.range(0, 10).forEach(i -> { + final Long timestamp = Instant.now().getEpochSecond(); + final String rowKey = "row-key-abc-" + i; + final String columnName = "column" + i; + final String content = "This is content for " + rowKey; + final String errorType = "TestErrorType"; + final ErrorRecord errorRecord = new ErrorRecord( + getAppId(), JOB_NAME, JOB_START_TIME, timestamp, + TABLE_NAME, rowKey, columnName, content, errorType); + recordList.add(errorRecord); + }); + return jsc.get().parallelize(recordList); + } + + private JavaRDD generateTestExceptionRecords() { + final List recordList = new ArrayList<>(); + IntStream.range(0, 10).forEach(i -> { + final Long timestamp = Instant.now().getEpochSecond(); + final String exception = "RuntimeException " + i; + final String exceptionMessage = "A RuntimeException message " + i; + final String stacktrace = "Very long stacktrace " + i; + final boolean is_driver = false; + final ExceptionRecord exceptionRecord = new ExceptionRecord( + getAppId(), JOB_NAME, JOB_START_TIME, timestamp, + exception, exceptionMessage, stacktrace, is_driver); + recordList.add(exceptionRecord); + }); + return jsc.get().parallelize(recordList); + } + + private JavaRDD generateTestMetricRecords() { + final List recordList = new ArrayList<>(); + IntStream.range(0, 10).forEach(i -> { + final Long timestamp = Instant.now().getEpochSecond(); + final String metricName = "MetricName " + i; + final long metricValue = i; + final String tags = ImmutableMap.of("tag1-" + i, "tagVal1-" + i, "tag2-" + i, "tagVal2-" + i).toString(); + final MetricRecord metricRecord = new MetricRecord( + getAppId(), JOB_NAME, JOB_START_TIME, timestamp, + metricName, metricValue, tags); + recordList.add(metricRecord); + }); + return jsc.get().parallelize(recordList); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/forkoperator/TestForkOperator.java b/marmaray/src/test/java/com/uber/marmaray/common/forkoperator/TestForkOperator.java new file mode 100644 index 0000000..37b1da0 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/forkoperator/TestForkOperator.java @@ -0,0 +1,209 @@ +/* + * 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.forkoperator; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.exceptions.ForkOperationException; +import com.uber.marmaray.common.data.ForkData; +import com.uber.marmaray.common.util.AbstractSparkTest; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.storage.StorageLevel; +import org.junit.Assert; +import org.junit.Test; + +class DummyForkFunction extends ForkFunction { + + public static final int INVALID_KEY = 0; + public static final int VALID_KEY = 1; + public static final int DELETE_KEY = 2; + + public DummyForkFunction() { + registerKeys(Arrays.asList(INVALID_KEY, VALID_KEY, DELETE_KEY)); + } + + @Override + protected List> process(final String record) { + final List keys = new LinkedList<>(); + if (record.toLowerCase().contains("error")) { + // errors. + keys.add(INVALID_KEY); + } else { + if (record.toLowerCase().contains("delete")) { + // deleted records. + keys.add(DELETE_KEY); + } + // all no error records are valid. + keys.add(VALID_KEY); + } + return Collections.singletonList(new ForkData<>(keys, record)); + } +} + +class InvalidDummyForkFunction extends DummyForkFunction { + + public static final int UNREGISTERED_KEY = -1; + + @Override + protected List> process(final String record) { + final List> forkData = super.process(record); + // Add invalid keys. + forkData.stream().forEach(fd -> fd.getKeys().add(UNREGISTERED_KEY)); + return forkData; + } +} + +public class TestForkOperator extends AbstractSparkTest { + + public List getTestData(int errorRecords, int deletedRecords, int correctRecords) { + final List testData = new LinkedList<>(); + while (errorRecords-- > 0) { + testData.add("error" + errorRecords); + } + while (deletedRecords-- > 0) { + testData.add("delete" + deletedRecords); + } + while (correctRecords-- > 0) { + testData.add("correct" + correctRecords); + } + return testData; + } + + public List getInterleavedTestData(int errorRecords, int deletedRecords, int correctRecords) { + final List testData = new LinkedList<>(); + + while (errorRecords > 0 || deletedRecords > 0 || correctRecords > 0) { + if (errorRecords-- > 0) { + testData.add("error" + errorRecords); + } + + if (deletedRecords-- > 0) { + testData.add("delete" + deletedRecords); + } + + if (correctRecords-- > 0) { + testData.add("correct" + correctRecords); + } + } + return testData; + } + + @Test + public void testPersistLevel() { + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new DummyForkFunction(); + final List testData = getTestData(0, 0, 0); + ForkOperator forkOperator; + forkOperator = new ForkOperator<>(jsc.get().parallelize(testData), forkFunction, conf); + // If nothing is specified then it should use default persist level. + Assert + .assertEquals(StorageLevel.fromString(forkOperator.DEFAULT_PERSIST_LEVEL), forkOperator.getPersistLevel()); + + conf.setProperty(forkOperator.PERSIST_LEVEL, "MEMORY_ONLY"); + forkOperator = new ForkOperator<>(jsc.get().parallelize(testData), forkFunction, conf); + Assert.assertEquals(StorageLevel.MEMORY_ONLY(), forkOperator.getPersistLevel()); + } + + @Test + public void invalidForkKeys() { + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new InvalidDummyForkFunction(); + final int errorRecords = 5, deletedRecords = 7, correctRecords = 9; + final List testData = getTestData(errorRecords,deletedRecords,correctRecords); + final JavaRDD testDataRDD = jsc.get().parallelize(testData); + final ForkOperator forkOperator = new ForkOperator<>(testDataRDD, forkFunction, conf); + try { + forkOperator.execute(); + forkOperator.getRDD(DummyForkFunction.DELETE_KEY).count(); + // exception is expected. + Assert.assertTrue(false); + } catch (Exception e) { + Assert.assertEquals(ForkOperationException.class, e.getCause().getClass()); + Assert.assertTrue( + e.getCause().getMessage().contains(Integer.toString(InvalidDummyForkFunction.UNREGISTERED_KEY))); + } + } + + @Test + public void testForkOperatorWithErrorDeletedAndCorrectRecords() { + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new DummyForkFunction(); + final int errorRecords = 5, deletedRecords = 7, correctRecords = 9; + final List testData = getTestData(errorRecords, deletedRecords, correctRecords); + final JavaRDD testDataRDD = jsc.get().parallelize(testData); + final ForkOperator forkOperator = new ForkOperator<>(testDataRDD, forkFunction, conf); + forkOperator.execute(); + Assert.assertEquals(errorRecords, forkOperator.getRDD(DummyForkFunction.INVALID_KEY).count()); + Assert.assertEquals(deletedRecords + correctRecords, forkOperator.getRDD(DummyForkFunction.VALID_KEY).count()); + Assert.assertEquals(deletedRecords, forkOperator.getRDD(DummyForkFunction.DELETE_KEY).count()); + Assert.assertEquals(errorRecords, forkOperator.getCount(DummyForkFunction.INVALID_KEY)); + Assert.assertEquals(deletedRecords + correctRecords, forkOperator.getCount(DummyForkFunction.VALID_KEY)); + } + + @Test + public void testForkOperatorWithInterleavedRecords() { + // We explicitly interleave error, deleted, and correct records in the list to ensure + // our fork operator is able to get the data out correctly if they are not explicitly contiguous + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new DummyForkFunction(); + final int errorRecords = 5, deletedRecords = 7, correctRecords = 9; + final List testData = getInterleavedTestData(errorRecords, deletedRecords, correctRecords); + final JavaRDD testDataRDD = jsc.get().parallelize(testData); + final ForkOperator forkOperator = new ForkOperator<>(testDataRDD, forkFunction, conf); + forkOperator.execute(); + Assert.assertEquals(errorRecords, forkOperator.getRDD(DummyForkFunction.INVALID_KEY).count()); + Assert.assertEquals(deletedRecords + correctRecords, forkOperator.getRDD(DummyForkFunction.VALID_KEY).count()); + Assert.assertEquals(deletedRecords, forkOperator.getRDD(DummyForkFunction.DELETE_KEY).count()); + Assert.assertEquals(errorRecords, forkOperator.getCount(DummyForkFunction.INVALID_KEY)); + Assert.assertEquals(deletedRecords + correctRecords, forkOperator.getCount(DummyForkFunction.VALID_KEY)); + } + + @Test + public void testForkOperatorWithOnlyErrorRecords() { + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new DummyForkFunction(); + final int errorRecords = 5, deletedRecords = 0, correctRecords = 0; + final List testData = getTestData(errorRecords, deletedRecords, correctRecords); + final JavaRDD testDataRDD = jsc.get().parallelize(testData); + final ForkOperator forkOperator = new ForkOperator<>(testDataRDD, forkFunction, conf); + forkOperator.execute(); + Assert.assertEquals(errorRecords, forkOperator.getRDD(DummyForkFunction.INVALID_KEY).count()); + Assert.assertEquals(0, forkOperator.getRDD(DummyForkFunction.VALID_KEY).count()); + Assert.assertEquals(0, forkOperator.getRDD(DummyForkFunction.DELETE_KEY).count()); + Assert.assertEquals(errorRecords, forkOperator.getCount(DummyForkFunction.INVALID_KEY)); + Assert.assertEquals(0, forkOperator.getCount(DummyForkFunction.VALID_KEY)); + } + + @Test + public void testForkOperatorWithOnlyErrorAndCorrectRecords() { + final Configuration conf = new Configuration(); + final DummyForkFunction forkFunction = new DummyForkFunction(); + final int errorRecords = 5, deletedRecords = 0, correctRecords = 3; + final List testData = getTestData(errorRecords, deletedRecords, correctRecords); + final JavaRDD testDataRDD = jsc.get().parallelize(testData); + final ForkOperator forkOperator = new ForkOperator<>(testDataRDD, forkFunction, conf); + forkOperator.execute(); + Assert.assertEquals(errorRecords, forkOperator.getRDD(DummyForkFunction.INVALID_KEY).count()); + Assert.assertEquals(correctRecords, forkOperator.getRDD(DummyForkFunction.VALID_KEY).count()); + Assert.assertEquals(0, forkOperator.getRDD(DummyForkFunction.DELETE_KEY).count()); + Assert.assertEquals(errorRecords, forkOperator.getCount(DummyForkFunction.INVALID_KEY)); + Assert.assertEquals(correctRecords, forkOperator.getCount(DummyForkFunction.VALID_KEY)); + } +} 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 new file mode 100644 index 0000000..aee0d90 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestExecutionTimeJobExecutionStrategy.java @@ -0,0 +1,113 @@ +/* + * 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.common.metadata.JobManagerMetadataTracker; +import org.apache.commons.lang.ArrayUtils; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestExecutionTimeJobExecutionStrategy { + + private final Map> defaultThings = ArrayUtils.toMap(new Object[][]{ + { "dag1", ArrayUtils.toMap(new Object[][]{ + { JobDag.LAST_RUNTIME_METADATA_KEY, "100" }, + { JobDag.LAST_EXECUTION_METADATA_KEY, String.valueOf(System.currentTimeMillis()) } }) }, + { "dag2", ArrayUtils.toMap(new Object[][]{ + { JobDag.LAST_RUNTIME_METADATA_KEY, "200" }, + { JobDag.LAST_EXECUTION_METADATA_KEY, String.valueOf(System.currentTimeMillis()) } }) }, + { "dag4", ArrayUtils.toMap(new Object[][]{ + { "someString", "someValue" } }) }, + { "dag5", ArrayUtils.toMap(new Object[][]{ + { JobDag.LAST_RUNTIME_METADATA_KEY, "1" }, + { JobDag.LAST_EXECUTION_METADATA_KEY, String.valueOf(System.currentTimeMillis() + - TimeUnit.DAYS.toMillis(1)) } }) } + }); + + @Test + 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 JobDag jobDag1 = mockJobDag("dag1"); + jobDagQueue.add(jobDag1); + final JobDag jobDag3 = mockJobDag("dag3"); + jobDagQueue.add(jobDag3); + final JobDag jobDag2 = mockJobDag("dag2"); + jobDagQueue.add(jobDag2); + final JobDag jobDag4 = mockJobDag("dag4"); + jobDagQueue.add(jobDag4); + final JobDag jobDag5 = mockJobDag("dag5"); + jobDagQueue.add(jobDag5); + 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); + expectedQueue.add(jobDag5); + // then sorted in descending order + expectedQueue.add(jobDag2); + expectedQueue.add(jobDag1); + assertListEquals(expectedQueue, resultQueue); + } + + 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++) { + assertEquals(String.format("Item %d of queues mismatched: %s vs %s", i, expectedList, resultList), + expectedList.get(i), resultList.get(i)); + } + } + + private final class ConfigurationAnswer implements Answer { + + @Override public Optional> answer(final InvocationOnMock invocation) throws Throwable { + final String dagName = (String) invocation.getArguments()[0]; + if (defaultThings.containsKey(dagName)) { + return Optional.of(defaultThings.get(dagName)); + } else { + return Optional.absent(); + } + } + } + + private JobDag mockJobDag(final String name) { + final JobDag mockResult = mock(JobDag.class); + when(mockResult.getDataFeedName()).thenReturn(name); + when(mockResult.toString()).thenReturn(String.format("JobDag(%s)", name)); + return mockResult; + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobSubDag.java b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobSubDag.java new file mode 100644 index 0000000..4fbb2d3 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestJobSubDag.java @@ -0,0 +1,161 @@ +/* + * 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.common.IPayload; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.JobMetrics; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.NonNull; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +class TestPayload implements IPayload> { + + @NonNull + private final List numbers; + + public TestPayload(@NonNull final List numbers) { + this.numbers = numbers; + } + + @Override + public List getData() { + return this.numbers; + } +} + +class TestSubDag extends JobSubDag { + + public static final String EVEN_TASK = "even"; + public static final String ODD_TASK = "odd"; + + private final boolean isEven; + private final AtomicInteger stepCounter; + + public TestSubDag(final boolean isEven, @NonNull final AtomicInteger stepCounter) { + super(isEven ? EVEN_TASK : ODD_TASK); + this.isEven = isEven; + this.stepCounter = stepCounter; + } + + @Override + protected void executeNode(final Optional data) { + Assert.assertTrue(data.isPresent() && data.get() instanceof TestPayload); + final int stepNumber = this.stepCounter.incrementAndGet(); + Assert.assertTrue(stepNumber > 1 && stepNumber < 4); + ((TestPayload) data.get()).getData().stream().forEach( + number -> { + Assert.assertTrue(number % 2 == (this.isEven ? 0 : 1)); + } + ); + } + + @Override + protected void commitNode() { + final int stepNumber = this.stepCounter.incrementAndGet(); + Assert.assertTrue(stepNumber == (this.isEven ? 4 : 5)); + } +} + +class TestParentSubDag extends JobSubDag { + + private final Map childData = new HashMap<>(); + private final AtomicInteger stepCounter; + + public TestParentSubDag(@NonNull final AtomicInteger stepCounter) { + super("parent"); + this.stepCounter = stepCounter; + addSubDag(0, new TestSubDag(true, stepCounter)); + addSubDag(1, new TestSubDag(false, stepCounter)); + final String jobName = "testJob"; + setJobMetrics(new JobMetrics(jobName)); + setDataFeedMetrics(new DataFeedMetrics(jobName, new HashMap<>())); + } + + @Override + protected void executeNode(final Optional data) { + Assert.assertTrue(this.stepCounter.incrementAndGet() == 1); + Assert.assertTrue(data.isPresent()); + final List evenList = new LinkedList<>(); + final List oddList = new LinkedList<>(); + ((TestPayload) data.get()).getData().stream().forEach( + number -> { + if (number % 2 == 0) { + evenList.add(number); + } else { + oddList.add(number); + } + }); + this.childData.put(TestSubDag.EVEN_TASK, new TestPayload(evenList)); + this.childData.put(TestSubDag.ODD_TASK, new TestPayload(oddList)); + } + + @Override + protected Optional getDataForChild(final JobSubDag childSubDag) { + final IPayload childPayload = this.childData.get(childSubDag.getName()); + if (childPayload == null) { + return Optional.absent(); + } else { + return Optional.of(childPayload); + } + } + + @Override + protected void commitNode() { + Assert.assertEquals(6, this.stepCounter.incrementAndGet()); + } +} + +public class TestJobSubDag { + + @BeforeClass + public static void setupClass() { + final Configuration conf = new Configuration(); + conf.setProperty(ThreadPoolService.NUM_THREADS, "4"); + conf.setProperty(ThreadPoolService.JOB_DAG_THREADS, "2"); + ThreadPoolService.init(conf); + } + + @AfterClass + public static void cleanupClass() { + ThreadPoolService.shutdown(true); + } + + @Test + public void testSubDag() { + final AtomicInteger stepCounter = new AtomicInteger(0); + final TestParentSubDag parentSubDag = new TestParentSubDag(stepCounter); + final int count = 100; + final List nums = new ArrayList(count); + for (int i =0; i < count; i++) { + nums.add(i); + } + parentSubDag.execute(Optional.of(new TestPayload(nums))); + parentSubDag.commit(); + Assert.assertEquals(6, stepCounter.get()); + } +} 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 new file mode 100644 index 0000000..17088fc --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/job/TestThreadPoolService.java @@ -0,0 +1,262 @@ +/* + * 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.configuration.Configuration; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import lombok.extern.slf4j.Slf4j; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; + +@Slf4j +public class TestThreadPoolService { + + @After + public void shutdown() { + // Making sure that thread pool service is stopped always after all tests. + ThreadPoolService.shutdown(true); + } + + // One giant test method required as the ThreadPoolService can only be initialized once + @Test + public void testService(){ + final Callable callable = () -> "done"; + + try { + // no thread pool service is running submit should throw exception. + ThreadPoolService.submit(callable, ThreadPoolServiceTier.JOB_DAG_TIER); + Assert.fail(); + } catch (IllegalStateException e) { + // expected. + } + + // Test for numThreads values. + final AtomicInteger exeptionCount = new AtomicInteger(0); + IntStream.of(0, -1).forEach( + numThreads -> { + try { + initService(numThreads, 1, 1); + Assert.fail(); + } catch (IllegalStateException e) { + exeptionCount.incrementAndGet(); + } + + } + ); + Assert.assertEquals(2, exeptionCount.get()); + + final int numThreads = 4; + final int numJobDagThreads = 2; + final int numActionsThreads = 2; + initService(numThreads, numJobDagThreads, numActionsThreads); + + // re-initialization should fail. + try { + initService(numThreads, numJobDagThreads, numActionsThreads); + Assert.fail(); + } catch (JobRuntimeException e) { + // expected. + } + + // Test that caller always waits till spawned tasks have finished. + final AtomicInteger runningCounter = new AtomicInteger(0); + final int attempts = 10; + final Queue> branchResults = new LinkedList<>(); + final Callable task = + () -> { + int pending = attempts; + while (pending-- > 0) { + runningCounter.incrementAndGet(); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + Assert.fail(); + } + } + return 0; + }; + for (int i = 0; i < numThreads; i++) { + branchResults.add(ThreadPoolService.submit(task, ThreadPoolServiceTier.JOB_DAG_TIER)); + } + while (!branchResults.isEmpty()) { + try { + final Future future = branchResults.poll(); + future.get(); + Assert.assertTrue(future.isDone()); + } catch (InterruptedException e) { + Assert.fail(); + } catch (ExecutionException e) { + Assert.fail(); + } + } + Assert.assertEquals(numThreads * attempts, runningCounter.get()); + + final Callable result = () -> 2; + + final AtomicInteger counter = new AtomicInteger(0); + final Queue> results = new LinkedList<>(); + final Callable failingTask = + () -> { + counter.incrementAndGet(); + try { + Thread.sleep(200); + } catch (InterruptedException e) { + log.error("I was interrupted???"); + } + return 3/0; + }; + final int jobDagCount = 100 * numJobDagThreads; + final int actionsCount = 100 * numActionsThreads; + for (int i = 0; i < jobDagCount; i++) { + results.add(ThreadPoolService.submit(failingTask, ThreadPoolServiceTier.JOB_DAG_TIER)); + } + for (int i = 0; i < actionsCount; i++) { + results.add(ThreadPoolService.submit(failingTask, ThreadPoolServiceTier.ACTIONS_TIER)); + } + // wait for threads to stop. + ThreadPoolService.shutdown(false); + // make sure we scheduled them all + Assert.assertEquals(jobDagCount + actionsCount, results.size()); + // make sure they all ran + Assert.assertEquals(jobDagCount + actionsCount, counter.get()); + exeptionCount.set(0); + while (!results.isEmpty()) { + final Future status = results.poll(); + try { + status.get(); + Assert.fail(); + } catch (InterruptedException e) { + Assert.fail(); + } catch (ExecutionException e) { + exeptionCount.incrementAndGet(); + // expected. + Assert.assertEquals(ArithmeticException.class, e.getCause().getClass()); + } + } + // make sure they all failed + Assert.assertEquals(jobDagCount + actionsCount, exeptionCount.get()); + } + + @Test + public void testTimeouts() throws Exception { + ThreadPoolService.init(new Configuration()); + final AtomicInteger counter = new AtomicInteger(); + final Callable sleepCallable = () -> { + Thread.sleep(10000); + counter.incrementAndGet(); + return 1; + }; + Future future = ThreadPoolService.submit(sleepCallable, ThreadPoolServiceTier.ACTIONS_TIER, 1); + try { + future.get(); + Assert.fail("Exception should have been thrown"); + } catch (JobRuntimeException e) { + // expected + assert(e.getCause() instanceof TimeoutException); + } + Assert.assertEquals(0, counter.get()); + future = ThreadPoolService.submit(sleepCallable, ThreadPoolServiceTier.ACTIONS_TIER, 1000); + try { + future.get(1, TimeUnit.SECONDS); + Assert.fail("Exception should have been thrown"); + } catch (TimeoutException e) { + // expected + } + Assert.assertEquals(0, counter.get()); + } + + @Test + public void testCancel() throws InterruptedException { + final Configuration conf = new Configuration(); + initService(2, 1, 1); + final AtomicInteger counter = new AtomicInteger(); + final Callable sleepCallable = () -> { + Thread.sleep(10000); + counter.incrementAndGet(); + return 1; + }; + // thread should actually run + final Future future1 = ThreadPoolService.submit(sleepCallable, ThreadPoolServiceTier.ACTIONS_TIER); + // thread should be queued and not run immediately + final Future future2 = ThreadPoolService.submit(sleepCallable, ThreadPoolServiceTier.ACTIONS_TIER); + Thread.sleep(1000); + Assert.assertTrue(future1.cancel(true)); + Assert.assertTrue(future2.cancel(true)); + ThreadPoolService.shutdown(false); + Assert.assertTrue(future1.isCancelled()); + Assert.assertTrue(future2.isCancelled()); + Assert.assertEquals(0, counter.get()); + Assert.assertTrue(future1.isDone()); + Assert.assertTrue(future2.isDone()); + } + + @Test + public void testActionsCanExpand() throws Exception{ + final int numThreads = 4; + final int numActionsThreads = 1; + final int numJobDagThreads = 1; + final int sleepMillis = 2000; + initService(numThreads, numJobDagThreads, numActionsThreads); + final AtomicInteger startedActions = new AtomicInteger(); + final AtomicInteger startedJobDags = new AtomicInteger(); + final Callable actionsCallable = () -> { + startedActions.incrementAndGet(); + Thread.sleep(sleepMillis); + return 1; + }; + final Callable jobDagCallable = () -> { + startedJobDags.incrementAndGet(); + Thread.sleep(sleepMillis); + return 1; + }; + // start two threads, taking up one space in the shared queue + for (int i = 0; i < 2; i++) { + ThreadPoolService.submit(jobDagCallable, ThreadPoolServiceTier.JOB_DAG_TIER); + } + for (int i = 0; i < 10; i++) { + ThreadPoolService.submit(actionsCallable, ThreadPoolServiceTier.ACTIONS_TIER); + } + // make sure things have started + Thread.sleep(1000); + // assert that we have 3 actions and 1 job dag running + Assert.assertEquals(2, startedActions.get()); + Assert.assertEquals(2, startedJobDags.get()); + // wait for one round to finish; we should have queued 3 more actions + Thread.sleep(sleepMillis); + Assert.assertEquals(5, startedActions.get()); + } + + private void initService(final int numThreads, final int numJobDagThreads, final int numActionsThreads) { + final Configuration conf = new Configuration(); + conf.setProperty(ThreadPoolService.NUM_THREADS, String.valueOf(numThreads)); + conf.setProperty(ThreadPoolService.JOB_DAG_THREADS, String.valueOf(numJobDagThreads)); + conf.setProperty(ThreadPoolService.ACTIONS_THREADS, String.valueOf(numActionsThreads)); + ThreadPoolService.init(conf); + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/HDFSTestConstants.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/HDFSTestConstants.java new file mode 100644 index 0000000..8079485 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/HDFSTestConstants.java @@ -0,0 +1,25 @@ +/* + * 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; + +public class HDFSTestConstants { + public static final String BASE_METADATA_PATH = "baseMetadataPath"; + public static final String JOBMANAGER_BASE_METADATA_BASEPATH = "jobManagerBasePath"; + public static final String BASE_RAW_DATA_PATH = "baseRawDataPath"; + public static final String LOCAL = "local"; + +} 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 new file mode 100644 index 0000000..260138c --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/MemoryMetadataManager.java @@ -0,0 +1,64 @@ +/* + * 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.base.Optional; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import lombok.extern.slf4j.Slf4j; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * This class will be helpful for testing. + */ +@Slf4j +public class MemoryMetadataManager implements IMetadataManager { + private final Map metadataMap = new HashMap<>(); + + @Override + public void set(@NotEmpty final String key, @NotEmpty final StringValue value) { + metadataMap.put(key, value); + } + + @Override + public Optional remove(@NotEmpty final String key) { + return Optional.fromNullable(this.metadataMap.remove(key)); + } + + @Override + public Optional get(@NotEmpty final String key) { + if (this.metadataMap.containsKey(key)) { + return Optional.of(this.metadataMap.get(key)); + } else { + return Optional.absent(); + } + } + + @Override + public void saveChanges() throws IOException { + // no-op + } + + @Override + public Set getAllKeys() { + return metadataMap.keySet(); + } +} 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 new file mode 100644 index 0000000..80108cf --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSDatePartitionManager.java @@ -0,0 +1,237 @@ +/* + * 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.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.HiveSourceConfiguration; +import com.uber.marmaray.utilities.FSUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.concurrent.atomic.AtomicBoolean; + +public class TestHDFSDatePartitionManager { + private static final String JOBNAME = "jobFoo"; + private static final String FILE1 = "file1"; + private static final String DATESTR = "datestr"; + private static final String RAW_DATA_PATH = HDFSTestConstants.BASE_RAW_DATA_PATH + File.separator + JOBNAME; + private FileSystem fs; + + @Before + public void setupTest() throws IOException { + this.fs = FSUtils.getFs(new Configuration()); + } + + @After + public void tearDownTest() throws IOException { + this.fs.delete(new Path(HDFSTestConstants.BASE_METADATA_PATH), true); + this.fs.delete(new Path(HDFSTestConstants.BASE_RAW_DATA_PATH), true); + this.fs.close(); + } + + @Test + public void testGetNextPartitionWithNonexistentCheckpoint() throws IOException { + final Path basePath = new Path(RAW_DATA_PATH, "datestr=2017-05-01"); + this.fs.mkdirs(basePath); + + final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + DATESTR, + Optional.absent(), + this.fs); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-05-01", partition.get()); + } + + @Test + public void testGetNextPartitionWithStartDateAndNoCheckpoint() throws IOException, ParseException { + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-05-01")); + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-06-01")); + + final SimpleDateFormat sdf = new SimpleDateFormat(HiveSourceConfiguration.HIVE_START_DATE_FORMAT); + 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), + this.fs); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-06-01", partition.get()); + } + + @Test + public void testGetNextPartitionWitMultipleDatePartitionsAndNoCheckpoint() + throws IOException { + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-05-01")); + 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(), + this.fs); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-05-01", partition.get()); + } + + @Test + public void testGetNextPartitionWithMultipleDatePartitionsAndOneCheckpoint() + throws IOException { + // Job has multiple data partitions, one is less than checkpoint and the other is larger + final Path partition1 = new Path(RAW_DATA_PATH, "datestr=2017-05-01"); + final Path partition2 = new Path(RAW_DATA_PATH, "datestr=2017-05-03"); + + this.fs.mkdirs(new Path(partition1, FILE1)); + this.fs.mkdirs(new Path(partition2, FILE1)); + + + 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(), + this.fs); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-05-03", partition.get()); + } + + @Test + public void testGetNextPartitionWithCheckpointLaterThanStartDate() + throws IOException, ParseException { + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-05-01")); + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-06-01")); + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-07-01")); + + final SimpleDateFormat sdf = new SimpleDateFormat(HiveSourceConfiguration.HIVE_START_DATE_FORMAT); + final Date startDate = sdf.parse("2017-05-03"); + + 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), + this.fs); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-07-01", partition.get()); + } + + @Test + public void testGetNextPartitionWithCheckpointBeforeThanStartDate() + throws IOException, ParseException { + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-05-01")); + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-06-01")); + this.fs.mkdirs(new Path(RAW_DATA_PATH, "datestr=2017-07-01")); + + final SimpleDateFormat sdf = new SimpleDateFormat(HiveSourceConfiguration.HIVE_START_DATE_FORMAT); + final Date startDate = sdf.parse("2017-06-01"); + + 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), + this.fs); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-07-01", partition.get()); + } + + + @Test + public void testGetNextPartitionWithSmallerExistentCheckpoint() throws IOException, InterruptedException { + final StringValue val1 = new StringValue("datestr=2017-05-01"); + + final Path partition1 = new Path(RAW_DATA_PATH, "datestr=2017-05-02"); + this.fs.mkdirs(new Path(partition1, FILE1)); + + final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + DATESTR, + Optional.absent(), + this.fs); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals("datestr=2017-05-02", partition.get()); + } + + @Test + public void testGetNextPartitionWithLargerExistentCheckpoint() throws IOException, InterruptedException { + // In this case the checkpoint is larger than the data partition so there is no "next" partition + final StringValue val1 = new StringValue("datestr=2017-05-02"); + + final Path partition1 = new Path(RAW_DATA_PATH, "datestr=2017-05-01"); + this.fs.mkdirs(new Path(partition1, FILE1)); + + final HDFSDatePartitionManager pm = new HDFSDatePartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + DATESTR, + Optional.absent(), + this.fs); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Optional partition = pm.getNextPartition(); + Assert.assertFalse(partition.isPresent()); + } +} 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 new file mode 100644 index 0000000..00f173f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSJobLevelMetadataTracker.java @@ -0,0 +1,90 @@ +/* + * 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.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +public class TestHDFSJobLevelMetadataTracker { + + private static final String JOB_NAME = "testJobMetadataManager"; + private static final String TEST_TOPIC_1 = "topic1"; + private static final String TEST_TOPIC_2 = "topic2"; + private Path basePath; + private FileSystem fileSystem; + private Configuration config; + private Optional tracker; + + @Before + public void setupTest() throws IOException { + config = new Configuration(); + config.setProperty(MetadataConstants.JOBMANAGER_METADATA_STORAGE, "hdfs"); + config.setProperty(MetadataConstants.JOBMANAGER_METADATA_HDFS_BASEPATH, + HDFSTestConstants.JOBMANAGER_BASE_METADATA_BASEPATH); + this.fileSystem = FSUtils.getFs(config); + final AtomicBoolean condition = new AtomicBoolean(true); + basePath = new Path(HDFSTestConstants.JOBMANAGER_BASE_METADATA_BASEPATH); + this.tracker = Optional.of(new JobManagerMetadataTracker(config)); + } + + @Test + public void testReadWriteJobManagerMetadata() throws IOException { + final Map testData1 = new HashMap(); + testData1.put("metaKey1","metaValue1"); + testData1.put("metaKey2","metaValue2"); + this.tracker.get().set(TEST_TOPIC_1, testData1 ); + final Map testData2 = new HashMap(); + testData2.put("metaKey1","metaValue1"); + testData2.put("metaKey2","metaValue2"); + this.tracker.get().set(TEST_TOPIC_2, testData2); + this.tracker.get().writeJobManagerMetadata(); + final FileStatus[] statuses = fileSystem.listStatus(basePath); + Assert.assertEquals(1,fileSystem.listStatus(basePath).length); + + this.tracker = Optional.absent(); + this.tracker = Optional.of(new JobManagerMetadataTracker(config)); + + Assert.assertEquals(this.tracker.get().contains(TEST_TOPIC_1), true); + Assert.assertEquals(this.tracker.get().contains(TEST_TOPIC_2), true); + Assert.assertEquals(this.tracker.get().get(TEST_TOPIC_2).get().keySet().size(),2); + Assert.assertEquals(this.tracker.get().get(TEST_TOPIC_1).get().keySet().size(),2); + Assert.assertEquals(this.tracker.get().get(TEST_TOPIC_1).get().get("metaKey1").toString(),"metaValue1"); + Assert.assertEquals(this.tracker.get().get(TEST_TOPIC_1).get().get("metaKey1").toString(),"metaValue1"); + } + + @After + public void clearTestSetup() throws IOException { + this.fileSystem.delete(basePath, true); + + if (this.fileSystem != null) { + this.fileSystem.close(); + } + } +} 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 new file mode 100644 index 0000000..f0a8b9b --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSMetadataManager.java @@ -0,0 +1,133 @@ +/* + * 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.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestHDFSMetadataManager { + + private static final String JOB_NAME = "jobName"; + private FileSystem fileSystem; + private HDFSMetadataManager metadataManager; + + @Before + public void setupTest() throws IOException { + this.fileSystem = FSUtils.getFs(new Configuration()); + 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); + } + + @After + public void tearDownTest() throws IOException { + this.fileSystem.delete(new Path(HDFSTestConstants.BASE_METADATA_PATH), true); + + if (this.fileSystem != null) { + this.fileSystem.close(); + } + } + + @Test + public void testHDFSReadWriteSingleMetadataFile() throws IOException { + // Test in memory + final StringValue val = new StringValue("testVal"); + this.metadataManager.set(MetadataConstants.CHECKPOINT_KEY, val); + final Optional readValue = this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY); + Assert.assertTrue(readValue.isPresent()); + Assert.assertTrue(readValue.get().getValue().equals("testVal")); + + this.metadataManager.set("foo", new StringValue("bar")); + + // Serialize the metadata map to a file + this.metadataManager.saveChanges(); + final Optional fs = this.metadataManager.getLatestMetadataFile(); + Assert.assertTrue(fs.isPresent()); + // Deserialize the metadata map and check contents are the same + final Map loadedMap = this.metadataManager.loadMetadata(fs.get().getPath()); + validateDeserializedMapEqualsInMemoryMap(loadedMap); + } + + @Test + public void testHDFSOverwriteCheckpointValue() throws IOException, InterruptedException { + 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 Optional fs = this.metadataManager.getLatestMetadataFile(); + Assert.assertTrue(fs.isPresent()); + final Map loadedMap = this.metadataManager.loadMetadata(fs.get().getPath()); + validateDeserializedMapEqualsInMemoryMap(loadedMap); + } + + @Test + public void testDeletionIsPropagated() throws Exception { + final StringValue val1 = new StringValue("testVal"); + this.metadataManager.set(MetadataConstants.CHECKPOINT_KEY, val1); + + this.metadataManager.saveChanges(); + Optional fs = this.metadataManager.getLatestMetadataFile(); + Assert.assertTrue(fs.isPresent()); + Map loadedMap = this.metadataManager.loadMetadata(fs.get().getPath()); + validateDeserializedMapEqualsInMemoryMap(loadedMap); + + // reload the configuration + setupTest(); + Assert.assertTrue(this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY).isPresent()); + + this.metadataManager.remove(MetadataConstants.CHECKPOINT_KEY); + Assert.assertFalse(this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY).isPresent()); + this.metadataManager.saveChanges(); + fs = this.metadataManager.getLatestMetadataFile(); + Assert.assertFalse(this.metadataManager.get(MetadataConstants.CHECKPOINT_KEY).isPresent()); + loadedMap = this.metadataManager.loadMetadata(fs.get().getPath()); + validateDeserializedMapEqualsInMemoryMap(loadedMap); + } + + private void validateDeserializedMapEqualsInMemoryMap(final Map deserializedMap) { + for (Map.Entry entry : deserializedMap.entrySet()) { + final Optional valueInMemory = this.metadataManager.get(entry.getKey()); + Assert.assertTrue(valueInMemory.isPresent()); + Assert.assertEquals(valueInMemory.get().getValue(), entry.getValue().getValue()); + } + + Assert.assertEquals(this.metadataManager.getAllKeys(), deserializedMap.keySet()); + } + + // T918085 - Todo: Add test to verify metadata files are culled +} 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 new file mode 100644 index 0000000..53b8fd9 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHDFSPartitionManager.java @@ -0,0 +1,245 @@ +/* + * 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.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestHDFSPartitionManager { + + private static final String JOBNAME = "jobFoo"; + private static final String PARTITION0 = "partition0"; + private static final String PARTITION1 = "partition1"; + private static final String PARTITION2 = "partition2"; + private static final String PARTITION3 = "partition3"; + private static final String FILE1 = "file1"; + private static final String METADATA_PATH = HDFSTestConstants.BASE_METADATA_PATH + File.separator + JOBNAME; + private static final String RAW_DATA_PATH = HDFSTestConstants.BASE_RAW_DATA_PATH + File.separator + JOBNAME; + private FileSystem fileSystem; + + @Before + public void setupTest() throws IOException { + this.fileSystem = FSUtils.getFs(new Configuration()); + } + + @After + public void tearDownTest() throws IOException { + this.fileSystem.delete(new Path(METADATA_PATH), true); + this.fileSystem.delete(new Path(HDFSTestConstants.BASE_RAW_DATA_PATH), true); + + if (this.fileSystem != null) { + this.fileSystem.close(); + } + } + + @Test + public void testGetNextPartitionWithNonExistentCheckpoint() throws InterruptedException, IOException { + final Path partitionPath = new Path(RAW_DATA_PATH, PARTITION1); + final Path filePath = new Path(partitionPath, FILE1); + + this.fileSystem.create(filePath); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals(PARTITION1, partition.get()); + } + + @Test + public void testGetNextPartitionWithOnlyTempFileCheckpoints() throws InterruptedException, IOException { + final Path partitionPath = new Path(RAW_DATA_PATH, PARTITION1); + final Path filePath = new Path(partitionPath, FILE1); + + this.fileSystem.create(filePath); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + // if this metadata was saved successfully we would say there's no partition to process + // but this will be in a temp file so it will be ignored + pm.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(PARTITION2)); + pm.saveChanges(); + + final Optional fs = pm.getLatestMetadataFile(); + Assert.assertTrue(fs.isPresent()); + + // move the metadata file back to a temp location + this.fileSystem.rename(fs.get().getPath(), + new Path(fs.get().getPath().toString() + MetadataConstants.TEMP_FILE_EXTENSION)); + + final Optional partition = pm.getNextPartition(); + Assert.assertTrue(partition.isPresent()); + Assert.assertEquals(PARTITION1, partition.get()); + } + + @Test + public void testGetNextPartitionSinglePartition() throws IOException, InterruptedException { + final Path partitionPath = new Path(RAW_DATA_PATH, PARTITION2); + final Path filePath = new Path(partitionPath, FILE1); + this.fileSystem.create(filePath); + + final StringValue val1 = new StringValue(PARTITION1); + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + Assert.assertTrue(pm.getNextPartition().isPresent()); + Assert.assertTrue(pm.getNextPartition().get().equals(PARTITION2)); + } + + @Test + public void testGetNextPartitionMultipleDataPartitions() throws IOException, InterruptedException { + final StringValue val1 = new StringValue(PARTITION1); + + final Path partition2Path = new Path(RAW_DATA_PATH, PARTITION2); + final Path partition3Path = new Path(RAW_DATA_PATH, PARTITION3); + this.fileSystem.create(new Path(partition2Path, FILE1)); + this.fileSystem.create(new Path(partition3Path, FILE1)); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + Assert.assertTrue(pm.getNextPartition().isPresent()); + Assert.assertTrue(pm.getNextPartition().get().equals(PARTITION2)); + } + + @Test + public void testGetLastCheckpointMultipleCheckpoints() throws IOException, InterruptedException { + final Path partition1Path = new Path(RAW_DATA_PATH, PARTITION1); + final Path partition2Path = new Path(RAW_DATA_PATH, PARTITION2); + this.fileSystem.create(new Path(partition1Path, FILE1)); + this.fileSystem.create(new Path(partition2Path, FILE1)); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + Assert.assertTrue(pm.getNextPartition().isPresent()); + Assert.assertEquals(pm.getNextPartition().get(), PARTITION1); + pm.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(PARTITION1)); + pm.saveChanges(); + + final HDFSPartitionManager pm2 = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + final Optional checkpoint = pm2.getLatestCheckpoint(); + Assert.assertTrue(checkpoint.isPresent()); + Assert.assertEquals(PARTITION1, checkpoint.get().getValue()); + Assert.assertTrue(pm2.getNextPartition().isPresent()); + Assert.assertEquals(pm2.getNextPartition().get(), PARTITION2); + + pm2.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(PARTITION2)); + pm2.saveChanges(); + + + final HDFSPartitionManager pm3 = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + final Optional latestCheckpoint = pm3.getLatestCheckpoint(); + Assert.assertTrue(latestCheckpoint.isPresent()); + Assert.assertEquals(PARTITION2, latestCheckpoint.get().getValue()); + Assert.assertFalse(pm3.getNextPartition().isPresent()); + } + + @Test + public void testGetNextPartitionCheckpointIsLargerThanPartition() throws InterruptedException, IOException { + final Path partition2Path = new Path(RAW_DATA_PATH, PARTITION2); + this.fileSystem.mkdirs(new Path(partition2Path, FILE1)); + + final StringValue val1 = new StringValue(PARTITION2); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + pm.set(MetadataConstants.CHECKPOINT_KEY, val1); + pm.saveChanges(); + + final Path partition1Path = new Path(RAW_DATA_PATH, PARTITION1); + this.fileSystem.mkdirs(new Path(partition1Path, FILE1)); + + // Checkpoint value is greater than the partitions in the data folder so nothing new to process + Assert.assertFalse(pm.getNextPartition().isPresent()); + } + + @Test + public void testGetExistingPartitions() throws IOException { + final Path partition0Path = new Path(RAW_DATA_PATH, PARTITION0); + final Path partition1Path = new Path(RAW_DATA_PATH, PARTITION1); + final Path partition2Path = new Path(RAW_DATA_PATH, PARTITION2); + this.fileSystem.create(new Path(partition0Path, FILE1)); + this.fileSystem.create(new Path(partition1Path, FILE1)); + this.fileSystem.create(new Path(partition2Path, FILE1)); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + Assert.assertEquals(3, pm.getExistingPartitions().size()); + + for (int i = 0; i < pm.getExistingPartitions().size(); i++) { + Assert.assertEquals("partition" + i, pm.getExistingPartitions().get(i)); + } + } + + @Test + public void testGetExistingPartitionsOnlyFilesExist() throws IOException { + final Path partition0File = new Path(RAW_DATA_PATH, PARTITION0); + this.fileSystem.create(partition0File); + + final HDFSPartitionManager pm = new HDFSPartitionManager(JOBNAME, + HDFSTestConstants.BASE_METADATA_PATH, + RAW_DATA_PATH, + this.fileSystem); + + Assert.assertEquals(1, pm.getExistingPartitions().size()); + Assert.assertEquals(RAW_DATA_PATH, pm.getNextPartition().get()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHoodieBasedMetadataManager.java b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHoodieBasedMetadataManager.java new file mode 100644 index 0000000..b1be2e4 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metadata/TestHoodieBasedMetadataManager.java @@ -0,0 +1,149 @@ +/* + * 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.hoodie.common.table.HoodieTableMetaClient; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.HadoopConfiguration; +import com.uber.marmaray.common.configuration.HoodieConfiguration; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.FileTestUtil; +import lombok.extern.slf4j.Slf4j; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.SchemaBuilder.FieldAssembler; +import org.apache.avro.SchemaBuilder.RecordBuilder; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Setting all the tests to ignore as the implementation of the job dag will require the + * refactoring of HoodieBasedMetadataManager + * + * In addition, the metadata manager is dependent on Hoodie Client which requires a + * non-backwards and old version of Google Guava (v.15). Hoodie Client needs to be updated + * to no longer use the Closeables.closeQuietly() method which has been deprecated. + */ +@Slf4j +public class TestHoodieBasedMetadataManager extends AbstractSparkTest { + + @Test + public void testHoodieMetadataManager() throws IOException { + final Path basePath = new Path(FileTestUtil.getTempFolder()); + final String tableName = "test-table"; + final String schemaStr = getSchema(10).toString(); + + final Configuration conf = new Configuration(); + final HoodieConfiguration hoodieConf = HoodieConfiguration.newBuilder(conf, tableName).withTableName(tableName) + .withBasePath(basePath.toString()).withSchema(schemaStr).withMetricsPrefix("hoodieMetricsPrefix") + .enableMetrics(false).build(); + final AtomicBoolean condition = new AtomicBoolean(true); + final HoodieBasedMetadataManager mgr = + new HoodieBasedMetadataManager(hoodieConf, condition, this.jsc.get()); + + // When no previous metadata is present then metadata map is expected to be empty. + Assert.assertEquals(0, mgr.getAll().size()); + Assert.assertTrue(mgr.getMetadataInfo().get(HoodieBasedMetadataManager.HOODIE_METADATA_KEY).isEmpty()); + + final String testKey = "partition1"; + final String testValue = "offset1"; + mgr.set(testKey, new StringValue(testValue)); + + // Now metadata map should have got updated. + Assert.assertEquals(1, mgr.getAll().size()); + Assert.assertFalse(mgr.getMetadataInfo().get(HoodieBasedMetadataManager.HOODIE_METADATA_KEY).isEmpty()); + + // Let's reset condition flag; so that saveChanges will not create any metadata / save metadata map. + condition.set(false); + mgr.saveChanges(); + // No hoodie commit files should have been created. + Assert.assertFalse(condition.get()); + Assert.assertTrue( + new HoodieTableMetaClient( + new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), basePath.toString(), true) + .getActiveTimeline().getCommitTimeline().filterCompletedInstants().empty()); + + // Now let's enable metadata creation. + condition.set(true); + mgr.saveChanges(); + Assert.assertFalse(condition.get()); + Assert.assertFalse( + new HoodieTableMetaClient( + new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), basePath.toString(), true) + .getActiveTimeline().getCommitTimeline().filterCompletedInstants().empty()); + + // Now let's create another + final AtomicBoolean condition2 = new AtomicBoolean(true); + final HoodieBasedMetadataManager mgr2 = + new HoodieBasedMetadataManager(hoodieConf, condition2, this.jsc.get()); + Assert.assertEquals(1, mgr2.getAll().size()); + Assert.assertFalse(mgr2.getMetadataInfo().get(HoodieBasedMetadataManager.HOODIE_METADATA_KEY).isEmpty()); + Assert.assertEquals(mgr.getMetadataInfo(), mgr2.getMetadataInfo()); + } + + @Test + public void testRemove() throws Exception { + final Path basePath = new Path(FileTestUtil.getTempFolder()); + final String tableName = "test-table"; + final String schemaStr = getSchema(10).toString(); + + final Configuration conf = new Configuration(); + final HoodieConfiguration hoodieConf = HoodieConfiguration.newBuilder(conf, tableName).withTableName(tableName) + .withBasePath(basePath.toString()).withSchema(schemaStr).withMetricsPrefix("hoodieMetricsPrefix") + .enableMetrics(false).build(); + final AtomicBoolean condition = new AtomicBoolean(true); + final HoodieBasedMetadataManager mgr = new HoodieBasedMetadataManager(hoodieConf, condition, this.jsc.get()); + + // set up default + final String testKey = "partition1"; + final String testValue = "offset1"; + mgr.set(testKey, new StringValue(testValue)); + mgr.saveChanges(); + + // mgr2 loads correctly + final HoodieBasedMetadataManager mgr2 = new HoodieBasedMetadataManager(hoodieConf, condition, this.jsc.get()); + Assert.assertEquals(testValue, mgr2.get(testKey).get().getValue()); + mgr2.remove(testKey); + Assert.assertFalse(mgr2.get(testKey).isPresent()); + + // mgr2 hasn't saved yet, so should still get old value + final HoodieBasedMetadataManager mgr3 = new HoodieBasedMetadataManager(hoodieConf, condition, this.jsc.get()); + Assert.assertEquals(testValue, mgr3.get(testKey).get().getValue()); + + // save remove + condition.set(true); + mgr2.saveChanges(); + + // new load shouldn't find it anymore + final HoodieBasedMetadataManager mgr4 = new HoodieBasedMetadataManager(hoodieConf, condition, this.jsc.get()); + Assert.assertFalse(mgr4.get(testKey).isPresent()); + + } + + private Schema getSchema(final int numOfSubFields) { + final RecordBuilder recordSchema = SchemaBuilder.builder().record("test"); + final FieldAssembler fields = recordSchema.fields(); + for (int i = 0; i < numOfSubFields; i++) { + fields.optionalString("test" + i); + } + return fields.endRecord(); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestDataFeedMetrics.java b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestDataFeedMetrics.java new file mode 100644 index 0000000..bf377e5 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestDataFeedMetrics.java @@ -0,0 +1,82 @@ +/* + * 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.google.common.base.Optional; +import org.junit.Test; +import org.spark_project.guava.collect.Maps; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link DataFeedMetrics} + */ +public class TestDataFeedMetrics { + private static DataFeedMetrics dataFeedMetrics = new DataFeedMetrics("test-job", + Collections.singletonMap(DataFeedMetrics.DATA_FEED_NAME, "test-topic")); + + @Test + public void testCreateTopicLongMetric() { + Metric topicMetric1 = dataFeedMetrics.createLongMetric(DataFeedMetricNames.OUTPUT_ROWCOUNT, + 100l, + Maps.newHashMap()); + Metric topicMetric2 = dataFeedMetrics.createLongMetric(DataFeedMetricNames.SIZE_BYTES, + 1028l, + Maps.newHashMap()); + + assertEquals("output_rowcount", topicMetric1.getMetricName()); + assertEquals("size_bytes", topicMetric2.getMetricName()); + + assertEquals(new Long(100), topicMetric1.getMetricValue()); + assertEquals(new Long(1028), topicMetric2.getMetricValue()); + + assertTrue(topicMetric1.getTags().containsKey("job")); + assertEquals(topicMetric1.getTags().get("job"), "test-job"); + assertTrue(topicMetric1.getTags().containsKey("datafeed")); + assertEquals(topicMetric1.getTags().get("datafeed"), "test-topic"); + assertFalse(topicMetric1.getTags().containsKey("hello")); + topicMetric1.addTag("hello", "world"); + assertTrue(topicMetric1.getTags().containsKey("hello")); + + assertTrue(dataFeedMetrics.getMetricSet().contains(topicMetric1)); + assertTrue(dataFeedMetrics.getMetricSet().contains(topicMetric2)); + } + + @Test + public void testCreateTopicTimerMetric() { + Metric topicMetric1 = dataFeedMetrics.createTimerMetric(DataFeedMetricNames.RUNTIME); + + assertEquals("runtime", topicMetric1.getMetricName()); + + assertTrue(topicMetric1 instanceof TimerMetric); + + assertTrue(topicMetric1.getTags().containsKey("job")); + assertEquals(topicMetric1.getTags().get("job"), "test-job"); + assertTrue(topicMetric1.getTags().containsKey("datafeed")); + assertEquals(topicMetric1.getTags().get("datafeed"), "test-topic"); + assertFalse(topicMetric1.getTags().containsKey("hello")); + topicMetric1.addTag("hello", "world"); + assertTrue(topicMetric1.getTags().containsKey("hello")); + + assertTrue(dataFeedMetrics.getMetricSet().contains(topicMetric1)); + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestJobMetrics.java b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestJobMetrics.java new file mode 100644 index 0000000..a5383ad --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestJobMetrics.java @@ -0,0 +1,76 @@ +/* + * 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.google.common.collect.Maps; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link JobMetrics} + */ +public class TestJobMetrics { + private static JobMetrics jobMetrics = new JobMetrics("test-job"); + + @Test + public void testCreateJobLongMetric() { + Metric jobMetric1 = jobMetrics.createJobLongMetric(JobMetricType.DRIVER_MEMORY, 100l, Maps.newHashMap()); + Metric jobMetric2 = jobMetrics.createJobLongMetric(JobMetricType.SIZE, 1028l, Maps.newHashMap()); + + assertEquals("job-driver_memory", jobMetric1.getMetricName()); + assertEquals("job-size", jobMetric2.getMetricName()); + + assertEquals(new Long(100), jobMetric1.getMetricValue()); + assertEquals(new Long(1028), jobMetric2.getMetricValue()); + + assertTrue(jobMetric1.getTags().containsKey("job")); + assertEquals(jobMetric1.getTags().get("job"), "test-job"); + assertFalse(jobMetric1.getTags().containsKey("topic")); + assertFalse(jobMetric1.getTags().containsKey("hello")); + jobMetric1.addTag("hello", "world"); + assertTrue(jobMetric1.getTags().containsKey("hello")); + + assertTrue(jobMetrics.getMetricSet().contains(jobMetric1)); + assertTrue(jobMetrics.getMetricSet().contains(jobMetric2)); + } + + @Test + public void testCreateJobTimerMetric() { + Metric jobMetric1 = jobMetrics.createJobTimerMetric(JobMetricType.RUNTIME, Maps.newHashMap()); + Metric jobMetric2 = jobMetrics.createJobTimerMetric(JobMetricType.STAGE_RUNTIME, Maps.newHashMap()); + + assertEquals("job-runtime", jobMetric1.getMetricName()); + assertEquals("job-stage_runtime", jobMetric2.getMetricName()); + + assertTrue(jobMetric1 instanceof TimerMetric); + assertTrue(jobMetric2 instanceof TimerMetric); + + assertTrue(jobMetric1.getTags().containsKey("job")); + assertEquals(jobMetric1.getTags().get("job"), "test-job"); + assertFalse(jobMetric1.getTags().containsKey("topic")); + assertFalse(jobMetric1.getTags().containsKey("hello")); + jobMetric1.addTag("hello", "world"); + assertTrue(jobMetric1.getTags().containsKey("hello")); + + assertTrue(jobMetrics.getMetricSet().contains(jobMetric1)); + assertTrue(jobMetrics.getMetricSet().contains(jobMetric2)); + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestTimerMetric.java b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestTimerMetric.java new file mode 100644 index 0000000..06bac4c --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/metrics/TestTimerMetric.java @@ -0,0 +1,58 @@ +/* + * 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; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** + * Test for TimerMetric + */ +public class TestTimerMetric { + + private TimerMetric timerMetric; + private final int SLEEP_TIME = 1000; + private final int THRESHOLD = 50; + + @Before + public void setupTestClass() { + this.timerMetric = new TimerMetric("timer-big-function"); + } + + @Test + public void testGetMetricValueSuccess() throws InterruptedException { + Thread.sleep(SLEEP_TIME); + this.timerMetric.stop(); + long diff = this.timerMetric.getMetricValue() - SLEEP_TIME; + assertTrue(diff < THRESHOLD); + } + + @Test(expected=JobRuntimeException.class) + public void testGetMetricValueFail() { + timerMetric.getMetricValue(); + } + + @Test + public void testAddTag() { + this.timerMetric.addTag("job", "test-job"); + assertTrue(this.timerMetric.getTags().containsKey("job")); + assertTrue(this.timerMetric.getTags().containsKey("metric-type")); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/retry/TestRetryableFunction.java b/marmaray/src/test/java/com/uber/marmaray/common/retry/TestRetryableFunction.java new file mode 100644 index 0000000..be40679 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/retry/TestRetryableFunction.java @@ -0,0 +1,100 @@ +/* + * 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.retry; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.RetryStrategyConfiguration; +import com.uber.marmaray.common.configuration.SimpleRetryStrategyConfiguration; + +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.File; +import lombok.NonNull; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; + + +class MockRetryableFunction extends RetryableFunction { + + private int count = 0; + + public MockRetryableFunction(IFunctionThrowsException func, + IRetryStrategy retryStrategy) { + super(func,spy(retryStrategy)); + when(super.retryStrategy.shouldRetry()).thenCallRealMethod(); + when(super.retryStrategy.retryMessage()).thenCallRealMethod(); + } + + @Override + protected IFunctionThrowsException getUserFunction() { + this.count++; + return super.getUserFunction(); + } + + public void verifyRetry(int numFuncApply, int numShouldRetry) { + Assert.assertEquals(count, numFuncApply); + Mockito.verify(this.retryStrategy, times(numShouldRetry)).shouldRetry(); + } +} + +public class TestRetryableFunction { + private final static String CONFIG_YAML = "src/test/resources/config.yaml"; + private final static Configuration conf = new Configuration(new File(CONFIG_YAML), Optional.absent()); + private final static int maxRetries = new SimpleRetryStrategyConfiguration(conf).getNumRetries(); + + private static int counter = 0; + private Integer divide(@NonNull final Integer divisor) { + counter++; + if (divisor == 0) { + throw new IllegalArgumentException("The divisor is 0."); + } else { + if (counter < maxRetries) + throw new RuntimeException("Failing for all attempts except last one"); + return (100 / divisor); + } + } + + @Test + public void testGoodRetryableFunction() throws Exception { + counter = 0; + final MockRetryableFunction divideBy = new MockRetryableFunction<>( + this::divide, + new RetryStrategyConfiguration(conf).getRetryStrategy()); + assertEquals(new Integer(50), divideBy.apply(2)); + divideBy.verifyRetry(maxRetries, maxRetries-1); + } + + @Test(expected = IllegalArgumentException.class) + public void testBadRetryableTask() throws Exception { + counter = 0; + final MockRetryableFunction divideBy = new MockRetryableFunction<>( + this::divide, + new RetryStrategyConfiguration(conf).getRetryStrategy()); + try { + divideBy.apply(0); + } + finally { + divideBy.verifyRetry(maxRetries, maxRetries); + } + } +} 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 new file mode 100644 index 0000000..0e106fa --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestCassandraSinkSchemaManager.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.schema.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.uber.marmaray.common.util.CassandraTestUtil; +import com.uber.marmaray.common.util.SchemaTestUtil; +import com.uber.marmaray.utilities.StringTypes; +import org.apache.avro.Schema; +import org.apache.thrift.transport.TTransportException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static com.uber.marmaray.common.util.CassandraTestConstants.KEY_SPACE; +import static com.uber.marmaray.common.util.CassandraTestConstants.TABLE; + + +public class TestCassandraSinkSchemaManager { + private static final String getColumnsQuery = String.format("SELECT column_name FROM system_schema.columns WHERE " + + "keyspace_name = '%s' AND table_name = '%s'", KEY_SPACE, TABLE); + + private static final List fields = Collections.unmodifiableList(Arrays.asList( + new CassandraSchemaField("country_code", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))), + new CassandraSchemaField("state_province", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))), + new CassandraSchemaField("city", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))), + new CassandraSchemaField("capacity", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.INT_TYPE))), + new CassandraSchemaField("gym_name", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))))); + + private static final List newFields = Collections.unmodifiableList(Arrays.asList( + new CassandraSchemaField("new_field1", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))), + new CassandraSchemaField("new_field2", CassandraSchemaField.convertFromAvroType( + SchemaTestUtil.getSchema(CassandraSchemaField.STRING_TYPE))))); + + @Before + public void setupTest() throws TTransportException, IOException { + CassandraTestUtil.setupCluster(); + } + + @After + public void teardownTest() { + CassandraTestUtil.teardownCluster(); + } + + @Test + public void testCreateTableWithSinglePrimaryKeyAndNoClusteringKey() { + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); + final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( + schema, + Collections.singletonList("country_code"), + Collections.EMPTY_LIST); + 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))) "; + Assert.assertEquals(expected, createTableStmt); + + try (final Session session = getSession()) { + session.execute(createTableStmt); + validateCreateTable(session); + } + } + + @Test + public void testCreateTableWithSinglePrimaryKeyAndOneClusteringKeyDesc() { + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); + final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( + schema, + Collections.singletonList("country_code"), + Collections.singletonList(new ClusterKey("state_province", ClusterKey.Order.DESC))); + 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))" + + " WITH CLUSTERING ORDER BY (state_province DESC)"; + Assert.assertEquals(expected, createTableStmt); + + try (final Session session = getSession()) { + session.execute(createTableStmt); + validateCreateTable(session); + } + } + + @Test + public void testCreateTableWithMultiplePrimaryKeys() { + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); + final CassandraSinkSchemaManager schemaManager = new CassandraSinkSchemaManager( + schema, + Arrays.asList("country_code", "state_province"), + Collections.EMPTY_LIST); + 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))) "; + Assert.assertEquals(expected, createTableStmt); + + try (final Session session = getSession()) { + session.execute(createTableStmt); + validateCreateTable(session); + } + } + + @Test + public void testCreateTableWithMultiplePrimaryKeysAndClusteringKeys() { + 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("city", ClusterKey.Order.DESC), + new ClusterKey("gym_name", ClusterKey.Order.ASC))); + 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)" + + ",city,gym_name)) WITH CLUSTERING ORDER BY (city DESC,gym_name ASC)"; + Assert.assertEquals(expected, createTableStmt); + + try (final Session session = getSession()) { + session.execute(createTableStmt); + validateCreateTable(session); + } + } + + @Test + public void testAlterTableWithOneNewColumn() { + createBasicTable(); + + final List joinedList = Lists.newArrayList(Iterables.concat(fields, + Collections.singletonList(new CassandraSchemaField("new_field1", "text")))); + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, joinedList); + 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))); + + final List alterTableQueries = + schemaManager.generateAlterTableStmt(fields.stream().map(f -> f.getFieldName()).collect(Collectors.toList())); + + Assert.assertTrue(alterTableQueries.size() == 1); + Assert.assertEquals("ALTER TABLE marmaray.crossfit_gyms ADD new_field1 text", alterTableQueries.get(0)); + try (final Session session = getSession()) { + alterTableQueries.stream().forEach(query -> session.execute(query)); + validateAlterTable(session, Collections.singletonList("new_field1")); + } + } + + @Test + public void testAlterTableWithOneMultipleColumns() { + createBasicTable(); + + final List joinedList = Lists.newArrayList(Iterables.concat(fields, newFields)); + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, joinedList); + 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))); + + final List alterTableQueries = + schemaManager.generateAlterTableStmt(fields.stream().map(f -> f.getFieldName()).collect(Collectors.toList())); + + Assert.assertTrue(alterTableQueries.size() == 2); + Assert.assertEquals("ALTER TABLE marmaray.crossfit_gyms ADD new_field1 text", alterTableQueries.get(0)); + Assert.assertEquals("ALTER TABLE marmaray.crossfit_gyms ADD new_field2 text", alterTableQueries.get(1)); + try (final Session session = getSession()) { + alterTableQueries.stream().forEach(query -> session.execute(query)); + validateAlterTable(session, newFields.stream().map(f -> f.getFieldName()).collect(Collectors.toList())); + } + } + + @Test + public void testGenerateColumnNameQueries() { + 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("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 " + + "keyspace_name = 'marmaray' AND columnfamily_name = 'crossfit_gyms'", cfQuery); + Assert.assertEquals("SELECT column_name FROM system_schema.columns WHERE " + + "keyspace_name = 'marmaray' AND table_name = 'crossfit_gyms'", tableQuery); + } + + @Test + public void testInsertStatement() { + final CassandraSchema schema = new CassandraSchema(KEY_SPACE, TABLE, fields); + + final CassandraSinkSchemaManager schemaManagerNoTTL = new CassandraSinkSchemaManager( + schema, + Arrays.asList("country_code", "state_province"), + Collections.EMPTY_LIST, + Optional.of(10000L)); + 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); + + final CassandraSinkSchemaManager schemaManagerWithTTL = new CassandraSinkSchemaManager( + schema, + Arrays.asList("country_code", "state_province"), + Collections.EMPTY_LIST, + Optional.absent()); + final String insertStmtWithTTL = schemaManagerWithTTL.generateInsertStmt(); + Assert.assertEquals(expected.replace("USING TTL 10000", StringTypes.EMPTY), insertStmtWithTTL); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + 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(); + } + + @Test(expected = IllegalStateException.class) + public void tesClusterKeyIsNotInFieldNames() { + 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(); + } + + @Test(expected = IllegalStateException.class) + 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); + Assert.fail(); + } + + private void validateCreateTable(final Session session) { + final ResultSet results = session.execute(getColumnsQuery); + + final List columns = results.all() + .stream() + .map(r -> r.getString("column_name")) + .collect(Collectors.toList()); + + Assert.assertTrue(columns.containsAll( + Arrays.asList("country_code", "state_province", "city", "capacity", "gym_name"))); + } + + private void validateAlterTable(final Session session, List newColumns) { + final ResultSet results = session.execute(getColumnsQuery); + + final List columns = results.all() + .stream() + .map(r -> r.getString("column_name")) + .collect(Collectors.toList()); + + Assert.assertTrue(columns.containsAll(newColumns)); + } + + /** + * Creates a basic table in Cassandra that can be used for more testing (i.e altering) + */ + private void createBasicTable() { + 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("city", ClusterKey.Order.DESC), + new ClusterKey("gym_name", ClusterKey.Order.ASC))); + final String createTableStmt = schemaManager.generateCreateTableStmt(); + + try (final Session session = getSession()) { + session.execute(createTableStmt); + } + } + + private Session getSession() { + final Cluster cluster = CassandraTestUtil.initCluster(); + final Session session = cluster.connect(); + return session; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestClusterKey.java b/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestClusterKey.java new file mode 100644 index 0000000..5366c66 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/schema/cassandra/TestClusterKey.java @@ -0,0 +1,47 @@ +/* + * 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 org.junit.Assert; +import org.junit.Test; + +public class TestClusterKey { + private final static String KEY = "testKey"; + @Test + public void testParseStringWithNoExplicitOrdering() { + final ClusterKey key = ClusterKey.parse(KEY); + Assert.assertEquals(KEY, key.getName()); + Assert.assertEquals(ClusterKey.Order.ASC, key.getOrder()); + Assert.assertEquals("testKey ASC", key.toString()); + } + + @Test + public void testParseStringWithDescOrder() { + final ClusterKey key = ClusterKey.parse(KEY + ":" + ClusterKey.Order.DESC); + Assert.assertEquals(KEY, key.getName()); + Assert.assertEquals(ClusterKey.Order.DESC, key.getOrder()); + Assert.assertEquals("testKey DESC", key.toString()); + } + + @Test + public void testParseStringWithAscOrder() { + final ClusterKey key = ClusterKey.parse(KEY + ":" + ClusterKey.Order.ASC); + Assert.assertEquals(KEY, key.getName()); + Assert.assertEquals(ClusterKey.Order.ASC, key.getOrder()); + Assert.assertEquals("testKey ASC", key.toString()); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/sinks/TestSinkStatManager.java b/marmaray/src/test/java/com/uber/marmaray/common/sinks/TestSinkStatManager.java new file mode 100644 index 0000000..a02019f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/TestSinkStatManager.java @@ -0,0 +1,83 @@ +/* + * 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; + +import com.uber.marmaray.common.metadata.IMetadataManager; +import com.uber.marmaray.common.metadata.MemoryMetadataManager; +import com.uber.marmaray.common.sinks.SinkStatManager.SinkStat; +import lombok.extern.slf4j.Slf4j; +import org.junit.Assert; +import org.junit.Test; + +@Slf4j +public class TestSinkStatManager { + + @Test + public void testSerDser() { + final String tableName = "testTable"; + final IMetadataManager metadataManager = new MemoryMetadataManager(); + final SinkStatManager sinkStatManager1 = new SinkStatManager(tableName, metadataManager); + + // Initially nothing will be found; it should not crash. + sinkStatManager1.init(); + Assert.assertEquals(0, sinkStatManager1.getAvgRecordSize()); + Assert.assertFalse(sinkStatManager1.isStatHistoryAvailable()); + + final int avgRecordSize1 = 30; + sinkStatManager1.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Integer.toString(avgRecordSize1)); + // nothing is saved to metadata manager before persist. + Assert.assertEquals(0, metadataManager.getAllKeys().size()); + sinkStatManager1.persist(); + Assert.assertEquals(1, metadataManager.getAllKeys().size()); + + final SinkStatManager sinkStatManager2 = new SinkStatManager(tableName, metadataManager); + sinkStatManager2.init(); + Assert.assertEquals(avgRecordSize1, sinkStatManager2.getAvgRecordSize()); + final int avgRecordSize2 = 20; + sinkStatManager2.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Integer.toString(avgRecordSize2)); + + sinkStatManager2.persist(); + + final SinkStatManager sinkStatManager3 = new SinkStatManager(tableName, metadataManager); + sinkStatManager3.init(); + Assert.assertEquals((avgRecordSize1 + avgRecordSize2) / 2, sinkStatManager3.getAvgRecordSize()); + } + + @Test + public void testMaxStatHistory() { + final String tableName = "testTable"; + final IMetadataManager metadataManager = new MemoryMetadataManager(); + final SinkStatManager sinkStatManager1 = new SinkStatManager(tableName, metadataManager); + sinkStatManager1.init(); + final int initialValue = SinkStatManager.MAX_HISTORY_SIZE * 2; + sinkStatManager1.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Integer.toString(initialValue)); + sinkStatManager1.persist(); + final int targetValue = 1; + for (int i = 0; i < SinkStatManager.MAX_HISTORY_SIZE; i++) { + final SinkStatManager tempSinkStatManager = new SinkStatManager(tableName, metadataManager); + tempSinkStatManager.init(); + tempSinkStatManager.getCurrentStat().put(SinkStat.AVG_RECORD_SIZE, Integer.toString(targetValue)); + Assert.assertNotEquals(targetValue, tempSinkStatManager.getAvgRecordSize()); + tempSinkStatManager.persist(); + } + + // After SinkStatManager.MAX_HISTORY_SIZE runs very first stat should get dropped. + final SinkStatManager sinkStatManager2 = new SinkStatManager(tableName, metadataManager); + sinkStatManager2.init(); + Assert.assertEquals(targetValue, sinkStatManager2.getAvgRecordSize()); + } +} 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 new file mode 100644 index 0000000..272abb2 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraClientSink.java @@ -0,0 +1,230 @@ +/* + * 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.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; +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.Test; + +@Slf4j +public class TestCassandraClientSink extends TestCassandraSinkUtil { + + private static final String TEST_TIMESTAMP = "10000"; + + @Before + public void setupTest() { + super.setupTest(); + try { + CassandraTestUtil.setupCluster(); + } catch (TTransportException | IOException e) { + throw new JobRuntimeException("Error while initializing Cassandra cluster for test: " + e.getMessage()); + } + } + + @After + public void teardownTest() { + super.teardownTest(); + CassandraTestUtil.teardownCluster(); + } + + @Test + public void testWriteAllFieldsMockDataToCassandraWithoutTimestamp() { + testWriteAllFieldsMockDataToCassandra(false); + } + + @Test + public void testWriteAllFieldsMockDataToCassandraWithTimestamp() { + testWriteAllFieldsMockDataToCassandra(true); + } + + /** + * We exclude the boolean field from the input schema and don't write it to Cassandra but do add a timestamp + */ + @Test + public void testWriteOnlySpecifiedFieldsMockDataToCassandraWithTimestamp() { + testWriteOnlySpecifiedFieldsMockDataToCassandra(true); + } + + /** + * We exclude the boolean field from the input schema and don't write it to Cassandra and do NOT add a timestamp + */ + @Test + public void testWriteOnlySpecifiedFieldsMockDataToCassandraWithoutTimestamp() { + testWriteOnlySpecifiedFieldsMockDataToCassandra(false); + } + + /* + * In this test, each Avro Payload is missing one of the partition keys in the Cassandra Schema + * The Avro Schema declares the field which is the partition key but does not have any data for it. + * Everything should be forked as an error record so no data is written to Cassandra and the job fails + */ + @Test + public void testWriteMockErrorDataToCassandra() { + final boolean includeTimestamp = false; + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + CassandraTestConstants.INT_FIELD); + + final List partitionKeys = Collections.singletonList(CassandraTestConstants.INT_FIELD); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(CassandraTestConstants.STRING_FIELD, ClusterKey.Order.DESC)); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(); + final List requiredFields = Arrays.asList(CassandraTestConstants.INT_FIELD, + CassandraTestConstants.STRING_FIELD); + + // 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 CassandraSinkCQLDataConverter converter = + new CassandraSinkCQLDataConverter(avroSchema, + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, Optional.absent()); + final CassandraSchema cassandraSchema = schemaConverter.convertToExternalSchema(avroSchema); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys); + + final CassandraSinkConfiguration conf = initializeConfiguration(false, includeTimestamp); + final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); + 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()); + } + } + + private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + StringTypes.EMPTY); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(); + + final List partitionKeys = Collections.singletonList(schemaFields.get(0)); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(schemaFields.get(1), ClusterKey.Order.DESC)); + + 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 CassandraSinkCQLDataConverter converter = + new CassandraSinkCQLDataConverter(AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY), + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + tsInfo, new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, tsInfo, Optional.absent()); + final CassandraSchema schema = schemaConverter.convertToExternalSchema( + AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY)); + + final Optional ttl = Optional.of(10000L); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl); + + final CassandraSinkConfiguration conf = initializeConfiguration(false, addLongTimestamp); + final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); + sink.write(testData); + validateCassandraTable(100, false, addLongTimestamp); + } + + private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addStringTimestamp) { + + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + StringTypes.EMPTY); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(CassandraTestConstants.BOOLEAN_FIELD); + + final List partitionKeys = Collections.singletonList(schemaFields.get(0)); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(schemaFields.get(1), ClusterKey.Order.DESC)); + + final List requiredFields = Arrays.asList(schemaFields.get(0), schemaFields.get(1)); + 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 CassandraSinkCQLDataConverter converter = + new CassandraSinkCQLDataConverter( + avroSchema, + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + tsInfo, new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, tsInfo, Optional.absent()); + final CassandraSchema schema = schemaConverter.convertToExternalSchema(avroSchema); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys); + + final CassandraSinkConfiguration conf = initializeConfiguration(true, addStringTimestamp); + final CassandraClientSink sink = new CassandraClientSink(converter, schemaManager, conf); + sink.write(testData); + validateCassandraTable(100, true, addStringTimestamp); + } +} 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 new file mode 100644 index 0000000..0cbf5fe --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSSTableSink.java @@ -0,0 +1,227 @@ +/* + * 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.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.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 org.apache.avro.Schema; +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.Test; + +public class TestCassandraSSTableSink extends TestCassandraSinkUtil { + + private static final String TEST_TIMESTAMP = "10000"; + + @Before + public void setupTest() { + super.setupTest(); + try { + CassandraTestUtil.setupCluster(); + } catch (TTransportException | IOException e) { + throw new JobRuntimeException("Error while initializing Cassandra cluster for test: " + e.getMessage()); + } + } + + @After + public void teardownTest() { + super.teardownTest(); + CassandraTestUtil.teardownCluster(); + } + + @Test + public void testWriteAllFieldsMockDataToCassandraWithoutTimestamp() { + testWriteAllFieldsMockDataToCassandra(false); + } + + @Test + public void testWriteAllFieldsMockDataToCassandraWithTimestamp() { + testWriteAllFieldsMockDataToCassandra(true); + } + + /** + * We exclude the boolean field from the input schema and don't write it to Cassandra but do add a timestamp + */ + @Test + public void testWriteOnlySpecifiedFieldsMockDataToCassandraWithTimestamp() { + testWriteOnlySpecifiedFieldsMockDataToCassandra(true); + } + + /** + * We exclude the boolean field from the input schema and don't write it to Cassandra and do NOT add a timestamp + */ + @Test + public void testWriteOnlySpecifiedFieldsMockDataToCassandraWithoutTimestamp() { + testWriteOnlySpecifiedFieldsMockDataToCassandra(false); + } + + /* + * In this test, each Avro Payload is missing one of the partition keys in the Cassandra Schema + * The Avro Schema declares the field which is the partition key but does not have any data for it. + * Everything should be forked as an error record so no data is written to Cassandra and the job fails + */ + @Test + public void testWriteMockErrorDataToCassandra() { + final boolean includeTimestamp = false; + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + CassandraTestConstants.INT_FIELD); + + final List partitionKeys = Collections.singletonList(CassandraTestConstants.INT_FIELD); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(CassandraTestConstants.STRING_FIELD, ClusterKey.Order.DESC)); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(); + final List requiredFields = Arrays.asList(CassandraTestConstants.INT_FIELD, + CassandraTestConstants.STRING_FIELD); + + // 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 CassandraSinkDataConverter converter = + new CassandraSinkDataConverter(avroSchema, + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + TimestampInfo.generateEmptyTimestampInfo(), new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, Optional.absent()); + final CassandraSchema cassandraSchema = schemaConverter.convertToExternalSchema(avroSchema); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(cassandraSchema, partitionKeys, clusteringKeys); + final CassandraSinkConfiguration conf = initializeConfiguration(false, includeTimestamp); + + final CassandraSSTableSink sink = new CassandraSSTableSink(converter, schemaManager, conf); + 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()); + } + + } + + private void testWriteAllFieldsMockDataToCassandra(boolean addLongTimestamp) { + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + StringTypes.EMPTY); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(); + + final List partitionKeys = Collections.singletonList(schemaFields.get(0)); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(schemaFields.get(1), ClusterKey.Order.DESC)); + + 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 CassandraSinkDataConverter dataconverter = + new CassandraSinkDataConverter(AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY), + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + tsInfo, new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, tsInfo, Optional.absent()); + final CassandraSchema schema = schemaConverter.convertToExternalSchema( + AvroPayloadUtil.getAvroTestDataSchema(StringTypes.EMPTY)); + + final Optional ttl = Optional.of(10000L); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys, ttl); + final CassandraSinkConfiguration conf = initializeConfiguration(false, addLongTimestamp); + + final CassandraSSTableSink sink = new CassandraSSTableSink(dataconverter, schemaManager, conf); + sink.write(testData); + validateCassandraTable(100, false, addLongTimestamp); + } + + private void testWriteOnlySpecifiedFieldsMockDataToCassandra(final boolean addStringTimestamp) { + + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + 100, + StringTypes.EMPTY); + + final List schemaFields = AvroPayloadUtil.getSchemaFields(CassandraTestConstants.BOOLEAN_FIELD); + + final List partitionKeys = Collections.singletonList(schemaFields.get(0)); + final List clusteringKeys = Collections.singletonList( + new ClusterKey(schemaFields.get(1), ClusterKey.Order.DESC)); + + final List requiredFields = Arrays.asList(schemaFields.get(0), schemaFields.get(1)); + 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 CassandraSinkDataConverter converter = + new CassandraSinkDataConverter( + avroSchema, + new Configuration(), + Optional.of(new HashSet<>(schemaFields)), + requiredFields, + tsInfo, new ErrorExtractor()); + + final CassandraSchemaConverter schemaConverter = new CassandraSchemaConverter(KEY_SPACE, TABLE, tsInfo, Optional.absent()); + final CassandraSchema schema = schemaConverter.convertToExternalSchema(avroSchema); + + final CassandraSinkSchemaManager schemaManager = + new CassandraSinkSchemaManager(schema, partitionKeys, clusteringKeys); + final CassandraSinkConfiguration conf = initializeConfiguration(true, addStringTimestamp); + + final CassandraSSTableSink sink = new CassandraSSTableSink(converter, schemaManager, conf); + sink.write(testData); + validateCassandraTable(100, true, addStringTimestamp); + } +} 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 new file mode 100644 index 0000000..10730eb --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/cassandra/TestCassandraSinkUtil.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.sinks.cassandra; + +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.CassandraSinkDataConverter; +import com.uber.marmaray.common.converters.schema.CassandraSchemaConverter; +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.SchemaUtil; +import com.uber.marmaray.utilities.StringTypes; +import com.uber.marmaray.utilities.TimestampInfo; +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; + +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; + +public class TestCassandraSinkUtil extends AbstractSparkTest { + + protected static final String TEST_TIMESTAMP = "10000"; + + protected CassandraSinkConfiguration initializeConfiguration(boolean excludeField, + boolean hasTimestampField) { + final Configuration conf = new Configuration(); + conf.setProperty(CassandraSinkConfiguration.DATACENTER, "test_dc"); + conf.setProperty(CassandraSinkConfiguration.DISABLE_QUERY_UNS, "true"); + conf.setProperty(CassandraSinkConfiguration.INITIAL_HOSTS, "localhost"); + 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"); + + // we always exclude the boolean field for now, can modify this in future to exclude a specific field + if (!excludeField) { + conf.setProperty(CassandraSinkConfiguration.COLUMN_LIST, + Joiner.on(",").join(AvroPayloadUtil.getSchemaFields())); + } else { + conf.setProperty(CassandraSinkConfiguration.COLUMN_LIST, + Joiner.on(",").join(AvroPayloadUtil.getSchemaFields(CassandraTestConstants.BOOLEAN_FIELD))); + } + + conf.setProperty(CassandraSinkConfiguration.NATIVE_TRANSPORT_PORT, "9142"); + return new CassandraSinkConfiguration(conf); + } + + protected void validateCassandraTable(final int expectedNumRows, + boolean excludeBoolField, + final boolean checkTimestampField) { + final Cluster cluster = CassandraTestUtil.initCluster(); + try (final Session session = cluster.connect()) { + final String query = "SELECT * FROM " + CassandraTestConstants.KEY_SPACE + ". " + CassandraTestConstants.TABLE; + final List rows = session.execute(query).all(); + Assert.assertEquals(expectedNumRows, rows.size()); + + for (int i = 0; i < expectedNumRows; i++) { + final Row row = rows.get(i); + final Integer intObj = (Integer) row.getObject(CassandraTestConstants.INT_FIELD); + final String stringObj = (String) row.getObject(CassandraTestConstants.STRING_FIELD); + Assert.assertEquals(Integer.valueOf(stringObj), intObj); + + if (!excludeBoolField) { + final Boolean boolObj = (Boolean) row.getObject(CassandraTestConstants.BOOLEAN_FIELD); + Assert.assertTrue(boolObj); + } + + if (checkTimestampField) { + final Object timestampObj = row.getObject(SchemaUtil.DISPERSAL_TIMESTAMP); + + if (timestampObj instanceof Long) { + Assert.assertEquals(Long.parseLong(TEST_TIMESTAMP), timestampObj); + } else { + Assert.assertEquals(TEST_TIMESTAMP, timestampObj.toString()); + } + } + } + } + } +} 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 new file mode 100644 index 0000000..4d7bd32 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/FileSinkTestUtil.java @@ -0,0 +1,94 @@ +/* + * 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.file; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.common.util.AbstractSparkTest; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.junit.Before; +import java.nio.file.FileSystemNotFoundException; + +@Slf4j +public class FileSinkTestUtil extends AbstractSparkTest { + protected String pathPrefix; + + @Before + public void setupTest() { + super.setupTest(); + try { + if (this.fileSystem.isPresent()) { + this.pathPrefix = this.fileSystem.get().getWorkingDirectory().toString(); + } + else { + throw new FileSystemNotFoundException("File System not found."); + } + + } catch (FileSystemNotFoundException e) { + log.error("Exception: {}", e.getMessage()); + throw new JobRuntimeException(e); + } + } + + protected Configuration initConfig(@NonNull final String pathPrefix, @NonNull final String fsPath, + @NonNull final String separator, @NonNull final String timeStamp, + @NonNull final String sourceSubPath, @NonNull final String dispersalType) { + final Configuration conf=new Configuration(); + conf.setProperty(FileSinkConfiguration.FS_PATH,fsPath); + conf.setProperty(FileSinkConfiguration.FILE_TYPE,"csv"); + conf.setProperty(FileSinkConfiguration.FILE_SIZE_MEGABYTE,"0.001"); + conf.setProperty(FileSinkConfiguration.SEPARATOR,separator); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, pathPrefix); + conf.setProperty(FileSinkConfiguration.SOURCE_TYPE, "hive"); + conf.setProperty(FileSinkConfiguration.TIMESTAMP, timeStamp); + conf.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test_trip.table"); + conf.setProperty(FileSinkConfiguration.DISPERSAL_TYPE, dispersalType); + conf.setProperty(FileSinkConfiguration.SOURCE_PARTITION_PATH, sourceSubPath); + conf.setProperty(FileSinkConfiguration.PARTITION_TYPE, "date"); + return conf; + } + + 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) { + final Configuration conf=new Configuration(); + conf.setProperty(FileSinkConfiguration.FILE_TYPE,"csv"); + conf.setProperty(FileSinkConfiguration.FILE_SIZE_MEGABYTE,"0.001"); + conf.setProperty(FileSinkConfiguration.SEPARATOR,","); + conf.setProperty(FileSinkConfiguration.PATH_PREFIX, pathPrefix); + conf.setProperty(FileSinkConfiguration.SOURCE_TYPE, "hive"); + conf.setProperty(FileSinkConfiguration.TIMESTAMP, timeStamp); + conf.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test_trip.table"); + conf.setProperty(FileSinkConfiguration.DISPERSAL_TYPE, dispersalType); + conf.setProperty(FileSinkConfiguration.SOURCE_PARTITION_PATH, sourceSubPath); + conf.setProperty(FileSinkConfiguration.PARTITION_TYPE, "date"); + //Aws Property + conf.setProperty(FileSinkConfiguration.FILE_SINK_TYPE, "S3"); + conf.setProperty(FileSinkConfiguration.AWS_LOCAL, awsLocal); + conf.setProperty(FileSinkConfiguration.BUCKET_NAME, bucketName); + conf.setProperty(FileSinkConfiguration.OBJECT_KEY, "marmaray_test/" + objectKey); + conf.setProperty(FileSinkConfiguration.AWS_REGION, "us-east-1"); + conf.setProperty(FileSinkConfiguration.AWS_ACCESS_KEY_ID, "username"); + conf.setProperty(FileSinkConfiguration.AWS_SECRET_ACCESS_KEY, "password"); + return conf; + } +} 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 new file mode 100644 index 0000000..9506b1c --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestAwsFileSink.java @@ -0,0 +1,267 @@ +/* + * 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.file; + +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.AnonymousAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.AwsConfiguration; +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.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; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Matchers; +import static org.mockito.Mockito.*; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.*; + +class MockAwsFileSink extends AwsFileSink { + private static final String S3_TEST_URL = "http://localhost:8181"; + private static final String S3_TEST_BUCKET_NAME = "aws-test"; + private static final String S3_TEST_REGION = "us-east-1"; + private EndpointConfiguration endpoint; + + public MockAwsFileSink(@NonNull final FileSinkConfiguration conf, + @NonNull final FileSinkDataConverter converter) throws IOException { + super(conf, converter); + } + + @Override + protected AmazonS3 getS3Connection() { + this.endpoint = new EndpointConfiguration(S3_TEST_URL, S3_TEST_REGION); + AmazonS3 s3ClientTest = AmazonS3ClientBuilder + .standard() + .withPathStyleAccessEnabled(true) + .withEndpointConfiguration(endpoint) + .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials())) + .build(); + s3ClientTest.createBucket(S3_TEST_BUCKET_NAME); + return spy(s3ClientTest); + } +} + +@Slf4j +public class TestAwsFileSink extends FileSinkTestUtil { + private static final String S3_TEST_URL = "http://localhost:8181"; + private static final String S3_TEST_REGION = "us-east-1"; + private static final String S3_TEST_BUCKET_NAME = "aws-test"; + private static final String S3_BUCKET_NAME = "uber-test"; + private static final String LOCAL1 = "/aws_test/test1"; + private static final String LOCAL2 = "/aws_test/test2"; + private static final String LOCAL3 = "/aws_test/test3"; + private static final String OBJ_KEY_1 = "test1"; + 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_INVOCATIONS = 1; + private static final String TIMESTAMP1 = "201808011025"; + private static final String TIMESTAMP2 = "201808012025"; + private static final String TIMESTAMP3 = "201808022025"; + private static final String SOURCE_SUB_PATH1 = "2018/08/01"; + private static final String SOURCE_SUB_PATH2 = "2018/08/02"; + private static final String VERSION = "version"; + private static final String OVERWRITE = "overwrite"; + private static final String S3_TEST_FOLDER = "s3-tests"; + private S3Mock s3mock; + private String port; + @Rule + public TemporaryFolder s3mockRoot = new TemporaryFolder(); + + @Before + public void setupTest() { + super.setupTest(); + try { + port = S3_TEST_URL.substring(S3_TEST_URL.lastIndexOf(":") + 1); + File s3mockDir = s3mockRoot.newFolder(S3_TEST_FOLDER); + s3mock = S3Mock.create(Integer.parseInt(port), s3mockDir.getCanonicalPath()); + s3mock.start(); + } catch (IOException e) { + log.error("Exception: {}", e.getMessage()); + throw new JobRuntimeException(e); + } + } + + @After + public void tearDownTest() throws IOException { + s3mock.stop(); + this.fileSystem.get().delete(new Path(PARENT_DIR), true); + super.teardownTest(); + } + + /**Todo: T1984925- Fix the bug while running mock aws client. + /** + * The following two tests are used for test of uploading to existed aws s3 bucket. + * If you need to test upload to a real aws s3 bucket, remove the ignore to test. + * Also add credential file named credentials at test directory:usually marmaray/marmaray/credentials. It will be needed in + * {@link FileSinkConfiguration} in {@link FileSinkTestUtil#initConfigWithAws} + */ + + /* Test bucket name: uber-test + * Test object key: marmaray_test/test1/SOURCE_PARTITION_SUB_PATH/FILENAME_PREFIX_TIMESTAMP_PARTITION_NUM + */ + @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); + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + NUM_RECORD, + StringTypes.EMPTY); + testWriteGeneral(testData, conf1); + testWriteGeneral(testData, conf2); + testWriteGeneral(testData, conf3); + } + + /* Test bucket name: uber-test + * Test object key: marmaray_test/test2/SOURCE_PARTITION_SUB_PATH/FILENAME_PREFIX_TIMESTAMP_PARTITION_NUM + */ + @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 JavaRDD testData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), + NUM_RECORD, + StringTypes.EMPTY); + testWriteGeneral(testData, conf1); + testWriteGeneral(testData, conf2); + testWriteGeneral(testData, conf3); + } + + //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); + } + @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); + } + @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 List filePrefix = new ArrayList<>(); + filePrefix.add(testWriteToMockS3General(conf1)); + filePrefix.add(testWriteToMockS3General(conf2)); + filePrefix.add(testWriteToMockS3General(conf3)); + final AmazonS3 MockClient = getMockS3Connection(); + assertTrue(MockClient.doesBucketExistV2(S3_TEST_BUCKET_NAME)); + for (String aFilePrefix : filePrefix) { + for (int j = 0; j < EXPECTED_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 List filePrefix = new ArrayList(); + filePrefix.add(testWriteToMockS3General(conf1)); + filePrefix.add(testWriteToMockS3General(conf2)); + filePrefix.add(testWriteToMockS3General(conf3)); + 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++) { + 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 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)); + assertTrue(MockClient.doesBucketExistV2(fileConf.getBucketName().get())); + for (int i = 0 ; i < EXPECTED_PARTITION_NUM ; i++) { + final Boolean objectExist = MockClient.doesObjectExist(fileConf.getBucketName().get(), awsConf.getS3FilePrefix()+ "_0000" + i); + assertTrue(objectExist); + } + return awsConf.getS3FilePrefix(); + } + + private void testWriteGeneral(@NonNull final JavaRDD testData, @NonNull final Configuration conf) throws IOException { + final FileSinkDataConverter converter = new FileSinkDataConverter(conf, new ErrorExtractor()); + final FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); + final FileSink awsSink = new AwsFileSink(fileConf, converter); + awsSink.write(testData); + } + + private AmazonS3 getMockS3Connection() { + final EndpointConfiguration endpoint = new EndpointConfiguration(S3_TEST_URL, S3_TEST_REGION); + AmazonS3 s3ClientTest = AmazonS3ClientBuilder + .standard() + .withPathStyleAccessEnabled(true) + .withEndpointConfiguration(endpoint) + .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials())) + .build(); + return s3ClientTest; + } +} 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 new file mode 100644 index 0000000..da48a99 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestFileSink.java @@ -0,0 +1,116 @@ +/* + * 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.file; + +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.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.JavaRDD; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Matchers; + +import static org.mockito.Mockito.*; + +import java.io.IOException; + +import static org.junit.Assert.*; + +@Slf4j +public class TestFileSink extends FileSinkTestUtil{ + private static final String PARENT_PATH = "/testpath"; + private static final String PATH1 = "/testpath/test1"; + private static final int NUM_RECORD1 = 100; + private static final int NUM_RECORD2 = 2000; + private static final String COMMA_SEPARATOR = ","; + private static final int EXPECTED_PARTITION_NUM = 1; + private static final long EXPECTED_SAMPLE_SIZE = 1084; + private static final String TIMESTAMP1 = "201808022025"; + private static final String SOURCE_SUB_PATH1 = "2018/08/01"; + private static final String VERSION = "version"; + private JavaRDD convertedData1; + private JavaRDD testData1; + private JavaRDD convertedData2; + private JavaRDD testData2; + private Configuration conf; + private FileSinkDataConverter converter; + private FileSink fileSink; + + @Before + public void setupTest() { + super.setupTest(); + this.testData1 = AvroPayloadUtil.generateTestData(this.jsc.get(), + NUM_RECORD1, + StringTypes.EMPTY); + this.testData2 = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), + NUM_RECORD2, + StringTypes.EMPTY); + this.conf = initConfig(pathPrefix, PATH1, COMMA_SEPARATOR, TIMESTAMP1, SOURCE_SUB_PATH1, VERSION); + this.converter = new FileSinkDataConverter(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); + } + + @After + public void tearDownTest() throws IOException { + this.fileSystem.get().delete(new Path(PARENT_PATH), true); + super.teardownTest(); + } + + @Test + public void testGetRepartitionNum() { + final int partitionNum = this.fileSink.getRepartitionNum(this.convertedData1); + assertEquals(EXPECTED_PARTITION_NUM, partitionNum); + } + + @Test + public void testGetRddSizeNoMoreThanSampleRow() { + final double rddSize = fileSink.getRddSizeInMegaByte(convertedData1); + final long sampleSize = fileSink.getSampleSizeInBytes(convertedData1); + final double sampleSizeInMB = (double) sampleSize / FileUtils.ONE_MB; + assertEquals(rddSize, sampleSizeInMB, 0.1); + } + + @Test + public void testGetRddSizeMoreThanSampleRow() { + final double rddSize = fileSink.getRddSizeInMegaByte(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; + /*File size is sampled result, so not accurate"*/ + log.info("Exact file size[MB]: {}", sampleSizeInMB); + log.info("Sample file size[MB]: {}", rddSize); + } + + @Test + public void testGetSampleSizeInBytes() { + final long sampleSize = fileSink.getSampleSizeInBytes(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 new file mode 100644 index 0000000..7018d49 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/file/TestHdfsFileSink.java @@ -0,0 +1,153 @@ +/* + * 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.file; + +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.util.AvroPayloadUtil; +import com.uber.marmaray.utilities.ErrorExtractor; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaRDD; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Mockito.*; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; + +import lombok.extern.slf4j.Slf4j; +import org.mockito.Matchers; +import static org.junit.Assert.*; + +@Slf4j +public class TestHdfsFileSink extends FileSinkTestUtil { + private static final String PATH1 = "/testpath/test1"; + private static final String PATH2 = "/testpath/test2"; + private static final String PATH3 = "/testpath/test3"; + private static final String PATH4 = "/testpath/test4"; + private static final String PARENT_PATH = "testpath"; + private static final String COMMA_SEPARATOR = ","; + private static final String SPECIAL_SEPARATOR = "\021"; + private static final int NUM_RECORD1 = 100; + private static final int NUM_RECORD2 = 200; + private static final int EXPECTED_PARTITION_NUM1 = 1; + private static final int EXPECTED_PARTITION_NUM2 = 2; + private static final String TIMESTAMP1 = "201808011025"; + private static final String TIMESTAMP2 = "201808012025"; + private static final String TIMESTAMP3 = "201808022025"; + private static final String SOURCE_SUB_PATH1 = "2018/08/01"; + private static final String SOURCE_SUB_PATH2 = "2018/08/02"; + private static final String VERSION = "version"; + private static final String OVERWRITE = "overwrite"; + @Before + public void setupTest() { + super.setupTest(); + } + + @After + public void tearDownTest() throws IOException { + this.fileSystem.get().delete(new Path(PARENT_PATH), true); + super.teardownTest(); + } + + @Test + public void testWriteToCsvWithCommaSeparatorWithVersion() throws Exception { + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + NUM_RECORD1, + StringTypes.EMPTY); + testWriteToCsvCommon(this.pathPrefix, PATH1, COMMA_SEPARATOR, testData, EXPECTED_PARTITION_NUM1, TIMESTAMP1, SOURCE_SUB_PATH1, VERSION); + testWriteToCsvCommon(this.pathPrefix, PATH1, COMMA_SEPARATOR, testData, EXPECTED_PARTITION_NUM2, TIMESTAMP2, SOURCE_SUB_PATH1, VERSION); + testWriteToCsvCommon(this.pathPrefix, PATH1, COMMA_SEPARATOR, testData, EXPECTED_PARTITION_NUM1, TIMESTAMP3, SOURCE_SUB_PATH2, VERSION); + } + + @Test + public void testWriteToCsvWithSpecialCharAndSeparatorWithOverwrite() throws Exception { + final JavaRDD testData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), + NUM_RECORD1, + StringTypes.EMPTY); + testWriteToCsvCommon(this.pathPrefix, PATH2, SPECIAL_SEPARATOR, testData, EXPECTED_PARTITION_NUM2, TIMESTAMP1, SOURCE_SUB_PATH1, VERSION); + testWriteToCsvCommon(this.pathPrefix, PATH2, COMMA_SEPARATOR, testData, EXPECTED_PARTITION_NUM2, TIMESTAMP3, SOURCE_SUB_PATH2, VERSION); + testWriteToCsvCommon(this.pathPrefix, PATH2, COMMA_SEPARATOR, testData, EXPECTED_PARTITION_NUM2, TIMESTAMP2, SOURCE_SUB_PATH1, OVERWRITE); + } + + @Test + public void testWriteToCsvWithOverwriteAtStart() throws Exception { + final JavaRDD testData = AvroPayloadUtil.generateTestData(this.jsc.get(), + NUM_RECORD1, + StringTypes.EMPTY); + testWriteToCsvCommon(this.pathPrefix, PATH3, SPECIAL_SEPARATOR, testData, EXPECTED_PARTITION_NUM1, TIMESTAMP1, SOURCE_SUB_PATH1, OVERWRITE); + } + + @Test + public void testWriteToCsvWithHeader() throws IOException { + final JavaRDD testData = AvroPayloadUtil.generateTestDataNew(this.jsc.get(), + NUM_RECORD2, + 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 FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); + final HdfsFileSink hdfsSink = spy(new HdfsFileSink(fileConf, converter)); + hdfsSink.write(testData); + verify(hdfsSink, times(1)).write(Matchers.any(JavaRDD.class)); + verify(hdfsSink, times(1)).addColumnHeader(Matchers.anyString(), Matchers.any(JavaRDD.class)); + final FileStatus[] status = this.fileSystem.get().listStatus(new Path(fileConf.getPathHdfs())); + for (final FileStatus fileStatus : status) { + if (fileStatus.isFile()) { + Path path = fileStatus.getPath(); + FSDataInputStream in = this.fileSystem.get().open(path); + BufferedReader d = new BufferedReader(new InputStreamReader(in)); + String header = d.readLine(); + Assert.assertEquals("int_field,string_field,boolean_field", header); + in.close(); + d.close(); + } + } + } + + private void testWriteToCsvCommon(@NonNull final String pathPrefix, @NonNull final String path, + @NonNull final String separator, @NonNull final JavaRDD testData, + @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 FileSinkConfiguration fileConf = new FileSinkConfiguration(conf); + final HdfsFileSink hdfsSink = spy(new HdfsFileSink(fileConf, converter)); + hdfsSink.write(testData); + verify(hdfsSink, times(1)).write(Matchers.any(JavaRDD.class)); + verify(hdfsSink, times(1)).getRepartitionNum(Matchers.any(JavaRDD.class)); + verify(hdfsSink, times(1)).getRddSizeInMegaByte(Matchers.any(JavaRDD.class)); + final FileStatus[] status = this.fileSystem.get().listStatus(new Path(fileConf.getPathHdfs())); + int fileNum = 0; + for (final FileStatus fileStatus : status) { + if (fileStatus.isFile()) { + fileNum++; + } + } + assertEquals(partitionNum, fileNum); + } +} 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 new file mode 100644 index 0000000..174cf0d --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sinks/hoodie/TestHoodieSink.java @@ -0,0 +1,464 @@ +/* + * 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; + +import com.google.common.annotations.VisibleForTesting; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.configuration.Configuration; +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.metadata.HoodieBasedMetadataManager; +import com.uber.marmaray.common.metadata.IMetadataManager; +import com.uber.marmaray.common.metadata.MemoryMetadataManager; +import com.uber.marmaray.common.metadata.NoOpMetadataManager; +import com.uber.marmaray.common.metadata.StringValue; +import com.uber.marmaray.common.metrics.DataFeedMetrics; +import com.uber.marmaray.common.metrics.Metric; +import com.uber.marmaray.common.sinks.hoodie.HoodieSink.HoodieWriteClientWrapper; +import com.uber.marmaray.common.util.AbstractSparkTest; +import com.uber.marmaray.common.util.FileTestUtil; +import com.uber.marmaray.utilities.FSUtils; +import lombok.Getter; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang.ArrayUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Matchers; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.uber.marmaray.common.sinks.hoodie.HoodieSink.HoodieSinkOp.BULK_INSERT; +import static com.uber.marmaray.common.sinks.hoodie.HoodieSink.HoodieSinkOp.DEDUP_INSERT; +import static com.uber.marmaray.common.sinks.hoodie.HoodieSink.HoodieSinkOp.INSERT; +import static com.uber.marmaray.common.sinks.hoodie.HoodieSink.HoodieSinkOp.UPSERT; +import static com.uber.marmaray.common.util.SchemaTestUtil.getRandomData; +import static com.uber.marmaray.common.util.SchemaTestUtil.getSchema; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * This is needed to add spy to HoodieWriteClientWrapper. + */ +class MockHoodieSink extends HoodieSink { + + @VisibleForTesting + @Getter + private HoodieWriteClientWrapper mockWriteClient; + + 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()); + } + + 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); + } + + @Override + protected HoodieWriteClientWrapper getHoodieWriteClient( + @NonNull final HoodieWriteConfig hoodieWriteConfig) { + this.mockWriteClient = spy(super.getHoodieWriteClient(hoodieWriteConfig)); + return this.mockWriteClient; + } +} + +@Slf4j +public class TestHoodieSink extends AbstractSparkTest { + + private static final String TS_KEY = "timestamp"; + private static final String RECORD_KEY = "primaryKey"; + private final Configuration conf = new Configuration(); + + // TODO(T933933) need to add tests for Error Records. + + @Test + public void testUpdateInsertParallelism() { + final String basePath = "/basePath"; + final String tableName = "test-table"; + final String schemaStr = getSchema("TS", "RECORD_KEY", 4, 8).toString(); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .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())); + when(mockSink.calculateNewBulkInsertParallelism(anyLong())).thenReturn(18); + Assert.assertTrue(mockSink.updateInsertParallelism(1000)); + Assert.assertEquals(18, hoodieConf.getInsertParallelism()); + Assert.assertEquals(HoodieConfiguration.DEFAULT_HOODIE_PARALLELISM, hoodieConf.getBulkInsertParallelism()); + } + + @Test + public void testUpdateBulkInsertParallelism() { + final String basePath = "/basePath"; + final String tableName = "test-table"; + final String schemaStr = getSchema("TS", "RECORD_KEY", 4, 8).toString(); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .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())); + when(mockSink.calculateNewBulkInsertParallelism(anyLong())).thenReturn(18); + Assert.assertTrue(mockSink.updateBulkInsertParallelism(1000)); + Assert.assertEquals(18, hoodieConf.getBulkInsertParallelism()); + Assert.assertEquals(HoodieConfiguration.DEFAULT_HOODIE_PARALLELISM, hoodieConf.getInsertParallelism()); + } + + @Test + public void testHoodieSinkWriteInsertWithoutMetadata() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), INSERT); + final JavaRDD inputRDD = + this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); + + // 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()); + /* + Expected function calls. + 1) startCommit (once). + 2) upsert (zero times). + 3) insert (once). + 4) commit (once with empty metadata). + 5) close (once). + 6) bulkInsert (zero). + */ + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).startCommit(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .upsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .insert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .commit(Matchers.anyString(), Matchers.any(JavaRDD.class), + Matchers.same(java.util.Optional.empty())); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).close(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .bulkInsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + } + + @Test + public void testHoodieSinkWriteUpsertWithoutMetadata() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(this.conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), UPSERT); + final JavaRDD inputRDD = + this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); + + // 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()); + /* + Expected function calls. + 1) startCommit (once). + 2) upsert (once). + 3) bulkInsert (zero times). + 4) commit (once with empty metadata). + 5) close (once). + 6) insert (zero times). + */ + + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).startCommit(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .upsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .bulkInsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .commit(Matchers.anyString(), Matchers.any(JavaRDD.class), + Matchers.same(java.util.Optional.empty())); + } + + @Test + public void testHoodieSinkWriteInsertWithMetadata() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(this.conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final HoodieBasedMetadataManager hoodieBasedMetadataManager = new HoodieBasedMetadataManager(hoodieConf, + new AtomicBoolean(true), this.jsc.get()); + hoodieBasedMetadataManager.set("randomKey", new StringValue("randomValue")); + final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), INSERT, + hoodieBasedMetadataManager); + final JavaRDD inputRDD = + this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); + + // 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()); + /* + Expected function calls. + 1) startCommit (once). + 2) upsert (zero times). + 3) insert (once). + 4) commit (once with metadata). + 5) close (once). + It should also reset flag for metadata manager. + */ + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).startCommit(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .upsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .insert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .commit(Matchers.anyString(), Matchers.any(JavaRDD.class), + Matchers.eq(java.util.Optional.of(hoodieBasedMetadataManager.getMetadataInfo()))); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).close(); + Assert.assertFalse(hoodieBasedMetadataManager.shouldSaveChanges().get()); + } + + @Test + public void testHoodieSinkWriteUpsertWithMetadata() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(this.conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final HoodieBasedMetadataManager hoodieBasedMetadataManager = new HoodieBasedMetadataManager(hoodieConf, + new AtomicBoolean(true), this.jsc.get()); + 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)); + + // 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()); + /* + Expected function calls. + 1) startCommit (once). + 2) upsert (once). + 3) bulkInsert (zero times). + 4) commit (once with metadata). + 5) close (once). + It should also reset flag for metadata manager. + */ + + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).startCommit(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .upsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .bulkInsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .commit(Matchers.anyString(), Matchers.any(JavaRDD.class), + Matchers.eq(java.util.Optional.of(hoodieBasedMetadataManager.getMetadataInfo()))); + Assert.assertFalse(hoodieBasedMetadataManager.shouldSaveChanges().get()); + } + + @Test + public void testHoodieSinkWriteDedupeInsert() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(this.conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr) + .withCombineBeforeInsert(true).withCombineBeforeUpsert(true).enableMetrics(false).build(); + final JavaRDD inputRDD = + this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); + + final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), DEDUP_INSERT); + // 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()); + /* + Expected function calls. + 1) startCommit (once). + 2) upsert (zero times). + 3) insert (once). + 4) filter exists (once) - needed for dedup. + 4) commit (once with empty metadata). + 5) close (once). + */ + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).startCommit(); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(0)) + .upsert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .insert(Matchers.any(JavaRDD.class), Matchers.anyString()); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .filterExists(Matchers.any(JavaRDD.class)); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)) + .commit(Matchers.anyString(), Matchers.any(JavaRDD.class), + Matchers.same(java.util.Optional.empty())); + Mockito.verify(hoodieWriteClientWrapper, Mockito.times(1)).close(); + + // If we try to re-insert then it should find all the records as a a part filterExists test and should not + // call bulkInsert. + final MockHoodieSink hoodieSink2 = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), DEDUP_INSERT); + hoodieSink.write(inputRDD); + final HoodieWriteClientWrapper hoodieWriteClientWrapper2 = hoodieSink.getMockWriteClient(); + + Assert.assertEquals(2, getCommitFiles(basePath, FSUtils.getFs(new Configuration())).size()); + final ArgumentCaptor rddCaputure = ArgumentCaptor.forClass(JavaRDD.class); + verify(hoodieWriteClientWrapper2).insert(rddCaputure.capture(), Matchers.any()); + // All records should get filtered out. + Assert.assertEquals(0, rddCaputure.getValue().count()); + } + + @Test + public void testHoodieSinkMetrics() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final String brokenSchemaStr = getSchema(TS_KEY, RECORD_KEY, 0, 0).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final MockHoodieSink hoodieSink = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), INSERT); + final Map emptyTags = new HashMap<>(); + final DataFeedMetrics dfm = new DataFeedMetrics(JOB_NAME, emptyTags); + hoodieSink.setDataFeedMetrics(dfm); + final Integer successRecordCount = 10; + final Integer failedRecordCount = 13; + final List data = getRandomData(schemaStr, TS_KEY, RECORD_KEY, successRecordCount); + data.addAll(getRandomData(brokenSchemaStr, TS_KEY, RECORD_KEY, failedRecordCount)); + final JavaRDD inputRDD = + this.jsc.get().parallelize(data); + hoodieSink.write(inputRDD); + + final Set ms = dfm.getMetricSet(); + final Map expected = ArrayUtils.toMap( + new Object[][] { + {"output_rowcount", successRecordCount.longValue()}, + {"error_rowcount", failedRecordCount.longValue()} + }); + Assert.assertEquals(expected.size(), ms.size()); + ms.forEach( metric -> { + final String key = metric.getMetricName(); + Assert.assertEquals("failure for metric " + key, expected.get(key), metric.getMetricValue()); + }); + } + + @Test + public void testUserDefinedCommitTime() throws IOException { + final String basePath = FileTestUtil.getTempFolder(); + final String tableName = "test-table"; + final String schemaStr = getSchema(TS_KEY, RECORD_KEY, 4, 8).toString(); + final HoodieSinkDataConverter hoodieKeyGenerator = + new TSBasedHoodieSinkDataConverter(this.conf, RECORD_KEY, TS_KEY, TimeUnit.MILLISECONDS); + final HoodieConfiguration hoodieConf = + HoodieConfiguration.newBuilder(tableName).withTableName(tableName).withMetricsPrefix("test") + .withBasePath(basePath).withSchema(schemaStr).enableMetrics(false).build(); + final JavaRDD inputRDD = + this.jsc.get().parallelize(getRandomData(schemaStr, TS_KEY, RECORD_KEY, 10)); + + final MockHoodieSink hoodieSink1 = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), BULK_INSERT); + hoodieSink1.write(inputRDD); + + final HoodieWriteClientWrapper hoodieWriteClientWrapper1 = hoodieSink1.getMockWriteClient(); + Mockito.verify(hoodieWriteClientWrapper1, Mockito.times(1)).startCommit(); + + final List commitFilesAfterFirstCommit = getCommitFiles(basePath, FSUtils.getFs(new Configuration())); + Assert.assertEquals(1, commitFilesAfterFirstCommit.size()); + + final String customCommit = + HoodieActiveTimeline.COMMIT_FORMATTER.format( + new Date(new Date().getTime() - TimeUnit.DAYS.toMillis(365))); + + final MockHoodieSink hoodieSink2 = new MockHoodieSink(hoodieConf, hoodieKeyGenerator, jsc.get(), BULK_INSERT); + 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())); + Assert.assertEquals(2, commitFilesAfterSecondCommit.size()); + + final String oldCommitTime = commitFilesAfterFirstCommit.get(0); + commitFilesAfterSecondCommit.removeAll(commitFilesAfterFirstCommit); + Assert.assertEquals(1, commitFilesAfterSecondCommit.size()); + final String newCommitTime = commitFilesAfterSecondCommit.get(0); + Assert.assertEquals(customCommit, newCommitTime.split(".commit")[0]); + } + + private List getCommitFiles(final String basePath, final FileSystem fs) throws IOException { + final List commitFiles = new ArrayList<>(); + final RemoteIterator filesI = fs.listFiles(new Path(basePath), true); + while (filesI.hasNext()) { + final LocatedFileStatus fileStatus = filesI.next(); + if (fileStatus.isFile() && fileStatus.getPath().getName().endsWith("commit")) { + commitFiles.add(fileStatus.getPath().getName()); + } + } + return commitFiles; + } +} 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 new file mode 100644 index 0000000..aab7083 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSource.java @@ -0,0 +1,93 @@ +/* + * 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.sources.hive; + +import com.google.common.collect.Sets; +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.converters.schema.DataFrameSchemaConverter; +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; +import com.uber.marmaray.common.util.FileTestUtil; +import com.uber.marmaray.common.util.HiveTestUtil; +import com.uber.marmaray.utilities.ErrorExtractor; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +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.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +public class TestHiveSource extends AbstractSparkTest { + + private static final String LEFT_FIELD = "left"; + private static final String RIGHT_FIELD = "right"; + private static final String METADATA_KEY = "testData"; + + private String metadataPath; + + @Before + public void setupTest() { + super.setupTest(); + this.metadataPath = FileTestUtil.getTempFolder(); + } + + @Test + public void testReadDataFromParquetFile() throws IOException { + final StructType dfSchema = DataTypes + .createStructType(new StructField[]{ + DataTypes.createStructField(LEFT_FIELD, DataTypes.StringType, false), + DataTypes.createStructField(RIGHT_FIELD, DataTypes.StringType, false)}); + final DataFrameSchemaConverter dfsc = new DataFrameSchemaConverter(); + final Schema avroSchema = dfsc.convertToCommonSchema(dfSchema); + + final String dataPath = FileHelperUtil.getResourcePath(getClass(), METADATA_KEY); + final HiveSourceConfiguration hiveConf = + HiveTestUtil.initializeConfig(JOB_NAME, dataPath, "testMetadataPath"); + 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); + final HDFSPartitionManager pm = new HDFSPartitionManager(METADATA_KEY, + this.metadataPath, + dataPath, + this.fileSystem.get()); + + final ParquetWorkUnitCalculator calculator = new ParquetWorkUnitCalculator(); + calculator.initPreviousRunState(pm); + final IWorkUnitCalculator.IWorkUnitCalculatorResult results + = calculator.computeWorkUnits(); + final JavaRDD rddData = + source.getData((ParquetWorkUnitCalculatorResult) results); + final List collectedData = rddData.collect(); + Assert.assertEquals(1, collectedData.size()); + + final GenericRecord record = collectedData.get(0).getData(); + + Assert.assertEquals("L", record.get(LEFT_FIELD).toString()); + Assert.assertEquals("R", record.get(RIGHT_FIELD).toString()); + } +} + 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 new file mode 100644 index 0000000..e919998 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestHiveSourceConfiguration.java @@ -0,0 +1,66 @@ +/* + * 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.sources.hive; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.HiveSourceConfiguration; +import com.uber.marmaray.common.exceptions.MissingPropertyException; +import org.junit.Assert; +import org.junit.Test; + +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) + public void testMissingHiveDataPath() { + final Configuration config = new Configuration(); + config.setProperty(HiveSourceConfiguration.JOB_NAME, JOB_NAME); + final HiveSourceConfiguration hiveConfig = new HiveSourceConfiguration(config); + Assert.fail(); + } + + @Test(expected = MissingPropertyException.class) + public void testMissingJobName() { + final Configuration config = new Configuration(); + config.setProperty(HiveSourceConfiguration.HIVE_DATA_PATH, DEFAULT_DATA_PATH); + final HiveSourceConfiguration hiveConfig = new HiveSourceConfiguration(config); + Assert.fail(); + } + + @Test + public void testBasicConfig() { + final Configuration config = getValidHiveSourceConfiguration(); + 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()); + } + + 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 new file mode 100644 index 0000000..6269d79 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/sources/hive/TestParquetWorkUnitCalculator.java @@ -0,0 +1,174 @@ +/* + * 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.sources.hive; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.metadata.HDFSPartitionManager; +import com.uber.marmaray.common.metadata.MetadataConstants; +import com.uber.marmaray.common.metadata.StringValue; +import com.uber.marmaray.common.sources.IWorkUnitCalculator; +import com.uber.marmaray.common.util.FileTestUtil; +import com.uber.marmaray.utilities.FSUtils; +import com.uber.marmaray.utilities.StringTypes; +import lombok.NonNull; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +public class TestParquetWorkUnitCalculator { + + private static final String JOB_NAME = "jobFoo"; + private static final String PARTITION_1 = "partition1"; + private static final String PARTITION_2 = "partition2"; + private static final String PARTITION_3 = "partition3"; + private FileSystem fileSystem; + private HDFSPartitionManager partitionManager; + private String dataPath; + private String metadataPath; + + + @Before + public void setupTest() throws IOException { + this.fileSystem = FSUtils.getFs(new Configuration()); + this.dataPath = FileTestUtil.getTempFolder(); + this.metadataPath = FileTestUtil.getTempFolder(); + } + + @After + public void tearDownTest() throws IOException { + if (this.fileSystem != null) { + this.fileSystem.close(); + } + } + + @Test + public void testComputeWorkUnitsWithNoPrexistentCheckpointsMultiplePartitions() throws IOException { + // No prexisting checkpoints for the workunit calculator + this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_2)); + this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_3)); + + this.partitionManager = new HDFSPartitionManager(JOB_NAME, + this.metadataPath, + this.dataPath, + this.fileSystem); + + Assert.assertFalse(this.partitionManager.isSinglePartition()); + Assert.assertFalse(this.partitionManager.getLatestCheckpoint().isPresent()); + virtuallyProcessPartition(this.partitionManager, Optional.absent(), PARTITION_2); + + final HDFSPartitionManager partitionManager2 = new HDFSPartitionManager(JOB_NAME, + this.metadataPath, + this.dataPath, + this.fileSystem); + Assert.assertTrue(partitionManager2.getLatestCheckpoint().isPresent()); + virtuallyProcessPartition(partitionManager2, Optional.of(new StringValue(PARTITION_2)), PARTITION_3); + } + + @Test + public void testComputeWorkUnitsWithNoPrexistentCheckpointsSinglePartition() throws IOException { + // No prexisting checkpoints for the workunit calculator + final String dataFileName = "data.parquet"; + this.fileSystem.create(new Path(this.dataPath, dataFileName)); + + this.partitionManager = new HDFSPartitionManager(JOB_NAME, + this.metadataPath, + this.dataPath, + this.fileSystem); + + Assert.assertTrue(this.partitionManager.isSinglePartition()); + Assert.assertFalse(this.partitionManager.getLatestCheckpoint().isPresent()); + + virtuallyProcessPartition(this.partitionManager, Optional.absent(), this.dataPath ); + + // 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); + Assert.assertTrue(pm2.getLatestCheckpoint().isPresent()); + Assert.assertEquals(this.dataPath, pm2.getLatestCheckpoint().get().getValue()); + + final ParquetWorkUnitCalculator calc = new ParquetWorkUnitCalculator(); + calc.initPreviousRunState(pm2); + Assert.assertTrue(calc.getNextPartition().isPresent()); + Assert.assertEquals(this.dataPath, calc.getNextPartition().get()); + + // explicitly remove the old checkpoint so we see it is set correctly to latest checkpoint when + // saving next run state + pm2.set(MetadataConstants.CHECKPOINT_KEY, new StringValue(StringTypes.EMPTY)); + calc.initPreviousRunState(pm2); + final IWorkUnitCalculator.IWorkUnitCalculatorResult iresult = calc.computeWorkUnits(); + calc.saveNextRunState(pm2, iresult.getNextRunState()); + Assert.assertEquals(this.dataPath, pm2.get(MetadataConstants.CHECKPOINT_KEY).get().getValue()); + } + + @Test + public void testComputeWorkUnitsWithExistentCheckpoint() throws IOException { + + this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_1)); + this.fileSystem.mkdirs(new Path(this.dataPath, PARTITION_3)); + + this.partitionManager = new HDFSPartitionManager(JOB_NAME, + this.metadataPath, + this.dataPath, + this.fileSystem); + + // partition 1 is in effect already processed since the checkpoint is larger + final StringValue val1 = new StringValue(PARTITION_2); + this.partitionManager.set(MetadataConstants.CHECKPOINT_KEY, val1); + this.partitionManager.saveChanges(); + + final ParquetWorkUnitCalculator calculator = new ParquetWorkUnitCalculator(); + calculator.initPreviousRunState(this.partitionManager); + final IWorkUnitCalculator.IWorkUnitCalculatorResult iresult = calculator.computeWorkUnits(); + Assert.assertTrue(iresult instanceof ParquetWorkUnitCalculatorResult); + final ParquetWorkUnitCalculatorResult result = + (ParquetWorkUnitCalculatorResult) iresult; + final List workUnits = result.getWorkUnits(); + Assert.assertEquals(1, workUnits.size()); + Assert.assertEquals(PARTITION_3, workUnits.get(0)); + Assert.assertTrue(result.getNextRunState().getPartition().isPresent()); + Assert.assertEquals(PARTITION_3, result.getNextRunState().getPartition().get()); + } + + private void virtuallyProcessPartition(@NonNull final HDFSPartitionManager partitionManager, + @NotEmpty final Optional expectedLatestCheckpoint, + @NotEmpty final String expectedNextPartition) { + Assert.assertEquals(expectedLatestCheckpoint, partitionManager.getLatestCheckpoint()); + + final ParquetWorkUnitCalculator calculator = new ParquetWorkUnitCalculator(); + calculator.initPreviousRunState(partitionManager); + final ParquetWorkUnitCalculatorResult result = calculator.computeWorkUnits(); + final List workUnits = result.getWorkUnits(); + Assert.assertEquals(1, workUnits.size()); + Assert.assertEquals(expectedNextPartition, workUnits.get(0)); + Assert.assertTrue(result.getNextRunState().getPartition().isPresent()); + Assert.assertEquals(expectedNextPartition, result.getNextRunState().getPartition().get()); + calculator.saveNextRunState(partitionManager, result.getNextRunState()); + Assert.assertEquals(expectedNextPartition, partitionManager.get(MetadataConstants.CHECKPOINT_KEY).get().getValue()); + partitionManager.saveChanges(); + } +} 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 new file mode 100644 index 0000000..b8a9514 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/spark/TestSparkFactory.java @@ -0,0 +1,91 @@ +/* + * 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.google.common.base.Optional; +import java.util.Arrays; +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); + assertExpectationsOnSparkContext(sparkArgs, sparkSession.sparkContext()); + + // should re-use existing SparkContext and not fail + final SparkContext sc2 = sparkFactory.get().getSparkContext(sparkArgs).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(); + } + + 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); + Assert.assertEquals(expectedAvroSchema.toString(), registeredAvroSchemaStr); + Assert.assertEquals("foo_bar", sc.appName()); + Assert.assertEquals("512", sc.hadoopConfiguration().get("mapreduce.map.memory.mb")); + } + + 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"); + + return new SparkArgs(schemas, Arrays.asList(), + overrideSparkProperties, hadoopConfiguration); + } +} 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 new file mode 100644 index 0000000..6baad4b --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/AbstractSparkTest.java @@ -0,0 +1,88 @@ +/* + * 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.util; + +import com.google.common.base.Optional; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; +import org.hibernate.validator.constraints.NotEmpty; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; + +@Slf4j +public class AbstractSparkTest { + + protected Optional jsc = Optional.absent(); + protected Optional spark = Optional.absent(); + protected Optional sqlContext = Optional.absent(); + protected Optional fileSystem = Optional.absent(); + protected static final String JOB_NAME = "test-job"; + + @Before + public void setupTest() { + this.spark = Optional.of(SparkTestUtil.getSparkSession(getSparkConf(AbstractSparkTest.class.getName()))); + 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())); + } catch (IOException e) { + log.error("Cannot initialize FileSystem object", e); + } + } + + protected SparkConf getSparkConf(@NotEmpty final String appName) { + return SparkTestUtil.getSparkConf(appName); + } + + @After + public void teardownTest() { + if (this.spark.isPresent()) { + this.spark.get().stop(); + this.spark = Optional.absent(); + } + + if (this.jsc.isPresent()) { + this.jsc.get().stop(); + this.jsc = Optional.absent(); + } + + if (this.sqlContext.isPresent()) { + this.sqlContext = Optional.absent(); + } + + if (this.fileSystem.isPresent()) { + try { + this.fileSystem.get().close(); + } catch (IOException e) { + log.error("Cannot close FileSystem object", e); + } + this.fileSystem = Optional.absent(); + } + } + + protected String getAppId() { + return jsc.get().getConf().getAppId(); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/AvroPayloadUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/AvroPayloadUtil.java new file mode 100644 index 0000000..176e997 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/AvroPayloadUtil.java @@ -0,0 +1,137 @@ +/* + * 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.util; + +import com.google.common.base.Strings; +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import com.uber.marmaray.utilities.StringTypes; +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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +public final class AvroPayloadUtil { + + private AvroPayloadUtil() { + throw new JobRuntimeException("Utility class should not be instantiated"); + } + + public static JavaRDD generateTestData(final JavaSparkContext jsc, + final int numRecords, + final String fieldToExclude) { + final List payloads = new ArrayList<>(); + + final Schema schema = getAvroTestDataSchema(fieldToExclude); + + for (int i = 1; i <= numRecords; i++) { + + final GenericRecord gr = new GenericData.Record(schema); + + if (!CassandraTestConstants.INT_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.INT_FIELD, i); + } + + if (!CassandraTestConstants.STRING_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.STRING_FIELD, Integer.toString(i)); + } + + if (!CassandraTestConstants.BOOLEAN_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.BOOLEAN_FIELD, true); + } + + final AvroPayload ap = new AvroPayload(gr); + payloads.add(ap); + } + + return jsc.parallelize(payloads); + } + + public static JavaRDD generateTestDataNew(final JavaSparkContext jsc, + final int numRecords, + final String fieldToExclude) { + final List payloads = new ArrayList<>(); + + final Schema schema = getAvroTestDataSchema(fieldToExclude); + + for (int i = 1; i <= numRecords; i++) { + + final GenericRecord gr = new GenericData.Record(schema); + + if (!CassandraTestConstants.INT_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.INT_FIELD, i); + } + + if (!CassandraTestConstants.STRING_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.STRING_FIELD, Integer.toString(i)+"\""+",try\\"); + } + + if (!CassandraTestConstants.BOOLEAN_FIELD.equals(fieldToExclude)) { + gr.put(CassandraTestConstants.BOOLEAN_FIELD, true); + } + + final AvroPayload ap = new AvroPayload(gr); + payloads.add(ap); + } + + return jsc.parallelize(payloads); + } + + public static Schema getAvroTestDataSchema(final String fieldToExclude) { + + SchemaBuilder.FieldAssembler fields = SchemaBuilder.record("commonSchema").fields(); + + if (!CassandraTestConstants.INT_FIELD.equals(fieldToExclude)) { + fields = fields.name(CassandraTestConstants.INT_FIELD).type().intType().noDefault(); + } + + if (!CassandraTestConstants.STRING_FIELD.equals(fieldToExclude)) { + fields = fields.name(CassandraTestConstants.STRING_FIELD).type().stringType().noDefault(); + } + + if (!CassandraTestConstants.BOOLEAN_FIELD.equals(fieldToExclude)) { + fields = fields.name(CassandraTestConstants.BOOLEAN_FIELD).type().booleanType().noDefault(); + } + + return fields.endRecord(); + } + + public static List getSchemaFields() { + return getSchemaFields(StringTypes.EMPTY); + } + + public static List getSchemaFields(final String fieldToExclude) { + List fields = new LinkedList<>(Arrays.asList( + CassandraTestConstants.INT_FIELD, + CassandraTestConstants.STRING_FIELD, + CassandraTestConstants.BOOLEAN_FIELD)); + + if (!Strings.isNullOrEmpty(fieldToExclude)) { + fields.remove(fieldToExclude); + } + + return Collections.unmodifiableList(fields); + } +} 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 new file mode 100644 index 0000000..b3c2490 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestConstants.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.util; + +public class CassandraTestConstants { + public static final String KEY_SPACE = "marmaray"; + public static final String TABLE = "crossfit_gyms"; + public static final String LOCALHOST = "localhost"; + public static final String INT_FIELD = "int_field"; + public static final String STRING_FIELD = "string_field"; + public static final String BOOLEAN_FIELD = "boolean_field"; + public static final int CASSANDRA_PORT = 9142; +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestUtil.java new file mode 100644 index 0000000..42fb045 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/CassandraTestUtil.java @@ -0,0 +1,57 @@ +/* + * 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.util; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.thrift.transport.TTransportException; +import org.cassandraunit.CQLDataLoader; +import org.cassandraunit.dataset.cql.ClassPathCQLDataSet; +import org.cassandraunit.utils.EmbeddedCassandraServerHelper; + +import java.io.IOException; + +public final class CassandraTestUtil { + + public static void setupCluster() throws ConfigurationException, TTransportException, IOException { + System.setProperty("cassandra.storagedir", "/tmp/cassandra" + System.nanoTime()); + EmbeddedCassandraServerHelper.startEmbeddedCassandra("/cassandra.yaml"); + final Cluster cluster = initCluster(); + try (final Session session = cluster.connect()) { + final CQLDataLoader loader = new CQLDataLoader(session); + loader.load(new ClassPathCQLDataSet("setupTable.cql")); + } + } + + public static void teardownCluster() { + final Cluster cluster = initCluster(); + try (final Session session = cluster.connect()) { + final CQLDataLoader loader = new CQLDataLoader(session); + loader.load(new ClassPathCQLDataSet("teardownTable.cql")); + } + } + + public static Cluster initCluster() { + final Cluster cluster = new Cluster.Builder() + .addContactPoints(CassandraTestConstants.LOCALHOST) + .withPort(CassandraTestConstants.CASSANDRA_PORT) + .build(); + return cluster; + } + +} 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 new file mode 100644 index 0000000..b352db6 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/FileHelperUtil.java @@ -0,0 +1,33 @@ +/* + * 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.util; + +import org.apache.commons.lang.CharEncoding; + +import java.io.File; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; + +public class FileHelperUtil { + public static String getResourcePath(Class clazz, String path) { + try { + return URLDecoder.decode(clazz.getResource(File.separator + path).getPath(), CharEncoding.UTF_8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/FileSinkConfigTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/FileSinkConfigTestUtil.java new file mode 100644 index 0000000..6d4bdf7 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/FileSinkConfigTestUtil.java @@ -0,0 +1,107 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.FileSinkConfiguration; +import lombok.NonNull; + +public class FileSinkConfigTestUtil extends AbstractSparkTest { + protected Configuration initS3(@NonNull final String sinkType) { + final Configuration conf = initCommon("/new_path/new_test", "csv", 1, ",", sinkType, "version"); + conf.setProperty(FileSinkConfiguration.AWS_LOCAL, "/aws_test"); + conf.setProperty(FileSinkConfiguration.AWS_REGION, "us-east-1"); + conf.setProperty(FileSinkConfiguration.AWS_ACCESS_KEY_ID, "username"); + conf.setProperty(FileSinkConfiguration.AWS_SECRET_ACCESS_KEY, "password"); + conf.setProperty(FileSinkConfiguration.BUCKET_NAME, "aws-test"); + conf.setProperty(FileSinkConfiguration.OBJECT_KEY, "marmaray_test/test1"); + return conf; + } + + protected Configuration initS3MissConfig(@NonNull final String sinkType, @NonNull final String propertyExclude) { + final Configuration conf = initCommon("/new_path/new_test", "csv", 1, ",", sinkType, "version"); + conf.setProperty(FileSinkConfiguration.AWS_LOCAL, "/aws_test"); + if (!propertyExclude.equals(FileSinkConfiguration.AWS_REGION)) { + conf.setProperty(FileSinkConfiguration.AWS_REGION, "us-east-1"); + } + if (!propertyExclude.equals(FileSinkConfiguration.AWS_ACCESS_KEY_ID)) { + conf.setProperty(FileSinkConfiguration.AWS_ACCESS_KEY_ID, "username"); + conf.setProperty(FileSinkConfiguration.AWS_SECRET_ACCESS_KEY, "password"); + } + if (!propertyExclude.equals(FileSinkConfiguration.BUCKET_NAME)) { + conf.setProperty(FileSinkConfiguration.BUCKET_NAME, "aws-test"); + } + if (!propertyExclude.equals(FileSinkConfiguration.OBJECT_KEY)) { + conf.setProperty(FileSinkConfiguration.OBJECT_KEY, "marmaray_test/test1"); + } + return conf; + } + + protected Configuration initCommon(@NonNull final String path, @NonNull final String fileType, + @NonNull final long fileMB, @NonNull final String separator, + @NonNull final String sinkType, @NonNull final String dispersalType) { + final Configuration c = initFileNameAndPath("date", true, dispersalType); + c.setProperty(FileSinkConfiguration.FS_PATH, path); + c.setProperty(FileSinkConfiguration.FILE_TYPE, fileType); + c.setProperty(FileSinkConfiguration.FILE_SIZE_MEGABYTE, String.valueOf(fileMB)); + c.setProperty(FileSinkConfiguration.SEPARATOR, separator); + c.setProperty(FileSinkConfiguration.FILE_SINK_TYPE, sinkType); + return c; + } + + protected Configuration initFileNameAndPath(@NonNull final String partitionType, @NonNull final boolean sourceSubPathExists, + @NonNull final String dispersalType) { + final Configuration c = initFileSystem(); + c.setProperty(FileSinkConfiguration.SOURCE_TYPE, "hive"); + c.setProperty(FileSinkConfiguration.TIMESTAMP, "201808011025"); + c.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test_trip.table"); + c.setProperty(FileSinkConfiguration.DISPERSAL_TYPE, dispersalType); + if (sourceSubPathExists) { + c.setProperty(FileSinkConfiguration.SOURCE_PARTITION_PATH, "2018/08/01"); + } + c.setProperty(FileSinkConfiguration.PARTITION_TYPE, partitionType); + return c; + } + + protected Configuration initFileNameAndPathMissConfig(@NonNull final String propertyExclude) { + Configuration c = new Configuration(); + if (!propertyExclude.equals(FileSinkConfiguration.PATH_PREFIX)) { + c = initFileSystem(); + } + if (!propertyExclude.equals(FileSinkConfiguration.SOURCE_TYPE)) { + c.setProperty(FileSinkConfiguration.SOURCE_TYPE, "hive"); + } + if (!propertyExclude.equals(FileSinkConfiguration.TIMESTAMP)) { + c.setProperty(FileSinkConfiguration.TIMESTAMP, "201808011025"); + } + if (!propertyExclude.equals(FileSinkConfiguration.SOURCE_NAME_PREFIX)) { + c.setProperty(FileSinkConfiguration.SOURCE_NAME_PREFIX, "test.db_test_trip.table"); + } + c.setProperty(FileSinkConfiguration.DISPERSAL_TYPE, "version"); + c.setProperty(FileSinkConfiguration.SOURCE_PARTITION_PATH, "2018/08/01"); + c.setProperty(FileSinkConfiguration.PARTITION_TYPE, "date"); + return c; + } + + protected Configuration initFileSystem() { + final Configuration c = new Configuration(); + final String filePrefix = this.fileSystem.get().getWorkingDirectory().toString(); + c.setProperty(FileSinkConfiguration.PATH_PREFIX, filePrefix); + return c; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/FileTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/FileTestUtil.java new file mode 100644 index 0000000..cd48637 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/FileTestUtil.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.util; + +import com.google.common.io.Files; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.io.File; + +public class FileTestUtil { + + private FileTestUtil() { + throw new JobRuntimeException("Utility class; don't instantiate it"); + } + + /** + * It creates temp directory. + * @return + */ + public static String getTempFolder() { + final File basePath = Files.createTempDir(); + // We want temp directory to delete after all tests have completed. + basePath.deleteOnExit(); + return basePath.getAbsolutePath(); + } +} 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 new file mode 100644 index 0000000..445c258 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/HiveTestUtil.java @@ -0,0 +1,37 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.HiveSourceConfiguration; + +public final class HiveTestUtil { + private HiveTestUtil() { + throw new RuntimeException("This test class should never be instantiated"); + } + + public static HiveSourceConfiguration initializeConfig(final String jobName, + final String dataPath, + final String metadataPath) + { + 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 new file mode 100644 index 0000000..519d1cb --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/KafkaTestHelper.java @@ -0,0 +1,242 @@ +/* + * 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.util; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.KafkaSourceConfiguration; +import com.uber.marmaray.common.exceptions.InvalidDataException; +import com.uber.marmaray.common.schema.ISchemaService; +import com.uber.marmaray.utilities.GenericRecordUtil; +import com.uber.marmaray.utilities.ScalaUtil; +import kafka.producer.KeyedMessage; +import kafka.producer.Producer; +import kafka.producer.ProducerConfig; +import kafka.serializer.DefaultEncoder; +import kafka.serializer.StringEncoder; +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.avro.generic.GenericRecordBuilder; +import org.apache.spark.streaming.kafka010.KafkaTestUtils; +import org.hibernate.validator.constraints.NotEmpty; + +import java.io.FileNotFoundException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +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_TOPIC_NAME; + +@Slf4j +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"; + private static final String STRING_VALUE_DEFAULT = "value"; + private static final Boolean BOOLEAN_VALUE_DEFAULT = true; + + public static void createTopicPartitions(@NonNull final KafkaTestUtils kafkaTestUtils, final String topicName, + final int partitions) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(topicName)); + // Create topic. + kafkaTestUtils.createTopic(topicName, partitions); + } + + public static ProducerConfig getProducerConfig(@NonNull final KafkaTestUtils kafkaTestUtils) { + final Properties props = new Properties(); + props.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + props.put("serializer.class", DefaultEncoder.class.getName()); + props.put("key.serializer.class", StringEncoder.class.getName()); + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1"); + return new ProducerConfig(props); + } + + /** + * It publishes kafka messages to specified partitions[index is used as partition number]. + */ + public static void publishMessages(@NonNull final KafkaTestUtils kafkaTestUtils, final String topicName, + @NonNull final List> messagesList) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(topicName)); + final Producer producer = new Producer<>(getProducerConfig(kafkaTestUtils)); + final Set> messageSet = new HashSet<>(); + for (int partition = 0; partition < messagesList.size(); partition++) { + final String key = Integer.toString(partition); + messagesList.get(partition).stream().forEach( + message -> { + messageSet.add(new KeyedMessage<>(topicName, key, message)); + } + ); + } + producer.send(ScalaUtil.toScalaSet(messageSet).toSeq()); + producer.close(); + } + + public static List> generateMessages(@NonNull final List messageCountList) + throws FileNotFoundException { + final Schema schema = getSchema("test_schema"); + final TestKafkaSchemaService schemaService = new TestKafkaSchemaService(); + final ISchemaService.ISchemaServiceWriter writer = + schemaService.getWriter("test_schema", 1); + + + 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); + } + } + ).collect(Collectors.toList())); + } + ); + return ret; + } + + public static Schema getSchema(@NotEmpty final String schemaName) { + return SchemaBuilder.record(schemaName).fields() + .name("testBoolean").type().optional().booleanType() + .name("testLong").type().optional().longType() + .name("testString").type().optional().stringType() + .endRecord(); + } + + public static List getTestData(@NonNull final Schema schema, final int numRecords) { + final List records = new ArrayList<>(numRecords); + for (int i = 0; i < numRecords; i++) { + final GenericRecord record = new GenericData.Record(schema); + schema.getFields().forEach(field -> record.put(field.name(), getDefaultValue(field))); + records.add(record); + } + return records; + } + + private static Object getDefaultValue(@NonNull final Schema.Field field) { + if (GenericRecordUtil.isOptional(field.schema())) { + final Iterator iter = field.schema().getTypes().iterator(); + while (iter.hasNext()) { + final Schema next = iter.next(); + if (next.getType() != Schema.Type.NULL) { + return getDefaultValue(next.getType()); + } + } + // no non-null schema type found, just return null + return null; + } else { + return getDefaultValue(field.schema().getType()); + } + } + + private static Object getDefaultValue(@NonNull final Schema.Type type) { + if (type == Schema.Type.STRING) { + return STRING_VALUE_DEFAULT; + } else if (type == Schema.Type.BOOLEAN) { + return BOOLEAN_VALUE_DEFAULT; + } else if (type == Schema.Type.LONG) { + return System.currentTimeMillis(); + } else if (type == Schema.Type.UNION) { + return null; + } else { + log.warn("Found unknown type {}, returning null", type.toString()); + return null; + } + + } + + public static void publishMessagesToKafkaTopics(@NonNull final KafkaTestUtils kafkaTestUtils, + @NotEmpty final String topicName) + throws FileNotFoundException { + 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); + KafkaTestHelper.publishMessages(kafkaTestUtils, topicName, messages); + } + + public static KafkaSourceConfiguration getKafkaSourceConfiguration(@NotEmpty final String topicName, + @NotEmpty final String brokerAddress, @NotEmpty final String startDate) { + 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), Arrays.asList(brokerAddress, topicName, TEST_KAFKA_CLUSTER_NAME, startDate)); + return new KafkaSourceConfiguration(conf); + } + + public static KafkaSourceConfiguration getKafkaSourceConfiguration(final String topicName, + final String brokerAddress) { + return getKafkaSourceConfiguration(topicName, brokerAddress, TEST_KAFKA_START_DATE); + } + + public static void setMandatoryConf(final Configuration conf, final List mandatoryProperties, + final List values) { + for (int i = 0; i < mandatoryProperties.size(); i++) { + conf.setProperty(mandatoryProperties.get(i), values.get(i)); + } + } + + public static class TestKafkaSchemaService implements ISchemaService, Serializable { + + + @Override public Schema getWrappedSchema(final String schemaName) { + return KafkaTestHelper.getSchema(schemaName); + } + + @Override public Schema getSchema(final String schemaName) { + return getWrappedSchema(schemaName); + } + + @Override public ISchemaServiceWriter getWriter(final String schemaName, final int schemaVersion) { + return new TestKafkaSchemaServiceWriter(); + } + + @Override public ISchemaServiceReader getReader(final String schemaName, final int schemaVersion) { + return new TestKafkaSchemaServiceReader(); + } + + private class TestKafkaSchemaServiceWriter implements ISchemaServiceWriter, Serializable { + @Override public byte[] write(final GenericRecord record) throws InvalidDataException { + return new byte[0]; + } + } + + private class TestKafkaSchemaServiceReader implements ISchemaServiceReader, Serializable { + @Override public GenericRecord read(final byte[] buffer) throws InvalidDataException { + return new GenericRecordBuilder(getSchema("test")).build(); + } + } + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/MultiThreadTestCoordinator.java b/marmaray/src/test/java/com/uber/marmaray/common/util/MultiThreadTestCoordinator.java new file mode 100644 index 0000000..4d19147 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/MultiThreadTestCoordinator.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.util; + +import java.util.concurrent.atomic.AtomicLong; + +public class MultiThreadTestCoordinator { + private final AtomicLong c; + + public MultiThreadTestCoordinator() { + this.c = new AtomicLong(0); + } + + public void nextStep() { + this.c.addAndGet(1); + } + + public void waitUntilStep(long s) throws RuntimeException { + while (c.get() != s) { + if (c.get() > s) { + throw new RuntimeException(String.format("Current Step %d has passed ExpectedStep %d", c.get(), s)); + } + } + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/ParquetWriterUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/ParquetWriterUtil.java new file mode 100644 index 0000000..2afbcb8 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/ParquetWriterUtil.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.util; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +public class ParquetWriterUtil { + /** + * Returns a ParquetWriter for Avro files. This functionality is provided mostly as a utility for tests + * where we may have to create parquet files to test ingestion or dispersal. + * + * @param dataFilePath + * @param schema + * @return + */ + public static ParquetWriter initializeAvroWriter(final Path dataFilePath, + final Schema schema, + final Configuration conf) throws IOException { + final ParquetWriter writer = AvroParquetWriter.builder(dataFilePath) + .withSchema(schema) + .withCompressionCodec(CompressionCodecName.UNCOMPRESSED) + .withConf(conf) + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0) + .build(); + return writer; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/SchemaTestUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/SchemaTestUtil.java new file mode 100644 index 0000000..100459c --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/SchemaTestUtil.java @@ -0,0 +1,100 @@ +/* + * 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.util; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.SchemaBuilder.FieldAssembler; +import org.apache.avro.SchemaBuilder.RecordBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.hibernate.validator.constraints.NotEmpty; + +/** + * Helper class to generate test data and {@link Schema}. + */ +public final class SchemaTestUtil { + private static final Random r = new Random(); + + private SchemaTestUtil() { + throw new JobRuntimeException("helper class; don't instantiate it"); + } + + public static Schema getSchema(@NotEmpty final String tsKey, @NotEmpty final String recordKey, + final int mandatoryFields, final int optionalFields) { + final RecordBuilder recordSchema = SchemaBuilder.builder().record("test"); + final FieldAssembler fields = recordSchema.fields(); + fields.requiredDouble(tsKey); + fields.requiredString(recordKey); + for (int i = 0; i < mandatoryFields; i++) { + fields.requiredString("testMandatory" + i); + } + for (int i = 0; i < optionalFields; i++) { + fields.optionalString("testOptional" + i); + } + return fields.endRecord(); + } + + public static List getRandomData(@NotEmpty final String schemaStr, + @NotEmpty final String tsKey, @NotEmpty final String recordKey, final int count) { + + final Schema schema = (new Schema.Parser()).parse(schemaStr); + final List records = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + final GenericRecord record = new GenericData.Record(schema); + for (Field field : schema.getFields()) { + record.put(field.name(), new String("value" + System.nanoTime())); + } + record.put(tsKey, new Double(System.currentTimeMillis())); + record.put(recordKey, "key" + r.nextLong()); + records.add(new AvroPayload(record)); + } + return records; + } + + public static Schema getSchema(@NotEmpty final String type) { + + final SchemaBuilder.TypeBuilder schema = SchemaBuilder.builder(); + + switch (type) { + case CassandraSchemaField.STRING_TYPE: + return schema.stringType(); + case CassandraSchemaField.BOOLEAN_TYPE: + return schema.booleanType(); + case CassandraSchemaField.DOUBLE_TYPE: + return schema.doubleType(); + case CassandraSchemaField.FLOAT_TYPE: + return schema.floatType(); + case CassandraSchemaField.LONG_TYPE: + return schema.longType(); + case CassandraSchemaField.INT_TYPE: + return schema.intType(); + default: + throw new IllegalArgumentException("Type " + type + " invalid for converting to Cassandra field"); + + } + } + +} 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 new file mode 100644 index 0000000..1c168fa --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/SparkTestUtil.java @@ -0,0 +1,51 @@ +/* + * 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.util; + +import lombok.NonNull; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +public class SparkTestUtil { + + public static JavaSparkContext getSparkContext(@NonNull final SparkConf sparkConf) { + return new JavaSparkContext(sparkConf); + } + + public static SparkConf getSparkConf(final String appName) { + return new SparkConf().setAppName(appName) + .setMaster("local[4]") + .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.kryoserializer.buffer.max", "512m"); + } + + public static SparkSession getSparkSession(@NonNull final SparkConf sparkConf) { + final SparkContext sc = SparkTestUtil.getSparkContext(sparkConf).sc(); + + final SparkSession session = org.apache.spark.sql.SparkSession.builder() + .master("local[*]") + .sparkContext(sc) + .getOrCreate(); + + return session; + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestConverterUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestConverterUtil.java new file mode 100644 index 0000000..0c393dd --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestConverterUtil.java @@ -0,0 +1,77 @@ +/* + * 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.util; + +import com.uber.marmaray.common.AvroPayload; +import com.uber.marmaray.utilities.ConverterUtil; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.JsonDecoder; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; + +public class TestConverterUtil extends AbstractSparkTest { + + @Test + public void testConvertAvroPayloadToBytes() throws IOException { + final Schema avroSchema = SchemaBuilder.record("commonSchema") + .fields() + .name("int").type().intType().noDefault() + .name("boolean").type().booleanType().noDefault() + .endRecord(); + + final GenericRecord gr = new GenericData.Record(avroSchema); + gr.put("int", Integer.MAX_VALUE); + gr.put("boolean", true); + + final AvroPayload ap = new AvroPayload(gr); + + byte[] bytes = ConverterUtil.convertData(ap); + + final String json = new String(bytes); + + Assert.assertTrue(json.contains("int")); + Assert.assertTrue(json.contains(String.valueOf(Integer.MAX_VALUE))); + Assert.assertTrue(json.contains("boolean")); + Assert.assertTrue(json.contains("true")); + + final JsonDecoder decoder = new DecoderFactory().jsonDecoder(avroSchema, new ByteArrayInputStream(bytes)); + final DatumReader reader = new GenericDatumReader<>(avroSchema); + GenericRecord entry; + while (true) { + try { + entry = reader.read(null, decoder); + System.out.println(entry); + Assert.assertTrue((Boolean) entry.get("boolean")); + Assert.assertEquals(Integer.MAX_VALUE, entry.get("int")); + } catch (EOFException exception) { + break; + } + } + + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestDateUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestDateUtil.java new file mode 100644 index 0000000..b703724 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestDateUtil.java @@ -0,0 +1,43 @@ +/* + * 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.util; + +import com.uber.marmaray.utilities.DateUtil; +import org.junit.Assert; +import org.junit.Test; + +import java.time.LocalDate; + +public class TestDateUtil { + + @Test + public void testConvertToUTCDate() { + // We should be able to handle both forward slashes and dashes as separators + // since we sanitize the input + final String dateStr = "2017-05-01"; + LocalDate ld = DateUtil.convertToUTCDate(dateStr); + Assert.assertEquals(2017, ld.getYear()); + Assert.assertEquals(5, ld.getMonth().getValue()); + Assert.assertEquals(1, ld.getDayOfMonth()); + + final String dateStr2 = "1998-06-10"; + LocalDate l2 = DateUtil.convertToUTCDate(dateStr2); + Assert.assertEquals(1998, l2.getYear()); + Assert.assertEquals(6, l2.getMonth().getValue()); + Assert.assertEquals(10, l2.getDayOfMonth()); + } +} 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 new file mode 100644 index 0000000..f24652b --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestFsUtils.java @@ -0,0 +1,83 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.stream.IntStream; + +public class TestFsUtils { + private FileSystem fileSystem; + + @Before + public void initClass() throws IOException { + final Configuration conf = new Configuration(); + this.fileSystem = FSUtils.getFs(conf); + } + + @Test + public void testDeleteMetadataFiles() throws IOException { + final String metadataPath = FileTestUtil.getTempFolder(); + Long currentTime = System.currentTimeMillis(); + + IntStream.range(0, 7).forEach(iteration -> { + try { + final Long newTime = currentTime + iteration; + final String fileLocation = new Path(metadataPath, newTime.toString()).toString(); + this.fileSystem.create(new Path(fileLocation)); + } catch (final IOException e) { + Assert.fail("IOException occurred while creating files"); + } + }); + + final FileStatus[] fileStatuses = this.fileSystem.listStatus(new Path(metadataPath)); + Assert.assertEquals(7, fileStatuses.length); + + FSUtils.deleteHDFSMetadataFiles(fileStatuses, this.fileSystem, 4, false); + + final int numExpectedRemaining = 4; + final FileStatus[] remaining = this.fileSystem.listStatus(new Path(metadataPath)); + Assert.assertEquals(numExpectedRemaining, remaining.length); + + IntStream.range(0, 7) + .forEach(iteration -> { + try { + final Long newTime = currentTime + iteration; + + if (iteration < numExpectedRemaining - 1) { + Assert.assertFalse(this.fileSystem.exists(new Path(metadataPath, newTime.toString()))); + } else { + Assert.assertTrue(this.fileSystem.exists(new Path(metadataPath, newTime.toString()))); + } + } catch (final IOException e) { + Assert.fail("IOException occurred while validating files"); + } + }); + + // fake delete, this just prints to console what would have been deleted if we really executed the deletion + FSUtils.deleteHDFSMetadataFiles(fileStatuses, this.fileSystem, 2, true); + Assert.assertEquals(numExpectedRemaining, remaining.length); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestJobUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestJobUtil.java new file mode 100644 index 0000000..c930cc6 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestJobUtil.java @@ -0,0 +1,37 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.JobUtil; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TestJobUtil { + + @Test + public void testGetDataCenter() throws IOException { + final Configuration conf = new Configuration(); + final String dcFile = FileHelperUtil.getResourcePath(getClass(), + new Path("datacenter", "datacenter").toString()); + final String dc = JobUtil.getDataCenterForJob(dcFile); + Assert.assertEquals("test_dc", dc); + } +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestLockManager.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestLockManager.java new file mode 100644 index 0000000..baa41fb --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestLockManager.java @@ -0,0 +1,298 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.common.configuration.LockManagerConfiguration; +import com.uber.marmaray.common.configuration.ZookeeperConfiguration; +import com.uber.marmaray.utilities.LockManager; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +import org.apache.curator.test.TestingServer; + + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@Slf4j +public class TestLockManager { + + @NonNull + private static final Configuration conf = new Configuration(); + + @NonNull + private static TestingServer zkServer; + + @BeforeClass + public static void startZK() throws Exception { + zkServer = new TestingServer(); + zkServer.start(); + conf.setProperty(LockManagerConfiguration.IS_ENABLED, "true"); + conf.setProperty(LockManagerConfiguration.ZK_BASE_PATH, "/////test/////lock_manager////"); + conf.setProperty(ZookeeperConfiguration.ZK_QUORUM, zkServer.getConnectString()); + conf.setProperty(ZookeeperConfiguration.ZK_PORT, Integer.toString(zkServer.getPort())); + 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)); + } + + @AfterClass + public static void closeZK() throws Exception { + zkServer.close(); + } + + @Test + public void testLockKeyGen() throws Exception { + final String lockKey = LockManager.getLockKey("part1", "part2", "part3"); + assertEquals("/part1/part2/part3", lockKey); + } + + @Test + public void testLockHappyUser() throws Exception { + + final MultiThreadTestCoordinator coordinator = new MultiThreadTestCoordinator(); + final String key1 = LockManager.getLockKey("test_lock", "key1"); + + final Callable task1 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(0); + assertTrue(lockManager.lock(key1, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(2); + assertTrue(lockManager.unlock(key1)); + } finally { + coordinator.nextStep(); + return null; + } + } + }; + + final Callable task2 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(1); + assertFalse(lockManager.lock(key1, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(3); + assertTrue(lockManager.lock(key1, "")); + } finally { + coordinator.nextStep(); + return null; + } + } + }; + + final ExecutorService service = Executors.newFixedThreadPool(2); + final Future result1 = service.submit(task1); + final Future result2 = service.submit(task2); + + service.shutdown(); + service.awaitTermination(2, TimeUnit.MINUTES); + + result1.get(); + result2.get(); + + final LockManager lockManager = new LockManager(conf); + coordinator.waitUntilStep(4); + assertTrue(lockManager.lock(key1, "")); + lockManager.close(); + } + + @Test + public void testLockLostNode() throws Exception { + final MultiThreadTestCoordinator coordinator = new MultiThreadTestCoordinator(); + final String key2 = LockManager.getLockKey("test_lock", "key2"); + final String FAIL_MESSAGE = "Failing on purpose"; + + final Callable task1 = new Callable() { + @Override + public Void call() throws Exception { + final LockManager lockManager = new LockManager(conf); + try { + coordinator.waitUntilStep(0); + assertTrue(lockManager.lock(key2, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(2); + throw new Exception(FAIL_MESSAGE); + } finally { + lockManager.close(); + coordinator.nextStep(); + } + } + }; + + final Callable task2 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(1); + assertFalse(lockManager.lock(key2, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(3); + assertTrue(lockManager.lock(key2, "")); + coordinator.nextStep(); + return null; + } + } + }; + + final ExecutorService service = Executors.newFixedThreadPool(2); + + final Future result1 = service.submit(task1); + final Future result2 = service.submit(task2); + + service.shutdown(); + service.awaitTermination(2, TimeUnit.MINUTES); + + try { + result1.get(); + fail("The first task should fail on purpose."); + } catch (Exception e) { + assertTrue(e.getMessage().contains(FAIL_MESSAGE)); + } + result2.get(); + } + + @Test + public void testBatchLock() throws Exception { + final String key1 = LockManager.getLockKey("test_lock", "batchkey1"); + final String key2 = LockManager.getLockKey("test_lock", "batchkey2"); + final List lockKeyList = Arrays.asList(key1, key2); + + try (final LockManager lockManager = new LockManager(conf)) { + assertTrue(lockManager.batchLock(lockKeyList, "")); + } + + final MultiThreadTestCoordinator coordinator = new MultiThreadTestCoordinator(); + + + // Order of these three tasks + // Step 0. task1: lock(key1)-success + // Step 1. task2: lock(key1,key2)-failure + // Step 2. task3: lock(key2)-success + // task3: unlock(key2) + // Step 3. task1: unlock(key1) + // Step 4. task2: lock(key1,key2)-success + + final Callable task1 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(0); + assertTrue(lockManager.lock(key1, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(3); + } + coordinator.nextStep(); + return null; + } + }; + + final Callable task2 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(1); + assertFalse(lockManager.batchLock(lockKeyList, "")); + coordinator.nextStep(); + coordinator.waitUntilStep(4); + assertTrue(lockManager.batchLock(lockKeyList, "")); + coordinator.nextStep(); + } + return null; + } + }; + + final Callable task3 = new Callable() { + @Override + public Void call() throws Exception { + try (final LockManager lockManager = new LockManager(conf)) { + coordinator.waitUntilStep(2); + assertTrue(lockManager.lock(key2,"")); + } + coordinator.nextStep(); + return null; + } + }; + + final ExecutorService service = Executors.newFixedThreadPool(3); + + final Future result1 = service.submit(task1); + final Future result2 = service.submit(task2); + final Future result3 = service.submit(task3); + + service.shutdown(); + service.awaitTermination(2, TimeUnit.MINUTES); + + result1.get(); + result2.get(); + result3.get(); + } + + @Test + public void testLockInfo() throws Exception { + + final String LOCK_INFO = "This is the information in the lock"; + final String key1 = LockManager.getLockKey("test_lock", "key1"); + + try (final LockManager lockManager = new LockManager(conf)) { + + assertTrue(lockManager.lock(key1, LOCK_INFO)); + + final LockManagerConfiguration lockConf = new LockManagerConfiguration(conf); + final ZookeeperConfiguration zkConf = new ZookeeperConfiguration(conf); + final CuratorFramework client = CuratorFrameworkFactory.builder() + .connectString(zkConf.getZkQuorum()) + .retryPolicy(new BoundedExponentialBackoffRetry(1000, 5000, 3)) + .namespace(lockConf.getZkBasePath()) + .sessionTimeoutMs(lockConf.getZkSessionTimeoutMs()) + .connectionTimeoutMs(lockConf.getZkConnectionTimeoutMs()) + .build(); + + client.start(); + final String node = client.getChildren().forPath(key1).get(0); + final String data = new String(client.getData().forPath(key1 + "/" + node)); + assertEquals(LOCK_INFO, data); + client.close(); + + lockManager.unlock(key1); + } + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/common/util/TestMapUtil.java b/marmaray/src/test/java/com/uber/marmaray/common/util/TestMapUtil.java new file mode 100644 index 0000000..fb2dd81 --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestMapUtil.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.util; + +import com.uber.marmaray.utilities.MapUtil; +import java.util.HashMap; +import java.util.Map; +import org.junit.Assert; +import org.junit.Test; + +public class TestMapUtil { + + @Test + public void testSerDeserMap() { + final Map map = new HashMap<>(); + final int mapEntries = 10; + for (int i = 0; i < mapEntries; i++) { + map.put("prefixKey" + i + MapUtil.KEY_VALUE_SEPARATOR + + MapUtil.KEYS_SEPARATOR + "suffix", + "prefixValue" + i + MapUtil.KEY_VALUE_SEPARATOR + + MapUtil.KEYS_SEPARATOR + "suffix"); + } + Assert.assertEquals(map.entrySet(), MapUtil.deserializeMap(MapUtil.serializeMap(map)).entrySet()); + } +} 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 new file mode 100644 index 0000000..ef5f2fb --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestParquetWriterUtil.java @@ -0,0 +1,80 @@ +/* + * 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.util; + +import com.uber.marmaray.utilities.FSUtils; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.ParquetWriter; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; + +public class TestParquetWriterUtil { + private FileSystem fileSystem; + + @Before + 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()); + } + + @Test + public void writeTestParquetFile() throws IOException { + final String filePath = FileTestUtil.getTempFolder(); + final Configuration conf = new Configuration(); + final String avscFile = FileHelperUtil.getResourcePath(getClass(), + new Path("schemas", "StringPair.avsc").toString()); + + final Path dataFilePath = new Path(filePath, "data.parquet"); + + try (final InputStream fis = new FileInputStream(avscFile)) { + final Schema.Parser parser = new Schema.Parser(); + final Schema schema = parser.parse(fis); + + + + final ParquetWriter writer = ParquetWriterUtil.initializeAvroWriter(dataFilePath, schema, conf); + final GenericRecord record = new GenericData.Record(schema); + record.put("left", "L"); + record.put("right", "R"); + + writer.write(record); + writer.close(); + + Assert.assertTrue(this.fileSystem.exists(dataFilePath)); + FileStatus[] fs = this.fileSystem.globStatus(dataFilePath); + + // Validate that the file isn't empty and data was actually written + Assert.assertTrue(fs[0].getLen() > 0); + } finally { + this.fileSystem.delete(dataFilePath); + } + } +} 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 new file mode 100644 index 0000000..c25eb9c --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/common/util/TestSchemaUtil.java @@ -0,0 +1,64 @@ +/* + * 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.util; + +import com.uber.marmaray.common.configuration.Configuration; +import com.uber.marmaray.utilities.FSUtils; +import com.uber.marmaray.utilities.SchemaUtil; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +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()); + } + + @Test + public void testGenerateSchemaFromParquet() throws IOException { + final StructType structType = SchemaUtil.generateSchemaFromParquet(fs, this.parquetDataPath); + Assert.assertEquals(2, structType.fields().length); + validate(structType); + } + + private void validate(final StructType structType) { + int i = 0; + + for (StructField f : structType.fields()) { + if (i == 0) { + Assert.assertEquals("left", f.name()); + } else if (i == 1) { + Assert.assertEquals("right", f.name()); + } + i++; + } + } + +} diff --git a/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java b/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java new file mode 100644 index 0000000..2c6717f --- /dev/null +++ b/marmaray/src/test/java/com/uber/marmaray/utilities/TestKafkaUtil.java @@ -0,0 +1,102 @@ +/* + * 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; + +import com.uber.marmaray.common.exceptions.JobRuntimeException; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.mockito.stubbing.Answer; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.HashMap; +import java.util.HashSet; +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.stream.IntStream; + +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({KafkaUtil.class}) +public class TestKafkaUtil { + + @Test + public void testRequestTimeout() { + mockStatic(KafkaUtil.class); + final String topicName = "testtopic"; + final Set topicPartitions = new HashSet(); + final List partitionInfos = new LinkedList<>(); + final int numPartitions = 5; + final int fetchTimeoutSec = 10; + final int fetchRetryCnt = 3; + when(KafkaUtil.getFetchOffsetRetryCnt()).thenReturn(fetchRetryCnt); + when(KafkaUtil.getFetchOffsetTimeoutSec()).thenReturn(fetchTimeoutSec); + + + IntStream.range(0, numPartitions).forEach( + i -> { + topicPartitions.add(new TopicPartition(topicName, i)); + partitionInfos.add(new PartitionInfo(topicName, i, null, null, null)); + } + ); + final Map attempts = new HashMap(); + final KafkaConsumer kafkaConsumer = mock(KafkaConsumer.class); + when(kafkaConsumer.partitionsFor(Matchers.anyString())).thenReturn(partitionInfos); + when(KafkaUtil.getTopicPartitionOffsets(Matchers.any(KafkaConsumer.class), + Matchers.anyString(), Matchers.any(Set.class))).thenCallRealMethod(); + when(kafkaConsumer.position(Matchers.any(TopicPartition.class))).thenAnswer( + (Answer) invocationOnMock -> { + 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))); + } + return tp.partition() * 2L; + } + ); + try { + KafkaUtil.getTopicPartitionOffsets(kafkaConsumer, topicName, topicPartitions); + Assert.fail("exception is expected"); + } catch (JobRuntimeException e) { + // ignore it. + } + Assert.assertEquals(fetchRetryCnt, attempts.get(numPartitions-1).get()); + IntStream.range(0, numPartitions - 1).forEach( + i -> { + Assert.assertTrue(!attempts.containsKey(i) || attempts.get(i).get() < 2); + } + ); + } +} diff --git a/marmaray/src/test/resources/cassandra.yaml b/marmaray/src/test/resources/cassandra.yaml new file mode 100644 index 0000000..977a383 --- /dev/null +++ b/marmaray/src/test/resources/cassandra.yaml @@ -0,0 +1,586 @@ +# Cassandra storage config YAML + +# NOTE: +# See http://wiki.apache.org/cassandra/StorageConfiguration for +# full explanations of configuration directives +# /NOTE + +# The name of the cluster. This is mainly used to prevent machines in +# one logical cluster from joining another. +cluster_name: 'Test Cluster' + +# You should always specify InitialToken when setting up a production +# cluster for the first time, and often when adding capacity later. +# The principle is that each node should be given an equal slice of +# the token ring; see http://wiki.apache.org/cassandra/Operations +# for more details. +# +# If blank, Cassandra will request a token bisecting the range of +# the heaviest-loaded existing node. If there is no load information +# available, such as is the case with a new cluster, it will pick +# a random token, which will lead to hot spots. +#initial_token: + +# See http://wiki.apache.org/cassandra/HintedHandoff +hinted_handoff_enabled: true +# this defines the maximum amount of time a dead host will have hints +# generated. After it has been dead this long, new hints for it will not be +# created until it has been seen alive and gone down again. +max_hint_window_in_ms: 10800000 # 3 hours +# Maximum throttle in KBs per second, per delivery thread. This will be +# reduced proportionally to the number of nodes in the cluster. (If there +# are two nodes in the cluster, each delivery thread will use the maximum +# rate; if there are three, each will throttle to half of the maximum, +# since we expect two nodes to be delivering hints simultaneously.) +hinted_handoff_throttle_in_kb: 1024 +# Number of threads with which to deliver hints; +# Consider increasing this number when you have multi-dc deployments, since +# cross-dc handoff tends to be slower +max_hints_delivery_threads: 2 + +# The following setting populates the page cache on memtable flush and compaction +# WARNING: Enable this setting only when the whole node's data fits in memory. +# Defaults to: false +# populate_io_cache_on_flush: false + +# Authentication backend, implementing IAuthenticator; used to identify users +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator, +# PasswordAuthenticator}. +# +# - AllowAllAuthenticator performs no checks - set it to disable authentication. +# - PasswordAuthenticator relies on username/password pairs to authenticate +# users. It keeps usernames and hashed passwords in system_auth.credentials table. +# Please increase system_auth keyspace replication factor if you use this authenticator. +authenticator: AllowAllAuthenticator + +# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer, +# CassandraAuthorizer}. +# +# - AllowAllAuthorizer allows any action to any user - set it to disable authorization. +# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +authorizer: AllowAllAuthorizer + +# Validity period for permissions cache (fetching permissions can be an +# expensive operation depending on the authorizer, CassandraAuthorizer is +# one example). Defaults to 2000, set to 0 to disable. +# Will be disabled automatically for AllowAllAuthorizer. +permissions_validity_in_ms: 2000 + + +# The partitioner is responsible for distributing rows (by key) across +# nodes in the cluster. Any IPartitioner may be used, including your/m +# own as long as it is on the classpath. Out of the box, Cassandra +# provides org.apache.cassandra.dht.{Murmur3Partitioner, RandomPartitioner +# ByteOrderedPartitioner, OrderPreservingPartitioner (deprecated)}. +# +# - RandomPartitioner distributes rows across the cluster evenly by md5. +# This is the default prior to 1.2 and is retained for compatibility. +# - Murmur3Partitioner is similar to RandomPartioner but uses Murmur3_128 +# Hash Function instead of md5. When in doubt, this is the best option. +# - ByteOrderedPartitioner orders rows lexically by key bytes. BOP allows +# scanning rows in key order, but the ordering can generate hot spots +# for sequential insertion workloads. +# - OrderPreservingPartitioner is an obsolete form of BOP, that stores +# - keys in a less-efficient format and only works with keys that are +# UTF8-encoded Strings. +# - CollatingOPP collates according to EN,US rules rather than lexical byte +# ordering. Use this as an example if you need custom collation. +# +# See http://wiki.apache.org/cassandra/Operations for more on +# partitioners and token selection. +partitioner: org.apache.cassandra.dht.Murmur3Partitioner + +# directories where Cassandra should store data on disk. +data_file_directories: + - target/embeddedCassandra/data + +# commit log +commitlog_directory: target/embeddedCassandra/commitlog + +# policy for data disk failures: +# stop: shut down gossip and Thrift, leaving the node effectively dead, but +# can still be inspected via JMX. +# best_effort: stop using the failed disk and respond to requests based on +# remaining available sstables. This means you WILL see obsolete +# data at CL.ONE! +# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra +disk_failure_policy: stop + + +# Maximum size of the key cache in memory. +# +# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the +# minimum, sometimes more. The key cache is fairly tiny for the amount of +# time it saves, so it's worthwhile to use it at large numbers. +# The row cache saves even more time, but must store the whole values of +# its rows, so it is extremely space-intensive. It's best to only use the +# row cache if you have hot rows or static rows. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache. +key_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# safe the keys cache. Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 14400 or 4 hours. +key_cache_save_period: 14400 + +# Number of keys from the key cache to save +# Disabled by default, meaning all keys are going to be saved +# key_cache_keys_to_save: 100 + +# Maximum size of the row cache in memory. +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is 0, to disable row caching. +row_cache_size_in_mb: 0 + +# Duration in seconds after which Cassandra should +# safe the row cache. Caches are saved to saved_caches_directory as specified +# in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 0 to disable saving the row cache. +row_cache_save_period: 0 + +# Number of keys from the row cache to save +# Disabled by default, meaning all keys are going to be saved +# row_cache_keys_to_save: 100 + +# saved caches +saved_caches_directory: target/embeddedCassandra/saved_caches + +# commitlog_sync may be either "periodic" or "batch." +# When in batch mode, Cassandra won't ack writes until the commit log +# has been fsynced to disk. It will wait up to +# commitlog_sync_batch_window_in_ms milliseconds for other writes, before +# performing the sync. +# +# commitlog_sync: batch +# commitlog_sync_batch_window_in_ms: 50 +# +# the other option is "periodic" where writes may be acked immediately +# and the CommitLog is simply synced every commitlog_sync_period_in_ms +# milliseconds. +commitlog_sync: periodic +commitlog_sync_period_in_ms: 10000 + +# The size of the individual commitlog file segments. A commitlog +# segment may be archived, deleted, or recycled once all the data +# in it (potentially from each columnfamily in the system) has been +# flushed to sstables. +# +# The default size is 32, which is almost always fine, but if you are +# archiving commitlog segments (see commitlog_archiving.properties), +# then you probably want a finer granularity of archiving; 8 or 16 MB +# is reasonable. +commitlog_segment_size_in_mb: 32 + +# any class that implements the SeedProvider interface and has a +# constructor that takes a Map of parameters will do. +seed_provider: + # Addresses of hosts that are deemed contact points. + # Cassandra nodes use this list of hosts to find each other and learn + # the topology of the ring. You must change this if you are running + # multiple nodes! + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + # seeds is actually a comma-delimited list of addresses. + # Ex: ",," + - seeds: "127.0.0.1" + + +# For workloads with more data than can fit in memory, Cassandra's +# bottleneck will be reads that need to fetch data from +# disk. "concurrent_reads" should be set to (16 * number_of_drives) in +# order to allow the operations to enqueue low enough in the stack +# that the OS and drives can reorder them. +# +# On the other hand, since writes are almost never IO bound, the ideal +# number of "concurrent_writes" is dependent on the number of cores in +# your system; (8 * number_of_cores) is a good rule of thumb. +concurrent_reads: 32 +concurrent_writes: 32 + +# Total memory to use for memtables. Cassandra will flush the largest +# memtable when this much memory is used. +# If omitted, Cassandra will set it to 1/3 of the heap. +# memtable_total_space_in_mb: 2048 + +# Total space to use for commitlogs. +# If space gets above this value (it will round up to the next nearest +# segment multiple), Cassandra will flush every dirty CF in the oldest +# segment and remove it. +# commitlog_total_space_in_mb: 4096 + +# This sets the amount of memtable flush writer threads. These will +# be blocked by disk io, and each one will hold a memtable in memory +# while blocked. If you have a large heap and many data directories, +# you can increase this value for better flush performance. +# By default this will be set to the amount of data directories defined. +#memtable_flush_writers: 1 + +# the number of full memtables to allow pending flush, that is, +# waiting for a writer thread. At a minimum, this should be set to +# the maximum number of secondary indexes created on a single CF. +#memtable_flush_queue_size: 4 + +# Whether to, when doing sequential writing, fsync() at intervals in +# order to force the operating system to flush the dirty +# buffers. Enable this to avoid sudden dirty buffer flushing from +# impacting read latencies. Almost always a good idea on SSD:s; not +# necessarily on platters. +trickle_fsync: false +trickle_fsync_interval_in_kb: 10240 + +# TCP port, for commands and data +storage_port: 7010 + +# SSL port, for encrypted communication. Unused unless enabled in +# encryption_options +ssl_storage_port: 7011 + +# Address to bind to and tell other Cassandra nodes to connect to. You +# _must_ change this if you want multiple nodes to be able to +# communicate! +# +# Leaving it blank leaves it up to InetAddress.getLocalHost(). This +# will always do the Right Thing *if* the node is properly configured +# (hostname, name resolution, etc), and the Right Thing is to use the +# address associated with the hostname (it might not be). +# +# Setting this to 0.0.0.0 is always wrong. +listen_address: 127.0.0.1 + +start_native_transport: true +# port for the CQL native transport to listen for clients on +native_transport_port: 9142 + +# Whether to start the thrift rpc server. +start_rpc: true + +# Address to broadcast to other Cassandra nodes +# Leaving this blank will set it to the same value as listen_address +# broadcast_address: 1.2.3.4 + +# The address to bind the Thrift RPC service to -- clients connect +# here. Unlike ListenAddress above, you *can* specify 0.0.0.0 here if +# you want Thrift to listen on all interfaces. +# +# Leaving this blank has the same effect it does for ListenAddress, +# (i.e. it will be based on the configured hostname of the node). +rpc_address: localhost +# port for Thrift to listen for clients on +rpc_port: 9171 + +# enable or disable keepalive on rpc connections +rpc_keepalive: true + +# Cassandra provides three options for the RPC Server: +# +# sync -> One connection per thread in the rpc pool (see below). +# For a very large number of clients, memory will be your limiting +# factor; on a 64 bit JVM, 128KB is the minimum stack size per thread. +# Connection pooling is very, very strongly recommended. +# +# async -> Nonblocking server implementation with one thread to serve +# rpc connections. This is not recommended for high throughput use +# cases. Async has been tested to be about 50% slower than sync +# or hsha and is deprecated: it will be removed in the next major release. +# +# hsha -> Stands for "half synchronous, half asynchronous." The rpc thread pool +# (see below) is used to manage requests, but the threads are multiplexed +# across the different clients. +# +# The default is sync because on Windows hsha is about 30% slower. On Linux, +# sync/hsha performance is about the same, with hsha of course using less memory. +rpc_server_type: sync + +# Uncomment rpc_min|max|thread to set request pool size. +# You would primarily set max for the sync server to safeguard against +# misbehaved clients; if you do hit the max, Cassandra will block until one +# disconnects before accepting more. The defaults for sync are min of 16 and max +# unlimited. +# +# For the Hsha server, the min and max both default to quadruple the number of +# CPU cores. +# +# This configuration is ignored by the async server. +# +# rpc_min_threads: 16 +# rpc_max_threads: 2048 + +# uncomment to set socket buffer sizes on rpc connections +# rpc_send_buff_size_in_bytes: +# rpc_recv_buff_size_in_bytes: + +# Frame size for thrift (maximum field length). +# 0 disables TFramedTransport in favor of TSocket. This option +# is deprecated; we strongly recommend using Framed mode. +thrift_framed_transport_size_in_mb: 15 + +# The max length of a thrift message, including all fields and +# internal thrift overhead. +thrift_max_message_length_in_mb: 16 + +# Set to true to have Cassandra create a hard link to each sstable +# flushed or streamed locally in a backups/ subdirectory of the +# Keyspace data. Removing these links is the operator's +# responsibility. +incremental_backups: false + +# Whether or not to take a snapshot before each compaction. Be +# careful using this option, since Cassandra won't clean up the +# snapshots for you. Mostly useful if you're paranoid when there +# is a data format change. +snapshot_before_compaction: false + +# Whether or not a snapshot is taken of the data before keyspace truncation +# or dropping of column families. The STRONGLY advised default of true +# should be used to provide data safety. If you set this flag to false, you will +# lose data on truncation or drop. +auto_snapshot: false + +# Add column indexes to a row after its contents reach this size. +# Increase if your column values are large, or if you have a very large +# number of columns. The competing causes are, Cassandra has to +# deserialize this much of the row to read a single column, so you want +# it to be small - at least if you do many partial-row reads - but all +# the index data is read for each access, so you don't want to generate +# that wastefully either. +column_index_size_in_kb: 64 + +# Size limit for rows being compacted in memory. Larger rows will spill +# over to disk and use a slower two-pass compaction process. A message +# will be logged specifying the row key. +#in_memory_compaction_limit_in_mb: 64 + +# Number of simultaneous compactions to allow, NOT including +# validation "compactions" for anti-entropy repair. Simultaneous +# compactions can help preserve read performance in a mixed read/write +# workload, by mitigating the tendency of small sstables to accumulate +# during a single long running compactions. The default is usually +# fine and if you experience problems with compaction running too +# slowly or too fast, you should look at +# compaction_throughput_mb_per_sec first. +# +# This setting has no effect on LeveledCompactionStrategy. +# +# concurrent_compactors defaults to the number of cores. +# Uncomment to make compaction mono-threaded, the pre-0.8 default. +#concurrent_compactors: 1 + +# Multi-threaded compaction. When enabled, each compaction will use +# up to one thread per core, plus one thread per sstable being merged. +# This is usually only useful for SSD-based hardware: otherwise, +# your concern is usually to get compaction to do LESS i/o (see: +# compaction_throughput_mb_per_sec), not more. +#multithreaded_compaction: false + +# Throttles compaction to the given total throughput across the entire +# system. The faster you insert data, the faster you need to compact in +# order to keep the sstable count down, but in general, setting this to +# 16 to 32 times the rate you are inserting data is more than sufficient. +# Setting this to 0 disables throttling. Note that this account for all types +# of compaction, including validation compaction. +compaction_throughput_mb_per_sec: 16 + +# Track cached row keys during compaction, and re-cache their new +# positions in the compacted sstable. Disable if you use really large +# key caches. +#compaction_preheat_key_cache: true + +# Throttles all outbound streaming file transfers on this node to the +# given total throughput in Mbps. This is necessary because Cassandra does +# mostly sequential IO when streaming data during bootstrap or repair, which +# can lead to saturating the network connection and degrading rpc performance. +# When unset, the default is 200 Mbps or 25 MB/s. +# stream_throughput_outbound_megabits_per_sec: 200 + +# How long the coordinator should wait for read operations to complete +read_request_timeout_in_ms: 5000 +# How long the coordinator should wait for seq or index scans to complete +range_request_timeout_in_ms: 20000 +# How long the coordinator should wait for writes to complete +write_request_timeout_in_ms: 2000 +# How long a coordinator should continue to retry a CAS operation +# that contends with other proposals for the same row +cas_contention_timeout_in_ms: 1000 +# How long the coordinator should wait for truncates to complete +# (This can be much longer, because unless auto_snapshot is disabled +# we need to flush first so we can snapshot before removing the data.) +truncate_request_timeout_in_ms: 60000 +# The default timeout for other, miscellaneous operations +request_timeout_in_ms: 20000 + +# Enable operation timeout information exchange between nodes to accurately +# measure request timeouts. If disabled, replicas will assume that requests +# were forwarded to them instantly by the coordinator, which means that +# under overload conditions we will waste that much extra time processing +# already-timed-out requests. +# +# Warning: before enabling this property make sure to ntp is installed +# and the times are synchronized between the nodes. +cross_node_timeout: false + +# Enable socket timeout for streaming operation. +# When a timeout occurs during streaming, streaming is retried from the start +# of the current file. This _can_ involve re-streaming an important amount of +# data, so you should avoid setting the value too low. +# Default value is 0, which never timeout streams. +# streaming_socket_timeout_in_ms: 0 + +# phi value that must be reached for a host to be marked down. +# most users should never need to adjust this. +# phi_convict_threshold: 8 + +# endpoint_snitch -- Set this to a class that implements +# IEndpointSnitch. The snitch has two functions: +# - it teaches Cassandra enough about your network topology to route +# requests efficiently +# - it allows Cassandra to spread replicas around your cluster to avoid +# correlated failures. It does this by grouping machines into +# "datacenters" and "racks." Cassandra will do its best not to have +# more than one replica on the same "rack" (which may not actually +# be a physical location) +# +# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER, +# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS +# ARE PLACED. +# +# Out of the box, Cassandra provides +# - SimpleSnitch: +# Treats Strategy order as proximity. This improves cache locality +# when disabling read repair, which can further improve throughput. +# Only appropriate for single-datacenter deployments. +# - PropertyFileSnitch: +# Proximity is determined by rack and data center, which are +# explicitly configured in cassandra-topology.properties. +# - RackInferringSnitch: +# Proximity is determined by rack and data center, which are +# assumed to correspond to the 3rd and 2nd octet of each node's +# IP address, respectively. Unless this happens to match your +# deployment conventions (as it did Facebook's), this is best used +# as an example of writing a custom Snitch class. +# - Ec2Snitch: +# Appropriate for EC2 deployments in a single Region. Loads Region +# and Availability Zone information from the EC2 API. The Region is +# treated as the Datacenter, and the Availability Zone as the rack. +# Only private IPs are used, so this will not work across multiple +# Regions. +# - Ec2MultiRegionSnitch: +# Uses public IPs as broadcast_address to allow cross-region +# connectivity. (Thus, you should set seed addresses to the public +# IP as well.) You will need to open the storage_port or +# ssl_storage_port on the public IP firewall. (For intra-Region +# traffic, Cassandra will switch to the private IP after +# establishing a connection.) +# +# You can use a custom Snitch by setting this to the full class name +# of the snitch, which will be assumed to be on your classpath. +endpoint_snitch: SimpleSnitch + +# controls how often to perform the more expensive part of host score +# calculation +dynamic_snitch_update_interval_in_ms: 100 +# controls how often to reset all host scores, allowing a bad host to +# possibly recover +dynamic_snitch_reset_interval_in_ms: 600000 +# if set greater than zero and read_repair_chance is < 1.0, this will allow +# 'pinning' of replicas to hosts in order to increase cache capacity. +# The badness threshold will control how much worse the pinned host has to be +# before the dynamic snitch will prefer other replicas over it. This is +# expressed as a double which represents a percentage. Thus, a value of +# 0.2 means Cassandra would continue to prefer the static snitch values +# until the pinned host was 20% worse than the fastest. +dynamic_snitch_badness_threshold: 0.1 + +# request_scheduler -- Set this to a class that implements +# RequestScheduler, which will schedule incoming client requests +# according to the specific policy. This is useful for multi-tenancy +# with a single Cassandra cluster. +# NOTE: This is specifically for requests from the client and does +# not affect inter node communication. +# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place +# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of +# client requests to a node with a separate queue for each +# request_scheduler_id. The scheduler is further customized by +# request_scheduler_options as described below. +request_scheduler: org.apache.cassandra.scheduler.NoScheduler + +# Scheduler Options vary based on the type of scheduler +# NoScheduler - Has no options +# RoundRobin +# - throttle_limit -- The throttle_limit is the number of in-flight +# requests per client. Requests beyond +# that limit are queued up until +# running requests can complete. +# The value of 80 here is twice the number of +# concurrent_reads + concurrent_writes. +# - default_weight -- default_weight is optional and allows for +# overriding the default which is 1. +# - weights -- Weights are optional and will default to 1 or the +# overridden default_weight. The weight translates into how +# many requests are handled during each turn of the +# RoundRobin, based on the scheduler id. +# +# request_scheduler_options: +# throttle_limit: 80 +# default_weight: 5 +# weights: +# Keyspace1: 1 +# Keyspace2: 5 + +# request_scheduler_id -- An identifer based on which to perform +# the request scheduling. Currently the only valid option is keyspace. +# request_scheduler_id: keyspace + +# index_interval controls the sampling of entries from the primrary +# row index in terms of space versus time. The larger the interval, +# the smaller and less effective the sampling will be. In technicial +# terms, the interval coresponds to the number of index entries that +# are skipped between taking each sample. All the sampled entries +# must fit in memory. Generally, a value between 128 and 512 here +# coupled with a large key cache size on CFs results in the best trade +# offs. This value is not often changed, however if you have many +# very small rows (many to an OS page), then increasing this will +# often lower memory usage without a impact on performance. +index_interval: 128 + +# Enable or disable inter-node encryption +# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that +# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher +# suite for authentication, key exchange and encryption of the actual data transfers. +# NOTE: No custom encryption options are enabled at the moment +# The available internode options are : all, none, dc, rack +# +# If set to dc cassandra will encrypt the traffic between the DCs +# If set to rack cassandra will encrypt the traffic between the racks +# +# The passwords used in these options must match the passwords used when generating +# the keystore and truststore. For instructions on generating these files, see: +# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# +encryption_options: + internode_encryption: none + keystore: conf/.keystore + keystore_password: cassandra + truststore: conf/.truststore + truststore_password: cassandra + # More advanced defaults below: + # protocol: TLS + # algorithm: SunX509 + # store_type: JKS + # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA] diff --git a/marmaray/src/test/resources/config.yaml b/marmaray/src/test/resources/config.yaml new file mode 100644 index 0000000..8509ec8 --- /dev/null +++ b/marmaray/src/test/resources/config.yaml @@ -0,0 +1,16 @@ +--- + marmaray: + scalar_configs: + integer: 1 + long: 1234567890123 + double: 1.23 + boolean: true + string_configs: + string1: "string1" + stringlist: + " string2 ": " string2 " + retry_strategy: + default_strategy: "SimpleRetryStrategy" + simple: + num_of_retries: 3 + wait_time_in_ms: 1000 diff --git a/marmaray/src/test/resources/configWithScopes.yaml b/marmaray/src/test/resources/configWithScopes.yaml new file mode 100644 index 0000000..7a8e5fe --- /dev/null +++ b/marmaray/src/test/resources/configWithScopes.yaml @@ -0,0 +1,59 @@ +scope_override_map: + bootstrap: default + incremental: default + +database: + connection: + type: "jdbc" + port: 5000 + +hbase: + connection: + host: ["hadoopzkfoo1", "hadoopzkfoo2"] + port: 4000 + +bootstrap: + spark_properties: + spark.executor.memory: "8g" + spark.driver.memory: "8g" + spark.network.timeout: "120s" + marmaray: + hoodie: + tables: + non_primitive_field_from_bootstrap: + heya: 3 + bye: 4 + target_table: + parallelism: 1000 + combine_before_insert: false + +incremental: + spark_properties: + spark.executor.memory: "4g" + spark.driver.memory: "4g" + spark.network.timeout: "100s" + marmaray: + hoodie: + tables: + target_table: + metrics_prefix: "hoover" + enable_metrics: true + parallelism: 100 + combine_before_insert: true + combine_before_upsert: true + +default: + marmaray: + hoodie: + tables: + non_primitive_field_from_default: + hello: 1 + hi: 2 + target_table: + parallelism: 10 + parquet_max_file_size: 2147483647 + +# parallelism is overridden +# parquet_max_file_size, foo, bar are a primitive fields inherited +# non_primitive_field_from_default is a non-primitive field inherited +# combine_before_insert, combine_before_upsert are primitive field present in override scopes diff --git a/marmaray/src/test/resources/datacenter/datacenter b/marmaray/src/test/resources/datacenter/datacenter new file mode 100644 index 0000000..e3c1d9b --- /dev/null +++ b/marmaray/src/test/resources/datacenter/datacenter @@ -0,0 +1 @@ +test_dc diff --git a/marmaray/src/test/resources/expectedConfigWithBootstrapScope.yaml b/marmaray/src/test/resources/expectedConfigWithBootstrapScope.yaml new file mode 100644 index 0000000..14a3f08 --- /dev/null +++ b/marmaray/src/test/resources/expectedConfigWithBootstrapScope.yaml @@ -0,0 +1,34 @@ +database: + connection: + type: "jdbc" + port: 5000 + +hbase: + connection: + host: ["hadoopzkfoo1", "hadoopzkfoo2"] + port: 4000 + +spark_properties: + spark.executor.memory: "8g" + spark.driver.memory: "8g" + spark.network.timeout: "120s" + +marmaray: + hoodie: + tables: + non_primitive_field_from_default: + hello: 1 + hi: 2 + non_primitive_field_from_bootstrap: + heya: 3 + bye: 4 + target_table: + parallelism: 1000 + combine_before_insert: false + parquet_max_file_size: 2147483647 + +# parallelism is overridden +# parquet_max_file_size, foo, bar are a primitive fields inherited +# non_primitive_field_from_default is a non-primitive field inherited +# combine_before_insert, combine_before_upsert are primitive fields retained in override scope +# non_primitive_field_from_default is a non-primitive field retained in override scope diff --git a/marmaray/src/test/resources/expectedConfigWithIncrementalScope.yaml b/marmaray/src/test/resources/expectedConfigWithIncrementalScope.yaml new file mode 100644 index 0000000..9cd61e7 --- /dev/null +++ b/marmaray/src/test/resources/expectedConfigWithIncrementalScope.yaml @@ -0,0 +1,33 @@ +database: + connection: + type: "jdbc" + port: 5000 + +hbase: + connection: + host: ["hadoopzkfoo1", "hadoopzkfoo2"] + port: 4000 + +spark_properties: + spark.executor.memory: "4g" + spark.driver.memory: "4g" + spark.network.timeout: "100s" + +marmaray: + hoodie: + tables: + non_primitive_field_from_default: + hello: 1 + hi: 2 + target_table: + metrics_prefix: "hoover" + enable_metrics: true + parallelism: 100 + combine_before_insert: true + combine_before_upsert: true + parquet_max_file_size: 2147483647 + +# parallelism is overridden +# parquet_max_file_size, foo, bar are a primitive fields inherited +# non_primitive_field_from_default is a non-primitive field inherited +# combine_before_insert, combine_before_upsert are primitive fields retained in override scope diff --git a/marmaray/src/test/resources/log4j-surefire.properties b/marmaray/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000..bc54b7f --- /dev/null +++ b/marmaray/src/test/resources/log4j-surefire.properties @@ -0,0 +1,10 @@ +log4j.rootLogger=INFO, A1 +log4j.category.com.uber=INFO +log4j.category.com.uber.hoodie.common.utils=WARN +log4j.category.org.apache.parquet.hadoop=WARN + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/marmaray/src/test/resources/schemas/StringPair.avsc b/marmaray/src/test/resources/schemas/StringPair.avsc new file mode 100644 index 0000000..d590b50 --- /dev/null +++ b/marmaray/src/test/resources/schemas/StringPair.avsc @@ -0,0 +1,9 @@ +{ + "type": "record", + "name": "StringPair", + "doc": "A pair of strings.", + "fields": [ + {"name": "left", "type": "string"}, + {"name": "right", "type": "string"} + ] +} diff --git a/marmaray/src/test/resources/setupTable.cql b/marmaray/src/test/resources/setupTable.cql new file mode 100644 index 0000000..03aa9da --- /dev/null +++ b/marmaray/src/test/resources/setupTable.cql @@ -0,0 +1,5 @@ +CREATE KEYSPACE IF NOT EXISTS marmaray WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; + +USE marmaray; + +CREATE TABLE IF NOT EXISTS devtable4 ( astr text PRIMARY KEY ) ; diff --git a/marmaray/src/test/resources/teardownTable.cql b/marmaray/src/test/resources/teardownTable.cql new file mode 100644 index 0000000..a87cf7a --- /dev/null +++ b/marmaray/src/test/resources/teardownTable.cql @@ -0,0 +1,4 @@ +USE marmaray; + +DROP TABLE IF EXISTS crossfit_gyms; +DROP KEYSPACE IF EXISTS marmaray; diff --git a/marmaray/src/test/resources/testData/testPartition/data.parquet b/marmaray/src/test/resources/testData/testPartition/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 new file mode 100644 index 0000000..47605ec --- /dev/null +++ b/pom.xml @@ -0,0 +1,724 @@ + + 4.0.0 + com.uber.marmaray + marmaray-base + pom + 1.0-SNAPSHOT + marmaray + http://maven.apache.org + + + marmaray + marmaray-tools + + + + 3.5 + 3.0.2 + 2.5.3 + 1.8 + 1.8 + 1.8 + 1.16.16 + 19.0 + 2.11 + 4.12 + 2.16 + 6.9 + 3.0.10 + 2.1.0 + + 2.1.0 + 1.2.17 + 2.19.1 + 1.10.19 + 1.7.3 + 5.7.2 + 2.6.0 + 1.1.0 + 2.6 + 1.2 + 0.4.1 + 4.5.2 + 3.1.3.2 + 3.2.0 + 3.2.0 + 1.2 + 2.8.2 + 2.0.0 + 0.10.2.1 + 0.7.9 + 1.0.5 + 2.3 + 1.3.2 + 3.21.0-GA + 4.2 + 1.11.327 + 0.2.4 + + + + + default + + true + + + test + + + + skipTestCompileAndRun + + + maven.test.skip + true + + + + none + + + + skipTestRun + + + skipTests + + + + none + + + + + + + + cloudera-repo-releases + https://repository.cloudera.com/artifactory/public/ + + + + + + central + http://artifactory.uber.internal:4587/artifactory/libs-release-local/ + + + snapshots + http://artifactory.uber.internal:4587/artifactory/libs-snapshot-local/ + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${maven.checkstyle.version} + + + com.puppycrawl.tools + checkstyle + ${com.puppycrawl.tools.version} + + + + checkstyles/marmaray-0.0.1.xml + checkstyle.suppression.file=checkstyles/suppressions.xml + true + true + false + **/gen-java/**,**/vehicles/**,**/treatment/**,**/products/**,**/generated/** + + + + process-sources + + check + + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + ${maven.compiler.source} + ${java.version} + + + + org.apache.maven.plugins + maven-surefire-plugin + ${maven-surefire-plugin.version} + + + file:${project.build.testOutputDirectory}/log4j-surefire.properties + + true + + + + **/IT*.java + + + + + org.apache.maven.plugins + maven-release-plugin + ${maven-release-plugin.version} + + true + false + release + deploy + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + + + test-jar + + test-compile + + + + + maven-assembly-plugin + + + package + + single + + + + + + jar-with-dependencies + + + + + org.jacoco + jacoco-maven-plugin + ${jacoco.version} + + + default-prepare-agent + + prepare-agent + + + + ${project.build.directory}/coverage-reports/jacoco-ut.exec + + + + default-report + test + + report + + + + ${project.build.directory}/coverage-reports/jacoco-ut.exec + + ${project.reporting.outputDirectory}/jacoco-ut + + + + + + org.fortasoft + gradle-maven-plugin + ${gradle-maven-plugin.version} + + + jacobo + + ${gradle.version} + + + + ${gradle-maven-plugin.phase} + + invoke + + + + + + + + + + + + com.amazonaws + aws-java-sdk-bom + ${aws.jdk.bom.version} + pom + import + + + + + + org.projectlombok + lombok + ${lombok.version} + + + org.javassist + javassist + ${javaassist.version} + + + log4j + log4j + ${log4j.version} + + + org.hamcrest + hamcrest-core + 1.3 + test + + + org.mockito + mockito-all + test + ${mockito.version} + + + org.hamcrest + hamcrest-core + + + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.hamcrest + hamcrest-core + + + + + org.powermock + powermock-api-mockito + ${powermock.version} + test + + + org.hamcrest + hamcrest-core + + + + + junit + junit + ${junit.version} + test + + + org.hamcrest + hamcrest-core + + + + + + com.google.guava + guava + ${google.guava.version} + + + + org.apache.spark + spark-core_${scala.libversion} + ${spark.version} + + + log4j + log4j + + + com.google.guava + guava + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-common + + + provided + + + + org.apache.parquet + parquet-avro + 1.8.1 + + + + org.apache.spark + spark-sql_${scala.libversion} + ${spark.version} + + + log4j + log4j + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-common + + + provided + + + org.apache.spark + spark-hive_${scala.libversion} + ${spark.version} + provided + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + org.apache.kafka + kafka_2.11 + ${kafka.version} + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.spark + spark-streaming_${scala.libversion} + ${spark.version} + provided + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.libversion} + ${spark.version} + + + + org.apache.hadoop + hadoop-common + ${hadoop.version}-cdh${cdh.version} + provided + + + com.google.guava + guava + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version}-cdh${cdh.version} + provided + + + com.google.guava + guava + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + org.apache.hadoop + hadoop-auth + ${hadoop.version}-cdh${cdh.version} + provided + + + org.apache.hive + hive-common + ${hive.version}-cdh${cdh.version} + provided + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version}-cdh${cdh.version} + provided + + + org.apache.hadoop + hadoop-mapreduce-client-common + ${hadoop.version}-cdh${cdh.version} + provided + + + org.apache.hive + hive-exec + ${hive.version}-cdh${cdh.version} + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + commons-logging + commons-logging + ${commons.logging.version} + + + + + org.apache.cassandra + cassandra-all + ${cassandra.version} + + + org.slf4j + slf4j-log4j12 + + + org.slf4j + log4j-over-slf4j + + + log4j + log4j + + + com.google.guava + guava + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + com.datastax.cassandra + cassandra-driver-core + ${cassandra.driver.version} + + + com.google.guava + guava + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-common + + + + + com.holdenkarau + spark-testing-base_2.11 + ${spark.test.version}_0.6.0 + test + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + + + + org.apache.hbase + hbase-client + ${hbase.client.version} + + + + + com.uber.hoodie + hoodie-client + ${hoodie.version} + + + com.google.guava + guava + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + org.apache.hbase + hbase-client + + + + + com.uber.hoodie + hoodie-common + ${hoodie.version} + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + + org.cassandraunit + cassandra-unit-shaded + ${cassandra.unit.version} + test + + + org.apache.httpcomponents + httpclient + ${http.client.version} + + + com.squareup.okhttp3 + okhttp + ${okhttp3.version} + + + commons-cli + commons-cli + ${apache.commons.cli.version} + + + org.apache.logging.log4j + log4j-iostreams + ${apache.log4j.iostreams.version} + + + com.github.rholder + guava-retrying + ${guava.retry.version} + + + + com.opencsv + opencsv + ${opencsv.version} + + + + com.amazonaws + aws-java-sdk-s3 + + + io.findify + s3mock_2.11 + ${mock.s3.version} + test + + +