From 09435994d3020446facba9e803e96fae6d6879e1 Mon Sep 17 00:00:00 2001 From: "Ma,Gang" Date: Mon, 10 Dec 2018 11:37:42 +0800 Subject: [PATCH] KYLIN-3690 New streaming backend implementation --- assembly/pom.xml | 4 + build/bin/kylin.sh | 109 ++ build/script/prepare-libs.sh | 3 + .../apache/kylin/common/KylinConfigBase.java | 133 ++ .../apache/kylin/common/KylinConfigExt.java | 2 +- .../kylin/common/debug/BackdoorToggles.java | 14 + .../common/persistence/ResourceStore.java | 1 + .../apache/kylin/common/util/ServerMode.java | 74 + .../apache/kylin/common/util/TimeUtil.java | 36 +- .../org/apache/kylin/cube/CubeSegment.java | 33 + .../org/apache/kylin/cube/model/CubeDesc.java | 4 + .../kylin/dict/DictionaryGenerator.java | 13 +- .../MultipleDictionaryValueEnumerator.java | 9 +- ...MultipleDictionaryValueEnumeratorTest.java | 13 +- .../job/constant/ExecutableConstants.java | 4 + .../kylin/measure/topn/TopNAggregator.java | 8 + .../kylin/measure/topn/TopNCounter.java | 6 + .../kylin/metadata/model/ISourceAware.java | 2 + .../kylin/metadata/model/IStorageAware.java | 1 + .../kylin/metadata/model/TableDesc.java | 8 + .../metadata/project/ProjectManager.java | 7 + .../apache/kylin/storage/StorageFactory.java | 13 + dev-support/checkstyle-suppressions.xml | 2 + .../cube/test_streaming_v2_cube.json | 17 + .../cube_desc/test_streaming_v2_cube.json | 139 ++ .../test_case_data/localmeta/kylin.properties | 1 + .../model_desc/test_streaming_v2_model.json | 30 + .../localmeta/project/default.json | 7 + .../table/DEFAULT.STREAMING_V2_TABLE.json | 47 + pom.xml | 34 +- server-base/pom.xml | 43 + .../StreamingCoordinatorController.java | 186 +++ .../controller/StreamingV2Controller.java | 524 +++++++ .../rest/request/StreamingRequestV2.java | 73 + .../kylin/rest/service/CubeService.java | 53 +- .../service/StreamingCoordinatorService.java | 117 ++ .../rest/service/StreamingV2Service.java | 570 ++++++++ server/src/main/resources/kylinSecurity.xml | 2 + .../hbase/util/DeployCoprocessorCLI.java | 2 +- storage-stream/pom.xml | 87 ++ .../kylin/storage/stream/StreamStorage.java | 65 + .../storage/stream/StreamStorageQuery.java | 130 ++ .../rpc/HttpStreamDataSearchClient.java | 328 +++++ .../stream/rpc/IStreamDataSearchClient.java | 45 + stream-coordinator/pom.xml | 83 ++ .../kylin/stream/coordinator/Coordinator.java | 1294 +++++++++++++++++ .../coordinator/HBaseStreamMetadataStore.java | 178 +++ .../coordinator/MockStreamMetadataStore.java | 178 +++ .../coordinator/StreamMetadataStore.java | 117 ++ .../StreamMetadataStoreFactory.java | 63 + .../stream/coordinator/StreamingCubeInfo.java | 66 + .../kylin/stream/coordinator/ZKUtils.java | 71 + .../ZookeeperStreamMetadataStore.java | 578 ++++++++ .../stream/coordinator/assign/Assigner.java | 49 + .../coordinator/assign/AssignmentUtil.java | 87 ++ .../coordinator/assign/AssignmentsCache.java | 83 ++ .../CubePartitionRoundRobinAssigner.java | 322 ++++ .../coordinator/assign/DefaultAssigner.java | 210 +++ .../coordinator/client/CoordinatorClient.java | 56 + .../client/CoordinatorClientFactory.java | 55 + .../client/CoordinatorResponse.java | 58 + .../client/HttpCoordinatorClient.java | 297 ++++ .../exception/CoordinateException.java | 39 + .../NotLeadCoordinatorException.java | 40 + .../coordinator/exception/StoreException.java | 36 + .../CubePartitionRoundRobinAssignerTest.java | 262 ++++ .../assign/DefaultAssignerTest.java | 183 +++ stream-core/pom.xml | 103 ++ .../core/client/HttpReceiverAdminClient.java | 254 ++++ .../core/client/ReceiverAdminClient.java | 63 + .../exception/IllegalStorageException.java | 39 + .../core/exception/StreamingException.java | 38 + .../stream/core/metrics/StreamingMetrics.java | 51 + .../stream/core/model/AssignRequest.java | 59 + .../core/model/ConsumerStatsResponse.java | 40 + .../stream/core/model/CubeAssignment.java | 147 ++ .../kylin/stream/core/model/DataRequest.java | 144 ++ .../kylin/stream/core/model/DataResponse.java | 41 + .../stream/core/model/HealthCheckInfo.java | 35 + .../apache/kylin/stream/core/model/Node.java | 106 ++ .../core/model/PauseConsumersRequest.java | 36 + .../model/RemoteStoreCompleteRequest.java | 58 + .../kylin/stream/core/model/ReplicaSet.java | 104 ++ .../model/ReplicaSetLeaderChangeRequest.java | 45 + .../core/model/ResumeConsumerRequest.java | 45 + .../stream/core/model/SegmentBuildState.java | 144 ++ .../core/model/StartConsumersRequest.java | 47 + .../core/model/StopConsumersRequest.java | 45 + .../core/model/StreamingCubeConsumeState.java | 23 + .../stream/core/model/StreamingMessage.java | 79 + .../stream/core/model/UnAssignRequest.java | 36 + .../stream/core/model/stats/ClusterState.java | 75 + .../core/model/stats/ConsumerStats.java | 94 ++ .../core/model/stats/CubeRealTimeState.java | 52 + .../core/model/stats/LongLatencyInfo.java | 85 ++ .../model/stats/PartitionConsumeStats.java | 76 + .../stats/ReceiverCubeRealTimeState.java | 47 + .../core/model/stats/ReceiverCubeStats.java | 100 ++ .../core/model/stats/ReceiverState.java | 82 ++ .../core/model/stats/ReceiverStats.java | 80 + .../core/model/stats/ReplicaSetState.java | 92 ++ .../stream/core/model/stats/SegmentStats.java | 91 ++ .../core/model/stats/SegmentStoreStats.java | 47 + .../core/query/HavingFilterChecker.java | 135 ++ .../core/query/IStreamingGTSearcher.java | 25 + .../core/query/IStreamingSearchResult.java | 59 + .../query/MultiThreadsResultCollector.java | 161 ++ .../stream/core/query/RecordsAggregator.java | 149 ++ .../core/query/ResponseResultSchema.java | 157 ++ .../stream/core/query/ResultCollector.java | 76 + .../query/SingleThreadResultCollector.java | 75 + .../StreamingBuiltInFunctionTransformer.java | 272 ++++ .../core/query/StreamingCubeDataSearcher.java | 196 +++ .../core/query/StreamingDataQueryPlanner.java | 108 ++ .../core/query/StreamingQueryProfile.java | 194 +++ .../core/query/StreamingSearchContext.java | 161 ++ .../core/query/StreamingTupleConverter.java | 121 ++ .../core/query/StreamingTupleIterator.java | 126 ++ .../stream/core/source/ISourcePosition.java | 46 + .../core/source/ISourcePositionHandler.java | 38 + .../core/source/IStreamingMessageParser.java | 25 + .../stream/core/source/IStreamingSource.java | 56 + .../core/source/MessageFormatException.java | 37 + .../stream/core/source/MessageParserInfo.java | 61 + .../kylin/stream/core/source/Partition.java | 96 ++ .../core/source/StreamingSourceConfig.java | 99 ++ .../source/StreamingSourceConfigManager.java | 185 +++ .../core/source/StreamingSourceFactory.java | 44 + .../core/source/StreamingTableSourceInfo.java | 49 + .../kylin/stream/core/storage/CheckPoint.java | 135 ++ .../stream/core/storage/CheckPointStore.java | 225 +++ .../core/storage/IStreamingSegmentStore.java | 57 + .../kylin/stream/core/storage/Record.java | 68 + .../core/storage/StreamingCubeSegment.java | 187 +++ .../storage/columnar/ColumnDataReader.java | 28 + .../storage/columnar/ColumnDataWriter.java | 27 + .../ColumnarMemoryStorePersister.java | 406 ++++++ .../columnar/ColumnarMetricsEncoding.java | 37 + .../ColumnarMetricsEncodingFactory.java | 181 +++ .../storage/columnar/ColumnarRecordCodec.java | 105 ++ .../columnar/ColumnarSegmentStore.java | 465 ++++++ .../ColumnarSegmentStoreFilesSearcher.java | 104 ++ .../storage/columnar/ColumnarStoreCache.java | 172 +++ .../columnar/ColumnarStoreCacheStats.java | 95 ++ .../columnar/ColumnarStoreDimDesc.java | 93 ++ .../columnar/ColumnarStoreMetricsDesc.java | 87 ++ .../storage/columnar/DataSegmentFragment.java | 159 ++ .../FSInputGeneralColumnDataReader.java | 77 + .../columnar/FragmentCuboidReader.java | 179 +++ .../core/storage/columnar/FragmentData.java | 111 ++ .../columnar/FragmentFileSearcher.java | 360 +++++ .../storage/columnar/FragmentFilesMerger.java | 624 ++++++++ .../core/storage/columnar/FragmentId.java | 89 ++ .../columnar/FragmentSearchResult.java | 483 ++++++ .../columnar/FragmentsMergeResult.java | 53 + .../columnar/GeneralColumnDataReader.java | 106 ++ .../columnar/GeneralColumnDataWriter.java | 60 + .../columnar/ParsedStreamingCubeInfo.java | 257 ++++ .../core/storage/columnar/RawRecord.java | 77 + .../columnar/StringArrayComparator.java | 50 + .../columnar/TimeDerivedColumnEncoding.java | 42 + .../columnar/compress/Compression.java | 23 + .../FSInputLZ4CompressedColumnReader.java | 115 ++ .../FSInputNoCompressedColumnReader.java | 83 ++ .../FSInputRLECompressedColumnReader.java | 124 ++ .../compress/LZ4CompressedColumnReader.java | 125 ++ .../compress/LZ4CompressedColumnWriter.java | 78 + .../columnar/compress/LZ4CompressorTest.java | 61 + .../compress/NoCompressedColumnReader.java | 82 ++ .../compress/NoCompressedColumnWriter.java | 40 + .../RunLengthCompressedColumnReader.java | 153 ++ .../RunLengthCompressedColumnWriter.java | 130 ++ .../invertindex/ColInvertIndexSearcher.java | 169 +++ .../invertindex/ColInvertIndexWriter.java | 43 + .../FixLenColInvertIndexWriter.java | 87 ++ .../invertindex/FixLenIIColumnDescriptor.java | 33 + .../invertindex/IIColumnDescriptor.java | 33 + .../invertindex/IndexSearchResult.java | 30 + .../invertindex/InvertIndexSearcher.java | 341 +++++ .../invertindex/SeqColInvertIndexWriter.java | 111 ++ .../invertindex/SeqIIColumnDescriptor.java | 33 + .../columnar/protocol/CuboidMetaInfo.java | 114 ++ .../protocol/DimDictionaryMetaInfo.java | 93 ++ .../columnar/protocol/DimensionMetaInfo.java | 140 ++ .../storage/columnar/protocol/Footer.java | 87 ++ .../columnar/protocol/FragmentMetaInfo.java | 171 +++ .../columnar/protocol/MetricMetaInfo.java | 142 ++ .../storage/rocksdb/RocksDBSegmentStore.java | 136 ++ .../util/CompareFilterTimeRangeChecker.java | 137 ++ .../kylin/stream/core/util/Constants.java | 24 + .../kylin/stream/core/util/HDFSUtil.java | 86 ++ .../stream/core/util/NamedThreadFactory.java | 45 + .../kylin/stream/core/util/NodeUtil.java | 58 + .../stream/core/util/RecordsSerializer.java | 126 ++ .../kylin/stream/core/util/RestService.java | 123 ++ .../kylin/stream/core/util/RetryCallable.java | 35 + .../kylin/stream/core/util/RetryCaller.java | 83 ++ .../stream/core/util/StreamFilterUtil.java | 56 + .../core/util/TimeDerivedColumnType.java | 219 +++ .../core/storage/CheckPointStoreTest.java | 142 ++ .../core/storage/MockPositionHandler.java | 72 + .../stream/core/storage/RecordsSerDeTest.java | 91 ++ .../storage/StreamingSegmentManagerTest.java | 180 +++ .../kylin/stream/core/storage/TestHelper.java | 158 ++ .../columnar/ColumnarSegmentStoreTest.java | 206 +++ .../columnar/FragmentCuboidReaderTest.java | 134 ++ .../columnar/FragmentFileSearcherTest.java | 208 +++ .../columnar/FragmentFilesMergerTest.java | 229 +++ .../columnar/GeneralColumnDataTest.java | 81 ++ .../columnar/SegmentMemoryStoreTest.java | 252 ++++ .../columnar/StreamingDataSimulator.java | 134 ++ .../compress/LZ4CompressColumnTest.java | 107 ++ .../compress/NoCompressColumnTest.java | 99 ++ .../compress/RunLengthCompressColumnTest.java | 168 +++ .../ColInvertIndexWriterWriterTest.java | 79 + .../SearchableColInvertIndexWriterTest.java | 40 + .../SimpleColInvertIndexWriterTest.java | 72 + .../FragmentCuboidReaderPerfTest.java | 203 +++ .../columnar/performance/PerfDataPrepare.java | 86 ++ .../columnar/performance/PerformanceTest.java | 163 +++ .../StreamingCubeDataSearcherPerfTest.java | 185 +++ .../kylin/stream/core/util/DataGenerator.java | 79 + .../core/util/TimeDerivedColumnTypeTest.java | 118 ++ stream-receiver/pom.xml | 308 ++++ .../server/ReplicaSetLeaderSelector.java | 100 ++ .../kylin/stream/server/ServerContext.java | 43 + .../stream/server/StreamingReceiver.java | 147 ++ .../kylin/stream/server/StreamingServer.java | 731 ++++++++++ .../rest/controller/AdminController.java | 165 +++ .../rest/controller/BasicController.java | 69 + .../rest/controller/DataController.java | 168 +++ .../rest/controller/QueryController.java | 104 ++ .../rest/controller/StatsController.java | 68 + .../rest/controller/SystemController.java | 87 ++ .../rest/exception/BadRequestException.java | 61 + .../rest/exception/ForbiddenException.java | 41 + .../exception/InternalErrorException.java | 63 + .../rest/exception/NotFoundException.java | 35 + .../server/rest/model/ErrorResponse.java | 38 + .../server/rest/model/PrepareSqlRequest.java | 121 ++ .../stream/server/rest/model/SQLRequest.java | 132 ++ .../stream/server/rest/model/SQLResponse.java | 167 +++ .../rest/security/StreamTableInterceptor.java | 64 + .../server/rest/service/BasicService.java | 55 + .../server/rest/service/QueryService.java | 274 ++++ .../stream/server/rest/util/QueryUtil.java | 225 +++ .../stream/server/retention/PolicyInfo.java | 51 + .../server/storage/LocalStreamStorage.java | 38 + .../storage/LocalStreamStorageQuery.java | 64 + .../src/main/resources/applicationContext.xml | 77 + .../src/main/resources/ehcache-test.xml | 37 + .../src/main/resources/ehcache.xml | 37 + .../stream-receiver-log4j.properties | 44 + .../src/main/webapp/WEB-INF/kylin-servlet.xml | 19 + stream-receiver/src/main/webapp/index.html | 23 + stream-source-kafka/pom.xml | 79 + .../source/kafka/BootstrapServerConfig.java | 54 + .../source/kafka/KafkaBatchSourceAdaptor.java | 43 + .../source/kafka/KafkaConsumerStartInfo.java | 43 + .../stream/source/kafka/KafkaPosition.java | 111 ++ .../source/kafka/KafkaPositionHandler.java | 86 ++ .../stream/source/kafka/KafkaSource.java | 285 ++++ .../source/kafka/KafkaTopicAssignment.java | 64 + .../source/kafka/TimedJsonStreamParser.java | 121 ++ .../source/kafka/consumer/KafkaConnector.java | 118 ++ .../KafkaSourcePositionHandlerTest.java | 70 + .../apache/kylin/tool/CubeMigrationCLI.java | 6 + .../kylin/tool/CubeMetaExtractorTest.java | 3 +- 268 files changed, 29831 insertions(+), 20 deletions(-) create mode 100644 core-common/src/main/java/org/apache/kylin/common/util/ServerMode.java create mode 100644 examples/test_case_data/localmeta/cube/test_streaming_v2_cube.json create mode 100644 examples/test_case_data/localmeta/cube_desc/test_streaming_v2_cube.json create mode 100644 examples/test_case_data/localmeta/model_desc/test_streaming_v2_model.json create mode 100644 examples/test_case_data/localmeta/table/DEFAULT.STREAMING_V2_TABLE.json create mode 100644 server-base/src/main/java/org/apache/kylin/rest/controller/StreamingCoordinatorController.java create mode 100644 server-base/src/main/java/org/apache/kylin/rest/controller/StreamingV2Controller.java create mode 100644 server-base/src/main/java/org/apache/kylin/rest/request/StreamingRequestV2.java create mode 100644 server-base/src/main/java/org/apache/kylin/rest/service/StreamingCoordinatorService.java create mode 100644 server-base/src/main/java/org/apache/kylin/rest/service/StreamingV2Service.java create mode 100644 storage-stream/pom.xml create mode 100644 storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorage.java create mode 100644 storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorageQuery.java create mode 100644 storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/HttpStreamDataSearchClient.java create mode 100644 storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/IStreamDataSearchClient.java create mode 100644 stream-coordinator/pom.xml create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/Coordinator.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/HBaseStreamMetadataStore.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/MockStreamMetadataStore.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStore.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStoreFactory.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamingCubeInfo.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZKUtils.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZookeeperStreamMetadataStore.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/Assigner.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentUtil.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentsCache.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssigner.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/DefaultAssigner.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClient.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClientFactory.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorResponse.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/HttpCoordinatorClient.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/CoordinateException.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/NotLeadCoordinatorException.java create mode 100644 stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/StoreException.java create mode 100644 stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssignerTest.java create mode 100644 stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/DefaultAssignerTest.java create mode 100644 stream-core/pom.xml create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/client/HttpReceiverAdminClient.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/client/ReceiverAdminClient.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/exception/IllegalStorageException.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/exception/StreamingException.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/metrics/StreamingMetrics.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/AssignRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/ConsumerStatsResponse.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/CubeAssignment.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/DataRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/DataResponse.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/HealthCheckInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/Node.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/PauseConsumersRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/RemoteStoreCompleteRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSet.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSetLeaderChangeRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/ResumeConsumerRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/SegmentBuildState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/StartConsumersRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/StopConsumersRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingCubeConsumeState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingMessage.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/UnAssignRequest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ClusterState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ConsumerStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/CubeRealTimeState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/LongLatencyInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/PartitionConsumeStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeRealTimeState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReplicaSetState.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStoreStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/HavingFilterChecker.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingGTSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingSearchResult.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/MultiThreadsResultCollector.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/RecordsAggregator.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/ResponseResultSchema.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/ResultCollector.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/SingleThreadResultCollector.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingBuiltInFunctionTransformer.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingCubeDataSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingDataQueryPlanner.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingQueryProfile.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingSearchContext.java create mode 100755 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleConverter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleIterator.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePosition.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePositionHandler.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingMessageParser.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingSource.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageFormatException.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageParserInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/Partition.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfig.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfigManager.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceFactory.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingTableSourceInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPoint.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPointStore.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/IStreamingSegmentStore.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/Record.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/StreamingCubeSegment.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMemoryStorePersister.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncoding.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncodingFactory.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarRecordCodec.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStore.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreFilesSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCache.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCacheStats.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreDimDesc.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreMetricsDesc.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/DataSegmentFragment.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentData.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMerger.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentId.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentSearchResult.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentsMergeResult.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ParsedStreamingCubeInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/RawRecord.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/StringArrayComparator.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/TimeDerivedColumnEncoding.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/Compression.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputNoCompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputRLECompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressorTest.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnReader.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenColInvertIndexWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenIIColumnDescriptor.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IIColumnDescriptor.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IndexSearchResult.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/InvertIndexSearcher.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqColInvertIndexWriter.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqIIColumnDescriptor.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/CuboidMetaInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimDictionaryMetaInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimensionMetaInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/Footer.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/FragmentMetaInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/MetricMetaInfo.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/storage/rocksdb/RocksDBSegmentStore.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/CompareFilterTimeRangeChecker.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/Constants.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/HDFSUtil.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/NamedThreadFactory.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/NodeUtil.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/RecordsSerializer.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/RestService.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCallable.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCaller.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/StreamFilterUtil.java create mode 100644 stream-core/src/main/java/org/apache/kylin/stream/core/util/TimeDerivedColumnType.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/CheckPointStoreTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/MockPositionHandler.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/RecordsSerDeTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/StreamingSegmentManagerTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/TestHelper.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReaderTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcherTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMergerTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/SegmentMemoryStoreTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/StreamingDataSimulator.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressColumnTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressColumnTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressColumnTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriterWriterTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SearchableColInvertIndexWriterTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SimpleColInvertIndexWriterTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/FragmentCuboidReaderPerfTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerfDataPrepare.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerformanceTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/StreamingCubeDataSearcherPerfTest.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/util/DataGenerator.java create mode 100644 stream-core/src/test/java/org/apache/kylin/stream/core/util/TimeDerivedColumnTypeTest.java create mode 100644 stream-receiver/pom.xml create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/ReplicaSetLeaderSelector.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/ServerContext.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingReceiver.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingServer.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/AdminController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/BasicController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/DataController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/QueryController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/StatsController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/SystemController.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/BadRequestException.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/ForbiddenException.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/InternalErrorException.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/NotFoundException.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/ErrorResponse.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/PrepareSqlRequest.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLRequest.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLResponse.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/security/StreamTableInterceptor.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/BasicService.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/QueryService.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/util/QueryUtil.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/retention/PolicyInfo.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorage.java create mode 100644 stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorageQuery.java create mode 100644 stream-receiver/src/main/resources/applicationContext.xml create mode 100644 stream-receiver/src/main/resources/ehcache-test.xml create mode 100644 stream-receiver/src/main/resources/ehcache.xml create mode 100644 stream-receiver/src/main/resources/stream-receiver-log4j.properties create mode 100644 stream-receiver/src/main/webapp/WEB-INF/kylin-servlet.xml create mode 100644 stream-receiver/src/main/webapp/index.html create mode 100644 stream-source-kafka/pom.xml create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/BootstrapServerConfig.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaBatchSourceAdaptor.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaConsumerStartInfo.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPosition.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPositionHandler.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaSource.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaTopicAssignment.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/TimedJsonStreamParser.java create mode 100644 stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/consumer/KafkaConnector.java create mode 100644 stream-source-kafka/src/test/java/org/apache/kylin/stream/coordinator/assign/KafkaSourcePositionHandlerTest.java diff --git a/assembly/pom.xml b/assembly/pom.xml index d8aac208d4c..5a1b9e1c0ed 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -50,6 +50,10 @@ org.apache.kylin kylin-storage-hbase + + org.apache.kylin + kylin-storage-stream + org.apache.kylin kylin-engine-mr diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh index edcc2ff6072..144ddc5a55e 100755 --- a/build/bin/kylin.sh +++ b/build/bin/kylin.sh @@ -193,6 +193,115 @@ then else quit "Kylin is not running" fi + +# streaming command +elif [ "$1" == "streaming" ] +then + if [ $# -lt 2 ] + then + echo "invalid input args $@" + exit -1 + fi + if [ "$2" == "start" ] + then + if [ -f "${KYLIN_HOME}/streaming_receiver_pid" ] + then + PID=`cat $KYLIN_HOME/streaming_receiver_pid` + if ps -p $PID > /dev/null + then + echo "Kylin is running, stop it first" + exit 1 + fi + fi + #retrive $hbase_dependency + source ${dir}/find-hbase-dependency.sh + #retrive $KYLIN_EXTRA_START_OPTS + if [ -f "${KYLIN_HOME}/conf/setenv.sh" ] + then source ${KYLIN_HOME}/conf/setenv.sh + fi + + mkdir -p ${KYLIN_HOME}/ext + HBASE_CLASSPATH=`hbase classpath` + #echo "hbase class path:"$HBASE_CLASSPATH + STREAM_CLASSPATH=${KYLIN_HOME}/lib/streaming/*:${KYLIN_HOME}/ext/*:${HBASE_CLASSPATH} + + # KYLIN_EXTRA_START_OPTS is for customized settings, checkout bin/setenv.sh + ${JAVA_HOME}/bin/java -cp $STREAM_CLASSPATH ${KYLIN_EXTRA_START_OPTS} \ + -Dlog4j.configuration=stream-receiver-log4j.properties\ + -DKYLIN_HOME=${KYLIN_HOME}\ + -Dkylin.hbase.dependency=${hbase_dependency} \ + org.apache.kylin.stream.server.StreamingReceiver $@ > ${KYLIN_HOME}/logs/streaming_receiver.out 2>&1 & echo $! > ${KYLIN_HOME}/streaming_receiver_pid & + exit 0 + elif [ "$2" == "stop" ] + then + if [ ! -f "${KYLIN_HOME}/streaming_receiver_pid" ] + then + echo "streaming is not running, please check" + exit 1 + fi + PID=`cat ${KYLIN_HOME}/streaming_receiver_pid` + if [ "$PID" = "" ] + then + echo "streaming is not running, please check" + exit 1 + else + echo "stopping streaming:$PID" + WAIT_TIME=2 + LOOP_COUNTER=20 + if ps -p $PID > /dev/null + then + echo "Stopping Kylin: $PID" + kill $PID + + for ((i=0; i<$LOOP_COUNTER; i++)) + do + # wait to process stopped + sleep $WAIT_TIME + if ps -p $PID > /dev/null ; then + echo "Stopping in progress. Will check after $WAIT_TIME secs again..." + continue; + else + break; + fi + done + + # if process is still around, use kill -9 + if ps -p $PID > /dev/null + then + echo "Initial kill failed, getting serious now..." + kill -9 $PID + sleep 1 #give kill -9 sometime to "kill" + if ps -p $PID > /dev/null + then + quit "Warning, even kill -9 failed, giving up! Sorry..." + fi + fi + + # process is killed , remove pid file + rm -rf ${KYLIN_HOME}/streaming_receiver_pid + echo "Kylin with pid ${PID} has been stopped." + exit 0 + else + quit "Kylin with pid ${PID} is not running" + fi + fi + elif [[ "$2" = org.apache.kylin.* ]] + then + source ${KYLIN_HOME}/conf/setenv.sh + HBASE_CLASSPATH=`hbase classpath` + #echo "hbase class path:"$HBASE_CLASSPATH + STREAM_CLASSPATH=${KYLIN_HOME}/lib/streaming/*:${KYLIN_HOME}/ext/*:${HBASE_CLASSPATH} + + shift + # KYLIN_EXTRA_START_OPTS is for customized settings, checkout bin/setenv.sh + ${JAVA_HOME}/bin/java -cp $STREAM_CLASSPATH ${KYLIN_EXTRA_START_OPTS} \ + -Dlog4j.configuration=stream-receiver-log4j.properties\ + -DKYLIN_HOME=${KYLIN_HOME}\ + -Dkylin.hbase.dependency=${hbase_dependency} \ + "$@" + exit 0 + fi + elif [ "$1" = "version" ] then retrieveDependency diff --git a/build/script/prepare-libs.sh b/build/script/prepare-libs.sh index 789a12047a2..d0773500ae6 100644 --- a/build/script/prepare-libs.sh +++ b/build/script/prepare-libs.sh @@ -30,11 +30,13 @@ echo "version ${version}" echo "copy lib file" rm -rf build/lib build/tool mkdir build/lib build/tool +mkdir build/lib/streaming cp assembly/target/kylin-assembly-${version}-job.jar build/lib/kylin-job-${version}.jar cp storage-hbase/target/kylin-storage-hbase-${version}-coprocessor.jar build/lib/kylin-coprocessor-${version}.jar cp jdbc/target/kylin-jdbc-${version}.jar build/lib/kylin-jdbc-${version}.jar cp tool-assembly/target/kylin-tool-assembly-${version}-assembly.jar build/tool/kylin-tool-${version}.jar cp datasource-sdk/target/kylin-datasource-sdk-${version}-lib.jar build/lib/kylin-datasource-sdk-${version}.jar +cp stream-receiver/target/kylin-stream-receiver-${version}-all.jar build/lib/streaming/kylin-stream-receiver-${version}-all.jar # Copied file becomes 000 for some env (e.g. my Cygwin) chmod 644 build/lib/kylin-job-${version}.jar @@ -42,3 +44,4 @@ chmod 644 build/lib/kylin-coprocessor-${version}.jar chmod 644 build/lib/kylin-jdbc-${version}.jar chmod 644 build/tool/kylin-tool-${version}.jar chmod 644 build/lib/kylin-datasource-sdk-${version}.jar +chmod 644 build/lib/streaming/kylin-stream-receiver-${version}-all.jar diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index c240e7e3c63..52211b695fc 100644 --- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -840,6 +840,8 @@ public Map getSourceEngines() { r.put(1, "org.apache.kylin.source.kafka.KafkaSource"); r.put(8, "org.apache.kylin.source.jdbc.JdbcSource"); r.put(16, "org.apache.kylin.source.jdbc.extensible.JdbcSource"); + r.put(20, "org.apache.kylin.stream.source.kafka.KafkaBatchSourceAdaptor"); + r.put(21, "org.apache.kylin.stream.source.kafka.KafkaBatchSourceAdaptor"); r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.source.provider."))); return r; } @@ -1031,6 +1033,7 @@ public Map getStorageEngines() { r.put(0, "org.apache.kylin.storage.hbase.HBaseStorage"); r.put(1, "org.apache.kylin.storage.hybrid.HybridStorage"); r.put(2, "org.apache.kylin.storage.hbase.HBaseStorage"); + r.put(3, "org.apache.kylin.storage.stream.StreamStorage"); r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider."))); return r; } @@ -1984,5 +1987,135 @@ public String getJdbcSourceAdaptor() { public boolean isLimitPushDownEnabled() { return Boolean.parseBoolean(getOptional("kylin.storage.limit-push-down-enabled", TRUE)); + + // ============================================================================ + // streaming + // ============================================================================ + public String getStreamingStoreClass() { + return getOptional("kylin.stream.store.class", "org.apache.kylin.stream.core.storage.columnar.ColumnarSegmentStore"); + } + + public String getStreamingBasicCuboidJobDFSBlockSize() { + return getOptional("kylin.stream.job.dfs.block.size", String.valueOf(16 * 1024 * 1024)); + } + + public String getStreamingIndexPath() { + return getOptional("kylin.stream.index.path", "stream_index"); + } + + public int getStreamingCubeConsumerTasksNum() { + return Integer.parseInt(getOptional("kylin.stream.cube-num-of-consumer-tasks", "3")); + } + + public int getStreamingCubeWindowInSecs() { + return Integer.parseInt(getOptional("kylin.stream.cube.window", "3600")); + } + + public int getStreamingCubeDurationInSecs() { + return Integer.parseInt(getOptional("kylin.stream.cube.duration", "7200")); + } + + public int getStreamingCubeMaxDurationInSecs() { + return Integer.parseInt(getOptional("kylin.stream.cube.duration.max", "43200")); + } + + public int getStreamingCheckPointFileMaxNum() { + return Integer.parseInt(getOptional("kylin.stream.checkpoint.file.max.num", "5")); + } + + public int getStreamingCheckPointIntervalsInSecs() { + return Integer.parseInt(getOptional("kylin.stream.index.checkpoint.intervals", "300")); + } + + public int getStreamingIndexMaxRows() { + return Integer.parseInt(getOptional("kylin.stream.index.maxrows", "50000")); + } + + public int getStreamingMaxImmutableSegments() { + return Integer.parseInt(getOptional("kylin.stream.immutable.segments.max.num", "100")); + } + + public boolean isStreamingConsumeFromLatestOffsets() { + return Boolean.parseBoolean(getOptional("kylin.stream.consume.offsets.latest", "true")); + } + + public String getStreamingNode() { + return getOptional("kylin.stream.node", null); + } + + public Map getStreamingNodeProperties() { + return getPropertiesByPrefix("kylin.stream.node"); + } + + public String getStreamingMetadataStoreType() { + return getOptional("kylin.stream.metadata.store.type", "zk"); + } + + public String getStreamingCoordinateZK() { + return getOptional("kylin.stream.zookeeper", null); + } + + public String getStreamingSegmentRetentionPolicy() { + return getOptional("kylin.stream.segment.retention.policy", "fullBuild"); + } + + public int getStreamingReceiverHttpMaxThreads() { + return Integer.parseInt(getOptional("kylin.stream.receiver.http.max.threads", "200")); + } + + public int getStreamingReceiverHttpMinThreads() { + return Integer.parseInt(getOptional("kylin.stream.receiver.http.min.threads", "10")); + } + + public int getStreamingReceiverQueryCoreThreads() { + return Integer.parseInt(getOptional("kylin.stream.receiver.query-core-threads", "50")); + } + + public int getStreamingReceiverQueryMaxThreads() { + return Integer.parseInt(getOptional("kylin.stream.receiver.query-max-threads", "200")); + } + + public int getStreamingReceiverUseThreadsPerQuery() { + return Integer.parseInt(getOptional("kylin.stream.receiver.use-threads-per-query", "8")); + } + + public int getStreamingRPCHttpConnTimeout() { + return Integer.parseInt(getOptional("kylin.stream.rpc.http.connect.timeout", "10000")); + } + + public int getStreamingRPCHttpReadTimeout() { + return Integer.parseInt(getOptional("kylin.stream.rpc.http.read.timeout", "60000")); + } + + public boolean isStreamingBuildAdditionalCuboids() { + return Boolean.parseBoolean(getOptional("kylin.stream.build.additional.cuboids", "false")); + } + + public Map getStreamingSegmentRetentionPolicyProperties(String policyName) { + return getPropertiesByPrefix("kylin.stream.segment.retention.policy." + policyName + "."); + } + + public int getStreamingMaxFragmentsInSegment() { + return Integer.parseInt(getOptional("kylin.stream.segment-max-fragments", "50")); + } + + public int getStreamingMinFragmentsInSegment() { + return Integer.parseInt(getOptional("kylin.stream.segment-min-fragments", "15")); + } + + public int getStreamingMaxFragmentSizeInMb() { + return Integer.parseInt(getOptional("kylin.stream.max-fragment-size-mb", "300")); + } + + public boolean isStreamingFragmentsAutoMergeEnabled() { + return Boolean.parseBoolean(getOptional("kylin.stream.fragments-auto-merge-enable", "true")); + } + + public boolean isStreamingConcurrentScanEnabled() { + return Boolean.parseBoolean(getOptional("kylin.stream.segment.concurrent.scan", "false")); + } + + public String getLocalStorageImpl() { + return getOptional("kylin.stream.settled.storage", null); } } diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigExt.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigExt.java index ae89200d51d..02c81068b98 100644 --- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigExt.java +++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigExt.java @@ -55,7 +55,7 @@ private KylinConfigExt(KylinConfigExt ext, Map overrides) { this.overrides = BCC.check(overrides); } - protected String getOptional(String prop, String dft) { + public String getOptional(String prop, String dft) { String value = overrides.get(prop); if (value != null) return StrSubstitutor.replace(value, System.getenv()); diff --git a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java index 210c1f86644..038e4432c6f 100644 --- a/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java +++ b/core-common/src/main/java/org/apache/kylin/common/debug/BackdoorToggles.java @@ -119,6 +119,10 @@ public static boolean getHtraceEnabled() { return getBoolean(DEBUG_TOGGLE_HTRACE_ENABLED); } + public static boolean isStreamingProfileEnable() { + return getBoolean(DEBUG_TOGGLE_STREAMING_DETAIL_PROFILE); + } + public static int getQueryTimeout() { String v = getString(DEBUG_TOGGLE_QUERY_TIMEOUT); if (v == null) @@ -369,4 +373,14 @@ public static Properties getJdbcDriverClientCalciteProps() { * extra calcite props from jdbc client */ public static final String JDBC_CLIENT_CALCITE_PROPS = "JDBC_CLIENT_CALCITE_PROPS"; + + /** + * set DEBUG_TOGGLE_STREAMING_PROFILE="true" to profile streaming query + * + example:(put it into request body) + "backdoorToggles": { + "DEBUG_TOGGLE_STREAMING_DETAIL_PROFILE": "true" + } + */ + public final static String DEBUG_TOGGLE_STREAMING_DETAIL_PROFILE = "DEBUG_TOGGLE_STREAMING_DETAIL_PROFILE"; } diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java index 5b3f196b7a5..67625ec31f5 100644 --- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java +++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java @@ -72,6 +72,7 @@ abstract public class ResourceStore { public static final String EXECUTE_RESOURCE_ROOT = "/execute"; public static final String EXECUTE_OUTPUT_RESOURCE_ROOT = "/execute_output"; public static final String STREAMING_RESOURCE_ROOT = "/streaming"; + public static final String STREAMING_V2_RESOURCE_ROOT = "/streaming_v2"; public static final String KAFKA_RESOURCE_ROOT = "/kafka"; public static final String STREAMING_OUTPUT_RESOURCE_ROOT = "/streaming_output"; public static final String CUBE_STATISTICS_ROOT = "/cube_statistics"; diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ServerMode.java b/core-common/src/main/java/org/apache/kylin/common/util/ServerMode.java new file mode 100644 index 00000000000..d58a44f5c03 --- /dev/null +++ b/core-common/src/main/java/org/apache/kylin/common/util/ServerMode.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.common.util; + +import java.util.List; +import java.util.Locale; + +import org.apache.kylin.common.KylinConfig; + +import com.google.common.collect.Lists; + +public class ServerMode { + public final static String SERVER_MODE_QUERY = "query"; + public final static String SERVER_MODE_JOB = "job"; + public final static String SERVER_MODE_STREAM_COORDINATOR = "stream_coordinator"; + public final static String SERVER_MODE_ALL = "all"; + + private List serverModes; + + public ServerMode(List serverModes) { + this.serverModes = serverModes; + } + + public boolean canServeQuery() { + return serverModes.contains(SERVER_MODE_ALL) || serverModes.contains(SERVER_MODE_QUERY); + } + + public boolean canServeJobBuild() { + return serverModes.contains(SERVER_MODE_ALL) || serverModes.contains(SERVER_MODE_JOB); + } + + public boolean canServeStreamingCoordinator() { + return serverModes.contains(SERVER_MODE_ALL) + || serverModes.contains(SERVER_MODE_STREAM_COORDINATOR); + } + + public boolean canServeAll() { + return serverModes.contains(SERVER_MODE_ALL); + } + + @Override + public String toString() { + return serverModes.toString(); + } + + public static ServerMode SERVER_MODE = getServerMode(); + + private static ServerMode getServerMode() { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + String serverModeStr = kylinConfig.getServerMode(); + List serverModes = Lists.newArrayList(); + String[] serverModeArray = serverModeStr.split("\\s*,\\s*"); + for (String serverMode : serverModeArray) { + serverModes.add(serverMode.toLowerCase(Locale.ROOT)); + } + return new ServerMode(serverModes); + } +} diff --git a/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java index bf3f3514400..e92d149beda 100644 --- a/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java +++ b/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java @@ -31,9 +31,9 @@ private TimeUtil() { } private static TimeZone gmt = TimeZone.getTimeZone("GMT"); - private static long ONE_MINUTE_TS = 60 * 1000L; - private static long ONE_HOUR_TS = 60 * ONE_MINUTE_TS; - private static long ONE_DAY_TS = 24 * ONE_HOUR_TS; + public static long ONE_MINUTE_TS = 60 * 1000L; + public static long ONE_HOUR_TS = 60 * ONE_MINUTE_TS; + public static long ONE_DAY_TS = 24 * ONE_HOUR_TS; public static long getMinuteStart(long ts) { return ts / ONE_MINUTE_TS * ONE_MINUTE_TS; @@ -83,4 +83,34 @@ public static long getYearStart(long ts) { return calendar.getTimeInMillis(); } + public static long getWeekEnd(long ts) { + Calendar calendar = Calendar.getInstance(gmt, Locale.ROOT); + calendar.setTimeInMillis(getWeekStart(ts)); + calendar.add(Calendar.DAY_OF_WEEK, 7); + return calendar.getTimeInMillis(); + } + + public static long getMonthEnd(long ts) { + Calendar calendar = Calendar.getInstance(gmt, Locale.ROOT); + calendar.setTimeInMillis(ts); + calendar.set(calendar.get(Calendar.YEAR), calendar.get(Calendar.MONDAY), calendar.get(Calendar.DAY_OF_MONTH), 0, 0, 0); + calendar.set(Calendar.DAY_OF_MONTH, calendar.getActualMaximum(Calendar.DAY_OF_MONTH)); + calendar.set(Calendar.HOUR_OF_DAY, 24); + return calendar.getTimeInMillis(); + } + + public static long getQuarterEnd(long ts) { + Calendar calendar = Calendar.getInstance(gmt, Locale.ROOT); + calendar.setTimeInMillis(getQuarterStart(ts)); + calendar.add(Calendar.MONTH, 3); + return calendar.getTimeInMillis(); + } + + public static long getYearEnd(long ts) { + Calendar calendar = Calendar.getInstance(gmt, Locale.ROOT); + calendar.setTimeInMillis(getYearStart(ts)); + calendar.add(Calendar.YEAR, 1); + return calendar.getTimeInMillis(); + } + } diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java index 90cce14d04e..7d336fb5ece 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java @@ -19,6 +19,7 @@ package org.apache.kylin.cube; import java.io.Serializable; +import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Collection; import java.util.LinkedHashMap; @@ -32,6 +33,7 @@ import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.common.util.Pair; import org.apache.kylin.common.util.ShardingHash; import org.apache.kylin.cube.cuboid.CuboidScheduler; import org.apache.kylin.cube.kv.CubeDimEncMap; @@ -116,6 +118,9 @@ public class CubeSegment implements IBuildable, ISegment, Serializable { @JsonInclude(JsonInclude.Include.NON_EMPTY) private Map sourcePartitionOffsetEnd = Maps.newHashMap(); + @JsonProperty("stream_source_checkpoint") + private String streamSourceCheckpoint; + @JsonProperty("additionalInfo") @JsonInclude(JsonInclude.Include.NON_EMPTY) private Map additionalInfo = new LinkedHashMap(); @@ -156,6 +161,26 @@ public static String makeSegmentName(TSRange tsRange, SegmentRange segRange, Dat return dateFormat.format(tsRange.start.v) + "_" + dateFormat.format(tsRange.end.v); } + public static Pair parseSegmentName(String segmentName) { + if ("FULL".equals(segmentName)) { + return new Pair<>(0L, 0L); + } + String[] startEnd = segmentName.split("_"); + if (startEnd.length != 2) { + throw new IllegalArgumentException("the segmentName is illegal: " + segmentName); + } + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.ROOT); + dateFormat.setTimeZone(TimeZone.getTimeZone("GMT")); + + try { + long dateRangeStart = dateFormat.parse(startEnd[0]).getTime(); + long dateRangeEnd = dateFormat.parse(startEnd[1]).getTime(); + return new Pair<>(dateRangeStart, dateRangeEnd); + } catch (ParseException e) { + throw new IllegalArgumentException("Invalid segmentName for CubeSegment, segmentName = " + segmentName); + } + } + // ============================================================================ public KylinConfig getConfig() { @@ -594,4 +619,12 @@ public Map getDimensionRangeInfoMap() { public void setDimensionRangeInfoMap(Map dimensionRangeInfoMap) { this.dimensionRangeInfoMap = dimensionRangeInfoMap; } + + public String getStreamSourceCheckpoint() { + return streamSourceCheckpoint; + } + + public void setStreamSourceCheckpoint(String streamSourceCheckpoint) { + this.streamSourceCheckpoint = streamSourceCheckpoint; + } } diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java index d1529c81e72..392c91543a8 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java @@ -1431,6 +1431,10 @@ public List getAllExtLookupSnapshotTypes() { return result; } + public boolean isStreamingCube() { + return getModel().getRootFactTable().getTableDesc().isStreamingTable(); + } + /** Get a column which can be used to cluster the source table. * To reduce memory footprint in base cuboid for global dict */ // TODO handle more than one ultra high cardinality columns use global dict in one cube diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java index 7c33b4a1e4b..6a501e9e123 100644 --- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java +++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java @@ -22,6 +22,8 @@ import java.util.ArrayList; import java.util.List; +import com.google.common.base.Function; +import com.google.common.collect.Lists; import org.apache.commons.lang.StringUtils; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.util.DateFormat; @@ -32,6 +34,8 @@ import com.google.common.base.Preconditions; +import javax.annotation.Nullable; + /** * @author yangli9 */ @@ -109,7 +113,14 @@ static Dictionary buildDictionary(IDictionaryBuilder builder, Dictionary } public static Dictionary mergeDictionaries(DataType dataType, List sourceDicts) throws IOException { - return buildDictionary(dataType, new MultipleDictionaryValueEnumerator(dataType, sourceDicts)); + List> dictList = Lists.transform(sourceDicts, new Function>() { + @Nullable + @Override + public Dictionary apply(@Nullable DictionaryInfo input) { + return input.dictionaryObject; + } + }); + return buildDictionary(dataType, new MultipleDictionaryValueEnumerator(dataType, dictList)); } private static class DateDictBuilder implements IDictionaryBuilder { diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java index c1686d33738..047b61c83ac 100644 --- a/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java +++ b/core-dictionary/src/main/java/org/apache/kylin/dict/MultipleDictionaryValueEnumerator.java @@ -36,13 +36,12 @@ public class MultipleDictionaryValueEnumerator implements IDictionaryValueEnumer private List> dictionaryList; private DataType dataType; - public MultipleDictionaryValueEnumerator(DataType dataType, List dictionaryInfoList) { + public MultipleDictionaryValueEnumerator(DataType dataType, List> dictionaryInfoList) { this.dataType = dataType; dictionaryList = Lists.newArrayListWithCapacity(dictionaryInfoList.size()); - for (DictionaryInfo dictInfo : dictionaryInfoList) { - Dictionary dictionary = (Dictionary) dictInfo.getDictionaryObject(); - dictionaryList.add((Dictionary) dictInfo.getDictionaryObject()); - curKeys.add(dictionary.getMinId()); + for (Dictionary dict : dictionaryInfoList) { + dictionaryList.add(dict); + curKeys.add(dict.getMinId()); } } diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java index a5e4564ba4d..d4a3fce5dd2 100644 --- a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java +++ b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java @@ -29,6 +29,8 @@ import java.util.ArrayList; import java.util.List; +import com.google.common.base.Function; +import com.google.common.collect.Lists; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.util.ClassUtil; import org.apache.kylin.common.util.Dictionary; @@ -37,6 +39,8 @@ import org.junit.BeforeClass; import org.junit.Test; +import javax.annotation.Nullable; + /** * Created by sunyerui on 16/8/2. */ @@ -58,7 +62,14 @@ private static DictionaryInfo createDictInfo(String[] values) { } private String[] enumerateDictInfoList(List dictionaryInfoList, String dataType) throws IOException { - enumerator = new MultipleDictionaryValueEnumerator(DataType.getType(dataType), dictionaryInfoList); + List> dictList = Lists.transform(dictionaryInfoList, new Function>() { + @Nullable + @Override + public Dictionary apply(@Nullable DictionaryInfo input) { + return input.dictionaryObject; + } + }); + enumerator = new MultipleDictionaryValueEnumerator(DataType.getType(dataType), dictList); List values = new ArrayList<>(); while (enumerator.moveNext()) { values.add(enumerator.current()); diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java index 5735a809754..a47ff182bb2 100644 --- a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java +++ b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java @@ -73,4 +73,8 @@ private ExecutableConstants() { public static final String STEP_NAME_MATERIALIZE_LOOKUP_TABLE_UPDATE_CUBE = "Update Cube Info"; public static final String SPARK_SPECIFIC_CONFIG_NAME_MERGE_DICTIONARY = "mergedict"; + + public static final String STEP_NAME_STREAMING_CREATE_DICTIONARY = "Build Dimension Dictionaries For Steaming Job"; + public static final String STEP_NAME_STREAMING_BUILD_BASE_CUBOID = "Build Base Cuboid Data For Streaming Job"; + public static final String STEP_NAME_STREAMING_SAVE_DICTS = "Save Cube Dictionaries"; } diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java index bc2bc3606c7..ae3cb2a14ef 100644 --- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java +++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java @@ -65,4 +65,12 @@ public int getMemBytesEstimate() { return 8 * capacity / 4; } + public TopNAggregator copy() { + TopNAggregator result = new TopNAggregator(); + result.capacity = this.capacity; + TopNCounter cpCounter = sum.copy(); + result.sum = cpCounter; + return result; + } + } diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java index 932248d47df..8037eac4401 100644 --- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java +++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java @@ -223,6 +223,12 @@ public double[] getCounters() { return counters; } + public TopNCounter copy() { + TopNCounter result = new TopNCounter(capacity); + result.counterMap = Maps.newHashMap(counterMap); + return result; + } + @Override public Iterator> iterator() { if (this.descending == true) { diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java index cfab0ed660f..b4e70d369c1 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java @@ -28,6 +28,8 @@ public interface ISourceAware { public static final int ID_EXTERNAL = 7; public static final int ID_JDBC = 8; public static final int ID_EXTENSIBLE_JDBC = 16; + public static final int ID_KAFKA = 20; + public static final int ID_KAFKA_HIVE = 21; int getSourceType(); diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java index e55257435bf..047d87f2ff2 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java @@ -23,6 +23,7 @@ public interface IStorageAware { public static final int ID_HBASE = 0; public static final int ID_HYBRID = 1; public static final int ID_SHARDED_HBASE = 2; + public static final int ID_REALTIME_AND_HBASE = 3; int getStorageType(); } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java index c6030983257..51fa6a0a448 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java @@ -426,4 +426,12 @@ public void setTableType(String tableType) { this.tableType = tableType; } + public boolean isStreamingTable() { + if (sourceType == ISourceAware.ID_KAFKA + || sourceType == ISourceAware.ID_KAFKA_HIVE) { + return true; + } + return false; + } + } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java index 77bad37acb3..62e33421de5 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java @@ -130,6 +130,13 @@ public ProjectInstance reloadProjectQuietly(String project) throws IOException { } } + public void reloadAll() throws IOException { + try (AutoLock lock = prjMapLock.lockForWrite()) { + crud.reloadAll(); + clearL2Cache(null); + } + } + public List listAllProjects() { try (AutoLock lock = prjMapLock.lockForRead()) { return new ArrayList(projectMap.values()); diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java index 55bdb4b913d..acdc2e949ff 100644 --- a/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java +++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java @@ -20,6 +20,7 @@ import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.threadlocal.InternalThreadLocal; +import org.apache.kylin.common.util.ClassUtil; import org.apache.kylin.common.util.ImplementationSwitch; import org.apache.kylin.metadata.model.IStorageAware; import org.apache.kylin.metadata.realization.IRealization; @@ -31,7 +32,19 @@ public class StorageFactory { // Use thread-local because KylinConfig can be thread-local and implementation might be different among multiple threads. private static InternalThreadLocal> storages = new InternalThreadLocal<>(); + private static IStorage configuredUseLocalStorage; + + static { + String localStorageImpl = KylinConfig.getInstanceFromEnv().getLocalStorageImpl(); + if (localStorageImpl != null){ + configuredUseLocalStorage = (IStorage) ClassUtil.newInstance(localStorageImpl); + } + } + public static IStorage storage(IStorageAware aware) { + if (configuredUseLocalStorage != null) { + return configuredUseLocalStorage; + } ImplementationSwitch current = storages.get(); if (storages.get() == null) { current = new ImplementationSwitch<>(KylinConfig.getInstanceFromEnv().getStorageEngines(), IStorage.class); diff --git a/dev-support/checkstyle-suppressions.xml b/dev-support/checkstyle-suppressions.xml index f110b7c3ef9..af30e61ff81 100644 --- a/dev-support/checkstyle-suppressions.xml +++ b/dev-support/checkstyle-suppressions.xml @@ -61,4 +61,6 @@ + + diff --git a/examples/test_case_data/localmeta/cube/test_streaming_v2_cube.json b/examples/test_case_data/localmeta/cube/test_streaming_v2_cube.json new file mode 100644 index 00000000000..feef9554a51 --- /dev/null +++ b/examples/test_case_data/localmeta/cube/test_streaming_v2_cube.json @@ -0,0 +1,17 @@ +{ + "uuid" : "3f81bc32-fa35-465e-92b0-a43195fa6de6", + "version" : "1.5.2", + "name" : "test_streaming_v2_cube", + "owner" : "ADMIN", + "cost" : 50, + "status" : "READY", + "segments" : [], + "cuboidBytes" : null, + "last_modified" : 1490005832153, + "descriptor" : "test_streaming_v2_cube", + "create_time_utc" : 1477870504616, + "cuboid_bytes_recommend" : null, + "input_records_size" : 242725, + "size_kb" : 9385, + "input_records_count" : 6588 +} \ No newline at end of file diff --git a/examples/test_case_data/localmeta/cube_desc/test_streaming_v2_cube.json b/examples/test_case_data/localmeta/cube_desc/test_streaming_v2_cube.json new file mode 100644 index 00000000000..7a40d1c3202 --- /dev/null +++ b/examples/test_case_data/localmeta/cube_desc/test_streaming_v2_cube.json @@ -0,0 +1,139 @@ +{ + "uuid" : "3f4a8124-d95d-4dbf-8888-d15d441de8f1", + "name" : "test_streaming_v2_cube", + "description" : "", + "dimensions" : [ { + "name" : "DEFAULT.STREAMING_V2_TABLE.SITE", + "table" : "DEFAULT.STREAMING_V2_TABLE", + "column" : "SITE", + "derived" : null + }, { + "name" : "DEFAULT.STREAMING_V2_TABLE.ITM", + "table" : "DEFAULT.STREAMING_V2_TABLE", + "column" : "ITM", + "derived" : null + }, { + "name" : "TIME", + "table" : "DEFAULT.STREAMING_V2_TABLE", + "column" : "DAY_START", + "derived" : null + }, { + "name" : "TIME", + "table" : "DEFAULT.STREAMING_V2_TABLE", + "column" : "HOUR_START", + "derived" : null + }, { + "name" : "TIME", + "table" : "DEFAULT.STREAMING_V2_TABLE", + "column" : "MINUTE_START", + "derived" : null + } ], + "measures" : [ { + "name" : "_COUNT_", + "function" : { + "expression" : "COUNT", + "parameter" : { + "type" : "constant", + "value" : "1", + "next_parameter" : null + }, + "returntype" : "bigint" + }, + "dependent_measure_ref" : null + }, { + "name" : "GMV_SUM", + "function" : { + "expression" : "SUM", + "parameter" : { + "type" : "column", + "value" : "GMV", + "next_parameter" : null + }, + "returntype" : "decimal(19,6)" + }, + "dependent_measure_ref" : null + }, { + "name": "GMV_PERCENTILE", + "function": { + "expression": "PERCENTILE", + "parameter": { + "type": "column", + "value": "GMV" + }, + "returntype": "percentile(100)" + } + }, { + "name" : "ITEM_COUNT_SUM", + "function" : { + "expression" : "SUM", + "parameter" : { + "type" : "column", + "value" : "ITEM_COUNT", + "next_parameter" : null + }, + "returntype" : "bigint" + }, + "dependent_measure_ref" : null + } ], + "rowkey" : { + "rowkey_columns" : [{ + "column" : "ITM", + "encoding" : "dict" + }, { + "column" : "DAY_START", + "encoding" : "dict" + }, { + "column" : "HOUR_START", + "encoding" : "dict" + }, { + "column" : "MINUTE_START", + "encoding" : "dict" + }, { + "column" : "SITE", + "encoding" : "dict" + } ] + }, + "signature" : null, + "last_modified" : 1448959801314, + "model_name" : "test_streaming_v2_model", + "null_string" : null, + "hbase_mapping" : { + "column_family" : [ { + "name" : "F1", + "columns" : [ { + "qualifier" : "M", + "measure_refs" : [ "_COUNT_", "GMV_SUM", "GMV_PERCENTILE", "ITEM_COUNT_SUM" ] + } ] + } ] + }, + "aggregation_groups" : [ { + "includes" : [ "DAY_START", "HOUR_START", "ITM", "MINUTE_START", "SITE" ], + "select_rule" : { + "hierarchy_dims" : [ [ "DAY_START", "HOUR_START", "MINUTE_START" ] ], + "mandatory_dims" : [ ], + "joint_dims" : [ ] + } + } ], + "mandatory_dimension_set_list": [ + [ + "SITE" + ], + [ + "ITM", + "SITE" + ] + ], + "override_kylin_properties": { + "kylin.stream.consume.offsets.latest": "true", + "kylin.stream.build.additional.cuboids": "false", + "kylin.stream.cube.window": "3600", + "kylin.stream.cube.duration": "3600" + }, + "notify_list" : [ ], + "status_need_notify" : [ ], + "auto_merge_time_ranges" : null, + "retention_range" : 0, + "engine_type" : 4, + "storage_type" : 2, + "partition_date_start": 0 +} \ No newline at end of file diff --git a/examples/test_case_data/localmeta/kylin.properties b/examples/test_case_data/localmeta/kylin.properties index be2ecd48274..ffa2fc48c82 100644 --- a/examples/test_case_data/localmeta/kylin.properties +++ b/examples/test_case_data/localmeta/kylin.properties @@ -143,6 +143,7 @@ kylin.job.lock=org.apache.kylin.job.lock.MockJobLockDup kylin.job.lock=org.apache.kylin.job.lock.MockJobLock kylin.env.zookeeper-connect-string=localhost:12181 +kylin.stream.metadata.store.type=mock kylin.engine.provider.0=org.apache.kylin.engine.mr.MRBatchCubingEngine diff --git a/examples/test_case_data/localmeta/model_desc/test_streaming_v2_model.json b/examples/test_case_data/localmeta/model_desc/test_streaming_v2_model.json new file mode 100644 index 00000000000..a43ed207d5d --- /dev/null +++ b/examples/test_case_data/localmeta/model_desc/test_streaming_v2_model.json @@ -0,0 +1,30 @@ +{ + "uuid": "84774301-f1ab-43f2-b3fc-21b5da49b113", + "name": "test_streaming_v2_model", + "dimensions": [ + { + "table": "DEFAULT.STREAMING_V2_TABLE", + "columns": [ + "minute_start", + "hour_start", + "day_start", + "itm", + "site" + ] + } + ], + "lookups": [], + "metrics": [ + "gmv", + "item_count" + ], + "capacity": "MEDIUM", + "last_modified": 0, + "fact_table": "DEFAULT.STREAMING_V2_TABLE", + "filter_condition": null, + "partition_desc": { + "partition_date_column": "DEFAULT.STREAMING_V2_TABLE.minute_start", + "partition_date_start": 0, + "partition_type": "APPEND" + } +} \ No newline at end of file diff --git a/examples/test_case_data/localmeta/project/default.json b/examples/test_case_data/localmeta/project/default.json index 606f16caba4..8ca7efec4ea 100644 --- a/examples/test_case_data/localmeta/project/default.json +++ b/examples/test_case_data/localmeta/project/default.json @@ -35,6 +35,11 @@ "name" : "test_streaming_join_table_cube", "type" : "CUBE", "realization" : "test_streaming_join_table_cube" + }, + { + "name": "test_streaming_v2_cube", + "type": "CUBE", + "realization": "test_streaming_v2_cube" } ], "tables": [ @@ -55,6 +60,7 @@ "DEFAULT.FIFTY_DIM", "DEFAULT.WIDE_TABLE", "DEFAULT.STREAMING_CATEGORY" + "DEFAULT.STREAMING_V2_TABLE" ], "models": [ "ssb", @@ -67,6 +73,7 @@ "twenty_dim", "ut_large_dimension_number", "test_streaming_join_table_model" + "test_streaming_v2_model" ], "override_kylin_properties" :{ "kylin.storage.hbase.owner-tag": "kylin@kylin.apache.org" diff --git a/examples/test_case_data/localmeta/table/DEFAULT.STREAMING_V2_TABLE.json b/examples/test_case_data/localmeta/table/DEFAULT.STREAMING_V2_TABLE.json new file mode 100644 index 00000000000..92fd3b89cb7 --- /dev/null +++ b/examples/test_case_data/localmeta/table/DEFAULT.STREAMING_V2_TABLE.json @@ -0,0 +1,47 @@ +{ + "uuid": "befdf555-9a6e-41ee-98f5-721cdbb04943", + "name": "STREAMING_V2_TABLE", + "columns": [ + { + "id": "1", + "name": "minute_start", + "datatype": "timestamp", + "index": "T" + }, + { + "id": "2", + "name": "hour_start", + "datatype": "timestamp", + "index": "T" + }, + { + "id": "3", + "name": "day_start", + "datatype": "date", + "index": "T" + }, + { + "id": "4", + "name": "itm", + "datatype": "string" + }, + { + "id": "5", + "name": "site", + "datatype": "string" + }, + { + "id": "6", + "name": "gmv", + "datatype": "decimal(19,6)" + }, + { + "id": "7", + "name": "item_count", + "datatype": "int" + } + ], + "database": "DEFAULT", + "last_modified": 0, + "source_type" : 20 +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index d745519b6c4..406649e2e47 100644 --- a/pom.xml +++ b/pom.xml @@ -380,6 +380,28 @@ ${project.version} test-jar + + + org.apache.kylin + kylin-stream-core + ${project.version} + + + org.apache.kylin + kylin-stream-coordinator + ${project.version} + + + org.apache.kylin + kylin-stream-source-kafka + ${project.version} + + + org.apache.kylin + kylin-storage-stream + ${project.version} + + org.apache.kylin kylin-datasource-sdk @@ -699,7 +721,6 @@ org.apache.kafka kafka_2.11 ${kafka.version} - provided @@ -927,7 +948,11 @@ org.eclipse.jetty jetty-server ${jetty.version} - test + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} org.eclipse.jetty @@ -1278,6 +1303,11 @@ metrics-reporter-kafka cache datasource-sdk + storage-stream + stream-receiver + stream-coordinator + stream-core + stream-source-kafka diff --git a/server-base/pom.xml b/server-base/pom.xml index 8fd037cb659..2b5cd743cbf 100644 --- a/server-base/pom.xml +++ b/server-base/pom.xml @@ -90,6 +90,49 @@ kylin-datasource-sdk framework + + org.apache.kylin + kylin-stream-coordinator + + + javax.servlet + servlet-api + + + javax.servlet.jsp + jsp-api + + + + + org.apache.kylin + kylin-stream-source-kafka + + + org.apache.kylin + kylin-storage-stream + + + org.apache.hive.hcatalog + hive-hcatalog-core + ${hive-hcatalog.version} + provided + + + javax.servlet + servlet-api + + + javax.servlet.jsp + jsp-api + + + org.apache.hive + hive-exec + + + + net.sf.ehcache ehcache diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingCoordinatorController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingCoordinatorController.java new file mode 100644 index 00000000000..bc3886fa970 --- /dev/null +++ b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingCoordinatorController.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.controller; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import javax.servlet.http.HttpServletRequest; + +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.rest.service.StreamingCoordinatorService; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.coordinator.client.CoordinatorResponse; +import org.apache.kylin.stream.coordinator.exception.NotLeadCoordinatorException; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.RemoteStoreCompleteRequest; +import org.apache.kylin.stream.core.model.ReplicaSetLeaderChangeRequest; +import org.apache.kylin.stream.core.source.Partition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.HttpStatus; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.ExceptionHandler; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; +import org.springframework.web.bind.annotation.ResponseStatus; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +/** + * StreamingCoordinatorController is defined as Restful API entrance for stream coordinator. + * + */ +@Controller +@RequestMapping(value = "/streaming_coordinator") +public class StreamingCoordinatorController extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(StreamingCoordinatorController.class); + + @Autowired + private StreamingCoordinatorService streamingCoordinartorService; + + @ResponseStatus(HttpStatus.OK) + @ExceptionHandler(NotLeadCoordinatorException.class) + @ResponseBody + CoordinatorResponse handleNotLeadCoordinator(HttpServletRequest req, Exception ex) { + CoordinatorResponse response = new CoordinatorResponse(); + response.setCode(CoordinatorResponse.NOT_LEAD_COORDINATOR); + response.setMsg(ex.getMessage()); + return response; + } + + @RequestMapping(value = "/balance/recommend", method = { RequestMethod.GET }) + @ResponseBody + public CoordinatorResponse reBalanceRecommend() { + Map>> result = streamingCoordinartorService.reBalanceRecommend(); + CoordinatorResponse response = new CoordinatorResponse(); + response.setData(result); + return response; + } + + @RequestMapping(value = "/balance", method = { RequestMethod.POST }) + @ResponseBody + public CoordinatorResponse reBalance(@RequestBody String reBalancePlanStr) { + Map>> reBalancePlan = deserializeRebalancePlan(reBalancePlanStr); + streamingCoordinartorService.reBalance(reBalancePlan); + return new CoordinatorResponse(); + } + + private Map>> deserializeRebalancePlan(String reBalancePlanStr) { + TypeReference>>> typeRef = new TypeReference>>>() { + }; + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.readValue(reBalancePlanStr, typeRef); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @RequestMapping(value = "/cubes/{cubeName}/assign", method = { RequestMethod.PUT }) + @ResponseBody + public CoordinatorResponse assignStreamingCube(@PathVariable String cubeName) { + streamingCoordinartorService.assignCube(cubeName); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/cubes/{cubeName}/unAssign", method = { RequestMethod.PUT }) + @ResponseBody + public CoordinatorResponse unAssignStreamingCube(@PathVariable String cubeName) { + streamingCoordinartorService.unAssignCube(cubeName); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/cubes/{cubeName}/reAssign", method = { RequestMethod.POST }) + @ResponseBody + public CoordinatorResponse reAssignStreamingCube(@PathVariable String cubeName, + @RequestBody CubeAssignment newAssignments) { + streamingCoordinartorService.reAssignCube(cubeName, newAssignments); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/replicaSet", method = { RequestMethod.POST }) + @ResponseBody + public CoordinatorResponse createReplicaSet(@RequestBody ReplicaSet rs) { + streamingCoordinartorService.createReplicaSet(rs); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}", method = { RequestMethod.DELETE }) + @ResponseBody + public CoordinatorResponse deleteReplicaSet(@PathVariable Integer replicaSetID) { + streamingCoordinartorService.removeReplicaSet(replicaSetID); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}/{nodeID:.+}", method = { RequestMethod.PUT }) + @ResponseBody + public CoordinatorResponse addNodeToReplicaSet(@PathVariable Integer replicaSetID, @PathVariable String nodeID) { + streamingCoordinartorService.addNodeToReplicaSet(replicaSetID, nodeID); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}/{nodeID:.+}", method = { RequestMethod.DELETE }) + @ResponseBody + public CoordinatorResponse removeNodeFromReplicaSet(@PathVariable Integer replicaSetID, @PathVariable String nodeID) { + streamingCoordinartorService.removeNodeFromReplicaSet(replicaSetID, nodeID); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/cubes/{cubeName}/pauseConsume", method = { RequestMethod.PUT }) + @ResponseBody + public CoordinatorResponse pauseCubeConsume(@PathVariable String cubeName) { + streamingCoordinartorService.pauseConsumers(cubeName); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/cubes/{cubeName}/resumeConsume", method = { RequestMethod.PUT }) + @ResponseBody + public CoordinatorResponse resumeCubeConsume(@PathVariable String cubeName) { + streamingCoordinartorService.resumeConsumers(cubeName); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/remoteStoreComplete", method = { RequestMethod.POST }) + @ResponseBody + public CoordinatorResponse segmentRemoteStoreComplete(@RequestBody RemoteStoreCompleteRequest request) { + Pair segmentRange = new Pair<>(request.getSegmentStart(), request.getSegmentEnd()); + Node receiver = request.getReceiverNode(); + logger.info( + "receive segment remote store complete request for cube:{}, segment:{}, try to find proper segment to build", + request.getCubeName(), segmentRange); + streamingCoordinartorService.onSegmentRemoteStoreComplete(request.getCubeName(), segmentRange, receiver); + return new CoordinatorResponse(); + } + + @RequestMapping(value = "/replicaSetLeaderChange", method = { RequestMethod.POST }) + @ResponseBody + public CoordinatorResponse replicaSetLeaderChange(@RequestBody ReplicaSetLeaderChangeRequest request) { + logger.info("receive replicaSet leader change:" + request); + streamingCoordinartorService.replicaSetLeaderChange(request.getReplicaSetID(), request.getNewLeader()); + return new CoordinatorResponse(); + } +} diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingV2Controller.java b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingV2Controller.java new file mode 100644 index 00000000000..03862cabaf3 --- /dev/null +++ b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingV2Controller.java @@ -0,0 +1,524 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.controller; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.HadoopUtil; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.model.ColumnDesc; +import org.apache.kylin.metadata.model.ISourceAware; +import org.apache.kylin.metadata.model.TableDesc; +import org.apache.kylin.metadata.project.ProjectInstance; +import org.apache.kylin.metadata.project.ProjectManager; +import org.apache.kylin.rest.exception.BadRequestException; +import org.apache.kylin.rest.exception.ForbiddenException; +import org.apache.kylin.rest.exception.InternalErrorException; +import org.apache.kylin.rest.exception.NotFoundException; +import org.apache.kylin.rest.request.StreamingRequest; +import org.apache.kylin.rest.request.StreamingRequestV2; +import org.apache.kylin.rest.response.ResponseCode; +import org.apache.kylin.rest.service.CubeService; +import org.apache.kylin.rest.service.StreamingV2Service; +import org.apache.kylin.rest.service.TableService; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.stats.ClusterState; +import org.apache.kylin.stream.core.model.stats.CubeRealTimeState; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingSourceConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.security.access.AccessDeniedException; +import org.springframework.security.core.context.SecurityContextHolder; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RequestParam; +import org.springframework.web.bind.annotation.ResponseBody; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * StreamingController is defined as Restful API entrance for UI. + * + */ +@Controller +@RequestMapping(value = "/streaming_v2") +public class StreamingV2Controller extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(StreamingV2Controller.class); + + @Autowired + private StreamingV2Service streamingService; + + @Autowired + private CubeService cubeMgmtService; + + @Autowired + @Qualifier("tableService") + private TableService tableService; + + @RequestMapping(value = "/getConfig", method = { RequestMethod.GET }) + @ResponseBody + public List getStreamings(@RequestParam(value = "table", required = false) String table, + @RequestParam(value = "limit", required = false) Integer limit, + @RequestParam(value = "offset", required = false) Integer offset) { + try { + return streamingService.getStreamingConfigs(table, limit, offset); + } catch (IOException e) { + logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e); + throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage()); + } + } + + /** + * + * create Streaming Schema + * + * @throws IOException + */ + @RequestMapping(value = "", method = { RequestMethod.POST }) + @ResponseBody + public StreamingRequestV2 saveStreamingConfig(@RequestBody StreamingRequestV2 streamingRequest) { + String project = streamingRequest.getProject(); + TableDesc tableDesc = deserializeTableDesc(streamingRequest); + StreamingSourceConfig streamingSourceConfig = deserializeStreamingConfig(streamingRequest.getStreamingConfig()); + + validateInput(tableDesc, streamingSourceConfig); + + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to saveStreamingConfig with table Identity {}", user, tableDesc.getIdentity()); + + boolean saveStreamingSuccess = false, saveTableSuccess = false; + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + ProjectInstance projectInstance = ProjectManager.getInstance(kylinConfig).getProject(project); + try { + try { + tableDesc.setUuid(UUID.randomUUID().toString()); + tableService.loadTableToProject(tableDesc, null, project); + saveTableSuccess = true; + } catch (IOException e) { + throw new BadRequestException("Failed to add streaming table."); + } + try { + streamingSourceConfig.setName(tableDesc.getIdentity()); + streamingSourceConfig.setUuid(UUID.randomUUID().toString()); + streamingService.createStreamingConfig(streamingSourceConfig, projectInstance); + saveStreamingSuccess = true; + } catch (IOException e) { + logger.error("Failed to save StreamingSourceConfig:" + e.getLocalizedMessage(), e); + throw new InternalErrorException("Failed to save StreamingSourceConfig: " + e.getLocalizedMessage()); + } + + } finally { + if (!saveTableSuccess || !saveStreamingSuccess) { + if (saveTableSuccess) { + try { + tableService.unloadHiveTable(tableDesc.getIdentity(), project); + } catch (IOException e) { + throw new InternalErrorException("Action failed and failed to rollback the create table " + + e.getLocalizedMessage(), e); + } + } + if (saveStreamingSuccess) { + try { + streamingService.dropStreamingConfig(streamingSourceConfig); + } catch (IOException e) { + throw new InternalErrorException( + "Action failed and failed to rollback the created streaming config: " + + e.getLocalizedMessage(), e); + } + } + } + + } + streamingRequest.setSuccessful(true); + return streamingRequest; + } + + private void validateInput(TableDesc tableDesc, StreamingSourceConfig streamingSourceConfig) { + if (StringUtils.isEmpty(tableDesc.getIdentity()) || StringUtils.isEmpty(streamingSourceConfig.getName())) { + logger.error("streamingSourceConfig name should not be empty."); + throw new BadRequestException("streamingSourceConfig name should not be empty."); + } + + // validate the compatibility for input table schema and the underline hive table schema + if (tableDesc.getSourceType() == ISourceAware.ID_KAFKA_HIVE) { + List fields; + try { + HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(new HiveConf()); + fields = metaStoreClient.getFields(tableDesc.getDatabase(), tableDesc.getName()); + } catch (NoSuchObjectException noObjectException) { + logger.info("table not exist in hive meta store for table:" + tableDesc.getIdentity(), + noObjectException); + throw new BadRequestException("table doesn't exist in hive meta store for table:" + + tableDesc.getIdentity(), ResponseCode.CODE_UNDEFINED, noObjectException); + } catch (Exception e) { + logger.error("error when get metadata from hive meta store for table:" + tableDesc.getIdentity(), e); + throw new BadRequestException("error when connect hive meta store", ResponseCode.CODE_UNDEFINED, e); + } + // check the data type compatibility for each column + Map fieldSchemaMap = Maps.newHashMap(); + for (FieldSchema field : fields) { + fieldSchemaMap.put(field.getName().toUpperCase(Locale.ROOT), field); + } + List incompatibleMsgs = Lists.newArrayList(); + for (ColumnDesc columnDesc : tableDesc.getColumns()) { + FieldSchema fieldSchema = fieldSchemaMap.get(columnDesc.getName().toUpperCase(Locale.ROOT)); + if (fieldSchema == null) { + incompatibleMsgs.add("column not exist in hive table:" + columnDesc.getName()); + continue; + } + if (!checkHiveTableFieldCompatible(fieldSchema, columnDesc)) { + String msg = String.format(Locale.ROOT, + "column:%s defined in hive type:%s is incompatible with the column definition:%s", + columnDesc.getName(), fieldSchema.getType(), columnDesc.getDatatype()); + incompatibleMsgs.add(msg); + } + } + if (incompatibleMsgs.size() > 0) { + logger.info("incompatible for hive and input table schema:{}", incompatibleMsgs); + throw new BadRequestException("incompatible for hive schema and input table schema:" + incompatibleMsgs); + } + } + } + + private static Map> COMPATIBLE_MAP = Maps.newHashMap(); + static { + COMPATIBLE_MAP.put("float", Sets.newHashSet("double")); + COMPATIBLE_MAP.put("string", Sets.newHashSet("varchar", "char", "varchar(256)")); + COMPATIBLE_MAP.put("varchar", Sets.newHashSet("string", "char")); + COMPATIBLE_MAP.put("varchar(256)", Sets.newHashSet("string", "char", "varchar")); + COMPATIBLE_MAP.put("long", Sets.newHashSet("bigint", "int", "smallint", "integer")); + COMPATIBLE_MAP.put("bigint", Sets.newHashSet("long", "int", "smallint", "integer")); + COMPATIBLE_MAP.put("int", Sets.newHashSet("smallint", "integer")); + } + + private boolean checkHiveTableFieldCompatible(FieldSchema fieldSchema, ColumnDesc columnDesc) { + DataType normalized = DataType.getType(columnDesc.getDatatype()); + String normalizedDataType = normalized == null ? columnDesc.getDatatype() : normalized.toString(); + if (fieldSchema.getType().equals(normalizedDataType)) { + return true; + } + Set compatibleSet = COMPATIBLE_MAP.get(fieldSchema.getType()); + if (compatibleSet != null && compatibleSet.contains(normalizedDataType)) { + return true; + } + return false; + } + + @RequestMapping(value = "/updateConfig", method = { RequestMethod.PUT }) + @ResponseBody + public StreamingRequest updateStreamingConfig(@RequestBody StreamingRequest streamingRequest) + throws JsonProcessingException { + StreamingSourceConfig streamingSourceConfig = deserializeStreamingConfig(streamingRequest.getStreamingConfig()); + + if (streamingSourceConfig == null) { + return streamingRequest; + } + + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to updateStreamingConfig.", user); + try { + streamingSourceConfig = streamingService.updateStreamingConfig(streamingSourceConfig); + } catch (AccessDeniedException accessDeniedException) { + throw new ForbiddenException("You don't have right to update this StreamingSourceConfig."); + } catch (Exception e) { + logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e); + throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage()); + } + streamingRequest.setSuccessful(true); + + return streamingRequest; + } + + @RequestMapping(value = "/{configName}", method = { RequestMethod.DELETE }) + @ResponseBody + public void deleteConfig(@PathVariable String configName) throws IOException { + StreamingSourceConfig config = streamingService.getStreamingManagerV2().getConfig(configName); + if (null == config) { + throw new NotFoundException("StreamingSourceConfig with name " + configName + " not found.."); + } + + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to delete config: {}", user, configName); + + try { + streamingService.dropStreamingConfig(config); + } catch (Exception e) { + logger.error(e.getLocalizedMessage(), e); + throw new InternalErrorException("Failed to delete StreamingSourceConfig. " + " Caused by: " + + e.getMessage(), e); + } + } + + @RequestMapping(value = "/parserTemplate", method = { RequestMethod.GET }) + @ResponseBody + public String getParserTemplate(@RequestParam(value = "sourceType") int sourceType, + @RequestParam(value = "streamingConfig") String streamingConfigStr) { + StreamingSourceConfig streamingSourceConfig = deserializeStreamingConfig(streamingConfigStr); + return streamingService.getParserTemplate(sourceType, streamingSourceConfig); + } + + @RequestMapping(value = "/cubeAssignments", method = { RequestMethod.GET }) + @ResponseBody + public List getCubeAssignments(@RequestParam(value = "cube", required = false) String cube) { + CubeInstance cubeInstance = null; + if (cube != null) { + cubeInstance = cubeMgmtService.getCubeManager().getCube(cube); + } + return streamingService.getStreamingCubeAssignments(cubeInstance); + } + + @RequestMapping(value = "/rsAssignments", method = { RequestMethod.GET }) + @ResponseBody + public Map>> getReplicaSetAssignments( + @RequestParam(value = "replicaSetID", required = false) Integer replicaSetID) { + return streamingService.getStreamingReplicaSetAssignments(replicaSetID); + } + + @RequestMapping(value = "/balance/recommend", method = { RequestMethod.GET }) + @ResponseBody + public Map>> reBalanceRecommend() { + return streamingService.reBalancePlan(); + } + + @RequestMapping(value = "/balance", method = { RequestMethod.POST }) + @ResponseBody + public void reBalance(@RequestBody String reBalancePlanStr) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to do reBalance.", user); + streamingService.reBalance(deserializeRebalancePlan(reBalancePlanStr)); + } + + private Map>> deserializeRebalancePlan(String reBalancePlanStr) { + TypeReference>>> typeRef = new TypeReference>>>() { + }; + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.readValue(reBalancePlanStr, typeRef); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @RequestMapping(value = "/cubeAssignments/{cubeName}", method = { RequestMethod.DELETE }) + @ResponseBody + public void removeCubeAssignment(@PathVariable String cubeName) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to remove CubeAssignment {}", user, cubeName); + streamingService.removeCubeAssignment(); + } + + @RequestMapping(value = "/cubes", method = { RequestMethod.GET }) + @ResponseBody + public List getStreamingCubes() { + return streamingService.getStreamingCubes(); + } + + @RequestMapping(value = "/cubes/{cubeName}/consumeState", method = { RequestMethod.GET }) + @ResponseBody + public String getStreamingCubeConsumeState(@PathVariable String cubeName) { + return streamingService.getStreamingCubeConsumeState(cubeName).toString(); + } + + @RequestMapping(value = "/cubes/{cubeName}/assign", method = { RequestMethod.PUT }) + @ResponseBody + public void assignStreamingCube(@PathVariable String cubeName) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to assign cube {}", user, cubeName); + CubeInstance cube = cubeMgmtService.getCubeManager().getCube(cubeName); + streamingService.assignCube(cube); + } + + @RequestMapping(value = "/cubes/{cubeName}/unAssign", method = { RequestMethod.PUT }) + @ResponseBody + public void unAssignStreamingCube(@PathVariable String cubeName) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to unAssign cube {}", user, cubeName); + CubeInstance cube = cubeMgmtService.getCubeManager().getCube(cubeName); + streamingService.unAssignCube(cube); + } + + @RequestMapping(value = "/cubes/{cubeName}/reAssign", method = { RequestMethod.POST }) + @ResponseBody + public void reAssignStreamingCube(@PathVariable String cubeName, @RequestBody CubeAssignment newAssignment) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to reAssign cube {}", user, cubeName); + streamingService.reAssignCube(cubeName, newAssignment); + } + + @RequestMapping(value = "/receivers", method = { RequestMethod.GET }) + @ResponseBody + public List getStreamingReceivers() { + return streamingService.getReceivers(); + } + + @RequestMapping(value = "/receivers/{receiverID:.+}", method = { RequestMethod.DELETE }) + @ResponseBody + public void removeStreamingReceiver(@PathVariable String receiverID) { + Node receiver = Node.fromNormalizeString(receiverID); + streamingService.removeReceiver(receiver); + } + + @RequestMapping(value = "/replicaSet", method = { RequestMethod.POST }) + @ResponseBody + public void createReplicaSet(@RequestBody ReplicaSet rs) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to create ReplicaSet {}", user, rs.getReplicaSetID()); + streamingService.createReplicaSet(rs); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}", method = { RequestMethod.DELETE }) + @ResponseBody + public void removeReplicaSet(@PathVariable Integer replicaSetID) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to remove ReplicaSet {}", user, replicaSetID); + streamingService.removeReplicaSet(replicaSetID); + } + + @RequestMapping(value = "/replicaSets", method = { RequestMethod.GET }) + @ResponseBody + public List getReplicaSets() { + return streamingService.getReplicaSets(); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}/{nodeID:.+}", method = { RequestMethod.PUT }) + @ResponseBody + public void addNodeToReplicaSet(@PathVariable Integer replicaSetID, @PathVariable String nodeID) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to add Node {} To ReplicaSet {}", user, nodeID, replicaSetID); + streamingService.addNodeToReplicaSet(replicaSetID, nodeID); + } + + @RequestMapping(value = "/replicaSet/{replicaSetID}/{nodeID:.+}", method = { RequestMethod.DELETE }) + @ResponseBody + public void removeNodeFromReplicaSet(@PathVariable Integer replicaSetID, @PathVariable String nodeID) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to remove Node {} from ReplicaSet {}", user, nodeID, replicaSetID); + streamingService.removeNodeFromReplicaSet(replicaSetID, nodeID); + } + + @RequestMapping(value = "/cubes/{cubeName}/suspendConsume", method = { RequestMethod.PUT }) + @ResponseBody + public void pauseCubeConsume(@PathVariable String cubeName) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to pause Consumers for cube {}", user, cubeName); + CubeInstance cube = cubeMgmtService.getCubeManager().getCube(cubeName); + streamingService.pauseConsumers(cube); + } + + @RequestMapping(value = "/cubes/{cubeName}/resumeConsume", method = { RequestMethod.PUT }) + @ResponseBody + public void resumeCubeConsume(@PathVariable String cubeName) { + final String user = SecurityContextHolder.getContext().getAuthentication().getName(); + logger.info("{} try to resume Consumers for cube {}", user, cubeName); + CubeInstance cube = cubeMgmtService.getCubeManager().getCube(cubeName); + streamingService.resumeConsumers(cube); + } + + @RequestMapping(value = "/cubes/{cubeName}/stats", method = { RequestMethod.GET }) + @ResponseBody + public CubeRealTimeState getCubeRealTimeState(@PathVariable String cubeName) { + CubeInstance cube = cubeMgmtService.getCubeManager().getCube(cubeName); + return streamingService.getCubeRealTimeState(cube); + } + + @RequestMapping(value = "/receivers/{receiverID:.+}/stats", method = { RequestMethod.GET }) + @ResponseBody + public ReceiverStats getReceiverStats(@PathVariable String receiverID) { + Node receiver = Node.fromNormalizeString(receiverID); + return streamingService.getReceiverStats(receiver); + } + + @RequestMapping(value = "/cluster/state", method = { RequestMethod.GET }) + @ResponseBody + public ClusterState getClusterState() { + return streamingService.getClusterState(); + } + + private TableDesc deserializeTableDesc(StreamingRequestV2 streamingRequest) { + TableDesc desc = null; + try { + logger.debug("Saving TableDesc " + streamingRequest.getTableData()); + desc = JsonUtil.readValue(streamingRequest.getTableData(), TableDesc.class); + } catch (JsonParseException e) { + logger.error("The TableDesc definition is invalid.", e); + updateRequest(streamingRequest, false, e.getMessage()); + } catch (JsonMappingException e) { + logger.error("The data TableDesc definition is invalid.", e); + updateRequest(streamingRequest, false, e.getMessage()); + } catch (IOException e) { + logger.error("Failed to deal with the request.", e); + throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e); + } + + String[] dbTable = HadoopUtil.parseHiveTableName(desc.getName()); + desc.setName(dbTable[1]); + desc.setDatabase(dbTable[0]); + desc.getIdentity(); + return desc; + } + + private StreamingSourceConfig deserializeStreamingConfig(String streamingConfigStr) { + try { + logger.debug("Saving StreamingSourceConfig " + streamingConfigStr); + return JsonUtil.readValue(streamingConfigStr, StreamingSourceConfig.class); + } catch (Exception e) { + logger.error("The StreamingSourceConfig definition is invalid.", e); + throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e); + } + } + + private void updateRequest(StreamingRequestV2 request, boolean success, String message) { + request.setSuccessful(success); + request.setMessage(message); + } + + public void setCubeService(CubeService cubeService) { + this.cubeMgmtService = cubeService; + } + +} diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/StreamingRequestV2.java b/server-base/src/main/java/org/apache/kylin/rest/request/StreamingRequestV2.java new file mode 100644 index 00000000000..2f995da69c3 --- /dev/null +++ b/server-base/src/main/java/org/apache/kylin/rest/request/StreamingRequestV2.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.request; + +public class StreamingRequestV2 { + + private String project; + + private String tableData; + + private String streamingConfig; + + private boolean successful; + + private String message; + + public String getProject() { + return project; + } + + public void setProject(String project) { + this.project = project; + } + + public String getTableData() { + return tableData; + } + + public void setTableData(String tableData) { + this.tableData = tableData; + } + + public boolean isSuccessful() { + return successful; + } + + public void setSuccessful(boolean successful) { + this.successful = successful; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + public String getStreamingConfig() { + return streamingConfig; + } + + public void setStreamingConfig(String streamingConfig) { + this.streamingConfig = streamingConfig; + } + +} diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java index 2a5ce26995a..df79a263f5b 100644 --- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java +++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java @@ -87,6 +87,9 @@ import org.apache.kylin.storage.hbase.HBaseConnection; import org.apache.kylin.storage.hbase.util.StorageCleanUtil; import org.apache.kylin.storage.hybrid.HybridInstance; +import org.apache.kylin.stream.coordinator.StreamMetadataStoreFactory; +import org.apache.kylin.stream.coordinator.client.CoordinatorClient; +import org.apache.kylin.stream.coordinator.client.CoordinatorClientFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.InitializingBean; @@ -410,7 +413,26 @@ public CubeInstance disableCube(CubeInstance cube) throws IOException { String.format(Locale.ROOT, msg.getDISABLE_NOT_READY_CUBE(), cubeName, ostatus)); } - return getCubeManager().updateCubeStatus(cube, RealizationStatusEnum.DISABLED); + boolean isStreamingCube = cube.getDescriptor().isStreamingCube(); + + boolean cubeStatusUpdated = false; + try { + CubeInstance cubeInstance = getCubeManager().updateCubeStatus(cube, RealizationStatusEnum.DISABLED); + cubeStatusUpdated = true; + // for streaming cube. + if (isStreamingCube) { + getStreamingCoordinator().unAssignCube(cubeName); + } + return cubeInstance; + } catch (Exception e) { + cube.setStatus(ostatus); + // roll back if cube status updated + if (cubeStatusUpdated) { + logger.info("roll back cube status to:{}", ostatus); + getCubeManager().updateCubeStatus(cube, ostatus); + } + throw e; + } } public void checkEnableCubeCondition(CubeInstance cube) { @@ -425,7 +447,7 @@ public void checkEnableCubeCondition(CubeInstance cube) { String.format(Locale.ROOT, msg.getENABLE_NOT_DISABLED_CUBE(), cubeName, ostatus)); } - if (cube.getSegments(SegmentStatusEnum.READY).size() == 0) { + if (cube.getSegments(SegmentStatusEnum.READY).size() == 0 && !cube.getDescriptor().isStreamingCube()) { throw new BadRequestException(String.format(Locale.ROOT, msg.getNO_READY_SEGMENT(), cubeName)); } @@ -442,7 +464,29 @@ public void checkEnableCubeCondition(CubeInstance cube) { * @throws IOException */ public CubeInstance enableCube(CubeInstance cube) throws IOException { - return getCubeManager().updateCubeStatus(cube, RealizationStatusEnum.READY); + boolean cubeStatusUpdated = false; + RealizationStatusEnum ostatus = cube.getStatus(); + try { + CubeInstance cubeInstance = getCubeManager().updateCubeStatus(cube, RealizationStatusEnum.READY); + cubeStatusUpdated = true; + // for streaming cube. + if (cube.getDescriptor().isStreamingCube()) { + getStreamingCoordinator().assignCube(cube.getName()); + } + return cubeInstance; + } catch (Exception e) { + cube.setStatus(ostatus); + // roll back if cube status updated + if (cubeStatusUpdated) { + logger.info("roll back cube status to:{}", ostatus); + getCubeManager().updateCubeStatus(cube, ostatus); + } + throw e; + } + } + + private CoordinatorClient getStreamingCoordinator() { + return CoordinatorClientFactory.createCoordinatorClient(StreamMetadataStoreFactory.getStreamMetaDataStore()); } public MetricsResponse calculateMetrics(MetricsRequest request) { @@ -486,8 +530,7 @@ public HBaseResponse getHTableInfo(String cubeName, String tableName) throws IOE hr = new HBaseResponse(); CubeInstance cube = CubeManager.getInstance(getConfig()).getCube(cubeName); - if (cube.getStorageType() == IStorageAware.ID_HBASE - || cube.getStorageType() == IStorageAware.ID_SHARDED_HBASE) { + if (cube.getStorageType() == IStorageAware.ID_HBASE || cube.getStorageType() == IStorageAware.ID_SHARDED_HBASE || cube.getStorageType() == IStorageAware.ID_REALTIME_AND_HBASE) { try { logger.debug("Loading HTable info " + cubeName + ", " + tableName); diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/StreamingCoordinatorService.java b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingCoordinatorService.java new file mode 100644 index 00000000000..b4f223fb34f --- /dev/null +++ b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingCoordinatorService.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.rest.service; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.stream.coordinator.Coordinator; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.coordinator.StreamMetadataStoreFactory; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.source.Partition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; + +import com.google.common.collect.Sets; + +@Component("streamingCoordinatorService") +public class StreamingCoordinatorService extends BasicService { + private static final Logger logger = LoggerFactory.getLogger(StreamingCoordinatorService.class); + + private StreamMetadataStore streamMetadataStore; + + private Coordinator streamingCoordinator; + + public StreamingCoordinatorService(){ + streamMetadataStore = StreamMetadataStoreFactory.getStreamMetaDataStore(); + //TODO coordinator operation should go to the only one lead coordinator + streamingCoordinator = Coordinator.getInstance(); + } + + public synchronized Map>> reBalanceRecommend() { + return streamingCoordinator.reBalanceRecommend(); + } + + public synchronized void reBalance(Map>> reBalancePlan) { + streamingCoordinator.reBalance(reBalancePlan); + } + + public void assignCube(String cubeName) { + streamingCoordinator.assignCube(cubeName); + } + + public void unAssignCube(String cubeName) { + streamingCoordinator.unAssignCube(cubeName); + } + + public void reAssignCube(String cubeName, CubeAssignment newAssignment) { + validateAssignment(newAssignment); + streamingCoordinator.reAssignCube(cubeName, newAssignment); + } + + private void validateAssignment(CubeAssignment newAssignment) { + Map> assignments = newAssignment.getAssignments(); + Set inputReplicaSetIDs = assignments.keySet(); + Set allReplicaSetIDs = Sets.newHashSet(streamMetadataStore.getReplicaSetIDs()); + for (Integer inputReplicaSetID : inputReplicaSetIDs) { + if (!allReplicaSetIDs.contains(inputReplicaSetID)) { + throw new IllegalArgumentException("the replica set id:" + inputReplicaSetID + " does not exist"); + } + } + } + + public void pauseConsumers(String cubeName) { + streamingCoordinator.pauseConsumers(cubeName); + } + + public void resumeConsumers(String cubeName) { + streamingCoordinator.resumeConsumers(cubeName); + } + + public void replicaSetLeaderChange(int replicaSetID, Node newLeader) { + streamingCoordinator.replicaSetLeaderChange(replicaSetID, newLeader); + } + + public void createReplicaSet(ReplicaSet rs){ + streamingCoordinator.createReplicaSet(rs); + } + + public void removeReplicaSet(int rsID){ + streamingCoordinator.removeReplicaSet(rsID); + } + + public void addNodeToReplicaSet(Integer replicaSetID, String nodeID){ + streamingCoordinator.addNodeToReplicaSet(replicaSetID, nodeID); + } + + public void removeNodeFromReplicaSet(Integer replicaSetID, String nodeID){ + streamingCoordinator.removeNodeFromReplicaSet(replicaSetID, nodeID); + } + + public void onSegmentRemoteStoreComplete(String cubeName, Pair segment, Node receiver) { + streamingCoordinator.segmentRemoteStoreComplete(receiver, cubeName, segment); + } + +} diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/StreamingV2Service.java b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingV2Service.java new file mode 100644 index 00000000000..996dbf54835 --- /dev/null +++ b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingV2Service.java @@ -0,0 +1,570 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.rest.service; + +import java.io.IOException; +import java.net.InetAddress; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.metadata.model.ISourceAware; +import org.apache.kylin.metadata.project.ProjectInstance; +import org.apache.kylin.rest.constant.Constant; +import org.apache.kylin.rest.exception.InternalErrorException; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.coordinator.StreamMetadataStoreFactory; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.coordinator.client.CoordinatorClient; +import org.apache.kylin.stream.coordinator.client.CoordinatorClientFactory; +import org.apache.kylin.stream.core.client.HttpReceiverAdminClient; +import org.apache.kylin.stream.core.client.ReceiverAdminClient; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.model.stats.ClusterState; +import org.apache.kylin.stream.core.model.stats.ConsumerStats; +import org.apache.kylin.stream.core.model.stats.CubeRealTimeState; +import org.apache.kylin.stream.core.model.stats.PartitionConsumeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeRealTimeState; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverState; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; +import org.apache.kylin.stream.core.model.stats.ReplicaSetState; +import org.apache.kylin.stream.core.source.IStreamingSource; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingSourceConfig; +import org.apache.kylin.stream.core.source.StreamingSourceConfigManager; +import org.apache.kylin.stream.core.source.StreamingSourceFactory; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; +import org.springframework.stereotype.Component; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +@Component("streamingServiceV2") +public class StreamingV2Service extends BasicService { + private static final Logger logger = LoggerFactory.getLogger(StreamingV2Service.class); + + private StreamMetadataStore streamMetadataStore; + + private ReceiverAdminClient receiverAdminClient; + + private Cache clusterStateCache = CacheBuilder.newBuilder() + .expireAfterWrite(10, TimeUnit.SECONDS).build(); + + private ExecutorService clusterStateExecutor = new ThreadPoolExecutor(0, 20, 60L, TimeUnit.SECONDS, + new LinkedBlockingQueue(), new NamedThreadFactory("fetch_receiver_state")); + + public StreamingV2Service() { + streamMetadataStore = StreamMetadataStoreFactory.getStreamMetaDataStore(); + receiverAdminClient = new HttpReceiverAdminClient(); + } + + public List listAllStreamingConfigs(final String table) throws IOException { + List streamingSourceConfigs = Lists.newArrayList(); + if (StringUtils.isEmpty(table)) { + streamingSourceConfigs = getStreamingManagerV2().listAllStreaming(); + } else { + StreamingSourceConfig config = getStreamingManagerV2().getConfig(table); + if (config != null) { + streamingSourceConfigs.add(config); + } + } + + return streamingSourceConfigs; + } + + public List getStreamingConfigs(final String table, final Integer limit, final Integer offset) + throws IOException { + List streamingSourceConfigs; + streamingSourceConfigs = listAllStreamingConfigs(table); + + if (limit == null || offset == null) { + return streamingSourceConfigs; + } + + if ((streamingSourceConfigs.size() - offset) < limit) { + return streamingSourceConfigs.subList(offset, streamingSourceConfigs.size()); + } + + return streamingSourceConfigs.subList(offset, offset + limit); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#project, 'ADMINISTRATION')") + public StreamingSourceConfig createStreamingConfig(StreamingSourceConfig config, ProjectInstance project) throws IOException { + if (getStreamingManagerV2().getConfig(config.getName()) != null) { + throw new InternalErrorException("The streamingSourceConfig named " + config.getName() + " already exists"); + } + StreamingSourceConfig streamingSourceConfig = getStreamingManagerV2().saveStreamingConfig(config); + return streamingSourceConfig; + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public StreamingSourceConfig updateStreamingConfig(StreamingSourceConfig config) throws IOException { + return getStreamingManagerV2().updateStreamingConfig(config); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void dropStreamingConfig(StreamingSourceConfig config) throws IOException { + getStreamingManagerV2().removeStreamingConfig(config); + } + + public String getParserTemplate(final int sourceType, StreamingSourceConfig config) { + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(new ISourceAware() { + @Override + public int getSourceType() { + return sourceType; + } + + @Override + public KylinConfig getConfig() { + return getConfig(); + } + }); + return streamingSource.getMessageTemplate(config); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public List getStreamingCubeAssignments(final CubeInstance cube) { + if (cube == null) { + return streamMetadataStore.getAllCubeAssignments(); + } + List result = Lists.newArrayList(); + CubeAssignment assignment = streamMetadataStore.getAssignmentsByCube(cube.getName()); + if (assignment != null) { + result.add(assignment); + } + return result; + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public Map>> getStreamingReplicaSetAssignments(Integer replicaSetID) { + if (replicaSetID == null) { + return streamMetadataStore.getAllReplicaSetAssignments(); + } + Map>> result = Maps.newHashMap(); + Map> assignment = streamMetadataStore.getAssignmentsByReplicaSet(replicaSetID); + if (assignment != null) { + result.put(replicaSetID, assignment); + } + return result; + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public Map>> reBalancePlan() { + return getCoordinatorClient().reBalanceRecommend(); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void reBalance(Map>> reBalancePlan) { + getCoordinatorClient().reBalance(reBalancePlan); + } + + public List getStreamingCubes() { + return streamMetadataStore.getCubes(); + } + + public StreamingCubeConsumeState getStreamingCubeConsumeState(String cubeName) { + return streamMetadataStore.getStreamingCubeConsumeState(cubeName); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public void assignCube(CubeInstance cube) { + getCoordinatorClient().assignCube(cube.getName()); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public void unAssignCube(CubeInstance cube) { + getCoordinatorClient().unAssignCube(cube.getName()); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void reAssignCube(String cubeName, CubeAssignment newAssignment) { + validateAssignment(newAssignment); + getCoordinatorClient().reAssignCube(cubeName, newAssignment); + } + + private void validateAssignment(CubeAssignment newAssignment) { + Map> assignments = newAssignment.getAssignments(); + Set inputReplicaSetIDs = assignments.keySet(); + Set allReplicaSetIDs = Sets.newHashSet(streamMetadataStore.getReplicaSetIDs()); + for (Integer inputReplicaSetID : inputReplicaSetIDs) { + if (!allReplicaSetIDs.contains(inputReplicaSetID)) { + throw new IllegalArgumentException("the replica set id:" + inputReplicaSetID + " does not exist"); + } + } + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public void pauseConsumers(CubeInstance cube) { + getCoordinatorClient().pauseConsumers(cube.getName()); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public void resumeConsumers(CubeInstance cube) { + getCoordinatorClient().resumeConsumers(cube.getName()); + } + + public void onSegmentRemoteStoreComplete(String cubeName, Pair segment, Node receiver) { + logger.info( + "segment remote store complete signal received for cube:{}, segment:{}, try to find proper segment to build", + cubeName, segment); + getCoordinatorClient().segmentRemoteStoreComplete(receiver, cubeName, segment); + } + + public StreamingSourceConfigManager getStreamingManagerV2() { + return StreamingSourceConfigManager.getInstance(getConfig()); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void removeCubeAssignment() { + + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public List getReceivers() { + List result = streamMetadataStore.getReceivers(); + return result; + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void removeReceiver(Node receiver) { + List replicaSets = streamMetadataStore.getReplicaSets(); + for (ReplicaSet replicaSet : replicaSets) { + Set receivers = replicaSet.getNodes(); + if (receivers != null && receivers.contains(receiver)) { + throw new IllegalStateException("Before remove receiver, it must be firstly removed from replica set:" + + replicaSet.getReplicaSetID()); + } + } + streamMetadataStore.removeReceiver(receiver); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void createReplicaSet(ReplicaSet rs) { + getCoordinatorClient().createReplicaSet(rs); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void removeReplicaSet(int rsID) { + getCoordinatorClient().removeReplicaSet(rsID); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void addNodeToReplicaSet(Integer replicaSetID, String nodeID) { + getCoordinatorClient().addNodeToReplicaSet(replicaSetID, nodeID); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public void removeNodeFromReplicaSet(Integer replicaSetID, String nodeID) { + getCoordinatorClient().removeNodeFromReplicaSet(replicaSetID, nodeID); + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN) + public List getReplicaSets() { + List result = streamMetadataStore.getReplicaSets(); + return result; + } + + public ReceiverStats getReceiverStats(Node receiver) { + try { + return receiverAdminClient.getReceiverStats(receiver); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public ReceiverCubeStats getReceiverCubeStats(Node receiver, String cubeName) { + try { + return receiverAdminClient.getReceiverCubeStats(receiver, cubeName); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')") + public CubeRealTimeState getCubeRealTimeState(CubeInstance cube) { + CubeRealTimeState result = new CubeRealTimeState(); + result.setCubeName(cube.getName()); + CubeAssignment cubeAssignment = streamMetadataStore.getAssignmentsByCube(cube.getName()); + Map> rsReceiverCubeStateMap = Maps.newHashMap(); + for (Integer replicaSetID : cubeAssignment.getReplicaSetIDs()) { + ReplicaSet replicaSet = streamMetadataStore.getReplicaSet(replicaSetID); + Map receiverCubeStateMap = Maps.newHashMap(); + Set receivers = replicaSet.getNodes(); + for (Node receiver : receivers) { + ReceiverCubeRealTimeState receiverCubeRealTimeState = new ReceiverCubeRealTimeState(); + try { + ReceiverCubeStats receiverCubeStats = receiverAdminClient.getReceiverCubeStats(receiver, + cube.getName()); + receiverCubeRealTimeState.setState(ReceiverState.State.HEALTHY); + receiverCubeRealTimeState.setReceiverCubeStats(receiverCubeStats); + } catch (IOException e) { + logger.error("exception when get receiver cube stats", e); + if (!isReceiverReachable(receiver)) { + receiverCubeRealTimeState.setState(ReceiverState.State.UNREACHABLE); + } else { + receiverCubeRealTimeState.setState(ReceiverState.State.DOWN); + } + } + receiverCubeStateMap.put(receiver, receiverCubeRealTimeState); + } + rsReceiverCubeStateMap.put(replicaSetID, receiverCubeStateMap); + } + result.setReceiverCubeStateMap(rsReceiverCubeStateMap); + + return result; + } + + public ClusterState getClusterState() { + ClusterState clusterState = clusterStateCache.getIfPresent("cluster_state"); + if (clusterState != null) { + return clusterState; + } + List replicaSets = streamMetadataStore.getReplicaSets(); + List allReceivers = streamMetadataStore.getReceivers(); + Map>> rsAssignments = streamMetadataStore.getAllReplicaSetAssignments(); + + Map> statsFuturesMap = Maps.newHashMap(); + for (final Node receiver : allReceivers) { + Future receiverStatsFuture = clusterStateExecutor.submit(new Callable() { + @Override + public ReceiverStats call() throws Exception { + return receiverAdminClient.getReceiverStats(receiver); + } + }); + statsFuturesMap.put(receiver, receiverStatsFuture); + } + + clusterState = new ClusterState(); + for (ReplicaSet replicaSet : replicaSets) { + ReplicaSetState replicaSetState = calReplicaSetState(replicaSet, + rsAssignments.get(replicaSet.getReplicaSetID()), statsFuturesMap); + clusterState.addReplicaSetState(replicaSetState); + allReceivers.removeAll(replicaSet.getNodes()); + } + + // left receivers are not assigned receivers + for (Node receiver : allReceivers) { + Future futureStats = statsFuturesMap.get(receiver); + ReceiverState receiverState = getReceiverStateFromStats(receiver, futureStats); + clusterState.addAvailableReveiverState(receiverState); + } + clusterState.setLastUpdateTime(System.currentTimeMillis()); + clusterStateCache.put("cluster_state", clusterState); + return clusterState; + } + + private ReplicaSetState calReplicaSetState(ReplicaSet replicaSet, Map> rsAssignment, + Map> statsFuturesMap) { + ReplicaSetState replicaSetState = new ReplicaSetState(); + replicaSetState.setRsID(replicaSet.getReplicaSetID()); + replicaSetState.setAssignment(rsAssignment); + Set receivers = replicaSet.getNodes(); + if (receivers == null || receivers.isEmpty()) { + return replicaSetState; + } + + Node leadReceiver = replicaSet.getLeader(); + replicaSetState.setLead(leadReceiver); + + Map receiverStatsMap = Maps.newHashMap(); + for (Node receiver : receivers) { + Future futureStats = statsFuturesMap.get(receiver); + try { + ReceiverStats receiverStats = futureStats.get(); + receiverStatsMap.put(receiver, receiverStats); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + replicaSetState.addReveiverState(getReceiverStateFromException(receiver, e)); + continue; + } + } + + Map cubeLatestEventMap = Maps.newHashMap(); + for (ReceiverStats receiverStats : receiverStatsMap.values()) { + Map cubeStatsMap = receiverStats.getCubeStatsMap(); + for (Map.Entry cubeStatsEntry : cubeStatsMap.entrySet()) { + String cubeName = cubeStatsEntry.getKey(); + ReceiverCubeStats cubeStats = cubeStatsEntry.getValue(); + Long latestEventTime = cubeLatestEventMap.get(cubeName); + if (latestEventTime != null && latestEventTime < cubeStats.getLatestEventTime()) { + cubeLatestEventMap.put(cubeName, cubeStats.getLatestEventTime()); + } else if (latestEventTime == null) { + cubeLatestEventMap.put(cubeName, cubeStats.getLatestEventTime()); + } + } + } + long consumeEventLagThreshold = 5 * 60 * 1000L; // default lag warning threshold is 5 minutes + for (Map.Entry receiverStatsEntry : receiverStatsMap.entrySet()) { + Node receiver = receiverStatsEntry.getKey(); + ReceiverStats receiverStats = receiverStatsEntry.getValue(); + ReceiverState receiverState = new ReceiverState(); + receiverState.setReceiver(receiver); + receiverState.setState(ReceiverState.State.HEALTHY); + Map> receiverAssignment = receiverStats.getAssignments(); + if (!assignmentEqual(receiverAssignment, rsAssignment)) { + ReceiverState.State state = ReceiverState.State.WARN; + receiverState.setState(state); + receiverState.addInfo("assignment is inconsistent"); + } + + if (receiverStats.isLead() && !receiver.equals(leadReceiver)) { + ReceiverState.State state = ReceiverState.State.WARN; + receiverState.setState(state); + receiverState.addInfo("lead state is inconsistent"); + } + + Map cubeStatsMap = receiverStats.getCubeStatsMap(); + for (Map.Entry cubeStatsEntry : cubeStatsMap.entrySet()) { + String cubeName = cubeStatsEntry.getKey(); + ReceiverCubeStats cubeStats = cubeStatsEntry.getValue(); + Long latestEventTime = cubeLatestEventMap.get(cubeName); + if ((latestEventTime - cubeStats.getLatestEventTime()) >= consumeEventLagThreshold) { + ReceiverState.State state = ReceiverState.State.WARN; + receiverState.setState(state); + receiverState.addInfo("cube:" + cubeName + " consuming is lagged"); + } + } + receiverState.setRateInOneMin(calConsumeRate(receiver, receiverStats)); + replicaSetState.addReveiverState(receiverState); + } + return replicaSetState; + } + + private boolean assignmentEqual(Map> receiverAssignment, + Map> rsAssignment) { + if (emptyMap(receiverAssignment) && emptyMap(rsAssignment)) { + return true; + } + if (receiverAssignment != null && receiverAssignment.equals(rsAssignment)) { + return true; + } + return false; + } + + private boolean emptyMap(Map map) { + if (map == null || map.isEmpty()) { + return true; + } + return false; + } + + private ReceiverState getReceiverStateFromException(Node receiver, ExecutionException e) { + ReceiverState receiverState = new ReceiverState(); + receiverState.setReceiver(receiver); + if (!isReceiverReachable(receiver)) { + receiverState.setState(ReceiverState.State.UNREACHABLE); + } else { + receiverState.setState(ReceiverState.State.DOWN); + } + return receiverState; + } + + private boolean isReceiverReachable(Node receiver) { + try { + InetAddress address = InetAddress.getByName(receiver.getHost());//ping this IP + boolean reachable = address.isReachable(1000); + if (!reachable) { + return false; + } + return true; + } catch (Exception exception) { + logger.error("exception when try ping host:" + receiver.getHost(), exception); + return false; + } + } + + private ReceiverState getReceiverStateFromStats(Node receiver, Future futureStats) { + ReceiverState receiverState = new ReceiverState(); + try { + ReceiverStats receiverStats = futureStats.get(); + receiverState.setReceiver(receiver); + receiverState.setState(ReceiverState.State.HEALTHY); + receiverState.setRateInOneMin(calConsumeRate(receiver, receiverStats)); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + receiverState = getReceiverStateFromException(receiver, e); + } + return receiverState; + } + + private double calConsumeRate(Node receiver, ReceiverStats receiverStats) { + double result = 0; + Map cubeStatsMap = receiverStats.getCubeStatsMap(); + for (Map.Entry receiverCubeStatsEntry : cubeStatsMap.entrySet()) { + ReceiverCubeStats cubeStats = receiverCubeStatsEntry.getValue(); + ConsumerStats consumerStats = cubeStats.getConsumerStats(); + if (consumerStats == null) { + logger.warn("no consumer stats exist for cube:{} in receiver:{}", receiverCubeStatsEntry.getKey(), + receiver); + continue; + } + Map partitionConsumeStatsMap = consumerStats.getPartitionConsumeStatsMap(); + for (PartitionConsumeStats partitionStats : partitionConsumeStatsMap.values()) { + result += partitionStats.getOneMinRate(); + } + } + return result; + } + + // private Node.NodeStatus getReceiverStatus(Node receiver) { + // try { + // HealthCheckInfo healthCheckInfo = receiverAdminClient.healthCheck(receiver); + // if (healthCheckInfo.getStatus() == HealthCheckInfo.Status.GOOD) { + // return Node.NodeStatus.HEALTHY; + // } else { + // return Node.NodeStatus.STOPPED; + // } + // } catch (IOException e) { + // return Node.NodeStatus.STOPPED; + // } + // } + + private synchronized CoordinatorClient getCoordinatorClient() { + return CoordinatorClientFactory.createCoordinatorClient(streamMetadataStore); + } +} diff --git a/server/src/main/resources/kylinSecurity.xml b/server/src/main/resources/kylinSecurity.xml index c08ae709c60..2329c8fd7a6 100644 --- a/server/src/main/resources/kylinSecurity.xml +++ b/server/src/main/resources/kylinSecurity.xml @@ -242,6 +242,7 @@ + @@ -288,6 +289,7 @@ + diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java index 0af00ef134c..853882e7a13 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java @@ -528,7 +528,7 @@ private static List getHTableNames(KylinConfig config) { ArrayList result = new ArrayList(); for (CubeInstance cube : cubeMgr.listAllCubes()) { - if (cube.getStorageType() == IStorageAware.ID_HBASE || cube.getStorageType() == IStorageAware.ID_SHARDED_HBASE) { + if (cube.getStorageType() == IStorageAware.ID_HBASE || cube.getStorageType() == IStorageAware.ID_SHARDED_HBASE || cube.getStorageType() == IStorageAware.ID_REALTIME_AND_HBASE) { for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) { String tableName = seg.getStorageLocationIdentifier(); if (StringUtils.isBlank(tableName) == false) { diff --git a/storage-stream/pom.xml b/storage-stream/pom.xml new file mode 100644 index 00000000000..bd626ab51b4 --- /dev/null +++ b/storage-stream/pom.xml @@ -0,0 +1,87 @@ + + + + + + kylin + org.apache.kylin + 2.6.0-SNAPSHOT + + 4.0.0 + kylin-storage-stream + jar + Apache Kylin - Stream Storage + + + + + + org.apache.kylin + kylin-core-common + ${project.parent.version} + + + + org.apache.kylin + kylin-core-metadata + ${project.parent.version} + + + + org.apache.kylin + kylin-core-storage + ${project.parent.version} + + + + org.apache.kylin + kylin-storage-hbase + ${project.parent.version} + + + + org.apache.kylin + kylin-stream-coordinator + ${project.parent.version} + + + + org.apache.kylin + kylin-stream-core + ${project.parent.version} + + + + + + junit + junit + test + + + + org.apache.hbase + hbase-common + provided + + + + + + diff --git a/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorage.java b/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorage.java new file mode 100644 index 00000000000..d327c1c3ee9 --- /dev/null +++ b/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorage.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.storage.stream; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.engine.mr.IMROutput2; +import org.apache.kylin.metadata.realization.IRealization; +import org.apache.kylin.metadata.realization.RealizationType; +import org.apache.kylin.storage.IStorage; +import org.apache.kylin.storage.IStorageQuery; +import org.apache.kylin.storage.hbase.steps.HBaseMROutput2Transition; +import org.apache.kylin.storage.stream.rpc.HttpStreamDataSearchClient; +import org.apache.kylin.storage.stream.rpc.IStreamDataSearchClient; + +//used by reflection +public class StreamStorage implements IStorage { + private volatile IStreamDataSearchClient realTimeSearchClient; + + @Override + public IStorageQuery createQuery(IRealization realization) { + if (realization.getType() == RealizationType.CUBE) { + CubeInstance cubeInstance = (CubeInstance) realization; + return new StreamStorageQuery(cubeInstance, getStreamingDataSearchClient()); + } else { + throw new IllegalArgumentException("Unknown realization type " + realization.getType()); + } + } + + private IStreamDataSearchClient getStreamingDataSearchClient() { + if (realTimeSearchClient == null) { + synchronized (this) { + if (realTimeSearchClient == null) { + realTimeSearchClient = new HttpStreamDataSearchClient(); + } + } + } + return realTimeSearchClient; + } + + @SuppressWarnings("unchecked") + @Override + public I adaptToBuildEngine(Class engineInterface) { + if (engineInterface == IMROutput2.class) { + return (I) new HBaseMROutput2Transition(); + } else { + throw new RuntimeException("Cannot adapt to " + engineInterface); + } + } +} diff --git a/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorageQuery.java b/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorageQuery.java new file mode 100644 index 00000000000..f2050c16416 --- /dev/null +++ b/storage-stream/src/main/java/org/apache/kylin/storage/stream/StreamStorageQuery.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.storage.stream; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeSegment; +import org.apache.kylin.cube.cuboid.Cuboid; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.SegmentRange.TSRange; +import org.apache.kylin.metadata.model.SegmentStatusEnum; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.realization.SQLDigest; +import org.apache.kylin.metadata.tuple.CompoundTupleIterator; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.storage.StorageContext; +import org.apache.kylin.storage.gtrecord.CubeSegmentScanner; +import org.apache.kylin.storage.gtrecord.GTCubeStorageQueryRequest; +import org.apache.kylin.storage.gtrecord.SequentialCubeTupleIterator; +import org.apache.kylin.storage.hbase.cube.v2.CubeStorageQuery; +import org.apache.kylin.storage.stream.rpc.IStreamDataSearchClient; +import org.apache.kylin.stream.core.query.StreamingDataQueryPlanner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +/** + * Streaming storage query + */ +public class StreamStorageQuery extends CubeStorageQuery { + + private static final Logger logger = LoggerFactory.getLogger(StreamStorageQuery.class); + + private final IStreamDataSearchClient realTimeSearchClient; + + public StreamStorageQuery(CubeInstance cube, IStreamDataSearchClient realTimeSearchClient) { + super(cube); + this.realTimeSearchClient = realTimeSearchClient; + } + + @Override + public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) { + GTCubeStorageQueryRequest request = getStorageQueryRequest(context, sqlDigest, returnTupleInfo); + + List scanners = Lists.newArrayList(); + long maxHistorySegmentTime = -1; + StreamingDataQueryPlanner segmentsPlanner = new StreamingDataQueryPlanner(cubeInstance.getDescriptor(), + request.getFilter()); + for (CubeSegment cubeSeg : cubeInstance.getSegments(SegmentStatusEnum.READY)) { + TSRange segmentRange = cubeSeg.getTSRange(); + if (segmentRange.end.v > maxHistorySegmentTime) { + maxHistorySegmentTime = cubeSeg.getTSRange().end.v; + } + CubeSegmentScanner scanner; + + if (cubeDesc.getConfig().isSkippingEmptySegments() && cubeSeg.getInputRecords() == 0) { + logger.info("Skip cube segment {} because its input record is 0", cubeSeg); + continue; + } + + if (segmentsPlanner.canSkip(segmentRange.start.v, segmentRange.end.v)) { + logger.info("Skip cube segment {} because of not satisfy filter:{}", cubeSeg, request.getFilter()); + continue; + } + + scanner = new CubeSegmentScanner(cubeSeg, request.getCuboid(), request.getDimensions(), + request.getGroups(), request.getDynGroups(), request.getDynGroupExprs(), request.getMetrics(), + request.getDynFuncs(), request.getFilter(), request.getHavingFilter(), request.getContext()); + if (!scanner.isSegmentSkipped()) + scanners.add(scanner); + } + + ITupleIterator historyResult; + if (scanners.isEmpty()) { + historyResult = ITupleIterator.EMPTY_TUPLE_ITERATOR; + } else { + historyResult = new SequentialCubeTupleIterator(scanners, request.getCuboid(), request.getDimensions(), + request.getDynGroups(), request.getGroups(), request.getMetrics(), returnTupleInfo, context, sqlDigest); + } + Set dimensionsD = request.getDimensions(); + if (dimensionsD.isEmpty()) { + dimensionsD = Sets.newHashSet(request.getCuboid().getColumns()); // temporary fix for query like: select count(1) from TABLE + } + + ITupleIterator realTimeResult; + if (segmentsPlanner.canSkip(maxHistorySegmentTime, Long.MAX_VALUE)) { + logger.info("Skip scan realTime data"); + realTimeResult = ITupleIterator.EMPTY_TUPLE_ITERATOR; + } else { + boolean isSelectAllQuery = isSelectAllQuery(request.getCuboid(), request.getGroups(), request.getFilter()); + int limitPushDown = isSelectAllQuery ? context.getFinalPushDownLimit() : Integer.MAX_VALUE; + realTimeResult = realTimeSearchClient.search(maxHistorySegmentTime, cubeInstance, returnTupleInfo, + request.getFilter(), dimensionsD, request.getGroups(), request.getMetrics(), limitPushDown, + !isSelectAllQuery); + } + return new CompoundTupleIterator(Arrays.asList(historyResult, realTimeResult)); + } + + // only 'select *' query don't need real time storage aggregation, and push down limit + private boolean isSelectAllQuery(Cuboid cuboid, Set groupsD, TupleFilter filterD) { + if (Cuboid.getBaseCuboidId(cubeDesc) == cuboid.getId() && filterD == null + && cuboid.getColumns().size() == groupsD.size()) { + return true; + } + return false; + } + +} diff --git a/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/HttpStreamDataSearchClient.java b/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/HttpStreamDataSearchClient.java new file mode 100644 index 00000000000..ec13485ec0f --- /dev/null +++ b/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/HttpStreamDataSearchClient.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.storage.stream.rpc; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.zip.DataFormatException; + +import org.apache.commons.codec.binary.Base64; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.QueryContext; +import org.apache.kylin.common.QueryContextFacade; +import org.apache.kylin.common.debug.BackdoorToggles; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.StringCodeSystem; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilterSerializer; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.ITuple; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.stream.coordinator.assign.AssignmentsCache; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.DataRequest; +import org.apache.kylin.stream.core.model.DataResponse; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.query.StreamingTupleConverter; +import org.apache.kylin.stream.core.query.StreamingTupleIterator; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.apache.kylin.stream.core.util.RecordsSerializer; +import org.apache.kylin.stream.core.util.RestService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * TODO use long connection rather than short connection + */ +public class HttpStreamDataSearchClient implements IStreamDataSearchClient { + public static final Logger logger = LoggerFactory.getLogger(HttpStreamDataSearchClient.class); + + private static ExecutorService executorService; + static { + executorService = new ThreadPoolExecutor(20, 100, 60L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue(), new NamedThreadFactory("stream-rpc-pool-t")); + } + private AssignmentsCache assignmentsCache; + private RestService restService; + private Map failedReceivers = Maps.newConcurrentMap(); + + public HttpStreamDataSearchClient() { + assignmentsCache = AssignmentsCache.getInstance(); + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + int connectionTimeout = kylinConfig.getStreamingRPCHttpConnTimeout(); + int readTimeout = kylinConfig.getStreamingRPCHttpReadTimeout(); + restService = new RestService(connectionTimeout, readTimeout); + } + + @Override + public ITupleIterator search(final long minSegmentTime, final CubeInstance cube, final TupleInfo tupleInfo, + final TupleFilter tupleFilter, final Set dimensions, final Set groups, + final Set metrics, final int storagePushDownLimit, final boolean allowStorageAggregation) { + List replicaSetsOfCube = assignmentsCache.getReplicaSetsByCube(cube.getName()); + int timeout = 120 * 1000; // timeout should be configurable + final QueuedStreamingTupleIterator result = new QueuedStreamingTupleIterator(replicaSetsOfCube.size(), timeout); + final QueryContext query = QueryContextFacade.current(); + + final CubeDesc cubeDesc = cube.getDescriptor(); + final ResponseResultSchema schema = new ResponseResultSchema(cubeDesc, dimensions, metrics); + final StreamingTupleConverter tupleConverter = new StreamingTupleConverter(schema, tupleInfo); + final RecordsSerializer recordsSerializer = new RecordsSerializer(schema); + final DataRequest dataRequest = createDataRequest(query.getQueryId(), cube.getName(), minSegmentTime, + tupleInfo, tupleFilter, dimensions, groups, metrics, storagePushDownLimit, allowStorageAggregation); + + logger.info("Query-{}:send request to stream receivers", query.getQueryId()); + for (final ReplicaSet rs : replicaSetsOfCube) { + executorService.submit(new Runnable() { + @Override + public void run() { + try { + Iterator tuplesBlock = search(dataRequest, cube, tupleConverter, recordsSerializer, rs, + tupleInfo); + result.addBlock(tuplesBlock); + } catch (Exception e) { + result.setEndpointException(e); + } + } + }); + } + + return result; + } + + public Iterator search(DataRequest dataRequest, CubeInstance cube, StreamingTupleConverter tupleConverter, + RecordsSerializer recordsSerializer, ReplicaSet rs, TupleInfo tupleInfo) throws Exception { + List receivers = Lists.newArrayList(rs.getNodes()); + Node leader = rs.getLeader(); + Node queryReceiver = findBestReceiverServeQuery(receivers, leader, cube.getName()); + IOException exception; + try { + return doSearch(dataRequest, cube, tupleConverter, recordsSerializer, queryReceiver, tupleInfo); + } catch (IOException e) { + exception = e; + failedReceivers.put(queryReceiver, System.currentTimeMillis()); + logger.error("exception throws for receiver:" + queryReceiver + " retry another receiver"); + } + + for (int i = 0; i < receivers.size(); i++) { + Node receiver = receivers.get(i); + if (receiver.equals(queryReceiver)) { + continue; + } + try { + return doSearch(dataRequest, cube, tupleConverter, recordsSerializer, receiver, tupleInfo); + } catch (IOException e) { + exception = e; + failedReceivers.put(receiver, System.currentTimeMillis()); + logger.error("exception throws for receiver:" + receiver + " retry another receiver"); + } + + } + throw exception; + } + + private Node findBestReceiverServeQuery(List receivers, Node lead, String cubeName) { + // stick to one receiver according to cube name + int receiversSize = receivers.size(); + int receiverNo = Math.abs(cubeName.hashCode()) % receiversSize; + Node foundReceiver = receivers.get(receiverNo); + Long lastFailTime = failedReceivers.get(foundReceiver); + if (lastFailTime == null) { + return foundReceiver; + } + + if (System.currentTimeMillis() - lastFailTime > 2 * 60 * 1000) { // retry every 2 minutes + return foundReceiver; + } + + return receivers.get((receiverNo + 1) % receiversSize); + } + + public Iterator doSearch(DataRequest dataRequest, CubeInstance cube, + StreamingTupleConverter tupleConverter, RecordsSerializer recordsSerializer, Node receiver, + TupleInfo tupleInfo) throws Exception { + String queryId = dataRequest.getQueryId(); + logger.info("send query to receiver " + receiver + " with query id:" + queryId); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/data/query"; + + try { + String content = JsonUtil.writeValueAsString(dataRequest); + Stopwatch sw = new Stopwatch(); + sw.start(); + int connTimeout = cube.getConfig().getStreamingRPCHttpConnTimeout(); + int readTimeout = cube.getConfig().getStreamingRPCHttpReadTimeout(); + String msg = restService.postRequest(url, content, connTimeout, readTimeout); + + logger.info("query-{}: receive response from {} take time:{}", queryId, receiver, sw.elapsedMillis()); + if (failedReceivers.containsKey(receiver)) { + failedReceivers.remove(receiver); + } + DataResponse response = JsonUtil.readValue(msg, DataResponse.class); + logger.info("query-{}: receiver {} profile info:{}", queryId, receiver, response.getProfile()); + return deserializeResponse(tupleConverter, recordsSerializer, cube.getName(), tupleInfo, response); + } catch (Exception e) { + logger.error("error when search data from receiver:" + url, e); + throw e; + } + } + + private Iterator deserializeResponse(final StreamingTupleConverter tupleConverter, + final RecordsSerializer recordsSerializer, String cubeName, TupleInfo tupleInfo, DataResponse response) + throws IOException, DataFormatException { + final Iterator records = recordsSerializer.deserialize(Base64.decodeBase64(response.getData())); + return new StreamingTupleIterator(records, tupleConverter, tupleInfo); + } + + private DataRequest createDataRequest(String queryId, String cubeName, long minSegmentTime, TupleInfo tupleInfo, + TupleFilter tupleFilter, Set dimensions, Set groups, Set metrics, + int storagePushDownLimit, boolean allowStorageAggregation) { + DataRequest request = new DataRequest(); + request.setCubeName(cubeName); + request.setQueryId(queryId); + request.setMinSegmentTime(minSegmentTime); + request.setTupleFilter(Base64.encodeBase64String(TupleFilterSerializer.serialize(tupleFilter, + StringCodeSystem.INSTANCE))); + request.setStoragePushDownLimit(storagePushDownLimit); + request.setAllowStorageAggregation(allowStorageAggregation); + request.setRequestSendTime(System.currentTimeMillis()); + request.setEnableDetailProfile(BackdoorToggles.isStreamingProfileEnable()); + request.setStorageBehavior(BackdoorToggles.getCoprocessorBehavior()); + + Set dimensionSet = Sets.newHashSet(); + for (TblColRef dimension : dimensions) { + dimensionSet.add(dimension.getCanonicalName()); + } + request.setDimensions(dimensionSet); + + Set groupSet = Sets.newHashSet(); + for (TblColRef group : groups) { + groupSet.add(group.getCanonicalName()); + } + request.setGroups(groupSet); + + request.setMetrics(metrics); + + return request; + } + + public static class QueuedStreamingTupleIterator implements ITupleIterator { + private BlockingQueue> queue; + + private Iterator currentBlock = Iterators.emptyIterator(); + + private int totalBlockNum; + private int numConsumeBlocks = 0; + + private int timeout; + private long timeoutTS; + private volatile Exception endpointException; + + public QueuedStreamingTupleIterator(int blockNum, int timeout) { + this.queue = new LinkedBlockingQueue<>(blockNum); + this.totalBlockNum = blockNum; + this.timeout *= 1.1; + this.timeoutTS = System.currentTimeMillis() + timeout; + } + + public void addBlock(Iterator tuples) { + try { + queue.put(tuples); + } catch (InterruptedException e) { + logger.error("interrupted", e); + throw new RuntimeException("interrupted", e); + } + } + + public void setEndpointException(Exception e) { + this.endpointException = e; + } + + private boolean hasEndpointFail() { + return endpointException != null; + } + + @Override + public void close() { + // do nothing + } + + @Override + public boolean hasNext() { + try { + if (currentBlock.hasNext()) { + return true; + } else if (numConsumeBlocks < totalBlockNum) { + while (numConsumeBlocks < totalBlockNum) { + if (hasEndpointFail()) { + throw new RuntimeException("endpoint fail", endpointException); + } + Iterator ret = null; + while (ret == null && endpointException == null && timeoutTS > System.currentTimeMillis()) { + ret = queue.poll(1000, TimeUnit.MILLISECONDS); + } + currentBlock = ret; + if (currentBlock == null) { + throw new RuntimeException("timeout when call stream rpc"); + } + numConsumeBlocks++; + if (currentBlock.hasNext()) { + return true; + } + } + + } + } catch (InterruptedException e) { + logger.error("interrupted", e); + throw new RuntimeException("interrupted", e); + } + + return false; + } + + @Override + public ITuple next() { + return currentBlock.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + +} diff --git a/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/IStreamDataSearchClient.java b/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/IStreamDataSearchClient.java new file mode 100644 index 00000000000..8d2308e963f --- /dev/null +++ b/storage-stream/src/main/java/org/apache/kylin/storage/stream/rpc/IStreamDataSearchClient.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.storage.stream.rpc; + +import java.util.Set; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.TupleInfo; + +public interface IStreamDataSearchClient { + /** + * + * @param minSegmentTime minimum segment time to search for steaming data + * @param cube + * @param tupleInfo + * @param tupleFilter + * @param dimensions + * @param groups + * @param metrics + * @return + */ + ITupleIterator search(long minSegmentTime, CubeInstance cube, TupleInfo tupleInfo, TupleFilter tupleFilter, + Set dimensions, Set groups, Set metrics, int storagePushDownLimit, + boolean allowStorageAggregation); +} diff --git a/stream-coordinator/pom.xml b/stream-coordinator/pom.xml new file mode 100644 index 00000000000..24b22da178e --- /dev/null +++ b/stream-coordinator/pom.xml @@ -0,0 +1,83 @@ + + + + + + kylin + org.apache.kylin + 2.6.0-SNAPSHOT + + 4.0.0 + kylin-stream-coordinator + jar + Apache Kylin - Stream Coordinator + + + + + + org.apache.kylin + kylin-core-common + ${project.parent.version} + + + org.apache.kylin + kylin-core-metadata + ${project.parent.version} + + + org.apache.kylin + kylin-storage-hbase + ${project.parent.version} + + + org.apache.kylin + kylin-stream-core + ${project.parent.version} + + + org.apache.hbase + hbase-common + provided + + + org.apache.hbase + hbase-client + provided + + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-recipes + + + + + junit + junit + test + + + + diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/Coordinator.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/Coordinator.java new file mode 100644 index 00000000000..bb16dd5d3e6 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/Coordinator.java @@ -0,0 +1,1294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.leader.LeaderSelector; +import org.apache.curator.framework.recipes.leader.LeaderSelectorListenerAdapter; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.HadoopUtil; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.common.util.ServerMode; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.CubeSegment; +import org.apache.kylin.engine.mr.CubingJob; +import org.apache.kylin.engine.mr.StreamingCubingEngine; +import org.apache.kylin.job.execution.DefaultChainedExecutable; +import org.apache.kylin.job.execution.ExecutableManager; +import org.apache.kylin.job.execution.ExecutableState; +import org.apache.kylin.metadata.model.SegmentRange.TSRange; +import org.apache.kylin.metadata.model.SegmentStatusEnum; +import org.apache.kylin.metadata.model.Segments; +import org.apache.kylin.metadata.realization.RealizationStatusEnum; +import org.apache.kylin.stream.coordinator.assign.Assigner; +import org.apache.kylin.stream.coordinator.assign.AssignmentUtil; +import org.apache.kylin.stream.coordinator.assign.AssignmentsCache; +import org.apache.kylin.stream.core.consumer.ConsumerStartProtocol; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.coordinator.assign.DefaultAssigner; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.coordinator.client.CoordinatorClient; +import org.apache.kylin.stream.coordinator.exception.CoordinateException; +import org.apache.kylin.stream.coordinator.exception.NotLeadCoordinatorException; +import org.apache.kylin.stream.core.client.HttpReceiverAdminClient; +import org.apache.kylin.stream.core.client.ReceiverAdminClient; +import org.apache.kylin.stream.core.model.AssignRequest; +import org.apache.kylin.stream.core.model.ConsumerStatsResponse; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.PauseConsumersRequest; +import org.apache.kylin.stream.core.model.ResumeConsumerRequest; +import org.apache.kylin.stream.core.model.SegmentBuildState; +import org.apache.kylin.stream.core.model.StartConsumersRequest; +import org.apache.kylin.stream.core.model.StopConsumersRequest; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.model.UnAssignRequest; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.source.ISourcePosition; +import org.apache.kylin.stream.core.source.ISourcePositionHandler; +import org.apache.kylin.stream.core.source.ISourcePositionHandler.MergeStrategy; +import org.apache.kylin.stream.core.source.IStreamingSource; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingTableSourceInfo; +import org.apache.kylin.stream.core.source.StreamingSourceFactory; +import org.apache.kylin.stream.core.util.HDFSUtil; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.apache.kylin.stream.core.util.NodeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.MapDifference; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import javax.annotation.Nullable; + +/** + * + * Each Kylin Streaming cluster has a coordinator to handle generic assignment, membership and streaming cube state management. + * + */ +public class Coordinator implements CoordinatorClient { + private static final Logger logger = LoggerFactory.getLogger(Coordinator.class); + private static final int DEFAULT_PORT = 7070; + private static volatile Coordinator instance = new Coordinator(); + + private StreamMetadataStore streamMetadataStore; + private Assigner assigner; + private ReceiverAdminClient receiverAdminClient; + private CuratorFramework zkClient; + private CoordinatorLeaderSelector selector; + private volatile boolean isLead = false; + + private ScheduledExecutorService streamingJobCheckExecutor; + + private StreamingBuildJobStatusChecker jobStatusChecker; + + private Coordinator() { + this.streamMetadataStore = StreamMetadataStoreFactory.getStreamMetaDataStore(); + this.receiverAdminClient = new HttpReceiverAdminClient(); + this.assigner = new DefaultAssigner(); + this.zkClient = ZKUtils.getZookeeperClient(); + this.selector = new CoordinatorLeaderSelector(); + this.jobStatusChecker = new StreamingBuildJobStatusChecker(); + this.streamingJobCheckExecutor = Executors.newScheduledThreadPool(1, new NamedThreadFactory( + "streaming_job_status_checker")); + if (ServerMode.SERVER_MODE.canServeStreamingCoordinator()) { + start(); + } + } + + public static Coordinator getInstance() { + return instance; + } + + public void start() { + selector.start(); + streamingJobCheckExecutor.scheduleAtFixedRate(jobStatusChecker, 0, 2, TimeUnit.MINUTES); + } + + private void restoreJobStatusChecker() { + logger.info("restore job status checker"); + List cubes = streamMetadataStore.getCubes(); + for (String cube : cubes) { + List segmentBuildStates = streamMetadataStore.getSegmentBuildStates(cube); + Collections.sort(segmentBuildStates); + for (SegmentBuildState segmentBuildState : segmentBuildStates) { + if (segmentBuildState.isInBuilding()) { + SegmentJobBuildInfo jobBuildInfo = new SegmentJobBuildInfo(cube, segmentBuildState.getSegmentName(), segmentBuildState.getState().getJobId()); + jobStatusChecker.addSegmentBuildJob(jobBuildInfo); + } + } + } + } + + /** + * Assign the streaming cube to replicaSets + * + * @param cubeName + * @return + */ + @Override + public synchronized void assignCube(String cubeName) { + checkLead(); + streamMetadataStore.addStreamingCube(cubeName); + StreamingCubeInfo cube = getStreamCubeInfo(cubeName); + CubeAssignment existAssignment = streamMetadataStore.getAssignmentsByCube(cube.getCubeName()); + if (existAssignment != null) { + logger.warn("cube " + cube.getCubeName() + " is already assigned."); + return; + } + List replicaSets = streamMetadataStore.getReplicaSets(); + if (replicaSets == null || replicaSets.isEmpty()) { + throw new IllegalStateException("no replicaSet is configured in system"); + } + CubeAssignment assignment = assigner.assign(cube, replicaSets, + streamMetadataStore.getAllCubeAssignments()); + doAssignCube(cubeName, assignment); + } + + @Override + public void unAssignCube(String cubeName) { + checkLead(); + CubeAssignment assignment = streamMetadataStore.getAssignmentsByCube(cubeName); + if (assignment == null) { + return; + } + List unAssignedFailReceivers = Lists.newArrayList(); + try { + logger.info("send unAssign cube:{} request to receivers", cubeName); + for (Integer replicaSetID : assignment.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(replicaSetID); + UnAssignRequest request = new UnAssignRequest(); + request.setCube(cubeName); + for (Node receiver : rs.getNodes()) { + try { + unAssignToReceiver(receiver, request); + } catch (Exception e) { + logger.error("exception throws when unAssign receiver", e); + // unAssignedFailReceivers.add(receiver); + } + } + } + logger.info("remove temp hdfs files"); + removeCubeHDFSFiles(cubeName); + logger.info("clear cube info from job check list"); + jobStatusChecker.clearCheckCube(cubeName); + logger.info("remove cube info in metadata store"); + streamMetadataStore.removeStreamingCube(cubeName); + AssignmentsCache.getInstance().clearCubeCache(cubeName); + } catch (Exception e) { + throw new CoordinateException(e); + } + if (unAssignedFailReceivers.size() > 0) { + String msg = "unAssign fail for receivers:" + unAssignedFailReceivers; + throw new CoordinateException(msg); + } + } + + @Override + public synchronized void reAssignCube(String cubeName, CubeAssignment assignments) { + checkLead(); + CubeAssignment preAssignments = streamMetadataStore.getAssignmentsByCube(cubeName); + if (preAssignments == null) { + logger.info("no previous cube assign exists, use the new assignment:{}", assignments); + doAssignCube(cubeName, assignments); + } else { + reassignCubeImpl(cubeName, preAssignments, assignments); + } + } + + @Override + public void segmentRemoteStoreComplete(Node receiver, String cubeName, Pair segment) { + checkLead(); + logger.info( + "segment remote store complete signal received for cube:{}, segment:{}, try to find proper segment to build", + cubeName, segment); + tryFindAndBuildSegment(cubeName); + } + + @Override + public Map>> reBalanceRecommend() { + checkLead(); + return reBalancePlan(getEnableStreamingCubes(), streamMetadataStore.getReplicaSets()); + } + + /** + * reBalance the cube and partitions + * @param newAssignmentsPlan Map> + * @return new assignments + */ + @Override + public synchronized void reBalance(Map>> newAssignmentsPlan) { + checkLead(); + List currCubeAssignments = streamMetadataStore.getAllCubeAssignments(); + List newCubeAssignments = AssignmentUtil.convertReplicaSetAssign2CubeAssign(newAssignmentsPlan); + doReBalance(currCubeAssignments, newCubeAssignments); + } + + @Override + public void pauseConsumers(String cubeName) { + checkLead(); + CubeAssignment assignment = streamMetadataStore.getAssignmentsByCube(cubeName); + PauseConsumersRequest request = new PauseConsumersRequest(); + request.setCube(cubeName); + try { + for (Integer rsID : assignment.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(rsID); + pauseConsumersInReplicaSet(rs, request); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + streamMetadataStore.saveStreamingCubeConsumeState(cubeName, StreamingCubeConsumeState.PAUSED); + } + + @Override + public void resumeConsumers(String cubeName) { + checkLead(); + CubeAssignment assignment = streamMetadataStore.getAssignmentsByCube(cubeName); + ResumeConsumerRequest request = new ResumeConsumerRequest(); + request.setCube(cubeName); + try { + for (Integer rsID : assignment.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(rsID); + resumeConsumersInReplicaSet(rs, request); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + streamMetadataStore.saveStreamingCubeConsumeState(cubeName, StreamingCubeConsumeState.RUNNING); + } + + @Override + public void replicaSetLeaderChange(int replicaSetID, Node newLeader) { + checkLead(); + Map> assignment = streamMetadataStore.getAssignmentsByReplicaSet(replicaSetID); + if (assignment == null || assignment.isEmpty()) { + return; + } + // clear assign cache for this group + for (String cubeName : assignment.keySet()) { + AssignmentsCache.getInstance().clearCubeCache(cubeName); + } + } + + private void checkLead() { + if (!isLead) { + throw new NotLeadCoordinatorException("coordinator is not lead"); + } + } + + private StreamingCubeInfo getStreamCubeInfo(String cubeName) { + CubeInstance cube = CubeManager.getInstance(getConfig()).getCube(cubeName); + if (cube == null) { + return null; + } + int numOfConsumerTasks = cube.getConfig().getStreamingCubeConsumerTasksNum(); + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cube); + StreamingTableSourceInfo tableSourceInfo = streamingSource.load(cubeName); + return new StreamingCubeInfo(cubeName, tableSourceInfo, numOfConsumerTasks); + } + + private KylinConfig getConfig() { + return KylinConfig.getInstanceFromEnv(); + } + + private void doAssignCube(String cubeName, CubeAssignment assignment) { + Set successRS = Sets.newHashSet(); + try { + for (Integer rsID : assignment.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(rsID); + assignCubeToReplicaSet(rs, cubeName, assignment.getPartitionsByReplicaSetID(rsID), true, false); + successRS.add(rs); + } + streamMetadataStore.saveNewCubeAssignment(assignment); + } catch (Exception e) { + // roll back the success group assignment + for (ReplicaSet rs : successRS) { + try { + UnAssignRequest request = new UnAssignRequest(); + request.setCube(cubeName); + unAssignFromReplicaSet(rs, request); + } catch (IOException e1) { + logger.error("error when roll back assignment", e); + } + } + throw new RuntimeException(e); + } + } + + private CubeAssignment reassignCubeImpl(String cubeName, CubeAssignment preAssignments, + CubeAssignment newAssignments) { + logger.info("start cube reBalance, cube:{}, previous assignments:{}, new assignments:{}", cubeName, + preAssignments, newAssignments); + if (newAssignments.equals(preAssignments)) { + logger.info("the new assignment is the same as the previous assignment, do nothing for this reassignment"); + return newAssignments; + } + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); + doReassign(cubeInstance, preAssignments, newAssignments); + MapDifference> assignDiff = Maps.difference(preAssignments.getAssignments(), + newAssignments.getAssignments()); + + // add empty partitions to the removed replica sets, means that there's still data in the replica set, but no new data will be consumed. + Map> removedAssign = assignDiff.entriesOnlyOnLeft(); + for (Integer removedReplicaSet : removedAssign.keySet()) { + newAssignments.addAssignment(removedReplicaSet, Lists. newArrayList()); + } + streamMetadataStore.saveNewCubeAssignment(newAssignments); + AssignmentsCache.getInstance().clearCubeCache(cubeName); + return newAssignments; + } + + // todo move to source specific implementation? + public void doReassign(CubeInstance cubeInstance, CubeAssignment preAssignments, CubeAssignment newAssignments) { + String cubeName = preAssignments.getCubeName(); + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cubeInstance); + MapDifference> assignDiff = Maps.difference(preAssignments.getAssignments(), + newAssignments.getAssignments()); + Map> sameAssign = assignDiff.entriesInCommon(); + Map> newAssign = assignDiff.entriesOnlyOnRight(); + Map> removedAssign = assignDiff.entriesOnlyOnLeft(); + + List allPositions = Lists.newArrayList(); + List successSyncReplicaSet = Lists.newArrayList(); + ISourcePosition consumePosition; + try { + for (Map.Entry> assignmentEntry : preAssignments.getAssignments().entrySet()) { + Integer replicaSetID = assignmentEntry.getKey(); + if (sameAssign.containsKey(replicaSetID)) { + logger.info("the assignment is not changed for cube:{}, replicaSet:{}", cubeName, replicaSetID); + continue; + } + ReplicaSet rs = getStreamMetadataStore().getReplicaSet(replicaSetID); + ISourcePosition position = syncAndStopConsumersInRs(streamingSource, cubeName, rs); + allPositions.add(position); + successSyncReplicaSet.add(rs); + } + consumePosition = streamingSource.getSourcePositionHandler().mergePositions(allPositions, MergeStrategy.KEEP_LARGE); + logger.info("the consumer position for cube:{} is:{}", cubeName, consumePosition); + } catch (Exception e) { + logger.error("fail to sync assign replicaSet for cube:" + cubeName, e); + // roll back the success group + for (ReplicaSet rs : successSyncReplicaSet) { + StartConsumersRequest request = new StartConsumersRequest(); + request.setCube(cubeName); + try { + startConsumersInReplicaSet(rs, request); + } catch (IOException e1) { + logger.error( + "fail to start consumers for cube:" + cubeName + " replicaSet:" + rs.getReplicaSetID(), e1); + } + } + throw new RuntimeException(e); + } + + List successAssigned = Lists.newArrayList(); + List successStarted = Lists.newArrayList(); + // the new assignment is break into two phrase to ensure transactional, first phrase is assign, and second phrase is start consumer. + try { + for (Map.Entry> cubeAssignmentEntry : newAssignments.getAssignments().entrySet()) { + Integer replicaSetID = cubeAssignmentEntry.getKey(); + if (sameAssign.containsKey(replicaSetID)) { + continue; + } + + ReplicaSet rs = getStreamMetadataStore().getReplicaSet(replicaSetID); + logger.info("assign cube:{} to replicaSet:{}", cubeName, replicaSetID); + assignCubeToReplicaSet(rs, cubeName, cubeAssignmentEntry.getValue(), false, true); + successAssigned.add(rs); + } + + for (Map.Entry> cubeAssignmentEntry : newAssignments.getAssignments().entrySet()) { + Integer replicaSetID = cubeAssignmentEntry.getKey(); + if (sameAssign.containsKey(replicaSetID)) { + continue; + } + + ConsumerStartProtocol consumerStartProtocol = new ConsumerStartProtocol(streamingSource.getSourcePositionHandler().serializePosition(consumePosition.advance())); + + ReplicaSet rs = getStreamMetadataStore().getReplicaSet(replicaSetID); + StartConsumersRequest startRequest = new StartConsumersRequest(); + startRequest.setCube(cubeName); + startRequest.setStartProtocol(consumerStartProtocol); + logger.info("start consumers for cube:{}, replicaSet:{}, startRequest:{}", cubeName, replicaSetID, + startRequest); + startConsumersInReplicaSet(rs, startRequest); + successStarted.add(rs); + } + + for (Map.Entry> removeAssignmentEntry : removedAssign.entrySet()) { + Integer replicaSetID = removeAssignmentEntry.getKey(); + logger.info("make cube immutable for cube:{}, replicaSet{}", cubeName, replicaSetID); + ReplicaSet rs = getStreamMetadataStore().getReplicaSet(replicaSetID); + makeCubeImmutableInReplicaSet(rs, cubeName); + } + logger.info("finish cube reBalance, cube:{}", cubeName); + } catch (IOException e) { + logger.error("fail to start consumers for cube:" + cubeName, e); + // roll back success started + for (ReplicaSet rs : successStarted) { + try { + StopConsumersRequest stopRequest = new StopConsumersRequest(); + stopRequest.setCube(cubeName); + // for new group assignment, need to stop the consumers and remove the cube data + if (newAssign.containsKey(rs.getReplicaSetID())) { + stopRequest.setRemoveData(true); + } + stopConsumersInReplicaSet(rs, stopRequest); + } catch (IOException e1) { + logger.error("fail to stop consumers for cube:" + cubeName + " replicaSet:" + rs.getReplicaSetID(), + e1); + } + } + + // roll back success assignment + for (ReplicaSet rs : successAssigned) { + try { + List partitions = preAssignments.getPartitionsByReplicaSetID(rs.getReplicaSetID()); + assignCubeToReplicaSet(rs, cubeName, partitions, true, true); + } catch (IOException e1) { + logger.error( + "fail to start consumers for cube:" + cubeName + " replicaSet:" + rs.getReplicaSetID(), e1); + } + } + + throw new RuntimeException(e); + } + + } + + /** + * sync the consumers in the replicaSet, ensure that all consumers in the group consume to the same position + * + * @param streamingSource + * @param cubeName + * @param replicaSet + * @return the consume position info. + */ + private ISourcePosition syncAndStopConsumersInRs(IStreamingSource streamingSource, String cubeName, ReplicaSet replicaSet) + throws IOException { + if (replicaSet.getNodes().size() > 1) { // when group nodes more than 1, force to sync the group + logger.info("sync consume for cube:{}, replicaSet:{}", cubeName, replicaSet.getReplicaSetID()); + + PauseConsumersRequest suspendRequest = new PauseConsumersRequest(); + suspendRequest.setCube(cubeName); + List allReceiverConsumeState = pauseConsumersInReplicaSet(replicaSet, + suspendRequest); + + List consumePositionList = Lists.transform(allReceiverConsumeState, new Function() { + @Nullable + @Override + public ISourcePosition apply(@Nullable ConsumerStatsResponse input) { + return streamingSource.getSourcePositionHandler().parsePosition(input.getConsumePosition()); + } + }); + ISourcePosition consumePosition = streamingSource.getSourcePositionHandler().mergePositions(consumePositionList, MergeStrategy.KEEP_LARGE); + ResumeConsumerRequest resumeRequest = new ResumeConsumerRequest(); + resumeRequest.setCube(cubeName); + resumeRequest.setResumeToPosition(streamingSource.getSourcePositionHandler().serializePosition(consumePosition)); + // assume that the resume will always succeed when the replica set can be paused successfully + resumeConsumersInReplicaSet(replicaSet, resumeRequest); + return consumePosition; + } else if (replicaSet.getNodes().size() == 1) { + Node receiver = replicaSet.getNodes().iterator().next(); + StopConsumersRequest request = new StopConsumersRequest(); + request.setCube(cubeName); + logger.info("stop consumers for cube:{}, receiver:{}", cubeName, receiver); + List stopResponse = stopConsumersInReplicaSet(replicaSet, request); + return streamingSource.getSourcePositionHandler().parsePosition(stopResponse.get(0).getConsumePosition()); + } else { + return null; + } + } + + public Map>> reBalancePlan(List allCubes, + List allReplicaSets) { + List currCubeAssignments = streamMetadataStore.getAllCubeAssignments(); + return assigner.reBalancePlan(allReplicaSets, allCubes, currCubeAssignments); + } + + private List getEnableStreamingCubes() { + List allCubes = getStreamingCubes(); + List result = Lists.newArrayList(); + for (StreamingCubeInfo cube : allCubes) { + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube( + cube.getCubeName()); + if (cubeInstance.getStatus() == RealizationStatusEnum.READY) { + result.add(cube); + } + } + return result; + } + + private List getStreamingCubes() { + List cubes = streamMetadataStore.getCubes(); + List result = Lists.newArrayList(); + for (String cubeName : cubes) { + StreamingCubeInfo cubeInfo = getStreamCubeInfo(cubeName); + if (cubeInfo != null) { + result.add(cubeInfo); + } + } + return result; + } + + private void removeCubeHDFSFiles(String cubeName) { + String segmentHDFSPath = HDFSUtil.getStreamingCubeFilePath(cubeName); + try { + FileSystem fs = HadoopUtil.getFileSystem(segmentHDFSPath); + fs.delete(new Path(segmentHDFSPath), true); + } catch (Exception e) { + logger.error("error when remove hdfs file, hdfs path:{}", segmentHDFSPath); + } + } + + public synchronized void createReplicaSet(ReplicaSet rs) { + int replicaSetID = streamMetadataStore.createReplicaSet(rs); + try { + for (Node receiver : rs.getNodes()) { + addReceiverToReplicaSet(receiver, replicaSetID); + } + } catch (IOException e) { + logger.warn("create replica set fail", e); + } + } + + public synchronized void removeReplicaSet(int rsID) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(rsID); + if (rs == null) { + return; + } + if (rs.getNodes() != null && rs.getNodes().size() > 0) { + throw new CoordinateException("cannot remove rs, because there are nodes in it"); + } + Map> assignment = streamMetadataStore.getAssignmentsByReplicaSet(rsID); + if (assignment != null && !assignment.isEmpty()) { + throw new CoordinateException("cannot remove rs, because there are assignments"); + } + streamMetadataStore.removeReplicaSet(rsID); + } + + public synchronized void addNodeToReplicaSet(Integer replicaSetID, String nodeID) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(replicaSetID); + Node receiver = Node.fromNormalizeString(nodeID); + List allReplicaSet = streamMetadataStore.getReplicaSets(); + for (ReplicaSet other : allReplicaSet) { + if (other.getReplicaSetID() != replicaSetID) { + if (other.getNodes().contains(receiver)) { + logger.error("error add Node {} to replicaSet {}, already exist in replicaSet {} ", nodeID, + replicaSetID, other.getReplicaSetID()); + throw new IllegalStateException("Node exists in ReplicaSet!"); + } + } + } + rs.addNode(receiver); + streamMetadataStore.updateReplicaSet(rs); + try { + Map> assignment = streamMetadataStore.getAssignmentsByReplicaSet(replicaSetID); + if (assignment == null || assignment.isEmpty()) { + return; + } + addReceiverToReplicaSet(receiver, replicaSetID); + // clear assign cache for this group + for (String cubeName : assignment.keySet()) { + AssignmentsCache.getInstance().clearCubeCache(cubeName); + } + } catch (IOException e) { + logger.warn("fail to add receiver to replicaSet ", e); + } + } + + public synchronized void removeNodeFromReplicaSet(Integer replicaSetID, String nodeID) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(replicaSetID); + Node receiver = Node.fromNormalizeString(nodeID); + rs.removeNode(receiver); + streamMetadataStore.updateReplicaSet(rs); + try { + Map> assignment = streamMetadataStore.getAssignmentsByReplicaSet(replicaSetID); + removeReceiverFromReplicaSet(receiver); + // clear assign cache for this group + if (assignment != null) { + for (String cubeName : assignment.keySet()) { + AssignmentsCache.getInstance().clearCubeCache(cubeName); + } + } + } catch (IOException e) { + logger.warn("remove node from replicaSet fail", e); + } + } + + public void stopConsumers(String cubeName) { + CubeAssignment assignment = streamMetadataStore.getAssignmentsByCube(cubeName); + StopConsumersRequest request = new StopConsumersRequest(); + request.setCube(cubeName); + try { + for (Integer rsID : assignment.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(rsID); + stopConsumersInReplicaSet(rs, request); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void doReBalance(List previousAssignments, List newAssignments) { + Map previousCubeAssignMap = Maps.newHashMap(); + Map newCubeAssignMap = Maps.newHashMap(); + for (CubeAssignment cubeAssignment : previousAssignments) { + previousCubeAssignMap.put(cubeAssignment.getCubeName(), cubeAssignment); + } + for (CubeAssignment cubeAssignment : newAssignments) { + newCubeAssignMap.put(cubeAssignment.getCubeName(), cubeAssignment); + } + try { + Set preCubes = previousCubeAssignMap.keySet(); + Set newCubes = newCubeAssignMap.keySet(); + if (!preCubes.equals(newCubes)) { + logger.error("previous assignment cubes:" + preCubes + ", new assignment cubes:" + newCubes); + throw new IllegalStateException("previous cube assignments"); + } + + MapDifference diff = Maps.difference(previousCubeAssignMap, newCubeAssignMap); + Map> changedAssignments = diff.entriesDiffering(); + + for (Map.Entry> changedAssignmentEntry : changedAssignments + .entrySet()) { + String cubeName = changedAssignmentEntry.getKey(); + MapDifference.ValueDifference cubeAssignDiff = changedAssignmentEntry.getValue(); + reassignCubeImpl(cubeName, cubeAssignDiff.leftValue(), cubeAssignDiff.rightValue()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void assignCubeToReplicaSet(ReplicaSet rs, String cubeName, List partitions, + boolean startConsumer, boolean mustAllSucceed) throws IOException { + boolean hasNodeAssigned = false; + IOException exception = null; + AssignRequest assignRequest = new AssignRequest(); + assignRequest.setCubeName(cubeName); + assignRequest.setPartitions(partitions); + assignRequest.setStartConsumers(startConsumer); + for (final Node node : rs.getNodes()) { + try { + assignToReceiver(node, assignRequest); + hasNodeAssigned = true; + } catch (IOException e) { + if (mustAllSucceed) { + throw e; + } + exception = e; + logger.error("cube:" + cubeName + " consumers start fail for node:" + node.toString(), e); + } + } + if (!hasNodeAssigned) { + if (exception != null) { + throw exception; + } + } + } + + private void assignToReceiver(final Node receiver, final AssignRequest request) throws IOException { + receiverAdminClient.assign(receiver, request); + } + + private void unAssignFromReplicaSet(final ReplicaSet rs, final UnAssignRequest unAssignRequest) throws IOException { + for (Node receiver : rs.getNodes()) { + unAssignToReceiver(receiver, unAssignRequest); + } + } + + private void unAssignToReceiver(final Node receiver, final UnAssignRequest request) throws IOException { + receiverAdminClient.unAssign(receiver, request); + } + + private void addReceiverToReplicaSet(final Node receiver, final int replicaSetID) throws IOException { + receiverAdminClient.addToReplicaSet(receiver, replicaSetID); + } + + private void removeReceiverFromReplicaSet(final Node receiver) throws IOException { + receiverAdminClient.removeFromReplicaSet(receiver); + } + + private void startConsumersForReceiver(final Node receiver, final StartConsumersRequest request) throws IOException { + receiverAdminClient.startConsumers(receiver, request); + } + + private ConsumerStatsResponse stopConsumersForReceiver(final Node receiver, final StopConsumersRequest request) + throws IOException { + return receiverAdminClient.stopConsumers(receiver, request); + } + + private ConsumerStatsResponse pauseConsumersForReceiver(final Node receiver, final PauseConsumersRequest request) + throws IOException { + return receiverAdminClient.pauseConsumers(receiver, request); + } + + public ConsumerStatsResponse resumeConsumersForReceiver(final Node receiver, final ResumeConsumerRequest request) + throws IOException { + return receiverAdminClient.resumeConsumers(receiver, request); + } + + private void makeCubeImmutableForReceiver(final Node receiver, final String cubeName) throws IOException { + receiverAdminClient.makeCubeImmutable(receiver, cubeName); + } + + public void startConsumersInReplicaSet(ReplicaSet rs, final StartConsumersRequest request) throws IOException { + for (final Node node : rs.getNodes()) { + startConsumersForReceiver(node, request); + } + } + + public void makeCubeImmutableInReplicaSet(ReplicaSet rs, String cubeName) throws IOException { + for (final Node node : rs.getNodes()) { + makeCubeImmutableForReceiver(node, cubeName); + } + } + + public List stopConsumersInReplicaSet(ReplicaSet rs, final StopConsumersRequest request) + throws IOException { + List consumerStats = Lists.newArrayList(); + for (final Node node : rs.getNodes()) { + consumerStats.add(stopConsumersForReceiver(node, request)); + } + return consumerStats; + } + + public List pauseConsumersInReplicaSet(ReplicaSet rs, final PauseConsumersRequest request) + throws IOException { + List consumerStats = Lists.newArrayList(); + List successReceivers = Lists.newArrayList(); + try { + for (final Node node : rs.getNodes()) { + consumerStats.add(pauseConsumersForReceiver(node, request)); + successReceivers.add(node); + } + } catch (IOException ioe) { + //roll back + logger.info("roll back pause consumers for receivers:" + successReceivers); + ResumeConsumerRequest resumeRequest = new ResumeConsumerRequest(); + resumeRequest.setCube(request.getCube()); + for (Node receiver : successReceivers) { + resumeConsumersForReceiver(receiver, resumeRequest); + } + throw ioe; + } + return consumerStats; + } + + public List resumeConsumersInReplicaSet(ReplicaSet rs, final ResumeConsumerRequest request) + throws IOException { + List consumerStats = Lists.newArrayList(); + for (final Node node : rs.getNodes()) { + consumerStats.add(resumeConsumersForReceiver(node, request)); + } + return consumerStats; + } + + public StreamMetadataStore getStreamMetadataStore() { + return streamMetadataStore; + } + + public ExecutableManager getExecutableManager() { + return ExecutableManager.getInstance(getConfig()); + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getConfig()); + } + + private synchronized boolean tryFindAndBuildSegment(String cubeName) { + List segmentStates = streamMetadataStore.getSegmentBuildStates(cubeName); + if (segmentStates.isEmpty()) { + logger.info("no segment build states for cube:{} found in the metadata store", cubeName); + return true; + } + boolean triggered = true; + List segmentsToBuild = findSegmentsCanBuild(cubeName); + if (segmentsToBuild != null && !segmentsToBuild.isEmpty()) { + logger.info("try to trigger cube building for cube:{}, segments:{}", cubeName, segmentsToBuild); + for (String segmentName : segmentsToBuild) { + triggered = triggered && triggerSegmentBuild(cubeName, segmentName); + } + } + + if (!triggered) { + jobStatusChecker.addPendingCube(cubeName); + } + return triggered; + } + + private void segmentBuildComplete(CubingJob cubingJob, CubeInstance cubeInstance, CubeSegment cubeSegment, SegmentJobBuildInfo segmentBuildInfo) throws IOException { + if (!checkPreviousSegmentReady(cubeSegment)) { + logger.warn("the segment:{}'s previous segment is not ready, will not set the segment to ready", + cubeSegment); + return; + } + if (!SegmentStatusEnum.READY.equals(cubeSegment.getStatus())) { + promoteNewSegment(cubingJob, cubeInstance, cubeSegment); + } + String cubeName = segmentBuildInfo.cubeName; + String segmentName = segmentBuildInfo.segmentName; + CubeAssignment assignments = streamMetadataStore.getAssignmentsByCube(cubeName); + for (int replicaSetID : assignments.getReplicaSetIDs()) { + ReplicaSet rs = streamMetadataStore.getReplicaSet(replicaSetID); + for (Node node : rs.getNodes()) { + try { + receiverAdminClient.segmentBuildComplete(node, cubeName, segmentName); + } catch (IOException e) { + logger.error("error when remove cube segment for receiver:" + node, e); + } + } + // for the assignment that doesn't have partitions, check if there is local segments exist + if (assignments.getPartitionsByReplicaSetID(replicaSetID).isEmpty()) { + logger.info( + "no partition is assign to the replicaSet:{}, check whether there are local segments on the rs.", + replicaSetID); + Node leader = rs.getLeader(); + try { + ReceiverCubeStats receiverCubeStats = receiverAdminClient.getReceiverCubeStats(leader, cubeName); + Set segments = receiverCubeStats.getSegmentStatsMap().keySet(); + if (segments.isEmpty()) { + logger.info("no local segments exist for replicaSet:{}, cube:{}, update assignments.", + replicaSetID, cubeName); + assignments.removeAssignment(replicaSetID); + streamMetadataStore.saveNewCubeAssignment(assignments); + } + } catch (IOException e) { + logger.error("error when get receiver cube stats from:" + leader, e); + } + } + } + streamMetadataStore.removeSegmentBuildState(cubeName, segmentName); + logger.info("try to remove the hdfs files for cube:{} segment:{}", cubeName, segmentName); + removeHDFSFiles(cubeName, segmentName); + logger.info("try to find segments for cube:{} build", cubeName); + tryFindAndBuildSegment(segmentBuildInfo.cubeName); // try to build new segment immediately after build complete + } + + private void promoteNewSegment(CubingJob cubingJob, CubeInstance cubeInstance, CubeSegment cubeSegment) throws IOException { + long sourceCount = cubingJob.findSourceRecordCount(); + long sourceSizeBytes = cubingJob.findSourceSizeBytes(); + long cubeSizeBytes = cubingJob.findCubeSizeBytes(); + Map sourceCheckpoint = streamMetadataStore.getSourceCheckpoint(cubeInstance.getName(), cubeSegment.getName()); + + ISourcePositionHandler positionOperator = StreamingSourceFactory.getStreamingSource(cubeInstance).getSourcePositionHandler(); + Collection sourcePositions = Collections2.transform(sourceCheckpoint.values(), new Function() { + @Nullable + @Override + public ISourcePosition apply(@Nullable String input) { + return positionOperator.parsePosition(input); + } + }); + ISourcePosition sourcePosition = positionOperator.mergePositions(sourcePositions, MergeStrategy.KEEP_SMALL); + cubeSegment.setLastBuildJobID(cubingJob.getId()); + cubeSegment.setLastBuildTime(System.currentTimeMillis()); + cubeSegment.setSizeKB(cubeSizeBytes / 1024); + cubeSegment.setInputRecords(sourceCount); + cubeSegment.setInputRecordsSize(sourceSizeBytes); + cubeSegment.setStreamSourceCheckpoint(positionOperator.serializePosition(sourcePosition)); + getCubeManager().promoteNewlyBuiltSegments(cubeInstance, cubeSegment); + } + + private boolean checkPreviousSegmentReady(CubeSegment currSegment) { + long currSegmentStart = currSegment.getTSRange().start.v; + CubeInstance cubeInstance = currSegment.getCubeInstance(); + Segments segments = cubeInstance.getSegments(); + long previousSegmentEnd = -1; + for (CubeSegment segment : segments) { + long segmentEnd = segment.getTSRange().end.v; + if (segmentEnd <= currSegmentStart && segmentEnd > previousSegmentEnd) { + previousSegmentEnd = segmentEnd; + } + } + + if (previousSegmentEnd == -1) {//no previous segment exist + return true; + } + + for (CubeSegment segment : segments) { + long segmentEnd = segment.getTSRange().end.v; + if (segmentEnd == previousSegmentEnd && SegmentStatusEnum.READY.equals(segment.getStatus())) { + return true; // any previous segment is ready return true + } + } + return false; + } + + private void removeHDFSFiles(String cubeName, String segmentName) { + String segmentHDFSPath = HDFSUtil.getStreamingSegmentFilePath(cubeName, segmentName); + try { + FileSystem fs = HadoopUtil.getFileSystem(segmentHDFSPath); + fs.delete(new Path(segmentHDFSPath), true); + } catch (Exception e) { + logger.error("error when remove hdfs file, hdfs path:{}", segmentHDFSPath); + } + } + + private boolean triggerSegmentBuild(String cubeName, String segmentName) { + CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()); + CubeInstance cubeInstance = cubeManager.getCube(cubeName); + try { + Pair segmentRange = CubeSegment.parseSegmentName(segmentName); + logger.info("submit streaming segment build, cube:{} segment:{}", cubeName, segmentName); + CubeSegment newSeg = getCubeManager().appendSegment(cubeInstance, new TSRange(segmentRange.getFirst(), + segmentRange.getSecond())); + DefaultChainedExecutable executable = new StreamingCubingEngine().createStreamingCubingBuilder(newSeg, + "SYSTEM"); + getExecutableManager().addJob(executable); + CubingJob cubingJob = (CubingJob) executable; + newSeg.setLastBuildJobID(cubingJob.getId()); + + SegmentJobBuildInfo segmentJobBuildInfo = new SegmentJobBuildInfo(cubeName, segmentName, cubingJob.getId()); + jobStatusChecker.addSegmentBuildJob(segmentJobBuildInfo); + SegmentBuildState.BuildState state = new SegmentBuildState.BuildState(); + state.setBuildStartTime(System.currentTimeMillis()); + state.setState(SegmentBuildState.BuildState.State.BUILDING); + state.setJobId(cubingJob.getId()); + streamMetadataStore.updateSegmentBuildState(cubeName, segmentName, state); + return true; + } catch (Exception e) { + logger.error("streaming job submit fail, cubeName:" + cubeName + " segment:" + segmentName, e); + return false; + } + } + + private List findSegmentsCanBuild(String cubeName) { + List result = Lists.newArrayList(); + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); + CubeSegment latestHistoryReadySegment = cubeInstance.getLatestReadySegment(); + long minSegmentStart = -1; + if (latestHistoryReadySegment != null) { + minSegmentStart = latestHistoryReadySegment.getTSRange().end.v; + } + int allowMaxBuildingSegments = cubeInstance.getConfig().getMaxBuildingSegments(); + + CubeAssignment assignments = streamMetadataStore.getAssignmentsByCube(cubeName); + Set cubeAssignedReplicaSets = assignments.getReplicaSetIDs(); + List segmentStates = streamMetadataStore.getSegmentBuildStates(cubeName); + Collections.sort(segmentStates); + // TODO need to check whether it is in optimization + int inBuildingSegments = cubeInstance.getBuildingSegments().size(); + int leftQuota = allowMaxBuildingSegments - inBuildingSegments; + + for (int i = 0; i < segmentStates.size(); i++) { + SegmentBuildState segmentState = segmentStates.get(i); + Pair segmentRange = CubeSegment.parseSegmentName(segmentState.getSegmentName()); + if (segmentRange.getFirst() < minSegmentStart) { + logger.warn("the cube segment state is not clear correctly, cube:{} segment:{}, clear it", cubeName, + segmentState.getSegmentName()); + streamMetadataStore.removeSegmentBuildState(cubeName, segmentState.getSegmentName()); + continue; + } + + if (segmentState.isInBuilding()) { + inBuildingSegments ++; + String jobId = segmentState.getState().getJobId(); + logger.info("there is segment in building, cube:{} segment:{} jobId:{}", cubeName, + segmentState.getSegmentName(), jobId); + long buildStartTime = segmentState.getState().getBuildStartTime(); + if (buildStartTime != 0 && jobId != null) { + long buildDuration = System.currentTimeMillis() - buildStartTime; + if (buildDuration < 40 * 60 * 1000) { // if build time larger than 40 minutes, check the job status + continue; + } + CubingJob cubingJob = (CubingJob) getExecutableManager().getJob(jobId); + ExecutableState jobState = cubingJob.getStatus(); + if (ExecutableState.SUCCEED.equals(jobState)) { // job is already succeed, remove the build state + CubeSegment cubeSegment = cubeInstance.getSegment(segmentState.getSegmentName(), null); + if (cubeSegment != null && SegmentStatusEnum.READY == cubeSegment.getStatus()) { + logger.info("job:{} is already succeed, and segment:{} is ready, remove segment build state", jobId, segmentState.getSegmentName()); + streamMetadataStore.removeSegmentBuildState(cubeName, segmentState.getSegmentName()); + } + continue; + } else if (ExecutableState.ERROR.equals(jobState)) { + logger.info("job:{} is error, resume the job", jobId); + getExecutableManager().resumeJob(jobId); + continue; + } else if (ExecutableState.DISCARDED.equals(jobState)) { + // if the job has been discard manually, just think that the segment is not in building + logger.info("job:{} is discard, reset the job state in metaStore", jobId); + SegmentBuildState.BuildState state = new SegmentBuildState.BuildState(); + state.setBuildStartTime(0); + state.setState(SegmentBuildState.BuildState.State.WAIT); + state.setJobId(cubingJob.getId()); + streamMetadataStore.updateSegmentBuildState(cubeName, segmentState.getSegmentName(), state); + segmentState.setState(state); + } else { + logger.info("job:{} is in running, job state: {}", jobId, jobState); + continue; + } + } + } + if (leftQuota <= 0) { + logger.info("No left quota to build segments for cube:" + cubeName); + return result; + } + if (!checkSegmentIsReadyToBuild(segmentStates, i, cubeAssignedReplicaSets)) { + break; + } + result.add(segmentState.getSegmentName()); + leftQuota--; + } + return result; + } + + private boolean checkSegmentIsReadyToBuild(List allSegmentStates, int checkedSegmentIdx, Set cubeAssignedReplicaSets) { + SegmentBuildState checkedSegmentState = allSegmentStates.get(checkedSegmentIdx); + Set notCompleteReplicaSets = Sets.newHashSet(Sets.difference(cubeAssignedReplicaSets, + checkedSegmentState.getCompleteReplicaSets())); + if (notCompleteReplicaSets.isEmpty()) { + return true; + } else { + for (int i = checkedSegmentIdx + 1; i < allSegmentStates.size(); i++) { + SegmentBuildState segmentBuildState = allSegmentStates.get(i); + Set completeReplicaSetsForNext = segmentBuildState.getCompleteReplicaSets(); + Iterator notCompleteRSItr = notCompleteReplicaSets.iterator(); + while (notCompleteRSItr.hasNext()) { + Integer rsID = notCompleteRSItr.next(); + if (completeReplicaSetsForNext.contains(rsID)) { + logger.info( + "the replica set:{} doesn't have data for segment:{}, but have data for later segment:{}", + rsID, checkedSegmentState.getSegmentName(), segmentBuildState.getSegmentName()); + notCompleteRSItr.remove(); + } + } + } + if (notCompleteReplicaSets.isEmpty()) { + return true; + } + } + return false; + } + + private class CoordinatorLeaderSelector extends LeaderSelectorListenerAdapter implements Closeable { + private LeaderSelector leaderSelector; + + public CoordinatorLeaderSelector() { + String path = ZKUtils.COORDINATOR_LEAD; + leaderSelector = new LeaderSelector(zkClient, path, this); + leaderSelector.autoRequeue(); + } + + @Override + public void close() throws IOException { + leaderSelector.close(); + } + + public void start() { + leaderSelector.start(); + } + + @Override + public void takeLeadership(CuratorFramework client) throws Exception { + logger.info("current node become the lead coordinator"); + streamMetadataStore.setCoordinatorNode(NodeUtil.getCurrentNode(DEFAULT_PORT)); + isLead = true; + // check job status every minute + restoreJobStatusChecker(); + while (true) { + try { + Thread.sleep(5 * 60 * 1000); + } catch (InterruptedException exception) { + Thread.interrupted(); + break; + } + if (!leaderSelector.hasLeadership()) { + break; + } + } + logger.info("become the follower coordinator"); + isLead = false; + } + + } + + private class StreamingBuildJobStatusChecker implements Runnable { + private int maxJobTryCnt = 5; + private CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()); + private ConcurrentMap> segmentBuildJobMap = Maps.newConcurrentMap(); + private CopyOnWriteArrayList pendingCubeName = Lists.newCopyOnWriteArrayList(); + + public void addSegmentBuildJob(SegmentJobBuildInfo segmentBuildJob) { + ConcurrentSkipListSet buildInfos = segmentBuildJobMap.get(segmentBuildJob.cubeName); + if (buildInfos == null) { + buildInfos = new ConcurrentSkipListSet<>(); + ConcurrentSkipListSet previousValue = segmentBuildJobMap.putIfAbsent(segmentBuildJob.cubeName, buildInfos); + if (previousValue != null) { + buildInfos = previousValue; + } + } + buildInfos.add(segmentBuildJob); + } + + public void addPendingCube(String cubeName) { + if (!pendingCubeName.contains(cubeName)) { + pendingCubeName.add(cubeName); + } + } + + public void clearCheckCube(String cubeName) { + if (pendingCubeName.contains(cubeName)) { + pendingCubeName.remove(cubeName); + } + segmentBuildJobMap.remove(cubeName); + } + + @Override + public void run() { + try { + if (isLead) { + doRun(); + } + } catch (Exception e) { + logger.error("error", e); + } + } + + private void doRun() { + List successJobs = Lists.newArrayList(); + for (ConcurrentSkipListSet buildInfos : segmentBuildJobMap.values()) { + if (buildInfos.isEmpty()) { + continue; + } + SegmentJobBuildInfo segmentBuildJob = buildInfos.first(); + logger.info("check the cube:{} segment:{} build status", segmentBuildJob.cubeName, + segmentBuildJob.segmentName); + try { + CubingJob cubingJob = (CubingJob) getExecutableManager().getJob(segmentBuildJob.jobID); + ExecutableState jobState = cubingJob.getStatus(); + if (ExecutableState.SUCCEED.equals(jobState)) { + logger.info("job:{} is complete", segmentBuildJob); + CubeInstance cubeInstance = cubeManager.getCube(segmentBuildJob.cubeName).latestCopyForWrite(); + CubeSegment cubeSegment = cubeInstance.getSegment(segmentBuildJob.segmentName, null); + logger.info("the cube:{} segment:{} is ready", segmentBuildJob.cubeName, + segmentBuildJob.segmentName); + segmentBuildComplete(cubingJob, cubeInstance, cubeSegment, segmentBuildJob); + successJobs.add(segmentBuildJob); + } else if (ExecutableState.ERROR.equals(jobState)) { + if (segmentBuildJob.retryCnt < maxJobTryCnt) { + logger.info("job:{} is error, resume the job", segmentBuildJob); + getExecutableManager().resumeJob(segmentBuildJob.jobID); + segmentBuildJob.retryCnt++; + } + } + } catch (Exception e) { + logger.error("error when check streaming segment job build state:" + segmentBuildJob, e); + } + } + + for (SegmentJobBuildInfo successJob : successJobs) { + ConcurrentSkipListSet buildInfos = segmentBuildJobMap.get(successJob.cubeName); + buildInfos.remove(successJob); + } + + List successCubes = Lists.newArrayList(); + for (String cubeName : pendingCubeName) { + logger.info("check the pending cube:{} ", cubeName); + try { + if (tryFindAndBuildSegment(cubeName)) { + successCubes.add(cubeName); + } + } catch (Exception e) { + logger.error("error when try to find and build cube segment:{}" + cubeName, e); + } + } + + for (String successCube : successCubes) { + pendingCubeName.remove(successCube); + } + } + } + + private class SegmentJobBuildInfo implements Comparable{ + public String cubeName; + public String segmentName; + public String jobID; + public int retryCnt = 0; + + public SegmentJobBuildInfo(String cubeName, String segmentName, String jobID) { + this.cubeName = cubeName; + this.segmentName = segmentName; + this.jobID = jobID; + } + + @Override + public String toString() { + return "SegmentJobBuildInfo{" + "cubeName='" + cubeName + '\'' + ", segmentName='" + segmentName + '\'' + + ", jobID='" + jobID + '\'' + ", retryCnt=" + retryCnt + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + SegmentJobBuildInfo that = (SegmentJobBuildInfo) o; + + if (cubeName != null ? !cubeName.equals(that.cubeName) : that.cubeName != null) + return false; + if (segmentName != null ? !segmentName.equals(that.segmentName) : that.segmentName != null) + return false; + return jobID != null ? jobID.equals(that.jobID) : that.jobID == null; + + } + + @Override + public int hashCode() { + int result = cubeName != null ? cubeName.hashCode() : 0; + result = 31 * result + (segmentName != null ? segmentName.hashCode() : 0); + result = 31 * result + (jobID != null ? jobID.hashCode() : 0); + return result; + } + + @Override + public int compareTo(SegmentJobBuildInfo o) { + if (!cubeName.equals(o.cubeName)) { + return cubeName.compareTo(o.cubeName); + } + return segmentName.compareTo(o.segmentName); + } + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/HBaseStreamMetadataStore.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/HBaseStreamMetadataStore.java new file mode 100644 index 00000000000..d2e3686869d --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/HBaseStreamMetadataStore.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.SegmentBuildState; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.source.Partition; + +//TODO implement this later +public class HBaseStreamMetadataStore implements StreamMetadataStore { + + @Override + public List getReceivers() { + return null; + } + + @Override + public List getCubes() { + return null; + } + + @Override + public void addStreamingCube(String cubeName) { + + } + + @Override + public void removeStreamingCube(String cubeName) { + + } + + @Override + public StreamingCubeConsumeState getStreamingCubeConsumeState(String cubeName) { + return null; + } + + @Override + public void saveStreamingCubeConsumeState(String cubeName, StreamingCubeConsumeState state) { + + } + + @Override + public void addReceiver(Node receiver) { + + } + + @Override + public void removeReceiver(Node receiver) { + + } + + @Override + public void removeCubeAssignment(String cubeName) { + + } + + @Override + public void saveNewCubeAssignment(CubeAssignment newCubeAssignment) { + + } + + @Override + public List getAllCubeAssignments() { + return null; + } + + @Override + public Map>> getAllReplicaSetAssignments() { + return null; + } + + @Override + public Map> getAssignmentsByReplicaSet(int replicaSetID) { + return null; + } + + @Override + public CubeAssignment getAssignmentsByCube(String cubeName) { + return null; + } + + @Override + public List getReplicaSets() { + return null; + } + + @Override + public List getReplicaSetIDs() { + return null; + } + + @Override + public ReplicaSet getReplicaSet(int rsID) { + return null; + } + + @Override + public void removeReplicaSet(int rsID) { + + } + + @Override + public int createReplicaSet(ReplicaSet rs) { + return -1; + } + + @Override + public void updateReplicaSet(ReplicaSet rs) { + + } + + @Override + public Node getCoordinatorNode() { + return null; + } + + @Override + public void setCoordinatorNode(Node coordinator) { + } + + @Override + public void saveSourceCheckpoint(String cubeName, String segmentName, int rsID, String sourceCheckpoint) { + + } + + @Override + public Map getSourceCheckpoint(String cubeName, String segmentName) { + return null; + } + + @Override + public void addCompleteReplicaSetForSegmentBuild(String cubeName, String segmentName, int rsID) { + + } + + @Override + public void updateSegmentBuildState(String cubeName, String segmentName, SegmentBuildState.BuildState state) { + + } + + @Override + public List getSegmentBuildStates(String cubeName) { + return null; + } + + @Override + public SegmentBuildState getSegmentBuildState(String cubeName, String segmentName) { + return null; + } + + @Override + public boolean removeSegmentBuildState(String cubeName, String segmentName) { + return true; + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/MockStreamMetadataStore.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/MockStreamMetadataStore.java new file mode 100644 index 00000000000..9d279f814a5 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/MockStreamMetadataStore.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.util.List; +import java.util.Map; + +import com.google.common.collect.Lists; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.SegmentBuildState; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.source.Partition; + +public class MockStreamMetadataStore implements StreamMetadataStore { + + @Override + public List getReceivers() { + return Lists.newArrayList(); + } + + @Override + public List getCubes() { + return Lists.newArrayList(); + } + + @Override + public void addStreamingCube(String cubeName) { + + } + + @Override + public void removeStreamingCube(String cubeName) { + + } + + @Override + public StreamingCubeConsumeState getStreamingCubeConsumeState(String cubeName) { + return null; + } + + @Override + public void saveStreamingCubeConsumeState(String cubeName, StreamingCubeConsumeState state) { + + } + + @Override + public void addReceiver(Node receiver) { + + } + + @Override + public void removeReceiver(Node receiver) { + + } + + @Override + public void removeCubeAssignment(String cubeName) { + + } + + @Override + public void saveNewCubeAssignment(CubeAssignment newCubeAssignment) { + + } + + @Override + public List getAllCubeAssignments() { + return null; + } + + @Override + public Map>> getAllReplicaSetAssignments() { + return null; + } + + @Override + public Map> getAssignmentsByReplicaSet(int replicaSetID) { + return null; + } + + @Override + public CubeAssignment getAssignmentsByCube(String cubeName) { + return null; + } + + @Override + public List getReplicaSets() { + return null; + } + + @Override + public List getReplicaSetIDs() { + return null; + } + + @Override + public ReplicaSet getReplicaSet(int rsID) { + return null; + } + + @Override + public void removeReplicaSet(int rsID) { + + } + + @Override + public int createReplicaSet(ReplicaSet rs) { + return -1; + } + + @Override + public void updateReplicaSet(ReplicaSet rs) { + + } + + @Override + public Node getCoordinatorNode() { + return null; + } + + @Override + public void setCoordinatorNode(Node coordinator) { + } + + @Override + public void saveSourceCheckpoint(String cubeName, String segmentName, int rsID, String sourceCheckpoint) { + + } + + @Override + public Map getSourceCheckpoint(String cubeName, String segmentName) { + return null; + } + + @Override + public void addCompleteReplicaSetForSegmentBuild(String cubeName, String segmentName, int rsID) { + + } + + @Override + public void updateSegmentBuildState(String cubeName, String segmentName, SegmentBuildState.BuildState state) { + + } + + @Override + public List getSegmentBuildStates(String cubeName) { + return null; + } + + @Override + public SegmentBuildState getSegmentBuildState(String cubeName, String segmentName) { + return null; + } + + @Override + public boolean removeSegmentBuildState(String cubeName, String segmentName) { + return true; + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStore.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStore.java new file mode 100644 index 00000000000..68f0cb28409 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStore.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.SegmentBuildState; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.source.Partition; + +public interface StreamMetadataStore { + List getReceivers(); + + List getCubes(); + + void addStreamingCube(String cubeName); + + void removeStreamingCube(String cubeName); + + StreamingCubeConsumeState getStreamingCubeConsumeState(String cubeName); + + void saveStreamingCubeConsumeState(String cubeName, StreamingCubeConsumeState state); + + void addReceiver(Node receiver); + + void removeReceiver(Node receiver); + + void removeCubeAssignment(String cubeName); + + void saveNewCubeAssignment(CubeAssignment newCubeAssignment); + + List getAllCubeAssignments(); + + Map>> getAllReplicaSetAssignments(); + + Map> getAssignmentsByReplicaSet(int replicaSetID); + + CubeAssignment getAssignmentsByCube(String cubeName); + + List getReplicaSets(); + + List getReplicaSetIDs(); + + ReplicaSet getReplicaSet(int rsID); + + void removeReplicaSet(int rsID); + + int createReplicaSet(ReplicaSet rs); + + void updateReplicaSet(ReplicaSet rs); + + Node getCoordinatorNode(); + + void setCoordinatorNode(Node coordinator); + + /** + * save the partition offset information + * @param cubeName + * @param sourceCheckpoint + */ + void saveSourceCheckpoint(String cubeName, String segmentName, int rsID, String sourceCheckpoint); + + /** + * get source checkpoint + * @param cubeName + * @return + */ + Map getSourceCheckpoint(String cubeName, String segmentName); + + /** + * add group id to the segment info, indicate that the segment data + * has been hand over to the remote store + * @param cubeName + * @param segmentName + * @param rsID + */ + void addCompleteReplicaSetForSegmentBuild(String cubeName, String segmentName, int rsID); + + void updateSegmentBuildState(String cubeName, String segmentName, SegmentBuildState.BuildState state); + + /** + * get segment build state + * @param cubeName + * @return + */ + List getSegmentBuildStates(String cubeName); + + /** + * get segment build state + * @param cubeName + * @param segmentName + * @return + */ + SegmentBuildState getSegmentBuildState(String cubeName, String segmentName); + + boolean removeSegmentBuildState(String cubeName, String segmentName); +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStoreFactory.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStoreFactory.java new file mode 100644 index 00000000000..a2a20deb2b8 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamMetadataStoreFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.kylin.common.KylinConfig; + +public class StreamMetadataStoreFactory { + public static final String STORE_TYPE_ZK = "zk"; + public static final String STORE_TYPE_HBASE = "hbase"; + public static final String STORE_TYPE_MOCK = "mock"; + + public static StreamMetadataStore getStreamMetaDataStore() { + KylinConfig config = KylinConfig.getInstanceFromEnv(); + return getStreamMetaDataStore(config.getStreamingMetadataStoreType()); + } + + public static StreamMetadataStore getStreamMetaDataStore(String storeType) { + switch (storeType) { + case STORE_TYPE_ZK: + return getZKStreamMetaDataStore(); + case STORE_TYPE_HBASE: + return getHBaseStreamMetaDataStore(); + case STORE_TYPE_MOCK: + return getMockStreamMetaDataStore(); + default: + throw new IllegalArgumentException("unSupport streaming metadata store type:" + storeType); + } + } + + public static StreamMetadataStore getZKStreamMetaDataStore() { + CuratorFramework client = ZKUtils.getZookeeperClient(); + StreamMetadataStore store = new ZookeeperStreamMetadataStore(client); + return store; + } + + public static StreamMetadataStore getHBaseStreamMetaDataStore() { + StreamMetadataStore store = new HBaseStreamMetadataStore(); + return store; + } + + public static StreamMetadataStore getMockStreamMetaDataStore() { + StreamMetadataStore store = new MockStreamMetadataStore(); + return store; + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamingCubeInfo.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamingCubeInfo.java new file mode 100644 index 00000000000..f8e6367aec7 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/StreamingCubeInfo.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import org.apache.kylin.stream.core.source.StreamingTableSourceInfo; + +public class StreamingCubeInfo { + private String cubeName; + private StreamingTableSourceInfo streamingTableSourceInfo; + private int numOfConsumerTasks; + + public StreamingCubeInfo(String cubeName, StreamingTableSourceInfo streamingTableSourceInfo, int numOfConsumerTasks) { + this.cubeName = cubeName; + this.streamingTableSourceInfo = streamingTableSourceInfo; + this.numOfConsumerTasks = numOfConsumerTasks; + } + + public String getCubeName() { + return cubeName; + } + + public StreamingTableSourceInfo getStreamingTableSourceInfo() { + return streamingTableSourceInfo; + } + + public int getNumOfConsumerTasks() { + return numOfConsumerTasks; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StreamingCubeInfo that = (StreamingCubeInfo) o; + + if (numOfConsumerTasks != that.numOfConsumerTasks) return false; + if (cubeName != null ? !cubeName.equals(that.cubeName) : that.cubeName != null) return false; + return streamingTableSourceInfo != null ? streamingTableSourceInfo.equals(that.streamingTableSourceInfo) : that.streamingTableSourceInfo == null; + + } + + @Override + public int hashCode() { + int result = cubeName != null ? cubeName.hashCode() : 0; + result = 31 * result + (streamingTableSourceInfo != null ? streamingTableSourceInfo.hashCode() : 0); + result = 31 * result + numOfConsumerTasks; + return result; + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZKUtils.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZKUtils.java new file mode 100644 index 00000000000..6f08b441082 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZKUtils.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.util.Arrays; + +import javax.annotation.Nullable; + +import org.apache.commons.lang3.StringUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.storage.hbase.HBaseConnection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; + +public class ZKUtils { + public static final String ZK_ROOT = "/kylin/stream/" + KylinConfig.getInstanceFromEnv().getDeployEnv(); + public static final String COORDINATOR_LEAD = ZK_ROOT + "/coordinator"; + public static final String REPLICASETS_LEADER_ELECT = ZK_ROOT + "/replica_sets_lead"; + private static final Logger logger = LoggerFactory.getLogger(ZKUtils.class); + + public static CuratorFramework getZookeeperClient() { + String zkString = KylinConfig.getInstanceFromEnv().getStreamingCoordinateZK(); + if (zkString == null) { + zkString = getHBaseZKConnString(); + logger.info("streaming zk is not config, use hbase zookeeper:{}", zkString); + } + CuratorFramework client = CuratorFrameworkFactory.builder().connectString(zkString) + .retryPolicy(new ExponentialBackoffRetry(1000, 3)).connectionTimeoutMs(15 * 1000) + .sessionTimeoutMs(60 * 1000).build(); + client.start(); + return client; + } + + private static String getHBaseZKConnString() { + Configuration conf = HBaseConnection.getCurrentHBaseConfiguration(); + final String serverList = conf.get(HConstants.ZOOKEEPER_QUORUM); + final String port = conf.get(HConstants.ZOOKEEPER_CLIENT_PORT); + return StringUtils.join( + Iterables.transform(Arrays.asList(serverList.split(",")), new Function() { + @Nullable + @Override + public String apply(String input) { + return input + ":" + port; + } + }), ","); + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZookeeperStreamMetadataStore.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZookeeperStreamMetadataStore.java new file mode 100644 index 00000000000..4cad0e5aa85 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/ZookeeperStreamMetadataStore.java @@ -0,0 +1,578 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.utils.ZKPaths; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.coordinator.assign.AssignmentUtil; +import org.apache.kylin.stream.coordinator.exception.StoreException; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.SegmentBuildState; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.source.Partition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class ZookeeperStreamMetadataStore implements StreamMetadataStore { + public static final String REPLICA_SET_ROOT = "/replica_sets"; + public static final String RECEIVER_ROOT = "/receivers"; + public static final String CUBE_ROOT = "/cubes"; + public static final String COORDINATOR_NODE = "/coordinator"; + public static final String CUBE_BUILD_STATE = "build_state"; + public static final String CUBE_CONSUME_STATE = "consume_state"; + public static final String CUBE_ASSIGNMENT = "assignment"; + public static final String CUBE_CONSUME_SRC_STATE = "consume_source_state"; + public static final String CUBE_SRC_CHECKPOINT = "source_checkpoint"; + private static final Logger logger = LoggerFactory.getLogger(ZookeeperStreamMetadataStore.class); + private CuratorFramework client; + private String zkRoot; + private String replicaSetRoot; + private String receiverRoot; + private String cubeRoot; + private String coordinatorRoot; + + public ZookeeperStreamMetadataStore(CuratorFramework client) { + this.client = client; + this.zkRoot = ZKUtils.ZK_ROOT; + init(); + } + + private void init() { + try { + replicaSetRoot = zkRoot + REPLICA_SET_ROOT; + receiverRoot = zkRoot + RECEIVER_ROOT; + cubeRoot = zkRoot + CUBE_ROOT; + coordinatorRoot = zkRoot + COORDINATOR_NODE; + + createZKNodeIfNotExist(zkRoot); + createZKNodeIfNotExist(replicaSetRoot); + createZKNodeIfNotExist(receiverRoot); + createZKNodeIfNotExist(cubeRoot); + createZKNodeIfNotExist(coordinatorRoot); + } catch (Exception e) { + logger.error("error when create zk nodes", e); + throw new StoreException(e); + } + } + + private void createZKNodeIfNotExist(String path) throws Exception { + if (client.checkExists().forPath(path) == null) { + client.create().creatingParentsIfNeeded().forPath(path); + } + } + + @Override + public void removeCubeAssignment(String cubeName) { + try { + client.delete().forPath(ZKPaths.makePath(cubeRoot, cubeName, CUBE_ASSIGNMENT)); + } catch (Exception e) { + logger.error("error when remove cube assignment", e); + throw new StoreException(e); + } + } + + @Override + public List getAllCubeAssignments() { + try { + List cubeAssignmentList = Lists.newArrayList(); + List cubes = client.getChildren().forPath(cubeRoot); + for (String cube : cubes) { + String cubeAssignmentPath = getCubeAssignmentPath(cube); + if (client.checkExists().forPath(cubeAssignmentPath) != null) { + byte[] data = client.getData().forPath(cubeAssignmentPath); + CubeAssignment assignment = CubeAssignment.deserializeCubeAssignment(data); + cubeAssignmentList.add(assignment); + } + } + return cubeAssignmentList; + } catch (Exception e) { + logger.error("error when get assignments"); + throw new StoreException(e); + } + } + + @Override + public Map>> getAllReplicaSetAssignments() { + try { + List cubeAssignmentList = getAllCubeAssignments(); + return AssignmentUtil.convertCubeAssign2ReplicaSetAssign(cubeAssignmentList); + } catch (Exception e) { + logger.error("error when get assignments"); + throw new StoreException(e); + } + } + + @Override + public Map> getAssignmentsByReplicaSet(int replicaSetID) { + try { + Map>> replicaSetAssignmentsMap = getAllReplicaSetAssignments(); + return replicaSetAssignmentsMap.get(replicaSetID); + } catch (Exception e) { + logger.error("error when get assignments"); + throw new StoreException(e); + } + } + + @Override + public CubeAssignment getAssignmentsByCube(String cubeName) { + try { + String cubeAssignmentPath = getCubeAssignmentPath(cubeName); + if (client.checkExists().forPath(cubeAssignmentPath) == null) { + return null; + } + byte[] data = client.getData().forPath(cubeAssignmentPath); + CubeAssignment assignment = CubeAssignment.deserializeCubeAssignment(data); + return assignment; + } catch (Exception e) { + logger.error("error when get cube assignment"); + throw new StoreException(e); + } + } + + @Override + public List getReplicaSets() { + List result = Lists.newArrayList(); + try { + List replicaSetIDs = client.getChildren().forPath(replicaSetRoot); + for (String replicaSetID : replicaSetIDs) { + ReplicaSet replicaSet = getReplicaSet(Integer.parseInt(replicaSetID)); + result.add(replicaSet); + } + } catch (Exception e) { + logger.error("error when get replica sets", e); + throw new StoreException(e); + } + return result; + } + + @Override + public List getReplicaSetIDs() { + try { + List replicaSetIDs = client.getChildren().forPath(replicaSetRoot); + return Lists.transform(replicaSetIDs, new Function() { + @Nullable + @Override + public Integer apply(@Nullable String input) { + return Integer.valueOf(input); + } + }); + } catch (Exception e) { + logger.error("error when get replica sets", e); + throw new StoreException(e); + } + } + + @Override + public int createReplicaSet(ReplicaSet rs) { + try { + List rsList = client.getChildren().forPath(replicaSetRoot); + List rsIDList = Lists.transform(rsList, new Function() { + @Nullable + @Override + public Integer apply(@Nullable String input) { + Integer result; + try { + result = Integer.valueOf(input); + } catch (Exception e) { + result = 0; + } + return result; + } + }); + int currMaxID = -1; + if (rsIDList != null && !rsIDList.isEmpty()) { + currMaxID = Collections.max(rsIDList); + } + int newReplicaSetID = currMaxID + 1; + rs.setReplicaSetID(newReplicaSetID); + String replicaSetPath = ZKPaths.makePath(replicaSetRoot, String.valueOf(newReplicaSetID)); + client.create().creatingParentsIfNeeded().forPath(replicaSetPath, serializeReplicaSet(rs)); + return newReplicaSetID; + } catch (Exception e) { + logger.error("error when create replicaSet:" + rs); + throw new StoreException(e); + } + } + + @Override + public void updateReplicaSet(ReplicaSet rs) { + try { + byte[] replicaSetData = serializeReplicaSet(rs); + client.setData().forPath(ZKPaths.makePath(replicaSetRoot, String.valueOf(rs.getReplicaSetID())), + replicaSetData); + } catch (Exception e) { + logger.error("error when update replicaSet:" + rs.getReplicaSetID()); + throw new StoreException(e); + } + } + + @Override + public Node getCoordinatorNode() { + try { + byte[] nodeData = client.getData().forPath(coordinatorRoot); + return JsonUtil.readValue(nodeData, Node.class); + } catch (Exception e) { + logger.error("error when get coordinator", e); + throw new StoreException(e); + } + } + + @Override + public void setCoordinatorNode(Node coordinator) { + try { + byte[] coordinatorBytes = JsonUtil.writeValueAsBytes(coordinator); + client.setData().forPath(coordinatorRoot, coordinatorBytes); + } catch (Exception e) { + logger.error("error when set coordinator", e); + throw new StoreException(e); + } + } + + @Override + public void saveSourceCheckpoint(String cubeName, String segmentName, int rsID, String sourceCheckpoint) { + try { + String path = ZKPaths.makePath(cubeRoot, cubeName, CUBE_SRC_CHECKPOINT, segmentName, + String.valueOf(rsID)); + if (client.checkExists().forPath(path) == null) { + client.create().creatingParentsIfNeeded().forPath(path); + } else { + logger.warn("checkpoint path already existed under path {}", path); + } + client.setData().forPath(path, Bytes.toBytes(sourceCheckpoint)); + } catch (Exception e) { + logger.error("fail to add replicaSet Id to segment build state", e); + throw new StoreException(e); + } + } + + @Override + public Map getSourceCheckpoint(String cubeName, String segmentName) { + try { + Map result = Maps.newHashMap(); + String ckRoot = ZKPaths.makePath(cubeRoot, cubeName, CUBE_SRC_CHECKPOINT, segmentName); + if (client.checkExists().forPath(ckRoot) == null) { + return null; + } + List children = client.getChildren().forPath(ckRoot); + if (children == null) { + return null; + } + for (String child : children) { + String rsPath = ZKPaths.makePath(ckRoot, child); + byte[] checkpointBytes = client.getData().forPath(rsPath); + String sourcePos = Bytes.toString(checkpointBytes); + result.put(Integer.valueOf(child), sourcePos); + } + return result; + } catch (Exception e) { + logger.error("fail to add replicaSet Id to segment build state", e); + throw new StoreException(e); + } + } + + private byte[] serializeReplicaSet(ReplicaSet rs) throws Exception { + String nodesStr = JsonUtil.writeValueAsString(rs); + return Bytes.toBytes(nodesStr); + } + + @Override + public ReplicaSet getReplicaSet(int rsID) { + try { + ReplicaSet result = new ReplicaSet(); + result.setReplicaSetID(rsID); + byte[] replicaSetData = client.getData().forPath(ZKPaths.makePath(replicaSetRoot, String.valueOf(rsID))); + if (replicaSetData != null && replicaSetData.length > 0) { + result = JsonUtil.readValue(Bytes.toString(replicaSetData), ReplicaSet.class); + } + + return result; + } catch (Exception e) { + logger.error("error when get replica set:" + rsID); + throw new StoreException(e); + } + } + + @Override + public void removeReplicaSet(int rsID) { + try { + client.delete().forPath(ZKPaths.makePath(replicaSetRoot, String.valueOf(rsID))); + } catch (Exception e) { + logger.error("error when remove replica set:" + rsID); + throw new StoreException(e); + } + } + + @Override + public List getReceivers() { + List result = Lists.newArrayList(); + try { + List receiverNames = client.getChildren().forPath(receiverRoot); + for (String receiverName : receiverNames) { + Node node = Node.from(receiverName.replace('_', ':')); + result.add(node); + } + } catch (Exception e) { + logger.error("error when fetch receivers", e); + throw new StoreException(e); + } + return result; + } + + @Override + public List getCubes() { + try { + return client.getChildren().forPath(cubeRoot); + } catch (Exception e) { + logger.error("error when fetch cubes", e); + throw new StoreException(e); + } + } + + @Override + public void addStreamingCube(String cube) { + try { + String path = ZKPaths.makePath(cubeRoot, cube); + if (client.checkExists().forPath(path) == null) { + client.create().creatingParentsIfNeeded().forPath(path); + } + } catch (Exception e) { + logger.error("error when add cube", e); + throw new StoreException(e); + } + } + + @Override + public void removeStreamingCube(String cube) { + try { + String path = ZKPaths.makePath(cubeRoot, cube); + if (client.checkExists().forPath(path) != null) { + client.delete().deletingChildrenIfNeeded().forPath(ZKPaths.makePath(cubeRoot, cube)); + } + } catch (Exception e) { + logger.error("error when remove cube", e); + throw new StoreException(e); + } + } + + @Override + public StreamingCubeConsumeState getStreamingCubeConsumeState(String cube) { + try { + String path = getCubeConsumeStatePath(cube); + if (client.checkExists().forPath(path) != null) { + byte[] cubeInfoData = client.getData().forPath(path); + if (cubeInfoData != null && cubeInfoData.length > 0) { + return JsonUtil.readValue(cubeInfoData, StreamingCubeConsumeState.class); + } else { + return StreamingCubeConsumeState.RUNNING; + } + } else { + return StreamingCubeConsumeState.RUNNING; + } + } catch (Exception e) { + logger.error("error when get streaming cube consume state", e); + throw new StoreException(e); + } + } + + @Override + public void saveStreamingCubeConsumeState(String cube, StreamingCubeConsumeState state) { + try { + String path = getCubeConsumeStatePath(cube); + if (client.checkExists().forPath(path) != null) { + client.setData().forPath(path, JsonUtil.writeValueAsBytes(state)); + } else { + client.create().creatingParentsIfNeeded().forPath(path, JsonUtil.writeValueAsBytes(state)); + } + } catch (Exception e) { + logger.error("error when save streaming cube consume state", e); + throw new StoreException(e); + } + } + + @Override + public void addReceiver(Node receiver) { + try { + String receiverPath = ZKPaths.makePath(receiverRoot, receiver.toNormalizeString()); + if (client.checkExists().forPath(receiverPath) == null) { + client.create().creatingParentsIfNeeded().forPath(receiverPath); + } + } catch (Exception e) { + logger.error("error when add new receiver", e); + throw new StoreException(e); + } + } + + @Override + public void removeReceiver(Node receiver) { + try { + String receiverPath = ZKPaths.makePath(receiverRoot, receiver.toNormalizeString()); + if (client.checkExists().forPath(receiverPath) != null) { + client.delete().deletingChildrenIfNeeded().forPath(receiverPath); + } + } catch (Exception e) { + logger.error("error when remove receiver:" + receiver, e); + throw new StoreException(e); + } + } + + @Override + public void saveNewCubeAssignment(CubeAssignment newCubeAssignment) { + logger.info("try saving new cube assignment for:" + newCubeAssignment.getCubeName()); + try { + String path = getCubeAssignmentPath(newCubeAssignment.getCubeName()); + if (client.checkExists().forPath(path) == null) { + client.create().creatingParentsIfNeeded() + .forPath(path, CubeAssignment.serializeCubeAssignment(newCubeAssignment)); + } else { + client.setData().forPath(path, CubeAssignment.serializeCubeAssignment(newCubeAssignment)); + } + } catch (Exception e) { + logger.error("fail to save cube assignment", e); + throw new StoreException(e); + } + } + + public void close() { + try { + client.close(); + } catch (Exception e) { + logger.error("exception throws when close assignmentManager", e); + } + } + + @Override + public void addCompleteReplicaSetForSegmentBuild(String cubeName, String segmentName, int rsID) { + try { + String path = ZKPaths.makePath(cubeRoot, cubeName, CUBE_BUILD_STATE, segmentName, "replica_sets", + String.valueOf(rsID)); + if (client.checkExists().forPath(path) == null) { + client.create().creatingParentsIfNeeded().forPath(path); + } else { + logger.warn("ReplicaSet id {} existed under path {}", rsID, path); + } + } catch (Exception e) { + logger.error("fail to add replicaSet Id to segment build state", e); + throw new StoreException(e); + } + } + + @Override + public void updateSegmentBuildState(String cubeName, String segmentName, SegmentBuildState.BuildState state) { + try { + String stateStr = JsonUtil.writeValueAsString(state); + String path = ZKPaths.makePath(cubeRoot, cubeName, CUBE_BUILD_STATE, segmentName); + client.setData().forPath(path, Bytes.toBytes(stateStr)); + } catch (Exception e) { + logger.error("fail to update segment build state", e); + throw new StoreException(e); + } + } + + @Override + public List getSegmentBuildStates(String cubeName) { + try { + String cubePath = getCubeBuildStatePath(cubeName); + if (client.checkExists().forPath(cubePath) == null) { + return Lists.newArrayList(); + } + List segments = client.getChildren().forPath(cubePath); + List result = Lists.newArrayList(); + for (String segment : segments) { + SegmentBuildState segmentState = doGetSegmentBuildState(cubePath, segment); + result.add(segmentState); + } + return result; + } catch (Exception e) { + logger.error("fail to get segment build states", e); + throw new StoreException(e); + } + } + + @Override + public SegmentBuildState getSegmentBuildState(String cubeName, String segmentName) { + try { + String cubePath = getCubeBuildStatePath(cubeName); + return doGetSegmentBuildState(cubePath, segmentName); + } catch (Exception e) { + logger.error("fail to get cube segment remote store state", e); + throw new StoreException(e); + } + } + + private SegmentBuildState doGetSegmentBuildState(String cubePath, String segmentName) throws Exception { + SegmentBuildState segmentState = new SegmentBuildState(segmentName); + String segmentPath = ZKPaths.makePath(cubePath, segmentName); + byte[] stateBytes = client.getData().forPath(segmentPath); + SegmentBuildState.BuildState state; + if (stateBytes != null && stateBytes.length > 0) { + String stateStr = Bytes.toString(stateBytes); + state = JsonUtil.readValue(stateStr, SegmentBuildState.BuildState.class); + segmentState.setState(state); + } + String replicaSetsPath = ZKPaths.makePath(segmentPath, "replica_sets"); + List replicaSets = client.getChildren().forPath(replicaSetsPath); + for (String replicaSetID : replicaSets) { + segmentState.addCompleteReplicaSet(Integer.valueOf(replicaSetID)); + } + return segmentState; + } + + @Override + public boolean removeSegmentBuildState(String cubeName, String segmentName) { + try { + String path = ZKPaths.makePath(cubeRoot, cubeName, CUBE_BUILD_STATE, segmentName); + if (client.checkExists().forPath(path) != null) { + client.delete().deletingChildrenIfNeeded().forPath(path); + return true; + } else { + logger.warn("cube segment deep store state does not exisit!, path {} ", path); + return false; + } + } catch (Exception e) { + logger.error("fail to remove cube segment deep store state", e); + throw new StoreException(e); + } + } + + private String getCubeAssignmentPath(String cubeName) { + return ZKPaths.makePath(cubeRoot, cubeName, CUBE_ASSIGNMENT); + } + + private String getCubeBuildStatePath(String cubeName) { + return ZKPaths.makePath(cubeRoot, cubeName, CUBE_BUILD_STATE); + } + + private String getCubeConsumeStatePath(String cubeName) { + return ZKPaths.makePath(cubeRoot, cubeName, CUBE_CONSUME_STATE); + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/Assigner.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/Assigner.java new file mode 100644 index 00000000000..74a935bbf71 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/Assigner.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.coordinator.StreamingCubeInfo; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.source.Partition; + +public interface Assigner { + /** + * Plan the rebalance for all the cube assignments among the list of replica set. + * + * @param replicaSets + * @param cubes + * @param existingAssignments + * @return + */ + Map>> reBalancePlan(List replicaSets, + List cubes, List existingAssignments); + + /** + * Assign the cube among the list of replica set. + * + * @param replicaSets + * @param existingAssignments + * @return + */ + CubeAssignment assign(StreamingCubeInfo cube, List replicaSets, List existingAssignments); +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentUtil.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentUtil.java new file mode 100644 index 00000000000..aaa8ad3c125 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentUtil.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.source.Partition; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class AssignmentUtil { + + /** + * + * @param cubeAssignmentList + * @return ReplicaSet assignment map, key is the replicaSet ID, value is the + * streaming cube and its assigned partition map. + */ + public static Map>> convertCubeAssign2ReplicaSetAssign( + List cubeAssignmentList) { + Map>> nodeAssignmentsMap = Maps.newHashMap(); + for (CubeAssignment cubeAssignment : cubeAssignmentList) { + String cubeName = cubeAssignment.getCubeName(); + for (Integer replicaSetID : cubeAssignment.getReplicaSetIDs()) { + List partitions = cubeAssignment.getPartitionsByReplicaSetID(replicaSetID); + Map> nodeAssignment = nodeAssignmentsMap.get(replicaSetID); + if (nodeAssignment == null) { + nodeAssignment = Maps.newHashMap(); + nodeAssignmentsMap.put(replicaSetID, nodeAssignment); + } + nodeAssignment.put(cubeName, partitions); + } + } + return nodeAssignmentsMap; + } + + /** + * + * @param replicaSetAssignmentsMap + * @return CubeAssignment list + */ + public static List convertReplicaSetAssign2CubeAssign( + Map>> replicaSetAssignmentsMap) { + Map>> cubeAssignmentsMap = Maps.newHashMap(); + for (Map.Entry>> entry : replicaSetAssignmentsMap.entrySet()) { + Integer replicaSetID = entry.getKey(); + Map> nodeAssignmentsInfo = entry.getValue(); + for (Map.Entry> assignment : nodeAssignmentsInfo.entrySet()) { + String cubeName = assignment.getKey(); + List partitions = assignment.getValue(); + Map> cubeAssignment = cubeAssignmentsMap.get(cubeName); + if (cubeAssignment == null) { + cubeAssignment = Maps.newHashMap(); + cubeAssignmentsMap.put(cubeName, cubeAssignment); + } + cubeAssignment.put(replicaSetID, partitions); + } + } + + List result = Lists.newArrayList(); + for (Map.Entry>> cubeAssignmentsEntry : cubeAssignmentsMap.entrySet()) { + CubeAssignment cubeAssignment = new CubeAssignment(cubeAssignmentsEntry.getKey(), + cubeAssignmentsEntry.getValue()); + result.add(cubeAssignment); + } + return result; + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentsCache.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentsCache.java new file mode 100644 index 00000000000..8cf3e6091b1 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/AssignmentsCache.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentMap; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.metadata.cachesync.Broadcaster; +import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.Broadcaster.Listener; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.coordinator.StreamMetadataStoreFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; + +public class AssignmentsCache { + private static volatile AssignmentsCache instance = new AssignmentsCache(); + private static final String ASSIGNMENT_ENTITY = "cube_assign"; + private StreamMetadataStore metadataStore; + private ConcurrentMap> cubeAssignmentCache; + + private AssignmentsCache() { + this.metadataStore = StreamMetadataStoreFactory.getStreamMetaDataStore(); + KylinConfig config = KylinConfig.getInstanceFromEnv(); + cubeAssignmentCache = Maps.newConcurrentMap(); + Broadcaster.getInstance(config).registerListener(new AssignCacheSyncListener(), ASSIGNMENT_ENTITY); + } + + public static AssignmentsCache getInstance() { + return instance; + } + + public List getReplicaSetsByCube(String cubeName) { + if (cubeAssignmentCache.get(cubeName) == null) { + synchronized (cubeAssignmentCache) { + if (cubeAssignmentCache.get(cubeName) == null) { + List result = Lists.newArrayList(); + + CubeAssignment assignment = metadataStore.getAssignmentsByCube(cubeName); + for (Integer replicaSetID : assignment.getReplicaSetIDs()) { + result.add(metadataStore.getReplicaSet(replicaSetID)); + } + cubeAssignmentCache.put(cubeName, result); + } + } + } + return cubeAssignmentCache.get(cubeName); + } + + public void clearCubeCache(String cubeName) { + Broadcaster.getInstance(KylinConfig.getInstanceFromEnv()).announce(ASSIGNMENT_ENTITY, + Broadcaster.Event.UPDATE.getType(), cubeName); + cubeAssignmentCache.remove(cubeName); + } + + private class AssignCacheSyncListener extends Listener { + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + cubeAssignmentCache.remove(cacheKey); + } + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssigner.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssigner.java new file mode 100644 index 00000000000..8232177e4de --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssigner.java @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.stream.coordinator.StreamingCubeInfo; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.source.Partition; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class CubePartitionRoundRobinAssigner implements Assigner { + + @Override + public Map>> reBalancePlan(List replicaSets, + List cubes, List existingAssignments) { + Map>> newPlan = Maps.newHashMap(); + if (replicaSets == null || cubes == null || cubes.size() == 0 || replicaSets.size() == 0) { + return newPlan; + } + + ImmutableSet rsIdSet = FluentIterable.from(replicaSets).transform(new Function() { + @Override + public Integer apply(ReplicaSet rs) { + return rs.getReplicaSetID(); + } + }).toImmutableSet(); + + Map>> existingRSAssignmentsMap = AssignmentUtil + .convertCubeAssign2ReplicaSetAssign(existingAssignments); + + //It is possible that there are new partitions coming from the streaming source that are never assigned before. + Set cubePartitions = expandCubePartitions(cubes); + int avgCubePartitionsPerNode = cubePartitions.size() / replicaSets.size(); + + Set cubePartitionsNeedReassign = Sets.newTreeSet(cubePartitions); + + for (Map.Entry>> groupAssignmentEntry : existingRSAssignmentsMap + .entrySet()) { + Integer replicaSetID = groupAssignmentEntry.getKey(); + Map> existNodeAssignment = groupAssignmentEntry.getValue(); + + // some groups are removed + if (!rsIdSet.contains(replicaSetID)) { + continue; + } + List existCubePartitions = expandAndIntersectCubePartitions(existNodeAssignment); + for (CubePartition existCubePartition : existCubePartitions) { + if (cubePartitions.contains(existCubePartition)) { + Map> newGroupAssignment = newPlan.get(replicaSetID); + if (newGroupAssignment == null) { + newGroupAssignment = Maps.newHashMap(); + newPlan.put(replicaSetID, newGroupAssignment); + } + int cubePartitionCnt = calCubePartitionCnt(newGroupAssignment.values()); + if (cubePartitionCnt < avgCubePartitionsPerNode + 1) { + addToGroupAssignment(newGroupAssignment, existCubePartition.cubeName, + existCubePartition.partition); + cubePartitionsNeedReassign.remove(existCubePartition); + } + } + } + } + int rsIdx = 0; + int rsSize = replicaSets.size(); + + LinkedList cubePartitionsNeedReassignList = Lists.newLinkedList(cubePartitionsNeedReassign); + while (!cubePartitionsNeedReassignList.isEmpty()) { + CubePartition cubePartition = cubePartitionsNeedReassignList.peek(); + String cubeName = cubePartition.cubeName; + + Integer replicaSetID = replicaSets.get(rsIdx).getReplicaSetID(); + Map> newGroupAssignment = newPlan.get(replicaSetID); + + if (newGroupAssignment == null) { + newGroupAssignment = Maps.newHashMap(); + newPlan.put(replicaSetID, newGroupAssignment); + } + int cubePartitionCnt = calCubePartitionCnt(newGroupAssignment.values()); + if (cubePartitionCnt < avgCubePartitionsPerNode + 1) { + addToGroupAssignment(newGroupAssignment, cubeName, cubePartition.partition); + cubePartitionsNeedReassignList.remove(); + } + + rsIdx = (rsIdx + 1) % rsSize; + } + return newPlan; + } + + @Override + public CubeAssignment assign(StreamingCubeInfo cube, List replicaSets, + List existingAssignments) { + int existingTotalPartitionNum = 0; + int totalPartitionNum = 0; + final Map replicaSetPartitionNumMap = Maps.newHashMap(); + for (CubeAssignment cubeAssignment : existingAssignments) { + Set replicaSetIDs = cubeAssignment.getReplicaSetIDs(); + for (Integer rsID : replicaSetIDs) { + int rsPartitionNum = cubeAssignment.getPartitionsByReplicaSetID(rsID).size(); + Integer replicaSetPartitionNum = replicaSetPartitionNumMap.get(rsID); + if (replicaSetPartitionNum == null) { + replicaSetPartitionNumMap.put(rsID, rsPartitionNum); + } else { + replicaSetPartitionNumMap.put(rsID, rsPartitionNum + replicaSetPartitionNum); + } + existingTotalPartitionNum += rsPartitionNum; + } + } + + List partitionsOfCube = cube.getStreamingTableSourceInfo().getPartitions(); + int cubePartitionNum = partitionsOfCube.size(); + totalPartitionNum += (existingTotalPartitionNum + cubePartitionNum); + int replicaSetsNum = replicaSets.size(); + int avgPartitionsPerRS = totalPartitionNum / replicaSetsNum; + + // Sort the ReplicaSet by partitions number on it + Collections.sort(replicaSets, new Comparator() { + @Override + public int compare(ReplicaSet o1, ReplicaSet o2) { + Integer partitionNum1Obj = replicaSetPartitionNumMap.get(o1); + int partitionNum1 = partitionNum1Obj == null ? 0 : partitionNum1Obj; + Integer partitionNum2Obj = replicaSetPartitionNumMap.get(o2); + int partitionNum2 = partitionNum2Obj == null ? 0 : partitionNum2Obj; + return partitionNum1 - partitionNum2; + } + }); + + int nextAssignPartitionIdx = 0; + Map> assignments = Maps.newHashMap(); + for (ReplicaSet rs : replicaSets) { + if (nextAssignPartitionIdx >= cubePartitionNum) { + break; + } + Integer replicaSetID = rs.getReplicaSetID(); + Integer partitionNumObj = replicaSetPartitionNumMap.get(replicaSetID); + int partitionNum = partitionNumObj == null ? 0 : partitionNumObj; + int availableRoom = avgPartitionsPerRS - partitionNum; + if (availableRoom <= 0) { + continue; + } + int end = (nextAssignPartitionIdx + availableRoom) < cubePartitionNum ? (nextAssignPartitionIdx + availableRoom) + : cubePartitionNum; + assignments.put(replicaSetID, Lists.newArrayList(partitionsOfCube.subList(nextAssignPartitionIdx, end))); + nextAssignPartitionIdx = end; + } + + if (nextAssignPartitionIdx < cubePartitionNum) { + for (ReplicaSet rs : replicaSets) { + if (nextAssignPartitionIdx >= cubePartitionNum) { + break; + } + Integer replicaSetID = rs.getReplicaSetID(); + Partition part = partitionsOfCube.get(nextAssignPartitionIdx); + List partitions = assignments.get(replicaSetID); + if (partitions == null) { + partitions = Lists.newArrayList(); + assignments.put(replicaSetID, partitions); + } + partitions.add(part); + nextAssignPartitionIdx++; + } + } + + CubeAssignment cubeAssignment = new CubeAssignment(cube.getCubeName(), assignments); + return cubeAssignment; + } + + private int calCubePartitionCnt(Collection> allPartitions) { + int size = 0; + for (List partitions : allPartitions) { + if (partitions != null) { + size += partitions.size(); + } + } + return size; + } + + private Set expandCubePartitions(List cubes) { + Set result = Sets.newHashSet(); + for (StreamingCubeInfo cube : cubes) { + String cubeName = cube.getCubeName(); + List partitionsOfCube = cube.getStreamingTableSourceInfo().getPartitions(); + for (Partition partition : partitionsOfCube) { + result.add(new CubePartition(cubeName, partition)); + } + } + return result; + } + + /** + * expand the node assignment to cube partition list, and the list is intersect by cube name, + * for example: + * + * input node assignment is: {cube1:[1,2,3], cube2:[1,2,3,4], cube3:[1]}, the output would be: + * [[cube1,1],[cube2,1],[cube3,1],[cube1,2],[cube2,2],[cube1,3],[cube2,3],[cube2,4]] + * @param nodeAssignment + * @return + */ + protected List expandAndIntersectCubePartitions(Map> nodeAssignment) { + List result = Lists.newArrayList(); + + Map> reverseMap = Maps.newTreeMap(); + for (Map.Entry> cubePartitionEntry : nodeAssignment.entrySet()) { + String cubeName = cubePartitionEntry.getKey(); + List partitions = cubePartitionEntry.getValue(); + for (Partition partition : partitions) { + Set cubes = reverseMap.get(partition); + if (cubes == null) { + cubes = Sets.newTreeSet(); + reverseMap.put(partition, cubes); + } + cubes.add(cubeName); + } + } + + for (Map.Entry> partitionCubesEntry : reverseMap.entrySet()) { + Partition partition = partitionCubesEntry.getKey(); + Set cubes = partitionCubesEntry.getValue(); + for (String cube : cubes) { + CubePartition cubePartition = new CubePartition(cube, partition); + result.add(cubePartition); + } + } + + return result; + } + + public void addToGroupAssignment(Map> groupAssignment, String cubeName, Partition partition) { + List partitions = groupAssignment.get(cubeName); + if (partitions == null) { + partitions = Lists.newArrayList(); + groupAssignment.put(cubeName, partitions); + } + partitions.add(partition); + } + + protected static class CubePartition implements Comparable { + public String cubeName; + public Partition partition; + + public CubePartition(String cubeName, Partition partition) { + this.cubeName = cubeName; + this.partition = partition; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((cubeName == null) ? 0 : cubeName.hashCode()); + result = prime * result + ((partition == null) ? 0 : partition.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + CubePartition other = (CubePartition) obj; + if (cubeName == null) { + if (other.cubeName != null) + return false; + } else if (!cubeName.equals(other.cubeName)) + return false; + if (partition == null) { + if (other.partition != null) + return false; + } else if (!partition.equals(other.partition)) + return false; + return true; + } + + @Override + public int compareTo(CubePartition other) { + int result = cubeName.compareTo(other.cubeName); + if (result != 0) { + return result; + } + + return partition.getPartitionId() - other.partition.getPartitionId(); + } + + @Override + public String toString() { + return "CubePartition{" + "cubeName='" + cubeName + '\'' + ", partition=" + partition + '}'; + } + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/DefaultAssigner.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/DefaultAssigner.java new file mode 100644 index 00000000000..6e095ada82c --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/assign/DefaultAssigner.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.kylin.stream.coordinator.StreamingCubeInfo; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.source.Partition; + +import com.google.common.collect.Maps; + +/** + * Default implementation for Assigner, assign according to the consumer task number for each cube + * + */ +public class DefaultAssigner implements Assigner { + @Override + public Map>> reBalancePlan(List replicaSets, + List cubes, List existingAssignments) { + Map>> newPlan = Maps.newHashMap(); + if (replicaSets == null || cubes == null || cubes.size() == 0 || replicaSets.size() == 0) { + return newPlan; + } + Map>> existingRSAssignmentsMap = AssignmentUtil + .convertCubeAssign2ReplicaSetAssign(existingAssignments); + Map cubeAssignmentMap = Maps.newHashMap(); + for (CubeAssignment existingAssignment : existingAssignments) { + cubeAssignmentMap.put(existingAssignment.getCubeName(), existingAssignment); + } + Set currReplicaSetIDs = Sets.newHashSet(); + for (ReplicaSet replicaSet : replicaSets) { + currReplicaSetIDs.add(replicaSet.getReplicaSetID()); + } + int totalTasks = 0; + + // find partitions changed cube + Map>> requireReassignTaskMap = Maps.newHashMap(); + Set partitionChangeCubes = Sets.newHashSet(); + for (StreamingCubeInfo cube : cubes) { + CubeAssignment existingAssignment = cubeAssignmentMap.get(cube.getCubeName()); + int prevPartitionNum = existingAssignment.getPartitionIDs().size(); + int currPartitionNum = cube.getStreamingTableSourceInfo().getPartitions().size(); + if (prevPartitionNum != currPartitionNum) { + List> cubeConsumeTasks = splitCubeConsumeTasks(cube, replicaSets.size()); + requireReassignTaskMap.put(cube.getCubeName(), cubeConsumeTasks); + partitionChangeCubes.add(cube.getCubeName()); + } + totalTasks += cube.getNumOfConsumerTasks(); + } + + int avgTasks = totalTasks / replicaSets.size(); + + for (Entry>> rsAssignmentEntry : existingRSAssignmentsMap.entrySet()) { + Integer rsId = rsAssignmentEntry.getKey(); + Map> rsAssignment = rsAssignmentEntry.getValue(); + if (!currReplicaSetIDs.contains(rsId)) { // handle removed replica set assignments + throw new IllegalStateException("current replica sets don't contain rs:" + rsId); + } + Map> newRsTaskMap = newPlan.get(rsId); + if (newRsTaskMap == null) { + newRsTaskMap = Maps.newHashMap(); + newPlan.put(rsId, newRsTaskMap); + } + for (Entry> taskEntry : rsAssignment.entrySet()) { + String cubeName = taskEntry.getKey(); + List partitions = taskEntry.getValue(); + if (partitionChangeCubes.contains(cubeName)) { + continue; + } + if (newRsTaskMap.size() < avgTasks) { + newRsTaskMap.put(cubeName, partitions); + } else { + List> cubeTasks = requireReassignTaskMap.get(cubeName); + if (cubeTasks == null) { + cubeTasks = Lists.newArrayList(); + requireReassignTaskMap.put(cubeName, cubeTasks); + } + cubeTasks.add(partitions); + } + } + } + + for (Entry>> requireReassignTaskEntry : requireReassignTaskMap.entrySet()) { + setNewPlanForCube(requireReassignTaskEntry.getKey(), requireReassignTaskEntry.getValue(), replicaSets, newPlan); + } + return newPlan; + } + + private void setNewPlanForCube(String cubeName, List> tasks, List replicaSets, final Map>> newPlan) { + Collections.sort(replicaSets, new Comparator() { + @Override + public int compare(ReplicaSet o1, ReplicaSet o2) { + Map> rs1Assign = newPlan.get(o1.getReplicaSetID()); + Map> rs2Assign = newPlan.get(o2.getReplicaSetID()); + int taskNum1 = rs1Assign == null ? 0 : rs1Assign.size(); + int taskNum2 = rs2Assign == null ? 0 : rs2Assign.size(); + return taskNum1 - taskNum2; + } + }); + + for (int i = 0; i < tasks.size(); i++) { + List task = tasks.get(i); + int rsId = replicaSets.get(i).getReplicaSetID(); + Map> cubeTaskMap = newPlan.get(rsId); + if (cubeTaskMap == null) { + cubeTaskMap = Maps.newHashMap(); + newPlan.put(rsId, cubeTaskMap); + } + cubeTaskMap.put(cubeName, task); + } + } + + @Override + public CubeAssignment assign(StreamingCubeInfo cube, List replicaSets, + List existingAssignments) { + final Map replicaSetTaskNumMap = Maps.newHashMap(); + for (CubeAssignment cubeAssignment : existingAssignments) { + Set rsIds = cubeAssignment.getReplicaSetIDs(); + for (Integer rsId : rsIds) { + Integer taskNum = replicaSetTaskNumMap.get(rsId); + if (taskNum != null) { + taskNum = taskNum + 1; + } else { + taskNum = 1; + } + replicaSetTaskNumMap.put(rsId, taskNum); + } + } + + Collections.sort(replicaSets, new Comparator() { + @Override + public int compare(ReplicaSet o1, ReplicaSet o2) { + Integer value1 = replicaSetTaskNumMap.get(o1.getReplicaSetID()); + Integer value2 = replicaSetTaskNumMap.get(o2.getReplicaSetID()); + int taskNum1 = value1 == null ? 0 : value1; + int taskNum2 = value2 == null ? 0 : value2; + return taskNum1 - taskNum2; + } + }); + + List> taskPartitions = splitCubeConsumeTasks(cube, replicaSets.size()); + + Map> assignment = Maps.newHashMap(); + for (int i = 0; i < taskPartitions.size(); i++) { + assignment.put(replicaSets.get(i).getReplicaSetID(), taskPartitions.get(i)); + } + CubeAssignment cubeAssignment = new CubeAssignment(cube.getCubeName(), assignment); + return cubeAssignment; + } + + private int getCubeConsumerTasks(StreamingCubeInfo cube, int replicaSetNum) { + int cubeConsumerTaskNum = cube.getNumOfConsumerTasks(); + if (cubeConsumerTaskNum <= 0) { + cubeConsumerTaskNum = 1; + } + List partitionsOfCube = cube.getStreamingTableSourceInfo().getPartitions(); + if (cubeConsumerTaskNum > replicaSetNum) { + cubeConsumerTaskNum = replicaSetNum; + } + if (cubeConsumerTaskNum > partitionsOfCube.size()) { + cubeConsumerTaskNum = partitionsOfCube.size(); + } + return cubeConsumerTaskNum; + } + + private List> splitCubeConsumeTasks(StreamingCubeInfo cube, int replicaSetNum) { + List partitionsOfCube = cube.getStreamingTableSourceInfo().getPartitions(); + int cubeConsumerTaskNum = getCubeConsumerTasks(cube, replicaSetNum); + List> result = Lists.newArrayListWithCapacity(cubeConsumerTaskNum); + for (int i = 0; i < cubeConsumerTaskNum; i++) { + result.add(Lists.newArrayList()); + } + for (int i = 0; i < partitionsOfCube.size(); i++) { + result.get(i % cubeConsumerTaskNum).add(partitionsOfCube.get(i)); + } + return result; + } + + private static class TaskInfo { + public String cubeName; + public List partitions; + + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClient.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClient.java new file mode 100644 index 00000000000..14d814d0b74 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClient.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.client; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.source.Partition; + +public interface CoordinatorClient { + void segmentRemoteStoreComplete(Node receiverNode, String cubeName, Pair segmentRange); + + void replicaSetLeaderChange(int replicaSetId, Node newLeader); + + Map>> reBalanceRecommend(); + + void reBalance(Map>> reBalancePlan); + + void assignCube(String cubeName); + + void unAssignCube(String cubeName); + + void reAssignCube(String cubeName, CubeAssignment newAssignments); + + void createReplicaSet(ReplicaSet rs); + + void removeReplicaSet(int rsID); + + void addNodeToReplicaSet(Integer replicaSetID, String nodeID); + + void removeNodeFromReplicaSet(Integer replicaSetID, String nodeID); + + void pauseConsumers(String cubeName); + + void resumeConsumers(String cubeName); +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClientFactory.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClientFactory.java new file mode 100644 index 00000000000..ef6fdbe9eef --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorClientFactory.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.client; + +import java.net.InetAddress; +import java.net.NetworkInterface; + +import org.apache.kylin.stream.coordinator.Coordinator; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.core.model.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CoordinatorClientFactory { + private static final Logger logger = LoggerFactory.getLogger(CoordinatorClientFactory.class); + + public static CoordinatorClient createCoordinatorClient(StreamMetadataStore streamMetadataStore) { + if (isCoordinatorCoLocate(streamMetadataStore)) { + return Coordinator.getInstance(); + } else { + return new HttpCoordinatorClient(streamMetadataStore); + } + } + + private static boolean isCoordinatorCoLocate(StreamMetadataStore streamMetadataStore) { + try { + Node coordinatorNode = streamMetadataStore.getCoordinatorNode(); + if (coordinatorNode == null) { + logger.warn("no coordinator node registered"); + return true; + } + InetAddress inetAddress = InetAddress.getByName(coordinatorNode.getHost()); + return NetworkInterface.getByInetAddress(inetAddress) != null; + } catch (Exception e) { + logger.error("error when "); + } + return true; + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorResponse.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorResponse.java new file mode 100644 index 00000000000..d46282d945a --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/CoordinatorResponse.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.client; + +public class CoordinatorResponse { + public static final int SUCCESS = 0; + public static final int NOT_LEAD_COORDINATOR = 1; + public static final int UNKNOW_ERROR = 99999; + + private int code; + private Object data; + private String msg; + + public int getCode() { + return code; + } + + public void setCode(int code) { + this.code = code; + } + + public Object getData() { + return data; + } + + public void setData(Object data) { + this.data = data; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + @Override + public String toString() { + return "CoordinatorResponse{" + "code=" + code + ", data=" + data + ", msg='" + msg + '\'' + '}'; + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/HttpCoordinatorClient.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/HttpCoordinatorClient.java new file mode 100644 index 00000000000..8ae3dd0d29a --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/client/HttpCoordinatorClient.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.client; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.RemoteStoreCompleteRequest; +import org.apache.kylin.stream.core.model.ReplicaSetLeaderChangeRequest; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.util.RestService; +import org.apache.kylin.stream.core.util.RetryCallable; +import org.apache.kylin.stream.core.util.RetryCaller; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class HttpCoordinatorClient implements CoordinatorClient { + private static final Logger logger = LoggerFactory.getLogger(HttpCoordinatorClient.class); + + private StreamMetadataStore streamMetadataStore; + private RestService restService; + private Node coordinatorNode; + private RetryCaller retryCaller; + + public HttpCoordinatorClient(StreamMetadataStore metadataStore) { + this.streamMetadataStore = metadataStore; + this.coordinatorNode = metadataStore.getCoordinatorNode(); + int maxRetry = 10; + + this.retryCaller = new RetryCaller(maxRetry, 1000); + int connectionTimeout = 5000; // default connection timeout is 5s, todo + // move to configuration + int readTimeout = 10000; + this.restService = new RestService(connectionTimeout, readTimeout); + } + + @Override + public void segmentRemoteStoreComplete(Node receiverNode, String cubeName, Pair segmentRange) { + logger.info("send receiver remote store complete message to coordinator"); + try { + RemoteStoreCompleteRequest completeRequest = new RemoteStoreCompleteRequest(); + completeRequest.setCubeName(cubeName); + completeRequest.setReceiverNode(receiverNode); + completeRequest.setSegmentStart(segmentRange.getFirst()); + completeRequest.setSegmentEnd(segmentRange.getSecond()); + + String content = JsonUtil.writeValueAsIndentString(completeRequest); + postRequest("/remoteStoreComplete", content); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void replicaSetLeaderChange(int replicaSetId, Node newLeader) { + logger.info("send replicaSet lead change notification to coordinator"); + try { + ReplicaSetLeaderChangeRequest changeRequest = new ReplicaSetLeaderChangeRequest(); + changeRequest.setReplicaSetID(replicaSetId); + changeRequest.setNewLeader(newLeader); + String content = JsonUtil.writeValueAsIndentString(changeRequest); + postRequest("/replicaSetLeaderChange", content); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public Map>> reBalanceRecommend() { + logger.info("send reBalance recommend request to coordinator"); + try { + Object response = getRequest("/balance/recommend"); + return (Map) response; + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void reBalance(Map>> reBalancePlan) { + logger.info("send reBalance request to coordinator"); + try { + String content = JsonUtil.writeValueAsIndentString(reBalancePlan); + postRequest("/balance", content); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void assignCube(String cubeName) { + logger.info("send assign request to coordinator"); + try { + putRequest("/cubes/" + cubeName + "/assign"); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void unAssignCube(String cubeName) { + logger.info("send unAssign request to coordinator"); + try { + putRequest("/cubes/" + cubeName + "/unAssign"); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void reAssignCube(String cubeName, CubeAssignment newAssignments) { + logger.info("send reassign request to coordinator"); + try { + String path = "/cubes/" + cubeName + "/reAssign"; + String content = JsonUtil.writeValueAsIndentString(newAssignments); + postRequest(path, content); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void createReplicaSet(ReplicaSet rs) { + logger.info("send create replicaSet request to coordinator"); + try { + String path = "/replicaSet"; + String content = JsonUtil.writeValueAsIndentString(rs); + postRequest(path, content); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void removeReplicaSet(int rsID) { + logger.info("send remove replicaSet request to coordinator"); + try { + String path = "/replicaSet/" + rsID; + deleteRequest(path); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void addNodeToReplicaSet(Integer replicaSetID, String nodeID) { + logger.info("send add node to replicaSet request to coordinator"); + try { + String path = "/replicaSet/" + replicaSetID + "/" + nodeID; + putRequest(path); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void removeNodeFromReplicaSet(Integer replicaSetID, String nodeID) { + logger.info("send remove node from replicaSet request to coordinator"); + try { + String path = "/replicaSet/" + replicaSetID + "/" + nodeID; + deleteRequest(path); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void pauseConsumers(String cubeName) { + logger.info("send cube pause request to coordinator: {}", cubeName); + try { + String path = "/cubes/" + cubeName + "/pauseConsume"; + putRequest(path); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public void resumeConsumers(String cubeName) { + logger.info("send cube resume request to coordinator: {}", cubeName); + try { + String path = "/cubes/" + cubeName + "/resumeConsume"; + putRequest(path); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + private Object postRequest(final String path, final String requestContent) throws IOException { + CoordinatorResponse response = retryCaller.call(new CoordinatorRetryCallable() { + @Override + public CoordinatorResponse call() throws Exception { + String url = getBaseUrl() + path; + String msg = restService.postRequest(url, requestContent); + return JsonUtil.readValue(msg, CoordinatorResponse.class); + } + }); + return response.getData(); + } + + private Object getRequest(String path) throws IOException { + final String url = getBaseUrl() + path; + CoordinatorResponse response = retryCaller.call(new CoordinatorRetryCallable() { + @Override + public CoordinatorResponse call() throws Exception { + String msg = restService.getRequest(url); + return JsonUtil.readValue(msg, CoordinatorResponse.class); + } + }); + return response.getData(); + } + + private Object putRequest(String path) throws IOException { + final String url = getBaseUrl() + path; + CoordinatorResponse response = retryCaller.call(new CoordinatorRetryCallable() { + @Override + public CoordinatorResponse call() throws Exception { + String msg = restService.putRequest(url); + return JsonUtil.readValue(msg, CoordinatorResponse.class); + } + }); + return response.getData(); + } + + private Object deleteRequest(String path) throws IOException { + final String url = getBaseUrl() + path; + CoordinatorResponse response = retryCaller.call(new CoordinatorRetryCallable() { + @Override + public CoordinatorResponse call() throws Exception { + String msg = restService.deleteRequest(url); + return JsonUtil.readValue(msg, CoordinatorResponse.class); + } + }); + return response.getData(); + } + + private String getBaseUrl() { + Node coordinatorNode = getCoordinator(); + return "http://" + coordinatorNode.getHost() + ":" + coordinatorNode.getPort() + + "/kylin/api/streaming_coordinator"; + } + + private void updateCoordinatorCache() { + this.coordinatorNode = streamMetadataStore.getCoordinatorNode(); + } + + private Node getCoordinator() { + return coordinatorNode; + } + + private abstract class CoordinatorRetryCallable implements RetryCallable { + + @Override + public boolean isResultExpected(CoordinatorResponse result) { + try { + if (result.getCode() == CoordinatorResponse.SUCCESS) { + return true; + } else { + return false; + } + } catch (Exception e) { + logger.error("result is not expected", e); + return false; + } + } + + @Override + public void update() { + updateCoordinatorCache(); + } + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/CoordinateException.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/CoordinateException.java new file mode 100644 index 00000000000..415933e949b --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/CoordinateException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.exception; + +import org.apache.kylin.stream.core.exception.StreamingException; + +public class CoordinateException extends StreamingException { + public CoordinateException() { + super(); + } + + public CoordinateException(String s) { + super(s); + } + + public CoordinateException(String message, Throwable cause) { + super(message, cause); + } + + public CoordinateException(Throwable cause) { + super(cause); + } +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/NotLeadCoordinatorException.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/NotLeadCoordinatorException.java new file mode 100644 index 00000000000..dfb42e81090 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/NotLeadCoordinatorException.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.exception; + +import org.apache.kylin.stream.core.exception.StreamingException; + +public class NotLeadCoordinatorException extends StreamingException { + public NotLeadCoordinatorException() { + super(); + } + + public NotLeadCoordinatorException(String s) { + super(s); + } + + public NotLeadCoordinatorException(String message, Throwable cause) { + super(message, cause); + } + + public NotLeadCoordinatorException(Throwable cause) { + super(cause); + } + +} diff --git a/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/StoreException.java b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/StoreException.java new file mode 100644 index 00000000000..a43951105e3 --- /dev/null +++ b/stream-coordinator/src/main/java/org/apache/kylin/stream/coordinator/exception/StoreException.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.exception; + +public class StoreException extends RuntimeException { + + private static final long serialVersionUID = -9149609663117728575L; + + public StoreException() { + super(); + } + + public StoreException(Throwable t) { + super(t); + } + + public StoreException(String message, Throwable t) { + super(message, t); + } +} diff --git a/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssignerTest.java b/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssignerTest.java new file mode 100644 index 00000000000..aaf0271e041 --- /dev/null +++ b/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/CubePartitionRoundRobinAssignerTest.java @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.coordinator.StreamingCubeInfo; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingTableSourceInfo; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class CubePartitionRoundRobinAssignerTest { + CubePartitionRoundRobinAssigner assigner; + + @Before + public void setup() { + assigner = new CubePartitionRoundRobinAssigner(); + } + + @Test + public void initBalanceTest() { + List receivers = Lists.newArrayList(); + Node node1 = new Node("host1", 9090); + Node node2 = new Node("host2", 9090); + Node node3 = new Node("host3", 9090); + receivers.add(node1); + receivers.add(node2); + receivers.add(node3); + + List replicaSets = Lists.newArrayList(); + ReplicaSet rs1 = new ReplicaSet(); + rs1.setReplicaSetID(1); + ReplicaSet rs2 = new ReplicaSet(); + rs2.setReplicaSetID(2); + ReplicaSet rs3 = new ReplicaSet(); + rs3.setReplicaSetID(3); + replicaSets.add(rs1); + replicaSets.add(rs2); + replicaSets.add(rs3); + + List cubeInfos = Lists.newArrayList(); + + createCubeInfoFromMeta("cube1", 3, cubeInfos); + createCubeInfoFromMeta("cube2", 4, cubeInfos); + createCubeInfoFromMeta("cube3", 2, cubeInfos); + createCubeInfoFromMeta("cube4", 1, cubeInfos); + createCubeInfoFromMeta("cube5", 7, cubeInfos); + + List existingAssignments = Lists.newArrayList(); + Map>> assignmentMap = assigner.reBalancePlan(replicaSets, cubeInfos, + existingAssignments); + + Map> node1Assignments = assignmentMap.get(rs1.getReplicaSetID()); + int node1PartitionCnt = calCubePartitionCnt(node1Assignments.values()); + assertTrue(node1PartitionCnt == 6 || node1PartitionCnt == 5); + + Map> node2Assignments = assignmentMap.get(rs2.getReplicaSetID()); + int node2PartitionCnt = calCubePartitionCnt(node2Assignments.values()); + assertTrue(node2PartitionCnt == 6 || node2PartitionCnt == 5); + + Map> node3Assignments = assignmentMap.get(rs3.getReplicaSetID()); + + int node3PartitionCnt = calCubePartitionCnt(node3Assignments.values()); + assertTrue(node3PartitionCnt == 6 || node3PartitionCnt == 5); + assertEquals(17, node1PartitionCnt + node2PartitionCnt + node3PartitionCnt); + } + + @Test + public void reBalanceTest2() { + List receivers = Lists.newArrayList(); + Node node1 = new Node("host1", 9090); + Node node2 = new Node("host2", 9090); + Node node3 = new Node("host3", 9090); + receivers.add(node1); + receivers.add(node2); + receivers.add(node3); + + List replicaSets = Lists.newArrayList(); + ReplicaSet rs1 = new ReplicaSet(); + rs1.setReplicaSetID(1); + ReplicaSet rs2 = new ReplicaSet(); + rs2.setReplicaSetID(2); + ReplicaSet rs3 = new ReplicaSet(); + rs3.setReplicaSetID(3); + replicaSets.add(rs1); + replicaSets.add(rs2); + replicaSets.add(rs3); + + List cubeInfos = Lists.newArrayList(); + createCubeInfoFromMeta("cube1", 3, cubeInfos); + createCubeInfoFromMeta("cube2", 4, cubeInfos); + createCubeInfoFromMeta("cube3", 2, cubeInfos); + createCubeInfoFromMeta("cube4", 1, cubeInfos); + createCubeInfoFromMeta("cube5", 7, cubeInfos); + + List existingAssignments = Lists.newArrayList(); + //Cube1 has 3 partitions + Map> cube1Assignment = Maps.newHashMap(); + cube1Assignment.put(1, Arrays.asList(new Partition(1), new Partition(3))); + cube1Assignment.put(2, Arrays.asList(new Partition(2))); + existingAssignments.add(new CubeAssignment("cube1", cube1Assignment)); + + //Cube2 has 4 partitions + Map> cube2Assignment = Maps.newHashMap(); + cube2Assignment.put(1, Arrays.asList(new Partition(2), new Partition(4))); + cube2Assignment.put(2, Arrays.asList(new Partition(1), new Partition(3))); + existingAssignments.add(new CubeAssignment("cube2", cube2Assignment)); + + //Cube3 has 2 partitions + Map> cube3Assignment = Maps.newHashMap(); + cube3Assignment.put(1, Arrays.asList(new Partition(2))); + cube3Assignment.put(2, Arrays.asList(new Partition(1))); + existingAssignments.add(new CubeAssignment("cube3", cube3Assignment)); + + //Cube4 has 1 partition + Map> cube4Assignment = Maps.newHashMap(); + cube4Assignment.put(2, Arrays.asList(new Partition(1))); + existingAssignments.add(new CubeAssignment("cube4", cube4Assignment)); + + //Cube5 has 7 partitions + Map> cube5Assignment = Maps.newHashMap(); + cube5Assignment.put(1, Arrays.asList(new Partition(1), new Partition(3), new Partition(5), new Partition(7))); + cube5Assignment.put(2, Arrays.asList(new Partition(2), new Partition(4), new Partition(6))); + existingAssignments.add(new CubeAssignment("cube5", cube5Assignment)); + + Map>> assignmentMap = assigner.reBalancePlan(replicaSets, cubeInfos, + existingAssignments); + + Map> node1Assignments = assignmentMap.get(rs1.getReplicaSetID()); + int node1PartitionCnt = calCubePartitionCnt(node1Assignments.values()); + assertTrue(node1PartitionCnt == 6 || node1PartitionCnt == 5); + + Map> node2Assignments = assignmentMap.get(rs2.getReplicaSetID()); + int node2PartitionCnt = calCubePartitionCnt(node2Assignments.values()); + assertTrue(node2PartitionCnt == 6 || node2PartitionCnt == 5); + + Map> node3Assignments = assignmentMap.get(rs3.getReplicaSetID()); + int node3PartitionCnt = calCubePartitionCnt(node3Assignments.values()); + assertTrue(node3PartitionCnt == 6 || node3PartitionCnt == 5); + + assertEquals(17, node1PartitionCnt + node2PartitionCnt + node3PartitionCnt); + + } + + @Test + public void assignCubeTest() { + List replicaSets = Lists.newArrayList(); + ReplicaSet rs1 = new ReplicaSet(); + rs1.setReplicaSetID(1); + ReplicaSet rs2 = new ReplicaSet(); + rs2.setReplicaSetID(2); + ReplicaSet rs3 = new ReplicaSet(); + rs3.setReplicaSetID(3); + replicaSets.add(rs1); + replicaSets.add(rs2); + replicaSets.add(rs3); + + List cubeInfos = Lists.newArrayList(); + createCubeInfoFromMeta("cube5", 7, cubeInfos); + StreamingCubeInfo cubeInfo = cubeInfos.get(0); + + List existingAssignments = Lists.newArrayList(); + + CubeAssignment cubeAssignment = assigner.assign(cubeInfo, replicaSets, existingAssignments); + assertEquals(cubeAssignment.getAssignments().get(rs1.getReplicaSetID()).size(), 3); + assertEquals(cubeAssignment.getAssignments().get(rs2.getReplicaSetID()).size(), 2); + assertEquals(cubeAssignment.getAssignments().get(rs3.getReplicaSetID()).size(), 2); + } + + @Test + public void testExpandAndIntersectCubePartitions() { + Map> nodeAssignment = Maps.newHashMap(); + nodeAssignment.put("cube1", Arrays.asList(new Partition(1), new Partition(3))); + nodeAssignment.put("cube2", Arrays.asList(new Partition(2), new Partition(4))); + nodeAssignment.put("cube3", Arrays.asList(new Partition(2))); + nodeAssignment.put("cube5", + Arrays.asList(new Partition(1), new Partition(3), new Partition(5), new Partition(7))); + List cubePartitions = assigner + .expandAndIntersectCubePartitions(nodeAssignment); + System.out.println(cubePartitions); + } + + private int calCubePartitionCnt(Collection> allPartitions) { + int size = 0; + for (List partitions : allPartitions) { + if (partitions != null) { + size += partitions.size(); + } + } + return size; + } + + @Test + public void testSplit() { + int calculatorNum = 8; + long[] cuboidIds = new long[2523]; + for (int i = 0; i < 2523; i++) { + cuboidIds[i] = i; + } + int splitSize = cuboidIds.length / calculatorNum; + if (splitSize <= 0) { + splitSize = 1; + } + for (int i = 0; i < calculatorNum; i++) { + long[] cuboidIdSplit; + int start = i * splitSize; + int end = (i + 1) * splitSize; + if (i == calculatorNum - 1) { + end = cuboidIds.length; + } + if (start > cuboidIds.length) { + break; + } + cuboidIdSplit = Arrays.copyOfRange(cuboidIds, start, end); + StringBuilder sb = new StringBuilder(); + for (long l : cuboidIdSplit) { + sb.append(l + ","); + } + System.out.println(i + ":" + sb.toString()); + } + + } + + private void createCubeInfoFromMeta(String cubeName, int partitionNum, List cubeInfos) { + List partitions = Lists.newArrayList(); + for (int i = 1; i <= partitionNum; i++) { + partitions.add(new Partition(i)); + } + StreamingCubeInfo cubeInfo = new StreamingCubeInfo(cubeName, new StreamingTableSourceInfo(partitions), 1); + cubeInfos.add(cubeInfo); + } + +} diff --git a/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/DefaultAssignerTest.java b/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/DefaultAssignerTest.java new file mode 100644 index 00000000000..55a922a7ffa --- /dev/null +++ b/stream-coordinator/src/test/java/org/apache/kylin/stream/coordinator/assign/DefaultAssignerTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.coordinator.StreamingCubeInfo; +import org.apache.kylin.stream.core.model.CubeAssignment; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingTableSourceInfo; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class DefaultAssignerTest { + DefaultAssigner assigner; + + @Before + public void setup() { + assigner = new DefaultAssigner(); + } + + @Test + public void initAssignTest() { + List rsList = Lists.newArrayList(); + ReplicaSet rs1 = new ReplicaSet(); + rs1.setReplicaSetID(0); + ReplicaSet rs2 = new ReplicaSet(); + rs2.setReplicaSetID(1); + ReplicaSet rs3 = new ReplicaSet(); + rs3.setReplicaSetID(2); + rsList.add(rs1); + rsList.add(rs2); + rsList.add(rs3); + + List existingAssignments = Lists.newArrayList(); + Map> cube1Assignment = Maps.newHashMap(); + cube1Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment1 = new CubeAssignment("cube1", cube1Assignment); + + Map> cube2Assignment = Maps.newHashMap(); + cube2Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1))); + cube2Assignment.put(1, Arrays.asList(new Partition(2))); + CubeAssignment assignment2 = new CubeAssignment("cube2", cube2Assignment); + + Map> cube3Assignment = Maps.newHashMap(); + cube3Assignment.put(2, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment3 = new CubeAssignment("cube3", cube3Assignment); + + Map> cube4Assignment = Maps.newHashMap(); + cube4Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment4 = new CubeAssignment("cube4", cube4Assignment); + + Map> cube5Assignment = Maps.newHashMap(); + cube5Assignment.put(2, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment5 = new CubeAssignment("cube5", cube5Assignment); + + existingAssignments.add(assignment1); + existingAssignments.add(assignment2); + existingAssignments.add(assignment3); + existingAssignments.add(assignment4); + existingAssignments.add(assignment5); + + StreamingCubeInfo cube5 = new StreamingCubeInfo("cube5", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1))), 2); + CubeAssignment cubeAssignment = assigner.assign(cube5, rsList, existingAssignments); + + assertEquals(cubeAssignment.getAssignments().size(), 2); + assertTrue(cubeAssignment.getReplicaSetIDs().contains(1)); + assertTrue(cubeAssignment.getReplicaSetIDs().contains(2)); + + cube5 = new StreamingCubeInfo("cube5", new StreamingTableSourceInfo(Arrays.asList(new Partition(0), + new Partition(1))), 3); + cubeAssignment = assigner.assign(cube5, rsList, existingAssignments); + + assertEquals(cubeAssignment.getAssignments().size(), 2); + assertTrue(cubeAssignment.getReplicaSetIDs().contains(1)); + assertTrue(cubeAssignment.getReplicaSetIDs().contains(2)); + + cube5 = new StreamingCubeInfo("cube5", new StreamingTableSourceInfo(Arrays.asList(new Partition(0), + new Partition(1), new Partition(2))), 5); + cubeAssignment = assigner.assign(cube5, rsList, existingAssignments); + + assertEquals(cubeAssignment.getAssignments().size(), 3); + } + + @Test + public void reBalanceTest() { + List rsList = Lists.newArrayList(); + ReplicaSet rs1 = new ReplicaSet(); + rs1.setReplicaSetID(0); + ReplicaSet rs2 = new ReplicaSet(); + rs2.setReplicaSetID(1); + ReplicaSet rs3 = new ReplicaSet(); + rs3.setReplicaSetID(2); + rsList.add(rs1); + rsList.add(rs2); + rsList.add(rs3); + + List existingAssignments = Lists.newArrayList(); + Map> cube1Assignment = Maps.newHashMap(); + cube1Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment1 = new CubeAssignment("cube1", cube1Assignment); + + Map> cube2Assignment = Maps.newHashMap(); + cube2Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1))); + cube2Assignment.put(1, Arrays.asList(new Partition(2))); + CubeAssignment assignment2 = new CubeAssignment("cube2", cube2Assignment); + + Map> cube3Assignment = Maps.newHashMap(); + cube3Assignment.put(2, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment3 = new CubeAssignment("cube3", cube3Assignment); + + Map> cube4Assignment = Maps.newHashMap(); + cube4Assignment.put(0, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment4 = new CubeAssignment("cube4", cube4Assignment); + + Map> cube5Assignment = Maps.newHashMap(); + cube5Assignment.put(2, Arrays.asList(new Partition(0), new Partition(1), new Partition(2))); + CubeAssignment assignment5 = new CubeAssignment("cube5", cube5Assignment); + + existingAssignments.add(assignment1); + existingAssignments.add(assignment2); + existingAssignments.add(assignment3); + existingAssignments.add(assignment4); + existingAssignments.add(assignment5); + + StreamingCubeInfo cube1 = new StreamingCubeInfo("cube1", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1), new Partition(2))), 1); + StreamingCubeInfo cube2 = new StreamingCubeInfo("cube2", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1), new Partition(2))), 2); + StreamingCubeInfo cube3 = new StreamingCubeInfo("cube3", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1), new Partition(2))), 1); + StreamingCubeInfo cube4 = new StreamingCubeInfo("cube4", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1), new Partition(2))), 1); + StreamingCubeInfo cube5 = new StreamingCubeInfo("cube5", new StreamingTableSourceInfo(Arrays.asList( + new Partition(0), new Partition(1), new Partition(2))), 1); + List cubes = Arrays.asList(cube1, cube2, cube3, cube4, cube5); + Map>> result = assigner.reBalancePlan(rsList, cubes, existingAssignments); + for (Map> rsAssignment : result.values()) { + assertEquals(2, rsAssignment.size()); + } + + ReplicaSet rs4 = new ReplicaSet(); + rs4.setReplicaSetID(3); + ReplicaSet rs5 = new ReplicaSet(); + rs5.setReplicaSetID(4); + ReplicaSet rs6 = new ReplicaSet(); + rs6.setReplicaSetID(5); + rsList.add(rs4); + rsList.add(rs5); + rsList.add(rs6); + result = assigner.reBalancePlan(rsList, cubes, existingAssignments); + for (Map> rsAssignment : result.values()) { + assertEquals(1, rsAssignment.size()); + } + + } +} diff --git a/stream-core/pom.xml b/stream-core/pom.xml new file mode 100644 index 00000000000..5198962e72c --- /dev/null +++ b/stream-core/pom.xml @@ -0,0 +1,103 @@ + + + + + + kylin + org.apache.kylin + 2.6.0-SNAPSHOT + + 4.0.0 + kylin-stream-core + jar + Apache Kylin - Stream Core + + + + + + org.apache.kylin + kylin-core-common + + + org.apache.kylin + kylin-core-metadata + + + org.apache.kylin + kylin-core-dictionary + + + org.apache.kylin + kylin-core-cube + + + + org.apache.hbase + hbase-common + provided + + + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-recipes + + + org.rocksdb + rocksdbjni + 5.0.1 + + + io.dropwizard.metrics + metrics-core + + + + net.jpountz.lz4 + lz4 + 1.3.0 + + + + + + junit + junit + test + + + org.apache.kylin + kylin-core-common + test-jar + ${project.parent.version} + test + + + org.apache.hadoop + hadoop-hdfs + ${hadoop2.version} + + + + diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/client/HttpReceiverAdminClient.java b/stream-core/src/main/java/org/apache/kylin/stream/core/client/HttpReceiverAdminClient.java new file mode 100644 index 00000000000..4a7b3d88847 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/client/HttpReceiverAdminClient.java @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.client; + +import java.io.IOException; + +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.model.AssignRequest; +import org.apache.kylin.stream.core.model.ConsumerStatsResponse; +import org.apache.kylin.stream.core.model.HealthCheckInfo; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.PauseConsumersRequest; +import org.apache.kylin.stream.core.model.ResumeConsumerRequest; +import org.apache.kylin.stream.core.model.StartConsumersRequest; +import org.apache.kylin.stream.core.model.StopConsumersRequest; +import org.apache.kylin.stream.core.model.UnAssignRequest; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; +import org.apache.kylin.stream.core.util.RestService; +import org.apache.kylin.stream.core.util.RetryCallable; +import org.apache.kylin.stream.core.util.RetryCaller; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class HttpReceiverAdminClient implements ReceiverAdminClient { + private static final Logger logger = LoggerFactory.getLogger(HttpReceiverAdminClient.class); + private RestService restService; + private int maxRetry; + private int retryPauseTime; + private RetryCaller retryCaller; + + public HttpReceiverAdminClient() { + int connectionTimeout = 5000; // default connection timeout is 5s, todo move to configuration + int readTimeout = 30000; + this.maxRetry = 3; + this.retryPauseTime = 1000; + this.retryCaller = new RetryCaller(maxRetry, retryPauseTime); + this.restService = new RestService(connectionTimeout, readTimeout); + } + + @Override + public void assign(Node receiver, AssignRequest assignRequest) throws IOException { + logger.info("send assign request:{} to receiver:{}", assignRequest, receiver); + final String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/assign"; + final String content = JsonUtil.writeValueAsString(assignRequest); + + retryPostRequest(url, content); + } + + @Override + public void unAssign(Node receiver, UnAssignRequest unAssignRequest) throws IOException { + logger.info("send unAssign request:{} to receiver:{}", unAssignRequest, receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/unAssign"; + String content = JsonUtil.writeValueAsString(unAssignRequest); + + retryPostRequest(url, content); + } + + @Override + public void startConsumers(Node receiver, StartConsumersRequest startRequest) throws IOException { + logger.info("send start request:{} to receiver:{}", startRequest, receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/consumers/start"; + String content = JsonUtil.writeValueAsString(startRequest); + + retryPostRequest(url, content); + } + + @Override + public ConsumerStatsResponse stopConsumers(Node receiver, StopConsumersRequest stopRequest) throws IOException { + logger.info("send stop consume request:{} to receiver:{}", stopRequest, receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/consumers/stop"; + String content = JsonUtil.writeValueAsString(stopRequest); + String retMsg = retryPostRequest(url, content); + return JsonUtil.readValue(retMsg, ConsumerStatsResponse.class); + } + + @Override + public ConsumerStatsResponse pauseConsumers(Node receiver, PauseConsumersRequest suspendRequest) throws IOException { + logger.info("send pause consumer request:{} to receiver:{}", suspendRequest, receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/consumers/pause"; + String content = JsonUtil.writeValueAsString(suspendRequest); + String retMsg = retryPostRequest(url, content); + return JsonUtil.readValue(retMsg, ConsumerStatsResponse.class); + } + + @Override + public ConsumerStatsResponse resumeConsumers(Node receiver, ResumeConsumerRequest resumeRequest) throws IOException { + logger.info("send resume consumer request:{} to receiver:{}", resumeRequest, receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/consumers/resume"; + String content = JsonUtil.writeValueAsString(resumeRequest); + String retMsg = retryPostRequest(url, content); + return JsonUtil.readValue(retMsg, ConsumerStatsResponse.class); + } + + @Override + public void removeCubeSegment(Node receiver, String cubeName, String segmentName) throws IOException { + logger.info("send request to receiver:{} to remove cube segment: {}", receiver, cubeName + "-" + segmentName); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/data/" + cubeName + + "/" + segmentName; + retryDeleteRequest(url); + } + + @Override + public void makeCubeImmutable(Node receiver, String cubeName) throws IOException { + logger.info("send request to receiver:{} to make cube immutable: {}", receiver, cubeName); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/data/" + cubeName + + "/immutable"; + + retryPutRequest(url); + } + + @Override + public void segmentBuildComplete(Node receiver, String cubeName, String segmentName) throws IOException { + logger.info("send request to receiver:{} to notify cube segment build complete: {}", receiver, cubeName + "-" + + segmentName); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + + "/kylin/api/admin/segment_build_complete/" + cubeName + "/" + segmentName; + + retryPutRequest(url); + } + + @Override + public void addToReplicaSet(Node receiver, int replicaSetID) throws IOException { + logger.info("send request to receiver:{} to add to replica set: {}", receiver, replicaSetID); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/replica_set/" + + replicaSetID + "/add"; + + retryPutRequest(url); + } + + @Override + public void removeFromReplicaSet(Node receiver) throws IOException { + logger.info("send request to receiver:{} to remove receiver from replica set", receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/admin/replica_set/remove"; + + retryPutRequest(url); + } + + @Override + public ReceiverStats getReceiverStats(Node receiver) throws IOException { + logger.info("send request to receiver:{} to get receiver stats ", receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/stats"; + + String msg = restService.getRequest(url); + return JsonUtil.readValue(msg, ReceiverStats.class); + + } + + @Override + public ReceiverCubeStats getReceiverCubeStats(Node receiver, String cubeName) throws IOException { + logger.info("send request to receiver:{} to get cube stats for cube:{}", receiver, cubeName); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/stats/cubes/" + cubeName; + + String msg = retryGetRequest(url); + return JsonUtil.readValue(msg, ReceiverCubeStats.class); + } + + @Override + public HealthCheckInfo healthCheck(Node receiver) throws IOException { + logger.info("send request to receiver:{} to do health check", receiver); + String url = "http://" + receiver.getHost() + ":" + receiver.getPort() + "/kylin/api/stats/healthCheck"; + + String msg = restService.getRequest(url); + return JsonUtil.readValue(msg, HealthCheckInfo.class); + } + + private String retryPostRequest(final String url, final String postContent) throws IOException { + return retryCaller.call(new RetryCallable() { + @Override + public String call() throws Exception { + return restService.postRequest(url, postContent); + } + + @Override + public boolean isResultExpected(String result) { + return true; + } + + @Override + public void update() { + } + }); + } + + private String retryDeleteRequest(final String url) throws IOException { + return retryCaller.call(new RetryCallable() { + @Override + public String call() throws Exception { + return restService.deleteRequest(url); + } + + @Override + public boolean isResultExpected(String result) { + return true; + } + + @Override + public void update() { + } + }); + } + + private String retryGetRequest(final String url) throws IOException { + return retryCaller.call(new RetryCallable() { + @Override + public String call() throws Exception { + return restService.getRequest(url); + } + + @Override + public boolean isResultExpected(String result) { + return true; + } + + @Override + public void update() { + } + }); + } + + private String retryPutRequest(final String url) throws IOException { + return retryCaller.call(new RetryCallable() { + @Override + public String call() throws Exception { + return restService.putRequest(url); + } + + @Override + public boolean isResultExpected(String result) { + return true; + } + + @Override + public void update() { + } + }); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/client/ReceiverAdminClient.java b/stream-core/src/main/java/org/apache/kylin/stream/core/client/ReceiverAdminClient.java new file mode 100644 index 00000000000..e080af7af2c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/client/ReceiverAdminClient.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.client; + +import java.io.IOException; + +import org.apache.kylin.stream.core.model.AssignRequest; +import org.apache.kylin.stream.core.model.ConsumerStatsResponse; +import org.apache.kylin.stream.core.model.HealthCheckInfo; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.PauseConsumersRequest; +import org.apache.kylin.stream.core.model.ResumeConsumerRequest; +import org.apache.kylin.stream.core.model.StartConsumersRequest; +import org.apache.kylin.stream.core.model.StopConsumersRequest; +import org.apache.kylin.stream.core.model.UnAssignRequest; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; + +public interface ReceiverAdminClient { + void assign(Node receiver, AssignRequest assignRequest) throws IOException; + + void unAssign(Node receiver, UnAssignRequest unAssignRequest) throws IOException; + + void startConsumers(Node receiver, StartConsumersRequest startRequest) throws IOException; + + ConsumerStatsResponse stopConsumers(Node receiver, StopConsumersRequest stopRequest) throws IOException; + + ConsumerStatsResponse pauseConsumers(Node receiver, PauseConsumersRequest request) throws IOException; + + ConsumerStatsResponse resumeConsumers(Node receiver, ResumeConsumerRequest request) throws IOException; + + void removeCubeSegment(Node receiver, String cubeName, String segmentName) throws IOException; + + void makeCubeImmutable(Node receiver, String cubeName) throws IOException; + + void segmentBuildComplete(Node receiver, String cubeName, String segmentName) throws IOException; + + void addToReplicaSet(Node receiver, int replicaSetID) throws IOException; + + void removeFromReplicaSet(Node receiver) throws IOException; + + ReceiverStats getReceiverStats(Node receiver) throws IOException; + + ReceiverCubeStats getReceiverCubeStats(Node receiver, String cubeName) throws IOException; + + HealthCheckInfo healthCheck(Node receiver) throws IOException; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/exception/IllegalStorageException.java b/stream-core/src/main/java/org/apache/kylin/stream/core/exception/IllegalStorageException.java new file mode 100644 index 00000000000..16d23ac9dc4 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/exception/IllegalStorageException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.exception; + +public class IllegalStorageException extends RuntimeException { + + public IllegalStorageException() { + super(); + } + + public IllegalStorageException(String s) { + super(s); + } + + public IllegalStorageException(String message, Throwable cause) { + super(message, cause); + } + + public IllegalStorageException(Throwable cause) { + super(cause); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/exception/StreamingException.java b/stream-core/src/main/java/org/apache/kylin/stream/core/exception/StreamingException.java new file mode 100644 index 00000000000..f1650cef982 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/exception/StreamingException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.exception; + +public class StreamingException extends RuntimeException { + + public StreamingException() { + super(); + } + + public StreamingException(String s) { + super(s); + } + + public StreamingException(String message, Throwable cause) { + super(message, cause); + } + + public StreamingException(Throwable cause) { + super(cause); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/metrics/StreamingMetrics.java b/stream-core/src/main/java/org/apache/kylin/stream/core/metrics/StreamingMetrics.java new file mode 100644 index 00000000000..adb5c465e72 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/metrics/StreamingMetrics.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.metrics; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricRegistry; + +public class StreamingMetrics { + public static final String CONSUME_RATE_PFX = "streaming.events.consume.cnt"; + private static StreamingMetrics instance = new StreamingMetrics(); + private final MetricRegistry metrics = new MetricRegistry(); + + private StreamingMetrics() { + } + + public static StreamingMetrics getInstance() { + return instance; + } + + public static Meter newMeter(String name) { + MetricRegistry metrics = getInstance().getMetrics(); + return metrics.meter(name); + } + + public MetricRegistry getMetrics() { + return metrics; + } + + public void start() { + // ConsoleReporter reporter = ConsoleReporter.forRegistry(metrics).convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS) + // .build(); + // reporter.start(5, TimeUnit.SECONDS); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/AssignRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/AssignRequest.java new file mode 100644 index 00000000000..d78ead05b14 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/AssignRequest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.util.List; + +import org.apache.kylin.stream.core.source.Partition; + +public class AssignRequest { + private String cubeName; + private List partitions; + private boolean startConsumers = false; + + public String getCubeName() { + return cubeName; + } + + public void setCubeName(String cubeName) { + this.cubeName = cubeName; + } + + public List getPartitions() { + return partitions; + } + + public void setPartitions(List partitions) { + this.partitions = partitions; + } + + public boolean isStartConsumers() { + return startConsumers; + } + + public void setStartConsumers(boolean startConsumers) { + this.startConsumers = startConsumers; + } + + @Override + public String toString() { + return "AssignRequest{" + "cubeName='" + cubeName + '\'' + ", partitions=" + partitions + ", startConsumers=" + + startConsumers + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/ConsumerStatsResponse.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ConsumerStatsResponse.java new file mode 100644 index 00000000000..eb75d914d88 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ConsumerStatsResponse.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class ConsumerStatsResponse { + private String cubeName; + private String consumePosition; + + public String getCubeName() { + return cubeName; + } + + public void setCubeName(String cubeName) { + this.cubeName = cubeName; + } + + public String getConsumePosition() { + return consumePosition; + } + + public void setConsumePosition(String consumePosition) { + this.consumePosition = consumePosition; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/CubeAssignment.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/CubeAssignment.java new file mode 100644 index 00000000000..d82b38dc64e --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/CubeAssignment.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.source.Partition; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/** + * Represents the Kylin streaming cube assignment. + * + */ +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class CubeAssignment { + @JsonProperty("cube_name") + private String cubeName; + + @JsonProperty("assignments") + // Map between the replicaSet Id and partitions + private Map> assignments; + + // Map between partition Id and replicaSet Id + private Map partitionReplicaSetIDMap; + + @JsonCreator + public CubeAssignment(@JsonProperty("cube_name") String cubeName, + @JsonProperty("assignments") Map> assignments) { + this.cubeName = cubeName; + this.assignments = assignments; + this.partitionReplicaSetIDMap = Maps.newHashMap(); + for (Map.Entry> assignEntry : assignments.entrySet()) { + for (Partition partition : assignEntry.getValue()) { + partitionReplicaSetIDMap.put(partition.getPartitionId(), assignEntry.getKey()); + } + } + } + + public static byte[] serializeCubeAssignment(CubeAssignment cubeAssignment) throws IOException { + return JsonUtil.writeValueAsBytes(cubeAssignment); + } + + public static CubeAssignment deserializeCubeAssignment(byte[] assignmentData) throws IOException { + return JsonUtil.readValue(assignmentData, CubeAssignment.class); + } + + public String getCubeName() { + return cubeName; + } + + public Map> getAssignments() { + return assignments; + } + + public Set getReplicaSetIDs() { + return assignments.keySet(); + } + + public Set getPartitionIDs() { + return partitionReplicaSetIDMap.keySet(); + } + + public List getPartitionsByReplicaSetID(Integer replicaSetID) { + List result = assignments.get(replicaSetID); + if (result == null) { + result = Lists.newArrayList(); + } + return result; + } + + public void addAssignment(Integer replicaSetID, List partitions) { + assignments.put(replicaSetID, partitions); + } + + public void removeAssignment(Integer replicaSetID) { + assignments.remove(replicaSetID); + } + + public Integer getReplicaSetIDByPartition(Integer partitionID) { + return partitionReplicaSetIDMap.get(partitionID); + } + + public Integer getPartitionNum() { + return partitionReplicaSetIDMap.size(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((assignments == null) ? 0 : assignments.hashCode()); + result = prime * result + ((cubeName == null) ? 0 : cubeName.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + CubeAssignment other = (CubeAssignment) obj; + if (assignments == null) { + if (other.assignments != null) + return false; + } else if (!assignments.equals(other.assignments)) + return false; + if (cubeName == null) { + if (other.cubeName != null) + return false; + } else if (!cubeName.equals(other.cubeName)) + return false; + return true; + } + + @Override + public String toString() { + return "CubeAssignment [cubeName=" + cubeName + ", assignments=" + assignments + "]"; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataRequest.java new file mode 100644 index 00000000000..dd8b58a4eeb --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataRequest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.util.Set; + +import org.apache.kylin.metadata.model.FunctionDesc; + +public class DataRequest { + private String cubeName; + private String queryId; + private long minSegmentTime; + private String tupleFilter; // Base64 encoded serialized TupleFilter + private String havingFilter; + private Set dimensions; // what contains in Pair is + private Set groups; + private Set metrics; + private int storagePushDownLimit = Integer.MAX_VALUE; + private boolean allowStorageAggregation; + + private long requestSendTime; + private boolean enableDetailProfile; + private String storageBehavior; + + public String getCubeName() { + return cubeName; + } + + public void setCubeName(String cubeName) { + this.cubeName = cubeName; + } + + public long getMinSegmentTime() { + return minSegmentTime; + } + + public void setMinSegmentTime(long minSegmentTime) { + this.minSegmentTime = minSegmentTime; + } + + public String getTupleFilter() { + return tupleFilter; + } + + public void setTupleFilter(String tupleFilter) { + this.tupleFilter = tupleFilter; + } + + public Set getDimensions() { + return dimensions; + } + + public void setDimensions(Set dimensions) { + this.dimensions = dimensions; + } + + public Set getGroups() { + return groups; + } + + public void setGroups(Set groups) { + this.groups = groups; + } + + public Set getMetrics() { + return metrics; + } + + public void setMetrics(Set metrics) { + this.metrics = metrics; + } + + public String getQueryId() { + return queryId; + } + + public void setQueryId(String queryId) { + this.queryId = queryId; + } + + public int getStoragePushDownLimit() { + return storagePushDownLimit; + } + + public void setStoragePushDownLimit(int storagePushDownLimit) { + this.storagePushDownLimit = storagePushDownLimit; + } + + public boolean isAllowStorageAggregation() { + return allowStorageAggregation; + } + + public void setAllowStorageAggregation(boolean allowStorageAggregation) { + this.allowStorageAggregation = allowStorageAggregation; + } + + public boolean isEnableDetailProfile() { + return enableDetailProfile; + } + + public void setEnableDetailProfile(boolean enableDetailProfile) { + this.enableDetailProfile = enableDetailProfile; + } + + public long getRequestSendTime() { + return requestSendTime; + } + + public void setRequestSendTime(long requestSendTime) { + this.requestSendTime = requestSendTime; + } + + public String getStorageBehavior() { + return storageBehavior; + } + + public void setStorageBehavior(String storageBehavior) { + this.storageBehavior = storageBehavior; + } + + public String getHavingFilter() { + return havingFilter; + } + + public void setHavingFilter(String havingFilter) { + this.havingFilter = havingFilter; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataResponse.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataResponse.java new file mode 100644 index 00000000000..9baebc68ad4 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/DataResponse.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class DataResponse { + private String data; // base64 encoded data, which is list of GTRecord + + private String profile; + + public String getData() { + return data; + } + + public void setData(String data) { + this.data = data; + } + + public String getProfile() { + return profile; + } + + public void setProfile(String profile) { + this.profile = profile; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/HealthCheckInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/HealthCheckInfo.java new file mode 100644 index 00000000000..3acc22f8da1 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/HealthCheckInfo.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class HealthCheckInfo { + private Status status; + + public Status getStatus() { + return status; + } + + public void setStatus(Status status) { + this.status = status; + } + + public enum Status { + GOOD, BAD + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/Node.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/Node.java new file mode 100644 index 00000000000..e013fa1bd49 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/Node.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class Node { + @JsonProperty("host") + private String host; + @JsonProperty("port") + private int port; + @JsonProperty("properties") + private Map properties; + + @JsonCreator + public Node(@JsonProperty("host") String host, @JsonProperty("port") int port) { + this.host = host; + this.port = port; + } + + public static Node from(String nodeString) { + return from(nodeString, ":"); + } + + public static Node fromNormalizeString(String nodeString) { + return from(nodeString, "_"); + } + + public static Node from(String nodeString, String separateStr) { + int lastIdx = nodeString.lastIndexOf(separateStr); + if (lastIdx == -1) { + throw new IllegalArgumentException("illegal host port string:" + nodeString); + } + String host = nodeString.substring(0, lastIdx); + int port = Integer.parseInt(nodeString.substring(lastIdx + separateStr.length())); + return new Node(host, port); + } + + public String getHost() { + return host; + } + + public int getPort() { + return port; + } + + public String toString() { + return host + ":" + port; + } + + public String toNormalizeString() { + return host + "_" + port; + } + + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + Node node = (Node) o; + + if (port != node.port) + return false; + return host != null ? host.equals(node.host) : node.host == null; + + } + + @Override + public int hashCode() { + int result = host != null ? host.hashCode() : 0; + result = 31 * result + port; + return result; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/PauseConsumersRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/PauseConsumersRequest.java new file mode 100644 index 00000000000..16ec0229e0a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/PauseConsumersRequest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class PauseConsumersRequest { + private String cube; + + public String getCube() { + return cube; + } + + public void setCube(String cube) { + this.cube = cube; + } + + @Override + public String toString() { + return "PauseConsumersRequest{" + "cube='" + cube + '\'' + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/RemoteStoreCompleteRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/RemoteStoreCompleteRequest.java new file mode 100644 index 00000000000..996cfd25382 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/RemoteStoreCompleteRequest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class RemoteStoreCompleteRequest { + private Node receiverNode; + private String cubeName; + private long segmentStart; + private long segmentEnd; + + public Node getReceiverNode() { + return receiverNode; + } + + public void setReceiverNode(Node receiverNode) { + this.receiverNode = receiverNode; + } + + public String getCubeName() { + return cubeName; + } + + public void setCubeName(String cubeName) { + this.cubeName = cubeName; + } + + public long getSegmentStart() { + return segmentStart; + } + + public void setSegmentStart(long segmentStart) { + this.segmentStart = segmentStart; + } + + public long getSegmentEnd() { + return segmentEnd; + } + + public void setSegmentEnd(long segmentEnd) { + this.segmentEnd = segmentEnd; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSet.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSet.java new file mode 100644 index 00000000000..6280b1724af --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSet.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.util.Set; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; + +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +/** + * + * A ReplicaSet ensures that a specified number of kylin receiver nodes “replicas” are running at any given time to provide HA service. + * + */ +public class ReplicaSet { + @JsonProperty("rs_id") + private int replicaSetID; + @JsonProperty("nodes") + private Set nodes; + @JsonProperty("leader") + private Node leader; + + public ReplicaSet() { + this.nodes = Sets.newHashSet(); + } + + public void addNode(Node node) { + nodes.add(node); + } + + public void removeNode(Node node) { + nodes.remove(node); + } + + public Set getNodes() { + return nodes; + } + + public void setNodes(Set nodes) { + this.nodes = nodes; + } + + public Node getLeader() { + return leader; + } + + public void setLeader(Node leader) { + this.leader = leader; + } + + public int getReplicaSetID() { + return replicaSetID; + } + + public void setReplicaSetID(int replicaSetID) { + this.replicaSetID = replicaSetID; + } + + public boolean containPhysicalNode(Node node) { + return nodes.contains(node); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + ReplicaSet that = (ReplicaSet) o; + + return replicaSetID == that.replicaSetID; + + } + + @Override + public int hashCode() { + return replicaSetID; + } + + @Override + public String toString() { + return "ReplicaSet{" + "replicaSetID=" + replicaSetID + ", nodes=" + nodes + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSetLeaderChangeRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSetLeaderChangeRequest.java new file mode 100644 index 00000000000..53ba95f570f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ReplicaSetLeaderChangeRequest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class ReplicaSetLeaderChangeRequest { + private Node newLeader; + private int replicaSetID; + + public Node getNewLeader() { + return newLeader; + } + + public void setNewLeader(Node newLeader) { + this.newLeader = newLeader; + } + + public int getReplicaSetID() { + return replicaSetID; + } + + public void setReplicaSetID(int replicaSetID) { + this.replicaSetID = replicaSetID; + } + + @Override + public String toString() { + return "ReplicaSetLeaderChangeRequest{" + "newLeader=" + newLeader + ", replicaSetID=" + replicaSetID + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/ResumeConsumerRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ResumeConsumerRequest.java new file mode 100644 index 00000000000..144d0bcdf1b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/ResumeConsumerRequest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class ResumeConsumerRequest { + private String cube; + private String resumeToPosition; + + public String getCube() { + return cube; + } + + public void setCube(String cube) { + this.cube = cube; + } + + public String getResumeToPosition() { + return resumeToPosition; + } + + public void setResumeToPosition(String resumeToPosition) { + this.resumeToPosition = resumeToPosition; + } + + @Override + public String toString() { + return "ResumeConsumerRequest{" + "cube='" + cube + '\'' + ", resumeToPosition='" + resumeToPosition + '\'' + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/SegmentBuildState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/SegmentBuildState.java new file mode 100644 index 00000000000..c217fa15a5b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/SegmentBuildState.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import java.util.HashSet; +import java.util.Set; + +public class SegmentBuildState implements Comparable { + + private String segmentName; + private Set completeReplicaSets = new HashSet<>(); + private BuildState state; + + public SegmentBuildState(String segmentName) { + this.segmentName = segmentName; + } + + public Set getCompleteReplicaSets() { + return completeReplicaSets; + } + + public void setCompleteReplicaSets(Set completeReplicaSets) { + this.completeReplicaSets = completeReplicaSets; + } + + public void addCompleteReplicaSet(int replicaSetID) { + this.completeReplicaSets.add(replicaSetID); + } + + public BuildState getState() { + return state; + } + + public void setState(BuildState state) { + this.state = state; + } + + public boolean isInBuilding() { + if (state == null) { + return false; + } + + return BuildState.State.BUILDING.equals(state.getState()); + } + + public boolean isInWaiting() { + return state == null || BuildState.State.WAIT.equals(state.getState()); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + SegmentBuildState that = (SegmentBuildState) o; + + if (segmentName != null ? !segmentName.equals(that.segmentName) : that.segmentName != null) + return false; + if (completeReplicaSets != null ? !completeReplicaSets.equals(that.completeReplicaSets) + : that.completeReplicaSets != null) + return false; + return state != null ? state.equals(that.state) : that.state == null; + + } + + @Override + public int hashCode() { + int result = segmentName != null ? segmentName.hashCode() : 0; + result = 31 * result + (completeReplicaSets != null ? completeReplicaSets.hashCode() : 0); + result = 31 * result + (state != null ? state.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "SegmentBuildState{" + "segmentName='" + segmentName + '\'' + ", completeReplicaSets=" + + completeReplicaSets + ", state=" + state + '}'; + } + + @Override + public int compareTo(SegmentBuildState o) { + return this.segmentName.compareTo(o.segmentName); + } + + public String getSegmentName() { + return segmentName; + } + + public void setSegmentName(String segmentName) { + this.segmentName = segmentName; + } + + public static class BuildState { + private State state = State.WAIT; + private long buildStartTime; + private String jobId; + + public State getState() { + return state; + } + + public void setState(State state) { + this.state = state; + } + + public long getBuildStartTime() { + return buildStartTime; + } + + public void setBuildStartTime(long buildStartTime) { + this.buildStartTime = buildStartTime; + } + + public String getJobId() { + return jobId; + } + + public void setJobId(String jobId) { + this.jobId = jobId; + } + + public enum State { + WAIT, BUILDING, COMPLETE + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/StartConsumersRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StartConsumersRequest.java new file mode 100644 index 00000000000..81383a696a9 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StartConsumersRequest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import org.apache.kylin.stream.core.consumer.ConsumerStartProtocol; + +public class StartConsumersRequest { + private String cube; + private ConsumerStartProtocol startProtocol; + + public String getCube() { + return cube; + } + + public void setCube(String cube) { + this.cube = cube; + } + + public ConsumerStartProtocol getStartProtocol() { + return startProtocol; + } + + public void setStartProtocol(ConsumerStartProtocol startProtocol) { + this.startProtocol = startProtocol; + } + + @Override + public String toString() { + return "StartConsumersRequest{" + "cube='" + cube + '\'' + ", startProtocol='" + startProtocol + '\'' + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/StopConsumersRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StopConsumersRequest.java new file mode 100644 index 00000000000..8e415b133bf --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StopConsumersRequest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class StopConsumersRequest { + private String cube; + private boolean removeData; // indicate remove the existing data or not + + public String getCube() { + return cube; + } + + public void setCube(String cube) { + this.cube = cube; + } + + public boolean isRemoveData() { + return removeData; + } + + public void setRemoveData(boolean removeData) { + this.removeData = removeData; + } + + @Override + public String toString() { + return "StopConsumersRequest{" + "cube='" + cube + '\'' + ", removeData=" + removeData + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingCubeConsumeState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingCubeConsumeState.java new file mode 100644 index 00000000000..317f2f2399b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingCubeConsumeState.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public enum StreamingCubeConsumeState { + RUNNING, PAUSED, STOPPED +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingMessage.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingMessage.java new file mode 100644 index 00000000000..c128bfcf5e8 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/StreamingMessage.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +import org.apache.kylin.stream.core.source.ISourcePosition.IPartitionPosition; + +import java.util.List; +import java.util.Map; + +/** + */ +public class StreamingMessage { + private final List data; + private IPartitionPosition sourcePosition; + private long timestamp; + private Map params; + private boolean filtered; + + public StreamingMessage(List data, IPartitionPosition partitionPosition, long timestamp, Map params) { + this.data = data; + this.sourcePosition = partitionPosition; + this.timestamp = timestamp; + this.params = params; + } + + public final List getData() { + return data; + } + + public IPartitionPosition getSourcePosition() { + return sourcePosition; + } + + public boolean isFiltered() { + return filtered; + } + + public void setFiltered(boolean filtered) { + this.filtered = filtered; + } + + public void setSourcePosition(IPartitionPosition sourcePosition) { + this.sourcePosition = sourcePosition; + } + + public final long getTimestamp() { + return timestamp; + } + + public Map getParams() { + return params; + } + + @Override + public String toString() { + return "StreamingMessage{" + + "data=" + data + + ", sourcePosition=" + sourcePosition + + ", timestamp=" + timestamp + + ", params=" + params + + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/UnAssignRequest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/UnAssignRequest.java new file mode 100644 index 00000000000..c8a04ebe6a3 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/UnAssignRequest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model; + +public class UnAssignRequest { + private String cube; + + public String getCube() { + return cube; + } + + public void setCube(String cube) { + this.cube = cube; + } + + @Override + public String toString() { + return "UnAssignRequest{" + "cube='" + cube + '\'' + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ClusterState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ClusterState.java new file mode 100644 index 00000000000..64fe3ad6899 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ClusterState.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ClusterState { + @JsonProperty("last_update_time") + private long lastUpdateTime; + + @JsonProperty("rs_states") + private List replicaSetStates = Lists.newArrayList(); + + @JsonProperty("available_receivers") + private List availableReceivers = Lists.newArrayList(); + + public long getLastUpdateTime() { + return lastUpdateTime; + } + + public void setLastUpdateTime(long lastUpdateTime) { + this.lastUpdateTime = lastUpdateTime; + } + + public List getReplicaSetStates() { + return replicaSetStates; + } + + public void setReplicaSetStates(List replicaSetStates) { + this.replicaSetStates = replicaSetStates; + } + + public List getAvailableReceivers() { + return availableReceivers; + } + + public void setAvailableReceivers(List availableReceivers) { + this.availableReceivers = availableReceivers; + } + + public void addReplicaSetState(ReplicaSetState replicaSetState) { + if (replicaSetStates == null) { + replicaSetStates = Lists.newArrayList(); + } + replicaSetStates.add(replicaSetState); + } + + public void addAvailableReveiverState(ReceiverState receiverState) { + if (availableReceivers == null) { + availableReceivers = Lists.newArrayList(); + } + availableReceivers.add(receiverState); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ConsumerStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ConsumerStats.java new file mode 100644 index 00000000000..732cf427d2a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ConsumerStats.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ConsumerStats { + @JsonProperty("total_incoming_events") + private long totalIncomingEvents; + + @JsonProperty("total_exception_events") + private long totalExceptionEvents; + + @JsonProperty("partition_consume_stats") + private Map partitionConsumeStatsMap = Maps.newHashMap(); + + @JsonProperty("paused") + private boolean paused; + + @JsonProperty("stopped") + private boolean stopped; + + @JsonProperty("consume_offset_info") + private String consumeOffsetInfo; + + public Map getPartitionConsumeStatsMap() { + return partitionConsumeStatsMap; + } + + public void setPartitionConsumeStatsMap(Map partitionConsumeStatsMap) { + this.partitionConsumeStatsMap = partitionConsumeStatsMap; + } + + public String getConsumeOffsetInfo() { + return consumeOffsetInfo; + } + + public void setConsumeOffsetInfo(String consumeOffsetInfo) { + this.consumeOffsetInfo = consumeOffsetInfo; + } + + public long getTotalIncomingEvents() { + return totalIncomingEvents; + } + + public void setTotalIncomingEvents(long totalIncomingEvents) { + this.totalIncomingEvents = totalIncomingEvents; + } + + public long getTotalExceptionEvents() { + return totalExceptionEvents; + } + + public void setTotalExceptionEvents(long totalExceptionEvents) { + this.totalExceptionEvents = totalExceptionEvents; + } + + public boolean isPaused() { + return paused; + } + + public void setPaused(boolean paused) { + this.paused = paused; + } + + public boolean isStopped() { + return stopped; + } + + public void setStopped(boolean stopped) { + this.stopped = stopped; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/CubeRealTimeState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/CubeRealTimeState.java new file mode 100644 index 00000000000..132601a36b8 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/CubeRealTimeState.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.Map; + +import org.apache.kylin.stream.core.model.Node; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class CubeRealTimeState { + @JsonProperty("cube_name") + private String cubeName; + + @JsonProperty("receiver_cube_real_time_states") + private Map> receiverCubeStateMap = Maps.newHashMap(); + + public String getCubeName() { + return cubeName; + } + + public void setCubeName(String cubeName) { + this.cubeName = cubeName; + } + + public Map> getReceiverCubeStateMap() { + return receiverCubeStateMap; + } + + public void setReceiverCubeStateMap(Map> receiverCubeStateMap) { + this.receiverCubeStateMap = receiverCubeStateMap; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/LongLatencyInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/LongLatencyInfo.java new file mode 100644 index 00000000000..55bf9e491ab --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/LongLatencyInfo.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class LongLatencyInfo { + @JsonProperty("long_latency_segment_events") + private Map longLatencyEventCnts = new TreeMap<>(); + @JsonProperty("total_long_latency_events") + private int totalLongLatencyEventCnt; + + public Map getLongLatencyEventCnts() { + return longLatencyEventCnts; + } + + public void setLongLatencyEventCnts(Map longLatencyEventCnts) { + this.longLatencyEventCnts = longLatencyEventCnts; + } + + public int getTotalLongLatencyEventCnt() { + return totalLongLatencyEventCnt; + } + + public void setTotalLongLatencyEventCnt(int totalLongLatencyEventCnt) { + this.totalLongLatencyEventCnt = totalLongLatencyEventCnt; + } + + public void incLongLatencyEvent(String segmentName) { + Integer llEventCnt = longLatencyEventCnts.get(segmentName); + if (llEventCnt == null) { + llEventCnt = 1; + } else { + llEventCnt = llEventCnt + 1; + } + longLatencyEventCnts.put(segmentName, llEventCnt); + totalLongLatencyEventCnt++; + } + + public LongLatencyInfo truncate(int maxSegments) { + int segmentNum = longLatencyEventCnts.size(); + if (segmentNum <= maxSegments) { + return this; + } else { + SortedMap sortMap = new TreeMap<>(longLatencyEventCnts); + int shouldRemoved = maxSegments - segmentNum; + for (String segmentName : sortMap.keySet()) { + if (shouldRemoved == 0) { + break; + } + longLatencyEventCnts.remove(segmentName); + shouldRemoved--; + } + } + return this; + } + + @Override + public String toString() { + return "LongLatencyInfo{" + "longLatencyEventCnts=" + longLatencyEventCnts + ", totalLongLatencyEventCnt=" + + totalLongLatencyEventCnt + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/PartitionConsumeStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/PartitionConsumeStats.java new file mode 100644 index 00000000000..2887bd5d81f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/PartitionConsumeStats.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class PartitionConsumeStats { + @JsonProperty("avg_rate") + private double avgRate; + @JsonProperty("one_min_rate") + private double oneMinRate; + @JsonProperty("five_min_rate") + private double fiveMinRate; + @JsonProperty("fifteen_min_rate") + private double fifteenMinRate; + @JsonProperty("total_consume") + private long totalConsume; + + public double getAvgRate() { + return avgRate; + } + + public void setAvgRate(double avgRate) { + this.avgRate = avgRate; + } + + public double getOneMinRate() { + return oneMinRate; + } + + public void setOneMinRate(double oneMinRate) { + this.oneMinRate = oneMinRate; + } + + public double getFiveMinRate() { + return fiveMinRate; + } + + public void setFiveMinRate(double fiveMinRate) { + this.fiveMinRate = fiveMinRate; + } + + public double getFifteenMinRate() { + return fifteenMinRate; + } + + public void setFifteenMinRate(double fifteenMinRate) { + this.fifteenMinRate = fifteenMinRate; + } + + public long getTotalConsume() { + return totalConsume; + } + + public void setTotalConsume(long totalConsume) { + this.totalConsume = totalConsume; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeRealTimeState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeRealTimeState.java new file mode 100644 index 00000000000..9236978abc4 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeRealTimeState.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ReceiverCubeRealTimeState { + @JsonProperty("receiver_state") + private ReceiverState.State state; + + @JsonProperty("receiver_cube_stats") + private ReceiverCubeStats receiverCubeStats; + + public ReceiverState.State getState() { + return state; + } + + public void setState(ReceiverState.State state) { + this.state = state; + } + + public ReceiverCubeStats getReceiverCubeStats() { + return receiverCubeStats; + } + + public void setReceiverCubeStats(ReceiverCubeStats receiverCubeStats) { + this.receiverCubeStats = receiverCubeStats; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeStats.java new file mode 100644 index 00000000000..d823f0a120b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverCubeStats.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ReceiverCubeStats { + @JsonProperty("segment_stats") + private Map segmentStatsMap = Maps.newHashMap(); + + //Consumer stats will not survive after streaming receiver restart + @JsonProperty("consumer_stats") + private ConsumerStats consumerStats; + + //Total ingest count will survive after streaming receiver restart + @JsonProperty("total_ingest") + private long totalIngest; + + @JsonProperty("latest_event_time") + private long latestEventTime; + + @JsonProperty("latest_event_ingest_time") + private long latestEventIngestTime; + + @JsonProperty("long_latency_info") + private LongLatencyInfo longLatencyInfo; + + public ConsumerStats getConsumerStats() { + return consumerStats; + } + + public void setConsumerStats(ConsumerStats consumerStats) { + this.consumerStats = consumerStats; + } + + public long getTotalIngest() { + return totalIngest; + } + + public void setTotalIngest(long totalIngest) { + this.totalIngest = totalIngest; + } + + public void addSegmentStats(String segmentName, SegmentStats segmentStats) { + segmentStatsMap.put(segmentName, segmentStats); + } + + public Map getSegmentStatsMap() { + return segmentStatsMap; + } + + public void setSegmentStatsMap(Map segmentStatsMap) { + this.segmentStatsMap = segmentStatsMap; + } + + public long getLatestEventTime() { + return latestEventTime; + } + + public void setLatestEventTime(long latestEventTime) { + this.latestEventTime = latestEventTime; + } + + public long getLatestEventIngestTime() { + return latestEventIngestTime; + } + + public void setLatestEventIngestTime(long latestEventIngestTime) { + this.latestEventIngestTime = latestEventIngestTime; + } + + public LongLatencyInfo getLongLatencyInfo() { + return longLatencyInfo; + } + + public void setLongLatencyInfo(LongLatencyInfo longLatencyInfo) { + this.longLatencyInfo = longLatencyInfo; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverState.java new file mode 100644 index 00000000000..0d976b16265 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverState.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.List; + +import org.apache.kylin.stream.core.model.Node; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ReceiverState { + @JsonProperty("receiver") + private Node receiver; + @JsonProperty("state") + private State state; + @JsonProperty("rate") + private double rateInOneMin; + @JsonProperty("info") + private List infoList; + + public Node getReceiver() { + return receiver; + } + + public void setReceiver(Node receiver) { + this.receiver = receiver; + } + + public State getState() { + return state; + } + + public void setState(State state) { + this.state = state; + } + + public double getRateInOneMin() { + return rateInOneMin; + } + + public void setRateInOneMin(double rateInOneMin) { + this.rateInOneMin = rateInOneMin; + } + + public List getInfoList() { + return infoList; + } + + public void setInfoList(List infoList) { + this.infoList = infoList; + } + + public void addInfo(String info) { + if (infoList == null) { + infoList = Lists.newArrayList(); + } + infoList.add(info); + } + + public enum State { + HEALTHY, WARN, UNREACHABLE, DOWN + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverStats.java new file mode 100644 index 00000000000..614798c64d6 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReceiverStats.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.storage.columnar.ColumnarStoreCacheStats; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ReceiverStats { + @JsonProperty("assignments") + private Map> assignments; + + @JsonProperty("is_lead") + private boolean isLead; + + @JsonProperty("cube_stats") + private Map cubeStatsMap = Maps.newHashMap(); + + @JsonProperty("cache_stats") + private ColumnarStoreCacheStats cacheStats; + + public void addCubeStats(String cubeName, ReceiverCubeStats cubeStats) { + cubeStatsMap.put(cubeName, cubeStats); + } + + public Map> getAssignments() { + return assignments; + } + + public void setAssignments(Map> assignments) { + this.assignments = assignments; + } + + public boolean isLead() { + return isLead; + } + + public void setLead(boolean lead) { + isLead = lead; + } + + public Map getCubeStatsMap() { + return cubeStatsMap; + } + + public void setCubeStatsMap(Map cubeStatsMap) { + this.cubeStatsMap = cubeStatsMap; + } + + public ColumnarStoreCacheStats getCacheStats() { + return cacheStats; + } + + public void setCacheStats(ColumnarStoreCacheStats cacheStats) { + this.cacheStats = cacheStats; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReplicaSetState.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReplicaSetState.java new file mode 100644 index 00000000000..0324416e753 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/ReplicaSetState.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import java.util.List; +import java.util.Map; + +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.source.Partition; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class ReplicaSetState { + @JsonProperty("rs_id") + private int rsID; + + @JsonProperty("lead") + private Node lead; + + @JsonProperty("assignment") + private Map> assignment; + + @JsonProperty("receiver_states") + private List receiverStates; + + public int getRsID() { + return rsID; + } + + public void setRsID(int rsID) { + this.rsID = rsID; + } + + public List getReceiverStates() { + return receiverStates; + } + + public void setReceiverStates(List receiverStates) { + this.receiverStates = receiverStates; + } + + public Node getLead() { + return lead; + } + + public void setLead(Node lead) { + this.lead = lead; + } + + public Map> getAssignment() { + return assignment; + } + + public void setAssignment(Map> assignment) { + this.assignment = assignment; + } + + public void addReveiverState(ReceiverState receiverState) { + if (receiverStates == null) { + receiverStates = Lists.newArrayList(); + } + receiverStates.add(receiverState); + } + + public ReceiverState getReceiverState(Node receiver) { + for (ReceiverState receiverState : receiverStates) { + if (receiverState.getReceiver().equals(receiver)) { + return receiverState; + } + } + return null; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStats.java new file mode 100644 index 00000000000..3bb2932ddc0 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStats.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class SegmentStats { + @JsonProperty("segment_state") + private String segmentState; + + @JsonProperty("segment_create_time") + private long segmentCreateTime; + + @JsonProperty("segment_last_update_time") + private long segmentLastUpdateTime; + + @JsonProperty("latest_event_time") + private long latestEventTime; + + @JsonProperty("latest_event_latency") + private long latestEventLatency; + + @JsonProperty("store_stats") + private SegmentStoreStats storeStats; + + public String getSegmentState() { + return segmentState; + } + + public void setSegmentState(String segmentState) { + this.segmentState = segmentState; + } + + public SegmentStoreStats getStoreStats() { + return storeStats; + } + + public void setStoreStats(SegmentStoreStats storeStats) { + this.storeStats = storeStats; + } + + public long getSegmentCreateTime() { + return segmentCreateTime; + } + + public void setSegmentCreateTime(long segmentCreateTime) { + this.segmentCreateTime = segmentCreateTime; + } + + public long getSegmentLastUpdateTime() { + return segmentLastUpdateTime; + } + + public void setSegmentLastUpdateTime(long segmentLastUpdateTime) { + this.segmentLastUpdateTime = segmentLastUpdateTime; + } + + public long getLatestEventTime() { + return latestEventTime; + } + + public void setLatestEventTime(long latestEventTime) { + this.latestEventTime = latestEventTime; + } + + public long getLatestEventLatency() { + return latestEventLatency; + } + + public void setLatestEventLatency(long latestEventLatency) { + this.latestEventLatency = latestEventLatency; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStoreStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStoreStats.java new file mode 100644 index 00000000000..ac89e293a74 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/model/stats/SegmentStoreStats.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.model.stats; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class SegmentStoreStats { + @JsonProperty("num_rows_in_mem") + private int numRowsInMem; + @JsonProperty("num_fragments") + private int numFragments; // for columnar store + + public int getNumRowsInMem() { + return numRowsInMem; + } + + public void setNumRowsInMem(int numRowsInMem) { + this.numRowsInMem = numRowsInMem; + } + + public int getNumFragments() { + return numFragments; + } + + public void setNumFragments(int numFragments) { + this.numFragments = numFragments; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/HavingFilterChecker.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/HavingFilterChecker.java new file mode 100644 index 00000000000..5dae1057fb2 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/HavingFilterChecker.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.kylin.measure.MeasureAggregator; +import org.apache.kylin.measure.bitmap.BitmapCounter; +import org.apache.kylin.measure.hllc.HLLCounter; +import org.apache.kylin.measure.percentile.PercentileCounter; +import org.apache.kylin.metadata.filter.IFilterCodeSystem; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.ITuple; + +public class HavingFilterChecker { + final HavingFilterTuple tuple = new HavingFilterTuple(); + final IFilterCodeSystem cs = new HavingFilterCodeSys(); + private TupleFilter havingFilter; + private ResponseResultSchema schema; + + public HavingFilterChecker(TupleFilter havingFilter, ResponseResultSchema schema) { + this.havingFilter = havingFilter; + this.schema = schema; + } + + public boolean check(MeasureAggregator[] aggregators) { + tuple.aggrValues = aggregators; + return havingFilter.evaluate(tuple, cs); + } + + private class HavingFilterCodeSys implements IFilterCodeSystem { + + Object o2Cache; + double n2Cache; + + @Override + public int compare(Object o1, Object o2) { + if (o1 == null && o2 == null) + return 0; + + if (o1 == null) // null is bigger to align with CubeCodeSystem + return 1; + + if (o2 == null) // null is bigger to align with CubeCodeSystem + return -1; + + // for the 'having clause', we only concern numbers and BigDecimal + // we try to cache the o2, which should be a constant according to CompareTupleFilter.evaluate() + + double n1; + if (o1 instanceof Number) { + n1 = ((Number) o1).doubleValue(); + } else if (o1 instanceof HLLCounter) { + n1 = ((HLLCounter) o1).getCountEstimate(); + } else if (o1 instanceof BitmapCounter) { + n1 = ((BitmapCounter) o1).getCount(); + } else if (o1 instanceof PercentileCounter) { + n1 = ((PercentileCounter) o1).getResultEstimate(); + } else { + throw new RuntimeException("Unknown datatype: value=" + o1 + ", class=" + o1.getClass()); + } + + double n2 = (o2Cache == o2) ? n2Cache : Double.parseDouble((String) o2); + + if (o2Cache == null) { + o2Cache = o2; + n2Cache = n2; + } + + return Double.compare(n1, n2); + } + + @Override + public boolean isNull(Object code) { + return code == null; + } + + @Override + public void serialize(Object code, ByteBuffer buf) { + throw new UnsupportedOperationException(); + } + + @Override + public Object deserialize(ByteBuffer buf) { + throw new UnsupportedOperationException(); + } + } + + private class HavingFilterTuple implements ITuple { + MeasureAggregator[] aggrValues; + + @Override + public Object getValue(TblColRef col) { + return aggrValues[schema.getIndexOfMetrics(col)].getState(); + } + + @Override + public List getAllFields() { + throw new UnsupportedOperationException(); + } + + @Override + public List getAllColumns() { + throw new UnsupportedOperationException(); + } + + @Override + public Object[] getAllValues() { + throw new UnsupportedOperationException(); + } + + @Override + public ITuple makeCopy() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingGTSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingGTSearcher.java new file mode 100644 index 00000000000..bdf6010ec57 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingGTSearcher.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.io.IOException; + +public interface IStreamingGTSearcher { + void search(final StreamingSearchContext searchContext, final ResultCollector collector) throws IOException; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingSearchResult.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingSearchResult.java new file mode 100644 index 00000000000..cd6c0d47b3d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/IStreamingSearchResult.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; + +import org.apache.kylin.stream.core.storage.Record; + +import com.google.common.collect.Iterators; + +public interface IStreamingSearchResult extends Iterable, Closeable { + IStreamingSearchResult EMPTY_RESULT = new IStreamingSearchResult() { + + @Override + public void close() throws IOException { + } + + @Override + public Iterator iterator() { + return Iterators.emptyIterator(); + } + + @Override + public void startRead() { + } + + @Override + public void endRead() { + } + }; + + /** + * start to read data + */ + void startRead(); + + /** + * finish reading data + */ + void endRead(); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/MultiThreadsResultCollector.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/MultiThreadsResultCollector.java new file mode 100644 index 00000000000..5ffe5f24103 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/MultiThreadsResultCollector.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +public class MultiThreadsResultCollector extends ResultCollector { + private static Logger logger = LoggerFactory.getLogger(MultiThreadsResultCollector.class); + private static ExecutorService executor; + static { + KylinConfig config = KylinConfig.getInstanceFromEnv(); + executor = new ThreadPoolExecutor(config.getStreamingReceiverQueryCoreThreads(), + config.getStreamingReceiverQueryMaxThreads(), 60L, TimeUnit.SECONDS, + new LinkedBlockingQueue(), new NamedThreadFactory("query-worker")); + } + + private int timeout; + private Semaphore workersSemaphore; + final BlockingQueue queue = new LinkedBlockingQueue<>(10000); + private AtomicInteger notCompletedWorkers; + + public MultiThreadsResultCollector(int numOfWorkers, int timeout) { + this.workersSemaphore = new Semaphore(numOfWorkers); + this.timeout = timeout; + } + + @Override + public Iterator iterator() { + notCompletedWorkers = new AtomicInteger(searchResults.size()); + executor.submit(new WorkSubmitter()); + + final int batchSize = 100; + final long startTime = System.currentTimeMillis(); + return new Iterator() { + List recordList = Lists.newArrayListWithExpectedSize(batchSize); + Iterator internalIT = recordList.iterator(); + + @Override + public boolean hasNext() { + boolean exits = (internalIT.hasNext() || queue.size() > 0); + if (!exits) { + while (notCompletedWorkers.get() > 0) { + Thread.yield(); + long takeTime = System.currentTimeMillis() - startTime; + if (takeTime > timeout) { + throw new RuntimeException("Timeout when iterate search result"); + } + if (internalIT.hasNext() || queue.size() > 0) { + return true; + } + } + } + + return exits; + } + + @Override + public Record next() { + try { + long takeTime = System.currentTimeMillis() - startTime; + if (takeTime > timeout) { + throw new RuntimeException("Timeout when iterate search result"); + } + if (!internalIT.hasNext()) { + recordList.clear(); + Record one = queue.poll(timeout - takeTime, TimeUnit.MILLISECONDS); + if (one == null) { + throw new RuntimeException("Timeout when iterate search result"); + } + recordList.add(one); + queue.drainTo(recordList, batchSize - 1); + internalIT = recordList.iterator(); + } + return internalIT.next(); + } catch (InterruptedException e) { + throw new RuntimeException("Error when waiting queue", e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not support"); + } + }; + } + + private class ResultIterateWorker implements Runnable { + IStreamingSearchResult result; + + public ResultIterateWorker(IStreamingSearchResult result) { + this.result = result; + } + + @Override + public void run() { + try { + result.startRead(); + for (Record record : result) { + try { + queue.put(record.copy()); + } catch (InterruptedException e) { + throw new RuntimeException("Timeout when visiting streaming segmenent", e); + } + } + result.endRead(); + } catch (Exception e) { + logger.error("error when iterate search result", e); + } finally { + notCompletedWorkers.decrementAndGet(); + workersSemaphore.release(); + } + } + } + + private class WorkSubmitter implements Runnable { + @Override + public void run() { + for (final IStreamingSearchResult result : searchResults) { + executor.submit(new ResultIterateWorker(result)); + try { + workersSemaphore.acquire(); + } catch (InterruptedException e) { + logger.error("interrupted", e); + } + } + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/RecordsAggregator.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/RecordsAggregator.java new file mode 100644 index 00000000000..d87fa780c28 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/RecordsAggregator.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedMap; + +import com.google.common.collect.Maps; +import org.apache.kylin.measure.MeasureAggregator; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.Record; + +public class RecordsAggregator implements Iterable{ + private SortedMap aggBufMap; + private ResponseResultSchema schema; + private int[] groupIndexes; + private int pushDownLimit; + private TupleFilter havingFilter; + + public RecordsAggregator(ResponseResultSchema schema, Set groups, TupleFilter havingFilter) { + this.schema = schema; + this.havingFilter = havingFilter; + this.groupIndexes = new int[groups.size()]; + int i = 0; + for (TblColRef group : groups) { + groupIndexes[i] = schema.getIndexOfDimension(group); + i++; + } + this.aggBufMap = Maps.newTreeMap(comparator); + this.pushDownLimit = Integer.MAX_VALUE; + } + + @Override + public Iterator iterator() { + Iterator> it = aggBufMap.entrySet().iterator(); + + final Iterator> input = it; + return new Iterator() { + Record oneRecord = new Record(schema.getDimensionCount(), schema.getMetricsCount()); + Entry returningEntry = null; + final HavingFilterChecker havingFilterChecker = (havingFilter == null) ? null + : new HavingFilterChecker(havingFilter, schema); + + @Override + public boolean hasNext() { + while (input.hasNext()) { + returningEntry = input.next(); + if (havingFilterChecker != null) { + if (havingFilterChecker.check(returningEntry.getValue())) { + return true; + } + } else { + return true; + } + } + return false; + } + + @Override + public Record next() { + String[] dimVals = returningEntry.getKey(); + for (int i = 0; i < dimVals.length; i++) { + oneRecord.setDimension(i, dimVals[i]); + } + MeasureAggregator[] measures = returningEntry.getValue(); + for (int i = 0; i < measures.length; i++) { + oneRecord.setMetric(i, measures[i].getState()); + } + return oneRecord; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("unSupportOperation!"); + } + }; + } + + public void aggregate(Iterator records) { + while (records.hasNext()) { + aggregate(records.next()); + } + } + + public void aggregate(Record record) { + String[] copyDimVals = new String[schema.getDimensionCount()]; + String[] dimVals = record.getDimensions(); + System.arraycopy(dimVals, 0, copyDimVals, 0, dimVals.length); + aggregate(copyDimVals, record.getMetrics()); + } + + public void aggregate(String[] dimVals, Object[] metricsVals) { + MeasureAggregator[] aggrs = aggBufMap.get(dimVals); + if (aggrs == null) { + //for storage push down limit + if (aggBufMap.size() >= pushDownLimit) { + return; + } + aggrs = newAggregators(); + aggBufMap.put(dimVals, aggrs); + } + for (int i = 0; i < aggrs.length; i++) { + aggrs[i].aggregate(metricsVals[i]); + } + } + + private MeasureAggregator[] newAggregators() { + String[] aggrFuncs = schema.getAggrFuncs(); + MeasureAggregator[] result = new MeasureAggregator[aggrFuncs.length]; + for (int i = 0; i < result.length; i++) { + result[i] = MeasureAggregator.create(aggrFuncs[i], schema.getMetricsDataType(i)); + } + return result; + } + + final Comparator comparator = new Comparator() { + @Override + public int compare(String[] o1, String[] o2) { + for (int i = 0; i < groupIndexes.length; i++) { + int groupIdx = groupIndexes[i]; + int result = o1[groupIdx].compareTo(o2[groupIdx]); + if (result != 0) { + return result; + } + } + return 0; + } + }; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResponseResultSchema.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResponseResultSchema.java new file mode 100644 index 00000000000..a0f26a62fe6 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResponseResultSchema.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.cube.model.RowKeyColDesc; +import org.apache.kylin.cube.model.RowKeyDesc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.dimension.IDimensionEncodingMap; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.google.common.collect.Maps; + +public class ResponseResultSchema { + private CubeDesc cubeDesc; + + private TblColRef[] dimensions; + private FunctionDesc[] metrics; + private MeasureDesc[] measures; + + private DataType[] dimDataTypes; + private DataType[] metricsDataTypes; + + private int nDimensions; + private Map dimColIdxMap; + + private int nMetrics; + private Map metricsColIdxMap; + + public ResponseResultSchema(CubeDesc cubeDesc, Set selectedDimensions, Set selectedMetrics) { + this.cubeDesc = cubeDesc; + init(selectedDimensions, selectedMetrics); + } + + private void init(Set selectedDimensions, Set selectedMetrics) { + this.dimensions = new TblColRef[selectedDimensions.size()]; + this.metrics = new FunctionDesc[selectedMetrics.size()]; + this.measures = new MeasureDesc[selectedMetrics.size()]; + this.dimDataTypes = new DataType[dimensions.length]; + this.metricsDataTypes = new DataType[metrics.length]; + // sort dimensions according to the rowKey definition + dimColIdxMap = Maps.newHashMap(); + RowKeyDesc rowKeyDesc = cubeDesc.getRowkey(); + int colIdx = 0; + for (RowKeyColDesc rowKeyColDesc : rowKeyDesc.getRowKeyColumns()) { + TblColRef dimension = rowKeyColDesc.getColRef(); + if (selectedDimensions.contains(dimension)) { + dimensions[colIdx] = dimension; + dimDataTypes[colIdx] = dimension.getType(); + dimColIdxMap.put(dimension, colIdx); + colIdx++; + } + } + + nDimensions = colIdx; + + colIdx = 0; + // metrics + metricsColIdxMap = Maps.newHashMap(); + for (MeasureDesc measure : cubeDesc.getMeasures()) { + FunctionDesc func = measure.getFunction(); + if (selectedMetrics.contains(func)) { + metrics[colIdx] = func; + measures[colIdx] = measure; + metricsColIdxMap.put(func.getParameter().getColRef(), colIdx); + metricsDataTypes[colIdx] = func.getReturnDataType(); + colIdx++; + } + } + + nMetrics = colIdx; + } + + public int getColumnCount() { + return nDimensions + nMetrics; + } + + public int getDimensionCount() { + return nDimensions; + } + + public int getMetricsCount() { + return nMetrics; + } + + public CubeDesc getCubeDesc() { + return cubeDesc; + } + + public DataType[] getMetricsDataTypes() { + return metricsDataTypes; + } + + public DataType getMetricsDataType(int i) { + return metricsDataTypes[i]; + } + + public int getIndexOfDimension(TblColRef dimension) { + Integer i = dimColIdxMap.get(dimension); + return i == null ? -1 : i.intValue(); + } + + public int getIndexOfMetrics(TblColRef metricsColumn) { + Integer i = metricsColIdxMap.get(metricsColumn); + return i == null ? -1 : i.intValue(); + } + + public TblColRef[] getDimensions() { + return dimensions; + } + + public FunctionDesc[] getMetrics() { + return metrics; + } + + public MeasureDesc[] getMeasureDescs() { + return measures; + } + + public String[] getAggrFuncs() { + String[] result = new String[metrics.length]; + for (int i = 0; i < metrics.length; i++) { + result[i] = metrics[i].getExpression(); + } + return result; + } + + public DimensionEncoding[] getDimensionEncodings(IDimensionEncodingMap dimEncMap) { + DimensionEncoding[] dimEncs = new DimensionEncoding[dimensions.length]; + for (int i = 0; i < dimEncs.length; i++) { + dimEncs[i] = dimEncMap.get(dimensions[i]); + } + return dimEncs; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResultCollector.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResultCollector.java new file mode 100644 index 00000000000..a70bdf0e348 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/ResultCollector.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.io.IOException; +import java.util.List; + +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class ResultCollector implements IStreamingSearchResult{ + private static Logger logger = LoggerFactory.getLogger(ResultCollector.class); + + protected List searchResults = Lists.newArrayList(); + private List closeListeners = Lists.newArrayList(); + + public void collectSearchResult(IStreamingSearchResult searchResult) { + searchResults.add(searchResult); + } + + public void addCloseListener(CloseListener listener) { + closeListeners.add(listener); + } + + @Override + public void close() throws IOException{ + Exception oneException = null; + for (CloseListener listener : closeListeners) { + try { + listener.onClose(); + } catch (Exception e) { + logger.error("exception throws when on close is called", e); + } + } + for (IStreamingSearchResult input : searchResults) { + try { + input.close(); + } catch (Exception e) { + oneException = e; + } + } + if (oneException != null) { + if (oneException instanceof IOException) { + throw (IOException)oneException; + } + throw new IOException(oneException); + } + } + + public void startRead(){ + } + + public void endRead(){ + } + + public interface CloseListener { + void onClose(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/SingleThreadResultCollector.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/SingleThreadResultCollector.java new file mode 100644 index 00000000000..68478c8076c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/SingleThreadResultCollector.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +import org.apache.kylin.stream.core.storage.Record; + +import com.google.common.collect.Iterators; + +public class SingleThreadResultCollector extends ResultCollector { + + @Override + public Iterator iterator() { + if (searchResults.isEmpty()) { + return Iterators.emptyIterator(); + } + Iterator resultIterator = searchResults.iterator(); + return new Iterator() { + Iterator current = Iterators.emptyIterator(); + IStreamingSearchResult prevResult = null; + + @Override + public boolean hasNext() { + boolean currentHasNext; + while (!(currentHasNext = checkNotNull(current).hasNext()) && resultIterator.hasNext()) { + if (prevResult != null) { + prevResult.endRead(); + } + IStreamingSearchResult currResult = resultIterator.next(); + currResult.startRead(); + prevResult = currResult; + current = currResult.iterator(); + } + if (!currentHasNext && prevResult != null) { + prevResult.endRead(); + } + return currentHasNext; + } + + @Override + public Record next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return current.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not support remove"); + } + }; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingBuiltInFunctionTransformer.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingBuiltInFunctionTransformer.java new file mode 100644 index 00000000000..109d980954f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingBuiltInFunctionTransformer.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.ListIterator; +import java.util.Set; + +import org.apache.kylin.metadata.filter.BuiltInFunctionTupleFilter; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.ConstantTupleFilter; +import org.apache.kylin.metadata.filter.DynamicTupleFilter; +import org.apache.kylin.metadata.filter.IFilterCodeSystem; +import org.apache.kylin.metadata.filter.ITupleFilterTransformer; +import org.apache.kylin.metadata.filter.LogicalTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.IEvaluatableTuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * only take effect when the compare filter has function + */ +public class StreamingBuiltInFunctionTransformer implements ITupleFilterTransformer { + public static final Logger logger = LoggerFactory.getLogger(StreamingBuiltInFunctionTransformer.class); + + private Set unEvaluableColumns; + + public StreamingBuiltInFunctionTransformer(Set unEvaluableColumns) { + this.unEvaluableColumns = unEvaluableColumns; + } + + @Override + public TupleFilter transform(TupleFilter tupleFilter) { + TupleFilter translated = null; + if (tupleFilter instanceof CompareTupleFilter) { + //normal case + translated = translateCompareTupleFilter((CompareTupleFilter) tupleFilter); + if (translated != null) { + logger.info("Translated {" + tupleFilter + "}"); + } + } else if (tupleFilter instanceof BuiltInFunctionTupleFilter) { + //like case + translated = translateFunctionTupleFilter((BuiltInFunctionTupleFilter) tupleFilter); + if (translated != null) { + logger.info("Translated {" + tupleFilter + "}"); + } + } else if (tupleFilter instanceof LogicalTupleFilter) { + @SuppressWarnings("unchecked") + ListIterator childIterator = (ListIterator) tupleFilter.getChildren() + .listIterator(); + while (childIterator.hasNext()) { + TupleFilter transformed = transform(childIterator.next()); + if (transformed != null) + childIterator.set(transformed); + } + } + + TupleFilter result = translated == null ? tupleFilter : translated; + if (result.getOperator() == TupleFilter.FilterOperatorEnum.NOT + && !TupleFilter.isEvaluableRecursively(result)) { + TupleFilter.collectColumns(result, unEvaluableColumns); + return ConstantTupleFilter.TRUE; + } + + // shortcut for unEvaluatable filter + if (!result.isEvaluable()) { + TupleFilter.collectColumns(result, unEvaluableColumns); + return ConstantTupleFilter.TRUE; + } + return result; + } + + private TupleFilter translateFunctionTupleFilter(BuiltInFunctionTupleFilter builtInFunctionTupleFilter) { + if (!builtInFunctionTupleFilter.isValid()) + return null; + + return new EvaluableBuildInFuncTupleFilter(builtInFunctionTupleFilter); + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + private TupleFilter translateCompareTupleFilter(CompareTupleFilter compTupleFilter) { + if (compTupleFilter.getFunction() == null + || (!(compTupleFilter.getFunction() instanceof BuiltInFunctionTupleFilter))) + return null; + + BuiltInFunctionTupleFilter buildInFunctionTupleFilter = (BuiltInFunctionTupleFilter) compTupleFilter + .getFunction(); + if (!buildInFunctionTupleFilter.isValid()) + return null; + + return new BuildInFuncCompareTupleFilter(compTupleFilter); + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + private static class BuildInFuncCompareTupleFilter extends TupleFilter { + private CompareTupleFilter delegate; + private BuiltInFunctionTupleFilter buildInFunctionTupleFilter; + + protected BuildInFuncCompareTupleFilter(CompareTupleFilter delegate) { + super(new ArrayList<>(delegate.getChildren()), delegate.getOperator()); + this.delegate = delegate; + this.buildInFunctionTupleFilter = (BuiltInFunctionTupleFilter) delegate.getFunction(); + } + + @Override + public boolean isEvaluable() { + return true; + } + + @Override + public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) { + // extract tuple value + Object tupleValue = null; + for (TupleFilter filter : this.buildInFunctionTupleFilter.getChildren()) { + if (!isConstant(filter)) { + filter.evaluate(tuple, cs); + tupleValue = filter.getValues().iterator().next(); + } + } + + // consider null case + if (cs.isNull(tupleValue)) { + if (operator == FilterOperatorEnum.ISNULL) + return true; + else + return false; + } + + if (operator == FilterOperatorEnum.ISNOTNULL) { + return true; + } + if (delegate.getFirstValue() == null || cs.isNull(delegate.getFirstValue())) { + return false; + } + + // tricky here -- order is ensured by string compare (even for number columns) + // because it's row key ID (not real value) being compared + Object computedVal = tupleValue; + try { + computedVal = buildInFunctionTupleFilter.invokeFunction(tupleValue); + } catch (Exception e) { + logger.error("exception when invoke buildIn function", e); + } + int comp = cs.compare(computedVal, delegate.getFirstValue()); + + boolean result; + switch (operator) { + case EQ: + result = comp == 0; + break; + case NEQ: + result = comp != 0; + break; + case LT: + result = comp < 0; + break; + case LTE: + result = comp <= 0; + break; + case GT: + result = comp > 0; + break; + case GTE: + result = comp >= 0; + break; + case IN: + result = delegate.getValues().contains(computedVal); + break; + case NOTIN: + result = !delegate.getValues().contains(computedVal); + break; + default: + result = false; + } + return result; + } + + private boolean isConstant(TupleFilter filter) { + return (filter instanceof ConstantTupleFilter) || (filter instanceof DynamicTupleFilter); + } + + @Override + public Collection getValues() { + return delegate.getValues(); + } + + @Override + public void serialize(IFilterCodeSystem cs, ByteBuffer buffer) { + throw new UnsupportedOperationException(); + } + + @Override + public void deserialize(IFilterCodeSystem cs, ByteBuffer buffer) { + throw new UnsupportedOperationException(); + } + } + + private static class EvaluableBuildInFuncTupleFilter extends TupleFilter { + private BuiltInFunctionTupleFilter buildInFunctionTupleFilter; + + protected EvaluableBuildInFuncTupleFilter(BuiltInFunctionTupleFilter builtInFunctionTupleFilter) { + super(new ArrayList<>(builtInFunctionTupleFilter.getChildren()), builtInFunctionTupleFilter.getOperator()); + this.buildInFunctionTupleFilter = builtInFunctionTupleFilter; + } + + @Override + public boolean isEvaluable() { + return true; + } + + @Override + public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) { + // extract tuple value + Object tupleValue = null; + for (TupleFilter filter : this.children) { + if (!isConstant(filter)) { + filter.evaluate(tuple, cs); + tupleValue = filter.getValues().iterator().next(); + } + } + if (tupleValue == null || cs.isNull(tupleValue)) { + return false; + } + + try { + return (Boolean) buildInFunctionTupleFilter.invokeFunction(tupleValue); + } catch (Exception e) { + logger.error("error when invoke build in function", e); + return false; + } + } + + @Override + public Collection getValues() { + return null; + } + + @Override + public void serialize(IFilterCodeSystem cs, ByteBuffer buffer) { + throw new UnsupportedOperationException(); + } + + @Override + public void deserialize(IFilterCodeSystem cs, ByteBuffer buffer) { + throw new UnsupportedOperationException(); + } + + private boolean isConstant(TupleFilter filter) { + return (filter instanceof ConstantTupleFilter) || (filter instanceof DynamicTupleFilter); + } + } +} \ No newline at end of file diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingCubeDataSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingCubeDataSearcher.java new file mode 100644 index 00000000000..f89ddecbc78 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingCubeDataSearcher.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.Set; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.gridtable.StorageSideBehavior; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.ITuple; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.Tuple; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.stream.core.storage.StreamingCubeSegment; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StreamingCubeDataSearcher { + private static Logger logger = LoggerFactory.getLogger(StreamingCubeDataSearcher.class); + + private static int TIMEOUT = Integer.MAX_VALUE; + + private StreamingSegmentManager streamingSegmentManager; + private String cubeName; + private CubeDesc cubeDesc; + + public StreamingCubeDataSearcher(StreamingSegmentManager streamingSegmentManager) { + this.streamingSegmentManager = streamingSegmentManager; + this.cubeName = streamingSegmentManager.getCubeInstance().getName(); + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + } + + public ITupleIterator search(TupleInfo returnTupleInfo, TupleFilter filter, TupleFilter havingFilter, + Set dimensions, Set groups, Set metrics, boolean allowStorageAggregation) { + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + filter, havingFilter); + IStreamingSearchResult searchResult = doSearch(searchRequest, -1, allowStorageAggregation); + StreamingTupleConverter tupleConverter = new StreamingTupleConverter(searchRequest.getRespResultSchema(), + returnTupleInfo); + return transformToTupleIterator(tupleConverter, searchResult, returnTupleInfo); + } + + public IStreamingSearchResult doSearch(StreamingSearchContext searchRequest, long minSegmentTime, + boolean allowStorageAggregation) { + StreamingQueryProfile queryProfile = StreamingQueryProfile.get(); + try { + logger.info("query-{}: use cuboid {} to serve the query", queryProfile.getQueryId(), + searchRequest.getHitCuboid()); + ResultCollector resultCollector = getResultCollector(); + Collection segments = streamingSegmentManager.getAllSegments(); + StreamingDataQueryPlanner scanRangePlanner = searchRequest.getQueryPlanner(); + for (StreamingCubeSegment queryableSegment : segments) { + if (!queryableSegment.isLongLatencySegment() && queryableSegment.getDateRangeStart() < minSegmentTime) { + String segmentName = queryableSegment.getSegmentName(); + queryProfile.skipSegment(segmentName); + logger.info("query-{}: skip segment {}, it is smaller than the min segment time:{}", + queryProfile.getQueryId(), segmentName, minSegmentTime); + continue; + } + + if (scanRangePlanner.canSkip(queryableSegment.getDateRangeStart(), queryableSegment.getDateRangeEnd())) { + String segmentName = queryableSegment.getSegmentName(); + queryProfile.skipSegment(segmentName); + logger.info("query-{}: skip segment {}", queryProfile.getQueryId(), + queryableSegment.getSegmentName()); + } else { + String segmentName = queryableSegment.getSegmentName(); + queryProfile.includeSegment(segmentName); + logger.info("query-{}: include segment {}", queryProfile.getQueryId(), segmentName); + + queryableSegment.getSegmentStore().search(searchRequest, resultCollector); + } + } + + return createFinalResult(resultCollector, searchRequest, allowStorageAggregation, queryProfile); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private ResultCollector getResultCollector() { + int useThreads = cubeDesc.getConfig().getStreamingReceiverUseThreadsPerQuery(); + if (useThreads > 1) { + return new MultiThreadsResultCollector(useThreads, TIMEOUT); + } else { + return new SingleThreadResultCollector(); + } + } + + private IStreamingSearchResult createFinalResult(final ResultCollector resultCollector, + final StreamingSearchContext searchRequest, boolean allowStorageAggregation, + StreamingQueryProfile queryProfile) throws IOException { + IStreamingSearchResult finalResult = resultCollector; + + if (queryProfile.getStorageBehavior().ordinal() <= StorageSideBehavior.SCAN.ordinal()) { + return finalResult; + } + if (allowStorageAggregation) { + finalResult = new StreamAggregateSearchResult(finalResult, searchRequest); + } + return finalResult; + } + + private ITupleIterator transformToTupleIterator(final StreamingTupleConverter tupleConverter, + final IStreamingSearchResult searchResult, final TupleInfo returnTupleInfo) { + final Tuple tuple = new Tuple(returnTupleInfo); + + final Iterator recordIterator = searchResult.iterator(); + return new ITupleIterator() { + @Override + public void close() { + try { + searchResult.close(); + } catch (IOException e) { + logger.warn("exception when close gtscanner", e); + } + } + + @Override + public boolean hasNext() { + return recordIterator.hasNext(); + } + + @Override + public ITuple next() { + tupleConverter.translateResult(recordIterator.next(), tuple); + return tuple; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not support"); + } + }; + } + + public class StreamAggregateSearchResult implements IStreamingSearchResult { + private IStreamingSearchResult inputSearchResult; + private RecordsAggregator recordsAggregator; + + public StreamAggregateSearchResult(IStreamingSearchResult inputSearchResult, + StreamingSearchContext searchRequest) { + this.inputSearchResult = inputSearchResult; + this.recordsAggregator = new RecordsAggregator(searchRequest.getRespResultSchema(), searchRequest.getAllGroups(), searchRequest.getHavingFilter()); + } + + @Override + public void startRead() { + + } + + @Override + public void endRead() { + + } + + @Override + public void close() throws IOException { + inputSearchResult.close(); + } + + @Override + public Iterator iterator() { + recordsAggregator.aggregate(inputSearchResult.iterator()); + return recordsAggregator.iterator(); + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingDataQueryPlanner.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingDataQueryPlanner.java new file mode 100644 index 00000000000..2b1c2860114 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingDataQueryPlanner.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.ConstantTupleFilter; +import org.apache.kylin.metadata.filter.LogicalTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.util.CompareFilterTimeRangeChecker; +import org.apache.kylin.stream.core.util.CompareFilterTimeRangeChecker.CheckResult; +import org.apache.kylin.stream.core.util.TimeDerivedColumnType; + +/** + * Scan planner for Streaming data segments, take derived time columns into consideration. + */ +public class StreamingDataQueryPlanner { + + protected CubeDesc cubeDesc; + protected TupleFilter filter; + protected TupleFilter flatFilter; + + public StreamingDataQueryPlanner(CubeDesc cubeDesc, TupleFilter filter) { + this.cubeDesc = cubeDesc; + this.filter = filter; + this.flatFilter = flattenToOrAndFilter(filter); + } + + public boolean canSkip(long timeStart, long timeEnd) { + if (flatFilter == null) { + return false; + } + CompareFilterTimeRangeChecker timeRangeChecker = new CompareFilterTimeRangeChecker(timeStart, timeEnd); + for (TupleFilter andFilter : flatFilter.getChildren()) { + if (andFilter.getOperator() != FilterOperatorEnum.AND) { + throw new IllegalStateException("Filter should be AND instead of " + andFilter); + } + if (!canSkipForAndFilter(andFilter, timeRangeChecker)) { + return false; + } + } + return true; + } + + private boolean canSkipForAndFilter(TupleFilter andFilter, CompareFilterTimeRangeChecker timeRangeChecker) { + for (TupleFilter filter : andFilter.getChildren()) { + if (!(filter instanceof CompareTupleFilter)) { + if (filter instanceof ConstantTupleFilter && !filter.evaluate(null, null)) { + return true; + } else { + continue; + } + } + CompareTupleFilter comp = (CompareTupleFilter) filter; + TblColRef column = comp.getColumn(); + if (column == null || !TimeDerivedColumnType.isTimeDerivedColumn(column.getName())) { + continue; + } + TimeDerivedColumnType timeDerivedColumnType = TimeDerivedColumnType.getTimeDerivedColumnType(column + .getName()); + + CheckResult checkResult = timeRangeChecker.check(comp, timeDerivedColumnType); + if (checkResult == CheckResult.EXCLUDED) { + return true; + } else { + continue; + } + } + return false; + } + + protected TupleFilter flattenToOrAndFilter(TupleFilter filter) { + if (filter == null) + return null; + + TupleFilter flatFilter = filter.flatFilter(); + + // normalize to OR-AND filter + if (flatFilter.getOperator() == FilterOperatorEnum.AND) { + LogicalTupleFilter f = new LogicalTupleFilter(FilterOperatorEnum.OR); + f.addChild(flatFilter); + flatFilter = f; + } + + if (flatFilter.getOperator() != FilterOperatorEnum.OR) + throw new IllegalStateException(); + + return flatFilter; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingQueryProfile.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingQueryProfile.java new file mode 100644 index 00000000000..d121139b1d3 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingQueryProfile.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.kylin.gridtable.StorageSideBehavior; + +public class StreamingQueryProfile { + private static ThreadLocal threadProfile = new ThreadLocal<>(); + private String queryId; + private boolean enableDetailProfile; + private long queryStartTime; + private long requestSendTime; + private int totalScanFiles = 0; + private long totalScanFileSize = 0; + private AtomicLong scanRows = new AtomicLong(0); + private AtomicLong filterRows= new AtomicLong(0); + private long finalRows; + + private StorageSideBehavior storageBehavior = StorageSideBehavior.SCAN_FILTER_AGGR_CHECKMEM; + private List includeSegments; + private List skippedSegments; + private List profileSteps; + private Map stepMap; + + public StreamingQueryProfile(String queryId, long requestSendTime) { + this.queryId = queryId; + this.includeSegments = Lists.newArrayList(); + this.skippedSegments = Lists.newArrayList(); + this.profileSteps = Lists.newArrayList(); + this.stepMap = Maps.newHashMap(); + this.requestSendTime = requestSendTime; + this.queryStartTime = System.currentTimeMillis(); + } + + public static StreamingQueryProfile get() { + return threadProfile.get(); + } + + public static void set(StreamingQueryProfile profile) { + threadProfile.set(profile); + } + + public boolean isDetailProfileEnable() { + return enableDetailProfile; + } + + public void enableDetailProfile() { + this.enableDetailProfile = true; + } + + public void includeSegment(String segmentName) { + includeSegments.add(segmentName); + } + + public void skipSegment(String segmentName) { + skippedSegments.add(segmentName); + } + + public void incScanFile(int fileSize) { + totalScanFiles++; + totalScanFileSize += fileSize; + } + + public ProfileStep startStep(String stepName) { + long startTime = System.currentTimeMillis(); + ProfileStep step = new ProfileStep(stepName, startTime); + profileSteps.add(step); + stepMap.put(stepName, step); + return step; + } + + public ProfileStep finishStep(String stepName) { + ProfileStep step = stepMap.get(stepName); + if (step != null) { + step.duration = System.currentTimeMillis() - step.startTime; + } + return step; + } + + public void addStepInfo(String stepName, String key, String val) { + ProfileStep step = stepMap.get(stepName); + if (step != null) { + step.stepInfo(key, val); + } + } + + public String getQueryId() { + return queryId; + } + + public void incScanRows(long n) { + scanRows.addAndGet(n); + } + + public void incFilterRows(long n) { + filterRows.addAndGet(n); + } + + public void setFinalRows(long n) { + this.finalRows = n; + } + + public StorageSideBehavior getStorageBehavior() { + return storageBehavior; + } + + public void setStorageBehavior(StorageSideBehavior storageBehavior) { + this.storageBehavior = storageBehavior; + } + + public String toString() { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + pw.println(); + pw.println("start: " + (queryStartTime - requestSendTime)); + pw.println("segments num: " + includeSegments.size()); + pw.println("segments: " + includeSegments); + pw.println("skip segments: " + skippedSegments); + pw.println("total files: " + totalScanFiles); + pw.println("total file size:" + totalScanFileSize); + pw.println("scan rows: " + scanRows); + pw.println("filter rows: " + filterRows); + pw.println("final rows: " + finalRows); + pw.println(); + if (enableDetailProfile) { + pw.println("details:"); + for (ProfileStep profileStep : profileSteps) { + pw.println(" " + profileStep.toString()); + } + } + return sw.toString(); + } + + public class ProfileStep { + String name; + Map properties; + long startTime; + long duration; + + ProfileStep(String name, long startTime) { + this.name = name; + this.properties = Maps.newHashMap(); + this.startTime = startTime; + } + + public ProfileStep stepInfo(String key, String val) { + this.properties.put(key, val); + return this; + } + + public long getDuration() { + return duration; + } + + @Override + public String toString() { + String start = String.valueOf(startTime - queryStartTime); + StringBuilder builder = new StringBuilder(); + builder.append(start); + builder.append(" "); + builder.append(name); + if (!properties.isEmpty()) { + builder.append(properties.toString()); + } + builder.append(" "); + builder.append(duration + "ms"); + return builder.toString(); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingSearchContext.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingSearchContext.java new file mode 100644 index 00000000000..9da8e5b6d35 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingSearchContext.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.query; + +import java.util.Set; + +import org.apache.kylin.cube.cuboid.Cuboid; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.google.common.collect.Sets; + +public class StreamingSearchContext { + private CubeDesc cubeDesc; + private TupleFilter filter; + private Set dimensions; + private Set groups; + private TupleFilter havingFilter; + private Set metrics; + + private Set addedGroups; + + private ResponseResultSchema respResultSchema; + + private StreamingDataQueryPlanner queryPlanner; + + private long hitCuboid; + private long basicCuboid; + + public StreamingSearchContext(CubeDesc cubeDesc, Set dimensions, Set groups, + Set metrics, TupleFilter filter, TupleFilter havingFilter) { + this.cubeDesc = cubeDesc; + this.dimensions = dimensions; + this.groups = groups; + this.metrics = metrics; + this.filter = filter; + this.havingFilter = havingFilter; + this.respResultSchema = new ResponseResultSchema(cubeDesc, dimensions, metrics); + this.queryPlanner = new StreamingDataQueryPlanner(cubeDesc, filter); + this.addedGroups = Sets.newHashSet(); + calculateHitCuboid(); + } + + public TupleFilter getFilter() { + return filter; + } + + public Set getGroups() { + return groups; + } + + public void addNewGroups(Set newGroups) { + addedGroups.addAll(newGroups); + } + + public Set getAllGroups() { + if (addedGroups.isEmpty()) { + return groups; + } + return Sets.union(groups, addedGroups); + } + + public Set getMetrics() { + return metrics; + } + + public Set getDimensions() { + return dimensions; + } + + public ResponseResultSchema getRespResultSchema() { + return respResultSchema; + } + + public TupleFilter getHavingFilter() { + return havingFilter; + } + + public void setRespResultSchema(ResponseResultSchema respResultSchema) { + this.respResultSchema = respResultSchema; + } + + public CubeDesc getCubeDesc() { + return cubeDesc; + } + + public long getHitCuboid() { + return hitCuboid; + } + + public void setHitCuboid(long hitCuboid) { + this.hitCuboid = hitCuboid; + } + + public void setBasicCuboid(long basicCuboid) { + this.basicCuboid = basicCuboid; + } + + public boolean hitBasicCuboid() { + return hitCuboid == basicCuboid; + } + + public StreamingDataQueryPlanner getQueryPlanner() { + return queryPlanner; + } + + public void setQueryPlanner(StreamingDataQueryPlanner queryPlanner) { + this.queryPlanner = queryPlanner; + } + + private void calculateHitCuboid() { + long basicCuboid = Cuboid.getBaseCuboidId(cubeDesc); + this.setBasicCuboid(basicCuboid); + if (!cubeDesc.getConfig().isStreamingBuildAdditionalCuboids()) { + this.setHitCuboid(basicCuboid); + return; + } + long targetCuboidID = identifyCuboid(dimensions); + Set mandatoryCuboids = getMandatoryCuboids(); + for (long cuboidID : mandatoryCuboids) { + if ((targetCuboidID & ~cuboidID) == 0) { + this.setHitCuboid(cuboidID); + return; + } + } + this.setHitCuboid(basicCuboid); + } + + private long identifyCuboid(Set dimensions) { + long cuboidID = 0; + for (TblColRef column : dimensions) { + int index = cubeDesc.getRowkey().getColumnBitIndex(column); + cuboidID |= 1L << index; + } + return cuboidID; + } + + private Set getMandatoryCuboids() { + Set sortedSet = Sets.newTreeSet(Cuboid.cuboidSelectComparator); + sortedSet.addAll(cubeDesc.getMandatoryCuboids()); + return sortedSet; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleConverter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleConverter.java new file mode 100755 index 00000000000..4788b2156f3 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleConverter.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.util.List; + +import org.apache.kylin.measure.MeasureType; +import org.apache.kylin.measure.MeasureType.IAdvMeasureFiller; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.Tuple; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.stream.core.storage.Record; + +import com.google.common.collect.Lists; + +/** + * Convert Streaming Record to Tuple + * + */ +public class StreamingTupleConverter { + final TupleInfo tupleInfo; + + final int[] dimTupleIdx; + final int[] metricsTupleIdx; + final int dimCnt; + final int metricsCnt; + final MeasureType[] measureTypes; + + final List advMeasureFillers; + final List advMeasureIndexInGTValues; + + + public StreamingTupleConverter(ResponseResultSchema schema, TupleInfo returnTupleInfo) { + this.tupleInfo = returnTupleInfo; + dimCnt = schema.getDimensionCount(); + metricsCnt = schema.getMetricsCount(); + dimTupleIdx = new int[dimCnt]; + metricsTupleIdx = new int[metricsCnt]; + + // measure types don't have this many, but aligned length make programming easier + measureTypes = new MeasureType[metricsCnt]; + + advMeasureFillers = Lists.newArrayListWithCapacity(1); + advMeasureIndexInGTValues = Lists.newArrayListWithCapacity(1); + + int idx = 0; + // pre-calculate dimension index mapping to tuple + for (TblColRef dim : schema.getDimensions()) { + dimTupleIdx[idx] = tupleInfo.hasColumn(dim) ? tupleInfo.getColumnIndex(dim) : -1; + idx++; + } + + idx = 0; + for (FunctionDesc metric : schema.getMetrics()) { + if (metric.needRewrite()) { + String rewriteFieldName = metric.getRewriteFieldName(); + metricsTupleIdx[idx] = tupleInfo.hasField(rewriteFieldName) ? tupleInfo.getFieldIndex(rewriteFieldName) : -1; + } else { // a non-rewrite metrics (like sum, or dimension playing as metrics) is like a dimension column + TblColRef col = metric.getParameter().getColRefs().get(0); + metricsTupleIdx[idx] = tupleInfo.hasColumn(col) ? tupleInfo.getColumnIndex(col) : -1; + } + + MeasureType measureType = metric.getMeasureType(); + if (measureType.needAdvancedTupleFilling()) { + advMeasureFillers.add(measureType.getAdvancedTupleFiller(metric, returnTupleInfo, null)); + advMeasureIndexInGTValues.add(idx); + } else { + measureTypes[idx] = measureType; + } + idx++; + } + } + + public List translateResult(Record record, Tuple tuple) { + // dimensions + String[] dimValues = record.getDimensions(); + Object[] metricsValues = record.getMetrics(); + for (int i = 0; i < dimCnt; i++) { + int ti = dimTupleIdx[i]; + if (ti >= 0) { + tuple.setDimensionValue(ti, dimValues[i]); + } + } + + // measures + for (int i = 0; i < metricsCnt; i++) { + int ti = metricsTupleIdx[i]; + if (ti >= 0 && measureTypes[i] != null) { + measureTypes[i].fillTupleSimply(tuple, ti, metricsValues[i]); + } + } + + // advanced measure filling, due to possible row split, will complete at caller side + if (advMeasureFillers.isEmpty()) { + return null; + } else { + for (int i = 0; i < advMeasureFillers.size(); i++) { + Object measureValue = metricsValues[advMeasureIndexInGTValues.get(i)]; + advMeasureFillers.get(i).reload(measureValue); + } + return advMeasureFillers; + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleIterator.java b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleIterator.java new file mode 100644 index 00000000000..70681795900 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/query/StreamingTupleIterator.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.query; + +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.kylin.measure.MeasureType.IAdvMeasureFiller; +import org.apache.kylin.metadata.tuple.ITuple; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.Tuple; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.stream.core.storage.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StreamingTupleIterator implements ITupleIterator { + private static final Logger logger = LoggerFactory.getLogger(StreamingTupleIterator.class); + + protected final TupleInfo tupleInfo; + protected final Tuple tuple; + + protected Iterator gtRecords; + protected StreamingTupleConverter tupleConverter; + protected Tuple next; + + private List advMeasureFillers; + private int advMeasureRowsRemaining; + private int advMeasureRowIndex; + + public StreamingTupleIterator(Iterator gtRecords, StreamingTupleConverter tupleConverter, + TupleInfo returnTupleInfo) { + this.gtRecords = gtRecords; + this.tupleConverter = tupleConverter; + this.tupleInfo = returnTupleInfo; + this.tuple = new Tuple(returnTupleInfo); + } + + @Override + public boolean hasNext() { + if (next != null) + return true; + + // consume any left rows from advanced measure filler + if (advMeasureRowsRemaining > 0) { + for (IAdvMeasureFiller filler : advMeasureFillers) { + filler.fillTuple(tuple, advMeasureRowIndex); + } + advMeasureRowIndex++; + advMeasureRowsRemaining--; + next = tuple; + return true; + } + + if (!gtRecords.hasNext()) { + return false; + } + // now we have a GTRecord + Record curRecord = gtRecords.next(); + + // translate into tuple + advMeasureFillers = tupleConverter.translateResult(curRecord, tuple); + + // the simple case + if (advMeasureFillers == null) { + next = tuple; + return true; + } + + // advanced measure filling, like TopN, will produce multiple tuples out + // of one record + advMeasureRowsRemaining = -1; + for (IAdvMeasureFiller filler : advMeasureFillers) { + if (advMeasureRowsRemaining < 0) + advMeasureRowsRemaining = filler.getNumOfRows(); + if (advMeasureRowsRemaining != filler.getNumOfRows()) + throw new IllegalStateException(); + } + if (advMeasureRowsRemaining < 0) + throw new IllegalStateException(); + + advMeasureRowIndex = 0; + return hasNext(); + } + + @Override + public ITuple next() { + // fetch next record + if (next == null) { + hasNext(); + if (next == null) + throw new NoSuchElementException(); + } + + ITuple result = next; + next = null; + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePosition.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePosition.java new file mode 100644 index 00000000000..f9a9736c75f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePosition.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.util.Map; + +public interface ISourcePosition { + void update(IPartitionPosition point); + + /** + * update the partition position info into the position + * when the partition position is not exist in the position. + * @param partitionPosition + */ + void updateWhenPartitionNotExist(IPartitionPosition partitionPosition); + + /** + * advance the source position + * @return + */ + ISourcePosition advance(); + + Map getPartitionPositions(); + + void copy(ISourcePosition other); + + interface IPartitionPosition extends Comparable { + int getPartition(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePositionHandler.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePositionHandler.java new file mode 100644 index 00000000000..77d6e8cfe71 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/ISourcePositionHandler.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.util.Collection; + +public interface ISourcePositionHandler { + enum MergeStrategy {KEEP_LARGE, KEEP_SMALL, KEEP_LATEST} + /** + * merge the input positions and return the result position according to the merge type + * @param positions + * @return + */ + ISourcePosition mergePositions(Collection positions, MergeStrategy mergeStrategy); + + ISourcePosition createEmptyPosition(); + + ISourcePosition parsePosition(String positionStr); + + String serializePosition(ISourcePosition position); + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingMessageParser.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingMessageParser.java new file mode 100644 index 00000000000..3e27d89e424 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingMessageParser.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import org.apache.kylin.stream.core.model.StreamingMessage; + +public interface IStreamingMessageParser { + StreamingMessage parse(T sourceMessage); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingSource.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingSource.java new file mode 100644 index 00000000000..f31d446e370 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/IStreamingSource.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.util.List; + +import org.apache.kylin.stream.core.consumer.ConsumerStartProtocol; +import org.apache.kylin.stream.core.consumer.IStreamingConnector; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; + +public interface IStreamingSource { + /** + * load the streaming source information for the cube + * @param cubeName + * @return + */ + StreamingTableSourceInfo load(String cubeName); + + /** + * fetch message template for specified streaming source + * @param streamingSourceConfig + * @return + */ + String getMessageTemplate(StreamingSourceConfig streamingSourceConfig); + + /** + * create streaming connector + * @param cubeName + * @param partitions + * @param startProtocol + * @param cubeSegmentManager + * @return + */ + IStreamingConnector createStreamingConnector(String cubeName, List partitions, ConsumerStartProtocol startProtocol, + StreamingSegmentManager cubeSegmentManager); + + + ISourcePositionHandler getSourcePositionHandler(); + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageFormatException.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageFormatException.java new file mode 100644 index 00000000000..69154f2968a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageFormatException.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +public class MessageFormatException extends RuntimeException { + public MessageFormatException() { + super(); + } + + public MessageFormatException(String s) { + super(s); + } + + public MessageFormatException(String message, Throwable cause) { + super(message, cause); + } + + public MessageFormatException(Throwable cause) { + super(cause); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageParserInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageParserInfo.java new file mode 100644 index 00000000000..89e36dc21d3 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/MessageParserInfo.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class MessageParserInfo { + @JsonProperty("ts_col_name") + private String tsColName; + + @JsonProperty("format_ts") + private boolean formatTs; + + @JsonProperty("field_mapping") + private Map columnToSourceFieldMapping; + + public String getTsColName() { + return tsColName; + } + + public void setTsColName(String tsColName) { + this.tsColName = tsColName; + } + + public boolean isFormatTs() { + return formatTs; + } + + public void setFormatTs(boolean formatTs) { + this.formatTs = formatTs; + } + + public Map getColumnToSourceFieldMapping() { + return columnToSourceFieldMapping; + } + + public void setColumnToSourceFieldMapping(Map columnToSourceFieldMapping) { + this.columnToSourceFieldMapping = columnToSourceFieldMapping; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/Partition.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/Partition.java new file mode 100644 index 00000000000..0fd8f844bee --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/Partition.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Kylin streaming partition, represents how the streaming cube is partitioned + * across the replicaSets. + * + */ +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class Partition implements Comparable { + @JsonProperty("partition_id") + private int partitionId; + @JsonProperty("partition_info") + private String partitionInfo; + + public Partition(int partitionId) { + this.partitionId = partitionId; + } + + public Partition() { + } + + public int getPartitionId() { + return partitionId; + } + + public void setPartitionId(int partitionId) { + this.partitionId = partitionId; + } + + public String getPartitionInfo() { + return partitionInfo; + } + + public void setPartitionInfo(String partitionInfo) { + this.partitionInfo = partitionInfo; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + partitionId; + result = prime * result + ((partitionInfo == null) ? 0 : partitionInfo.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Partition other = (Partition) obj; + if (partitionId != other.partitionId) + return false; + if (partitionInfo == null) { + if (other.partitionInfo != null) + return false; + } else if (!partitionInfo.equals(other.partitionInfo)) + return false; + return true; + } + + @Override + public String toString() { + return "Partition [partitionId=" + partitionId + ", partitionInfo=" + partitionInfo + "]"; + } + + @Override + public int compareTo(Partition other) { + return getPartitionId() - other.getPartitionId(); + } +} \ No newline at end of file diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfig.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfig.java new file mode 100644 index 00000000000..1cf9f2ffe6e --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfig.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Map; + +import org.apache.kylin.common.persistence.JsonSerializer; +import org.apache.kylin.common.persistence.ResourceStore; +import org.apache.kylin.common.persistence.RootPersistentEntity; +import org.apache.kylin.common.persistence.Serializer; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; + +/** + */ +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class StreamingSourceConfig extends RootPersistentEntity { + + private static final long serialVersionUID = 4931334770020441831L; + + public static Serializer SERIALIZER = new JsonSerializer( + StreamingSourceConfig.class); + + @JsonProperty("name") + private String name; + + @JsonProperty("parser_info") + private MessageParserInfo parserInfo; + + @JsonProperty("properties") + private Map properties = Maps.newLinkedHashMap(); + + public static String concatResourcePath(String name) { + return ResourceStore.STREAMING_V2_RESOURCE_ROOT + "/" + name + ".json"; + } + + public String getResourcePath() { + return concatResourcePath(name); + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + public MessageParserInfo getParserInfo() { + return parserInfo; + } + + public void setParserInfo(MessageParserInfo parserInfo) { + this.parserInfo = parserInfo; + } + + @Override + public StreamingSourceConfig clone() { + try { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + SERIALIZER.serialize(this, new DataOutputStream(baos)); + return SERIALIZER.deserialize(new DataInputStream(new ByteArrayInputStream(baos.toByteArray()))); + } catch (IOException e) { + throw new RuntimeException(e);//in mem, should not happen + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfigManager.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfigManager.java new file mode 100644 index 00000000000..2ce8fa4a387 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceConfigManager.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.lang3.StringUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.persistence.JsonSerializer; +import org.apache.kylin.common.persistence.ResourceStore; +import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +/** + */ +public class StreamingSourceConfigManager { + + public static final Serializer STREAMING_SERIALIZER = new JsonSerializer( + StreamingSourceConfig.class); + private static final Logger logger = LoggerFactory.getLogger(StreamingSourceConfigManager.class); + // static cached instances + private static final ConcurrentHashMap CACHE = new ConcurrentHashMap(); + private KylinConfig config; + + private StreamingSourceConfigManager(KylinConfig config) throws IOException { + this.config = config; + } + + public static StreamingSourceConfigManager getInstance(KylinConfig config) { + StreamingSourceConfigManager r = CACHE.get(config); + if (r != null) { + return r; + } + + synchronized (StreamingSourceConfigManager.class) { + r = CACHE.get(config); + if (r != null) { + return r; + } + try { + r = new StreamingSourceConfigManager(config); + CACHE.put(config, r); + if (CACHE.size() > 1) { + logger.warn("More than one streamingManager singleton exist"); + } + return r; + } catch (IOException e) { + throw new IllegalStateException("Failed to init StreamingManager from " + config, e); + } + } + } + + private ResourceStore getStore() { + return ResourceStore.getStore(this.config); + } + + public List listAllStreaming() throws IOException { + List results = Lists.newArrayList(); + ResourceStore store = getStore(); + logger.info("Load all streaming metadata from folder " + + store.getReadableResourcePath(ResourceStore.STREAMING_V2_RESOURCE_ROOT)); + + List paths = store.collectResourceRecursively(ResourceStore.STREAMING_V2_RESOURCE_ROOT, + MetadataConstants.FILE_SURFIX); + for (String path : paths) { + StreamingSourceConfig streamingSourceConfig; + try { + streamingSourceConfig = loadStreamingConfigAt(path); + } catch (Exception e) { + logger.error("Error loading streaming desc " + path, e); + continue; + } + if (path.equals(streamingSourceConfig.getResourcePath()) == false) { + logger.error("Skip suspicious desc at " + path + ", " + streamingSourceConfig + " should be at " + + streamingSourceConfig.getResourcePath()); + continue; + } + results.add(streamingSourceConfig); + } + + logger.debug("Loaded " + results.size() + " StreamingSourceConfig(s)"); + return results; + } + + /** + * Reload StreamingSourceConfig from resource store It will be triggered by an desc + * update event. + * + * @param name + * @throws IOException + */ + public StreamingSourceConfig reloadStreamingConfigLocal(String name) throws IOException { + + // Save Source + String path = StreamingSourceConfig.concatResourcePath(name); + + // Reload the StreamingSourceConfig + StreamingSourceConfig ndesc = loadStreamingConfigAt(path); + return ndesc; + } + + // remove streamingSourceConfig + public void removeStreamingConfig(StreamingSourceConfig streamingSourceConfig) throws IOException { + String path = streamingSourceConfig.getResourcePath(); + getStore().deleteResource(path); + } + + public StreamingSourceConfig getConfig(String name) { + name = name.toUpperCase(Locale.ROOT); + try { + return reloadStreamingConfigLocal(name); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + /** + * + * @param desc + * @return + * @throws IOException + */ + public StreamingSourceConfig updateStreamingConfig(StreamingSourceConfig desc) throws IOException { + // Validate CubeDesc + if (desc.getUuid() == null || desc.getName() == null) { + throw new IllegalArgumentException("SteamingConfig Illegal."); + } + + // Save Source + String path = desc.getResourcePath(); + getStore().putResource(path, desc, System.currentTimeMillis(), STREAMING_SERIALIZER); + + // Reload the StreamingSourceConfig + StreamingSourceConfig ndesc = loadStreamingConfigAt(path); + + return ndesc; + } + + public StreamingSourceConfig saveStreamingConfig(StreamingSourceConfig streamingSourceConfig) throws IOException { + if (streamingSourceConfig == null || StringUtils.isEmpty(streamingSourceConfig.getName())) { + throw new IllegalArgumentException(); + } + + String path = StreamingSourceConfig.concatResourcePath(streamingSourceConfig.getName()); + getStore().putResource(path, streamingSourceConfig, System.currentTimeMillis(), + StreamingSourceConfig.SERIALIZER); + return streamingSourceConfig; + } + + private StreamingSourceConfig loadStreamingConfigAt(String path) throws IOException { + ResourceStore store = getStore(); + StreamingSourceConfig streamingDesc = store.getResource(path, STREAMING_SERIALIZER); + if (streamingDesc == null) { + return null; + } + if (StringUtils.isBlank(streamingDesc.getName())) { + throw new IllegalStateException("StreamingSourceConfig name must not be blank"); + } + return streamingDesc; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceFactory.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceFactory.java new file mode 100644 index 00000000000..52ed47a63b9 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingSourceFactory.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import static org.apache.kylin.metadata.model.ISourceAware.ID_KAFKA; +import static org.apache.kylin.metadata.model.ISourceAware.ID_KAFKA_HIVE; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.kylin.common.util.ImplementationSwitch; +import org.apache.kylin.metadata.model.ISourceAware; + +public class StreamingSourceFactory { + private static String KAFKA_SOURCE_CLAZZ = "org.apache.kylin.stream.source.kafka.KafkaSource"; + + private static ImplementationSwitch sources; + static { + Map impls = new HashMap<>(); + impls.put(ID_KAFKA, KAFKA_SOURCE_CLAZZ); + impls.put(ID_KAFKA_HIVE, KAFKA_SOURCE_CLAZZ); + sources = new ImplementationSwitch<>(impls, IStreamingSource.class); + } + + public static IStreamingSource getStreamingSource(ISourceAware aware) { + return sources.get(aware.getSourceType()); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingTableSourceInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingTableSourceInfo.java new file mode 100644 index 00000000000..7e0600f6701 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/source/StreamingTableSourceInfo.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.source; + +import java.util.List; + +public class StreamingTableSourceInfo { + private List partitions; + + public StreamingTableSourceInfo(List partitions) { + this.partitions = partitions; + } + + public List getPartitions() { + return partitions; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StreamingTableSourceInfo that = (StreamingTableSourceInfo) o; + + return partitions != null ? partitions.equals(that.partitions) : that.partitions == null; + + } + + @Override + public int hashCode() { + return partitions != null ? partitions.hashCode() : 0; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPoint.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPoint.java new file mode 100644 index 00000000000..c6bf5e42956 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPoint.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.util.Map; + +import org.apache.kylin.stream.core.model.stats.LongLatencyInfo; + +public class CheckPoint { + private String sourceConsumePosition; + private Map persistedIndexes; + private LongLatencyInfo longLatencyInfo; + // Map + private Map segmentSourceStartPosition; + private long checkPointTime; + private long totalCount; + private long checkPointCount; + + public String getSourceConsumePosition() { + return sourceConsumePosition; + } + + public void setSourceConsumePosition(String sourceConsumePosition) { + this.sourceConsumePosition = sourceConsumePosition; + } + + public Map getPersistedIndexes() { + return persistedIndexes; + } + + public void setPersistedIndexes(Map persistedIndexes) { + this.persistedIndexes = persistedIndexes; + } + + public long getCheckPointTime() { + return checkPointTime; + } + + public void setCheckPointTime(long checkPointTime) { + this.checkPointTime = checkPointTime; + } + + public long getTotalCount() { + return totalCount; + } + + public void setTotalCount(long totalCount) { + this.totalCount = totalCount; + } + + public long getCheckPointCount() { + return checkPointCount; + } + + public void setCheckPointCount(long checkPointCount) { + this.checkPointCount = checkPointCount; + } + + public LongLatencyInfo getLongLatencyInfo() { + return longLatencyInfo; + } + + public void setLongLatencyInfo(LongLatencyInfo longLatencyInfo) { + this.longLatencyInfo = longLatencyInfo; + } + + public Map getSegmentSourceStartPosition() { + return segmentSourceStartPosition; + } + + public void setSegmentSourceStartPosition(Map segmentSourceStartPosition) { + this.segmentSourceStartPosition = segmentSourceStartPosition; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CheckPoint that = (CheckPoint) o; + + if (checkPointTime != that.checkPointTime) return false; + if (totalCount != that.totalCount) return false; + if (checkPointCount != that.checkPointCount) return false; + if (sourceConsumePosition != null ? !sourceConsumePosition.equals(that.sourceConsumePosition) : that.sourceConsumePosition != null) + return false; + if (persistedIndexes != null ? !persistedIndexes.equals(that.persistedIndexes) : that.persistedIndexes != null) + return false; + if (longLatencyInfo != null ? !longLatencyInfo.equals(that.longLatencyInfo) : that.longLatencyInfo != null) + return false; + return segmentSourceStartPosition != null ? segmentSourceStartPosition.equals(that.segmentSourceStartPosition) : that.segmentSourceStartPosition == null; + + } + + @Override + public int hashCode() { + int result = sourceConsumePosition != null ? sourceConsumePosition.hashCode() : 0; + result = 31 * result + (persistedIndexes != null ? persistedIndexes.hashCode() : 0); + result = 31 * result + (longLatencyInfo != null ? longLatencyInfo.hashCode() : 0); + result = 31 * result + (segmentSourceStartPosition != null ? segmentSourceStartPosition.hashCode() : 0); + result = 31 * result + (int) (checkPointTime ^ (checkPointTime >>> 32)); + result = 31 * result + (int) (totalCount ^ (totalCount >>> 32)); + result = 31 * result + (int) (checkPointCount ^ (checkPointCount >>> 32)); + return result; + } + + @Override + public String toString() { + return "CheckPoint{" + + "sourceConsumePosition='" + sourceConsumePosition + '\'' + + ", persistedIndexes=" + persistedIndexes + + ", longLatencyInfo=" + longLatencyInfo + + ", segmentSourceStartPosition=" + segmentSourceStartPosition + + ", checkPointTime=" + checkPointTime + + ", totalCount=" + totalCount + + ", checkPointCount=" + checkPointCount + + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPointStore.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPointStore.java new file mode 100644 index 00000000000..4ce875e2cbc --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/CheckPointStore.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.io.File; +import java.io.FileFilter; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.LinkedList; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.input.ReversedLinesFileReader; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.exception.IllegalStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; + +/** + * checkpoint + */ +public class CheckPointStore { + static final int CP_FILE_MAX_NUM = 5; + private static final String CP_START = "===================="; + private static final String CP_END = "####################"; + private static final long DAY_TIMESTAMP_BASE = 24 * 3600 * 1000L; + private static final String CP_FILE_PREFIX = "CP-"; + private static Logger logger = LoggerFactory.getLogger(CheckPointStore.class); + private final File checkPointFolder; + private final String cubeName; + private final int maxNumOfCPFile; + + public CheckPointStore(String cubeName, File checkPointParent) { + this(cubeName, checkPointParent, CP_FILE_MAX_NUM); + } + + public CheckPointStore(String cubeName, File checkPointParent, int keepCPFileNum) { + this.cubeName = cubeName; + this.checkPointFolder = new File(checkPointParent, ".cp"); + if (checkPointFolder.exists() && !checkPointFolder.isDirectory()) { + checkPointFolder.delete(); + } + if (!checkPointFolder.exists()) { + checkPointFolder.mkdirs(); + } + this.maxNumOfCPFile = keepCPFileNum; + } + + @VisibleForTesting + File[] getCheckPointFiles() { + File[] cpFiles = checkPointFolder.listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return pathname.getName().startsWith(CP_FILE_PREFIX); + } + }); + if (cpFiles.length == 0) { + return null; + } + return cpFiles; + } + + private File getLatestCheckPointFile() { + File[] cpFiles = getCheckPointFiles(); + if (cpFiles == null || cpFiles.length == 0) { + return null; + } + + File latestCheckPointFile = cpFiles[0]; + long latestTimestamp = getTimestampFromFileName(latestCheckPointFile.getName()); + for (int i = 1; i < cpFiles.length; i++) { + long curTimestamp = getTimestampFromFileName(cpFiles[i].getName()); + if (curTimestamp > latestTimestamp) { + latestTimestamp = curTimestamp; + latestCheckPointFile = cpFiles[i]; + } + } + return latestCheckPointFile; + } + + private File getCheckPointFile(CheckPoint cp) { + File checkPointFile = new File(checkPointFolder, getFileNameFromTimestamp(cp.getCheckPointTime())); + if (!checkPointFile.exists()) { + try { + checkPointFile.createNewFile(); + deleteOldCPFiles(); + } catch (IOException e) { + throw new IllegalStorageException(e); + } + } + return checkPointFile; + } + + @VisibleForTesting + void deleteOldCPFiles() { + File[] cpFiles = getCheckPointFiles(); + if (cpFiles == null || cpFiles.length <= maxNumOfCPFile) { + return; + } + + ArrayList cpFileList = Lists.newArrayList(cpFiles); + Collections.sort(cpFileList, new Comparator() { + @Override + public int compare(File o1, File o2) { + return o1.getName().compareTo(o2.getName()); + } + }); + Iterator fileIterator = cpFileList.subList(0, cpFileList.size() - maxNumOfCPFile).iterator(); + while (fileIterator.hasNext()) { + File file = fileIterator.next(); + logger.info("going to delete checkpoint file " + file.getName()); + System.out.println("going to delete checkpoint file " + file.getName()); + file.delete(); + } + } + + private String getFileNameFromTimestamp(long timestamp) { + return CP_FILE_PREFIX + (timestamp / DAY_TIMESTAMP_BASE) * DAY_TIMESTAMP_BASE; + } + + private long getTimestampFromFileName(String cpFileName) { + return Long.valueOf(cpFileName.substring(CP_FILE_PREFIX.length())); + } + + public void saveCheckPoint(CheckPoint cp) { + try (FileOutputStream outputStream = FileUtils.openOutputStream(getCheckPointFile(cp), true)) { + String jsonCP = JsonUtil.writeValueAsIndentString(cp); + outputStream.write(Bytes.toBytes(wrapCheckPointString(jsonCP))); + outputStream.flush(); + } catch (Exception e) { + logger.error("CheckPoint error for cube " + cubeName, e); + } + } + + private String wrapCheckPointString(String checkpoint) { + String lineSeparator = System.lineSeparator(); + return CP_START + lineSeparator + checkpoint + lineSeparator + CP_END + lineSeparator; + } + + /** + * + * @return null if there is no valid checkpoint + */ + public CheckPoint getLatestCheckPoint() { + return getLatestCheckPoint(getLatestCheckPointFile()); + } + + private CheckPoint getLatestCheckPoint(File checkPointFile) { + if (checkPointFile == null) { + return null; + } + CheckPoint cp = null; + try (ReversedLinesFileReader fileReader = new ReversedLinesFileReader(checkPointFile, 4096, + Charset.forName("UTF-8"))) { + String line = fileReader.readLine(); + while (!CP_END.equals(line) && line != null) { + line = fileReader.readLine(); + } + + if (line == null) { //not found the checkpoint end line + return null; + } + + LinkedList stack = Lists.newLinkedList(); + line = fileReader.readLine(); + while (!CP_START.equals(line) && line != null) { + stack.push(line); + line = fileReader.readLine(); + } + if (line == null) { + return null; + } + StringBuilder cpJson = new StringBuilder(); + while (!stack.isEmpty()) { + cpJson.append(stack.pop()); + } + cp = JsonUtil.readValue(cpJson.toString(), CheckPoint.class); + } catch (IOException e) { + logger.error("error when parse checkpoint"); + } + return cp; + } + + public void removeAllCheckPoints() { + File[] cpFiles = getCheckPointFiles(); + if (cpFiles == null || cpFiles.length == 0) { + return; + } + for (File cpFile : cpFiles) { + removeCheckPoints(cpFile); + } + } + + private void removeCheckPoints(File checkPointFile) { + try { + FileUtils.write(checkPointFile, ""); + } catch (IOException e) { + logger.error("error when remove all checkpoints"); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/IStreamingSegmentStore.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/IStreamingSegmentStore.java new file mode 100644 index 00000000000..a5e082b3bea --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/IStreamingSegmentStore.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.io.File; +import java.io.IOException; + +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.model.stats.SegmentStoreStats; +import org.apache.kylin.stream.core.query.IStreamingGTSearcher; + +public interface IStreamingSegmentStore extends IStreamingGTSearcher { + void init(); + + int addEvent(StreamingMessage event); + + File getStorePath(); + + void persist(); + + /** + * the latest store state, checkpoint implementation need this info + * to save the store state + * @return + */ + Object checkpoint(); + + void purge(); + + void restoreFromCheckpoint(Object checkpoint); + + String getSegmentName(); + + StreamingCubeSegment.State getSegmentState(); + + void setSegmentState(StreamingCubeSegment.State state); + + SegmentStoreStats getStoreStats(); + + void close() throws IOException; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/Record.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/Record.java new file mode 100644 index 00000000000..7333fa91406 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/Record.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.util.Arrays; + +public class Record { + private String[] dimensions; + private Object[] metrics; + + public Record(int dimNum, int metricNum) { + this.dimensions = new String[dimNum]; + this.metrics = new Object[metricNum]; + } + + public Record copy() { + Record record = new Record(dimensions.length, metrics.length); + System.arraycopy(dimensions, 0, record.dimensions, 0, dimensions.length); + System.arraycopy(metrics, 0, record.metrics, 0, metrics.length); + return record; + } + + public void setDimensions(String[] dimensions) { + this.dimensions = dimensions; + } + + public void setMetrics(Object[] metrics) { + this.metrics = metrics; + } + + public String[] getDimensions() { + return dimensions; + } + + public void setDimension(int idx, String value) { + this.dimensions[idx] = value; + } + + public Object[] getMetrics() { + return metrics; + } + + public void setMetric(int idx, Object value) { + this.metrics[idx] = value; + } + + @Override + public String toString() { + return "Record{" + "dimensions=" + Arrays.toString(dimensions) + ", metrics=" + Arrays.toString(metrics) + + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/StreamingCubeSegment.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/StreamingCubeSegment.java new file mode 100644 index 00000000000..dbcdf19391e --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/StreamingCubeSegment.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeSegment; +import org.apache.kylin.metadata.model.SegmentRange.TSRange; +import org.apache.kylin.stream.core.model.StreamingMessage; + +public class StreamingCubeSegment implements Comparable { + private volatile State state; + private IStreamingSegmentStore segmentStore; + private long createTime; + private long lastUpdateTime; + private long latestEventTimeStamp = 0; + private long latestEventLatecy = 0; + private String cubeName; + private CubeInstance cubeInstance; + private String segmentName; + private long dateRangeStart; + private long dateRangeEnd; + + public StreamingCubeSegment(CubeInstance cubeInstance, IStreamingSegmentStore segmentStore, long segmentStart, + long segmentEnd) { + this.cubeInstance = cubeInstance; + + this.cubeName = cubeInstance.getName(); + this.dateRangeStart = segmentStart; + this.dateRangeEnd = segmentEnd; + this.createTime = System.currentTimeMillis(); + this.lastUpdateTime = System.currentTimeMillis(); + this.state = State.ACTIVE; + this.segmentStore = segmentStore; + this.segmentName = CubeSegment.makeSegmentName(new TSRange(segmentStart, segmentEnd), null, cubeInstance.getModel()); + } + + public static StreamingCubeSegment parseSegment(CubeInstance cubeInstance, File segmentFolder, + IStreamingSegmentStore segmentStore) { + Pair segmentStartEnd = CubeSegment.parseSegmentName(segmentFolder.getName()); + StreamingCubeSegment segment = new StreamingCubeSegment(cubeInstance, segmentStore, segmentStartEnd.getFirst(), + segmentStartEnd.getSecond()); + + State state = segmentStore.getSegmentState(); + segment.saveState(state); + return segment; + } + + public IStreamingSegmentStore getSegmentStore() { + return segmentStore; + } + + public void immutable() { + segmentStore.persist(); + saveState(State.IMMUTABLE); + } + + public boolean isActive() { + return State.ACTIVE == state; + } + + public boolean isImmutable() { + return State.IMMUTABLE == state || State.REMOTE_PERSISTED == state; + } + + public boolean isPersistToRemote() { + return State.REMOTE_PERSISTED == state; + } + + public long getCreateTime() { + return createTime; + } + + public long getLastUpdateTime() { + return lastUpdateTime; + } + + public void setLastUpdateTime(long lastUpdateTime) { + this.lastUpdateTime = lastUpdateTime; + } + + public long getLatestEventTimeStamp() { + return latestEventTimeStamp; + } + + public long getLatestEventLatecy() { + return latestEventLatecy; + } + + public long getDateRangeStart() { + return dateRangeStart; + } + + public long getDateRangeEnd() { + return dateRangeEnd; + } + + public boolean isLongLatencySegment() { + return dateRangeStart == 0; + } + + public Pair getSegmentRange() { + return new Pair<>(dateRangeStart, dateRangeEnd); + } + + public File getDataSegmentFolder() { + return segmentStore.getStorePath(); + } + + public CubeInstance getCubeInstance() { + return cubeInstance; + } + + public String getCubeName() { + return cubeName; + } + + public String getSegmentName() { + return segmentName; + } + + public State getState() { + return state; + } + + public void saveState(State state) { + this.segmentStore.setSegmentState(state); + this.state = state; + } + + public void addEvent(StreamingMessage event) { + segmentStore.addEvent(event); + latestEventTimeStamp = event.getTimestamp(); + latestEventLatecy = System.currentTimeMillis() - event.getTimestamp(); + } + + public void purge() { + segmentStore.purge(); + } + + public void close() throws IOException { + segmentStore.close(); + } + + @Override + public int compareTo(StreamingCubeSegment o) { + if (!this.getCubeName().equals(o.getCubeName())) { + return this.getCubeName().compareTo(o.getCubeName()); + } + + return Long.compare(getDateRangeStart(), o.getDateRangeStart()); + } + + @Override + public String toString() { + return "StreamingCubeSegment [cubeName=" + cubeName + ", segmentName=" + segmentName + "]"; + } + + public enum State { + ACTIVE, IMMUTABLE, REMOTE_PERSISTED + } + + public static class SegmentInfo { + private Map sourceStartOffsets; + + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataReader.java new file mode 100644 index 00000000000..e5cd87a5c53 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataReader.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.Closeable; +import java.util.Iterator; + +public interface ColumnDataReader extends Iterable, Closeable{ + Iterator iterator(); + + byte[] read(int rowNum); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataWriter.java new file mode 100644 index 00000000000..fc1832f8795 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnDataWriter.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; + +public interface ColumnDataWriter { + void write(byte[] valBytes) throws IOException; + + void flush() throws IOException; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMemoryStorePersister.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMemoryStorePersister.java new file mode 100644 index 00000000000..65d4c8fd287 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMemoryStorePersister.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; + +import javax.annotation.Nullable; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.cube.model.RowKeyColDesc; +import org.apache.kylin.dict.DictionaryGenerator; +import org.apache.kylin.dict.DictionarySerializer; +import org.apache.kylin.dict.IterableDictionaryValueEnumerator; +import org.apache.kylin.dict.TrieDictionary; +import org.apache.kylin.dimension.DictionaryDimEnc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.dimension.DimensionEncodingFactory; +import org.apache.kylin.measure.MeasureAggregator; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.columnar.ParsedStreamingCubeInfo.CuboidInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimDictionaryMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimensionMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.MetricMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.invertindex.FixLenIIColumnDescriptor; +import org.apache.kylin.stream.core.storage.columnar.invertindex.IIColumnDescriptor; +import org.apache.kylin.stream.core.storage.columnar.invertindex.SeqIIColumnDescriptor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Function; +import com.google.common.base.Stopwatch; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.io.CountingOutputStream; + +public class ColumnarMemoryStorePersister { + private static Logger logger = LoggerFactory.getLogger(ColumnarMemoryStorePersister.class); + private CubeDesc cubeDesc; + private CubeInstance cubeInstance; + private String segmentName; + + protected final TblColRef[] dimensions; + protected final MeasureDesc[] measures; + protected final Set dimensionsUseDictEncoding; + + protected final long baseCuboidId; + + public ColumnarMemoryStorePersister(ParsedStreamingCubeInfo parsedCubeInfo, String segmentName) { + this.cubeInstance = parsedCubeInfo.cubeInstance; + this.cubeDesc = cubeInstance.getDescriptor(); + this.segmentName = segmentName; + + this.baseCuboidId = parsedCubeInfo.basicCuboid.getId(); + this.dimensions = parsedCubeInfo.dimensions; + this.measures = parsedCubeInfo.measureDescs; + this.dimensionsUseDictEncoding = Sets.newHashSet(parsedCubeInfo.dimensionsUseDictEncoding); + } + + /** + * Build dictionary, Inverted indexes and persist streaming columnar data structure to disk. + * + */ + public void persist(SegmentMemoryStore memoryStore, DataSegmentFragment fragment) { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + logger.info("Start persist memory store for cube:{}, segment:{}, rowCnt:{}", cubeInstance.getName(), segmentName, memoryStore.getRowCount()); + try { + persistDataFragment(memoryStore, fragment); + stopwatch.stop(); + logger.info("Finish persist memory store for cube:{} segment:{}, take: {}ms", cubeInstance.getName(), + segmentName, stopwatch.elapsedMillis()); + } catch (Exception e) { + logger.error("Error persist DataSegment.", e); + } + } + + private void persistDataFragment(SegmentMemoryStore memoryStore, DataSegmentFragment fragment) throws Exception { + FragmentMetaInfo fragmentMeta = new FragmentMetaInfo(); + Map cuboidMetaInfoMap = Maps.newHashMap(); + + fragmentMeta.setFragmentId(fragment.getFragmentId().toString()); + fragmentMeta.setMinEventTime(memoryStore.getMinEventTime()); + fragmentMeta.setMaxEventTime(memoryStore.getMaxEventTime()); + fragmentMeta.setOriginNumOfRows(memoryStore.getOriginRowCount()); + FileOutputStream fragmentOutputStream = FileUtils.openOutputStream(fragment.getDataFile()); + + try (CountingOutputStream fragmentOut = new CountingOutputStream(new BufferedOutputStream(fragmentOutputStream))) { + ConcurrentSkipListMap basicCuboidData = memoryStore.getBasicCuboidData(); + List> basicCuboidColumnarValues = transformToColumnar(baseCuboidId, dimensions.length, + basicCuboidData); + // persist dictionaries + Map> dictMap = buildAndPersistDictionaries(fragmentMeta, + basicCuboidColumnarValues, fragmentOut); + // persist basic cuboid + CuboidMetaInfo basicCuboidMeta = persistCuboidData(baseCuboidId, dimensions, dictMap, + basicCuboidColumnarValues, fragmentOut); + + fragmentMeta.setBasicCuboidMetaInfo(basicCuboidMeta); + long totalRowCnt = basicCuboidMeta.getNumberOfRows(); + + // persist additional cuboids + Map> additionalCuboidsData = memoryStore + .getAdditionalCuboidsData(); + if (additionalCuboidsData != null && additionalCuboidsData.size() > 0) { + for (Entry> cuboidDataEntry : additionalCuboidsData + .entrySet()) { + CuboidInfo cuboidInfo = cuboidDataEntry.getKey(); + ConcurrentSkipListMap cuboidData = cuboidDataEntry.getValue(); + List> cuboidColumnarValues = transformToColumnar(cuboidInfo.getCuboidID(), + cuboidInfo.getDimCount(), cuboidData); + CuboidMetaInfo cuboidMeta = persistCuboidData(cuboidInfo.getCuboidID(), cuboidInfo.getDimensions(), + dictMap, cuboidColumnarValues, fragmentOut); + cuboidMetaInfoMap.put(String.valueOf(cuboidInfo.getCuboidID()), cuboidMeta); + totalRowCnt += cuboidMeta.getNumberOfRows(); + } + } + + fragmentMeta.setNumberOfRows(totalRowCnt); + fragmentMeta.setCuboidMetaInfoMap(cuboidMetaInfoMap); + } + + FileOutputStream metaOutputStream = FileUtils.openOutputStream(fragment.getMetaFile()); + JsonUtil.writeValueIndent(metaOutputStream, fragmentMeta); + metaOutputStream.flush(); + metaOutputStream.close(); + } + + /** + * Transform the internal aggBufMap to columnar format which includes all the dimensions and metrics. + * + * @return + */ + private List> transformToColumnar(long cuboidId, int dimCnt, + ConcurrentSkipListMap aggBufMap) { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + int columnsNum = dimCnt + measures.length; + List> columnarValues = Lists.newArrayListWithExpectedSize(columnsNum); + for (int i = 0; i <= columnsNum; i++) { + List valueList = Lists.newLinkedList(); + columnarValues.add(valueList); + } + + for (Entry entry : aggBufMap.entrySet()) { + String[] row = entry.getKey(); + MeasureAggregator[] measures = entry.getValue(); + + for (int i = 0; i < row.length; i++) { + String cell = row[i]; + List dimValueList = columnarValues.get(i); + dimValueList.add(cell); // todo consider null case + } + + for (int j = 0; j < measures.length; j++) { + MeasureAggregator measure = measures[j]; + List measureValueList = columnarValues.get(dimCnt + j); + measureValueList.add(measure.getState()); + } + } + stopwatch.stop(); + if (logger.isDebugEnabled()) { + logger.debug("cuboid-{} transform to columnar, take {} ms", cuboidId, stopwatch.elapsedMillis()); + } + return columnarValues; + } + + private Map> buildAndPersistDictionaries(FragmentMetaInfo fragmentMetaInfo, + List> allColumnarValues, CountingOutputStream fragmentOut) throws IOException { + Map> dictMaps = Maps.newHashMap(); + List dimDictionaryMetaInfos = Lists.newArrayList(); + for (int i = 0; i < dimensions.length; i++) { + TblColRef dimension = dimensions[i]; + List dimValueList = allColumnarValues.get(i); + Dictionary dict; + DimDictionaryMetaInfo dimDictionaryMetaInfo = new DimDictionaryMetaInfo(); + if (dimensionsUseDictEncoding.contains(dimension)) { + dict = buildDictionary(dimension, dimValueList); + dictMaps.put(dimension, dict); + + dimDictionaryMetaInfo.setDimName(dimension.getName()); + dimDictionaryMetaInfo.setDictType(dict.getClass().getName()); + dimDictionaryMetaInfo.setStartOffset((int) fragmentOut.getCount()); + + DictionarySerializer.serialize(dict, fragmentOut); + dimDictionaryMetaInfo.setDictLength((int) fragmentOut.getCount() + - dimDictionaryMetaInfo.getStartOffset()); + dimDictionaryMetaInfos.add(dimDictionaryMetaInfo); + } + } + fragmentMetaInfo.setDimDictionaryMetaInfos(dimDictionaryMetaInfos); + return dictMaps; + } + + private CuboidMetaInfo persistCuboidData(long cuboidID, TblColRef[] dimensions, + Map> dictMaps, List> columnarCuboidValues, + CountingOutputStream fragmentOutput) throws Exception { + CuboidMetaInfo cuboidMeta = new CuboidMetaInfo(); + int dimCnt = dimensions.length; + List dimensionMetaList = Lists.newArrayListWithExpectedSize(dimCnt); + cuboidMeta.setDimensionsInfo(dimensionMetaList); + cuboidMeta.setNumberOfDim(dimCnt); + List metricMetaInfoList = Lists.newArrayListWithCapacity(measures.length); + cuboidMeta.setMetricsInfo(metricMetaInfoList); + cuboidMeta.setNumberOfMetrics(measures.length); + + long rowNum = -1; + for (int i = 0; i < dimCnt; i++) { + if (rowNum == -1) { + rowNum = columnarCuboidValues.get(i).size(); + } + persistDimension(cuboidID, columnarCuboidValues.get(i), dimensionMetaList, fragmentOutput, + dimensions[i], dictMaps); + } + + for (int i = 0; i < measures.length; i++) { + persistMetric(cuboidID, columnarCuboidValues.get(dimCnt + i), metricMetaInfoList, i, fragmentOutput); + } + cuboidMeta.setNumberOfRows(rowNum); + return cuboidMeta; + } + + /** + * This method is used to persist the dimension data to disk file, first part is the dictionary, second part is the dimension value, third part is the index. + * + * @param dimValueList + * @param dimensionMetaList + * @param indexOut + * @param dimension + * @param dictMaps + * @throws IOException + */ + private void persistDimension(long cuboidId, List dimValueList, List dimensionMetaList, + CountingOutputStream indexOut, TblColRef dimension, Map> dictMaps) + throws IOException { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + + DimensionMetaInfo dimensionMeta = new DimensionMetaInfo(); + dimensionMetaList.add(dimensionMeta); + + DimensionEncoding encoding; + IIColumnDescriptor columnDescriptor; + if (dimensionsUseDictEncoding.contains(dimension)) { + Dictionary dict = dictMaps.get(dimension); + encoding = new DictionaryDimEnc(dict); + if (dict instanceof TrieDictionary) { + columnDescriptor = new SeqIIColumnDescriptor(dimension.getName(), dict.getMinId(), dict.getMaxId()); + } else { + columnDescriptor = new FixLenIIColumnDescriptor(dimension.getName(), encoding.getLengthOfEncoding()); + } + } else { + RowKeyColDesc colDesc = cubeDesc.getRowkey().getColDesc(dimension); + encoding = DimensionEncodingFactory.create(colDesc.getEncodingName(), colDesc.getEncodingArgs(), + colDesc.getEncodingVersion()); + columnDescriptor = new FixLenIIColumnDescriptor(dimension.getName(), encoding.getLengthOfEncoding()); + } + dimensionMeta.setName(dimension.getName()); + dimensionMeta.setStartOffset((int) indexOut.getCount()); + int fixEncodingLen = encoding.getLengthOfEncoding(); + + DataOutputStream dataOut = new DataOutputStream(indexOut); + ColumnarStoreDimDesc cStoreDimDesc = getColumnarStoreDimDesc(dimension, encoding); + ColumnDataWriter columnDataWriter = cStoreDimDesc.getDimWriter(dataOut, dimValueList.size()); + + //Raw values are stored on disk files with fixed length encoding to make it easy for inverted index to search and scan. + for (Object cell : dimValueList) { + byte[] fixLenBytes = new byte[fixEncodingLen]; + if (cell != null) { + encoding.encode((String) cell, fixLenBytes, 0); + } else { + encoding.encode(null, fixLenBytes, 0); + dimensionMeta.setHasNull(true); + } + columnDescriptor.getWriter().addValue(fixLenBytes); + columnDataWriter.write(fixLenBytes); + } + columnDataWriter.flush(); + dimensionMeta.setDataLength(dataOut.size()); + columnDescriptor.getWriter().write(indexOut); + dimensionMeta.setIndexLength((int) indexOut.getCount() - dimensionMeta.getStartOffset() + - dimensionMeta.getDataLength()); + dimensionMeta.setCompression(cStoreDimDesc.getCompression().name()); + + stopwatch.stop(); + if (logger.isDebugEnabled()) { + logger.debug("cuboid-{} saved dimension:{}, took: {}ms", cuboidId, dimension.getName(), + stopwatch.elapsedMillis()); + } + } + + private ColumnarStoreDimDesc getColumnarStoreDimDesc(TblColRef dimension, DimensionEncoding encoding) { + return ColumnarStoreDimDesc.getDefaultCStoreDimDesc(cubeDesc, dimension.getName(), encoding); + } + + /** + * This method is used to persist the metrics data to disk file. + * + * @param metricValueList + * @param metricMetaInfoList + * @param indexOut + * @throws IOException + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + private void persistMetric(long cuboidId, List metricValueList, List metricMetaInfoList, + int metricIdx, CountingOutputStream indexOut) throws IOException { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + + MetricMetaInfo metricMeta = new MetricMetaInfo(); + metricMetaInfoList.add(metricMeta); + String measureName = measures[metricIdx].getName(); + metricMeta.setName(measureName); + metricMeta.setCol(metricIdx); + metricMeta.setStartOffset((int) indexOut.getCount()); + + DataType type = measures[metricIdx].getFunction().getReturnDataType(); + + ColumnarMetricsEncoding metricsEncoding = ColumnarMetricsEncodingFactory.create(type); + DataTypeSerializer serializer = metricsEncoding.asDataTypeSerializer(); + DataOutputStream metricsOut = new DataOutputStream(indexOut); + + int maxLength = serializer.maxLength(); + metricMeta.setMaxSerializeLength(maxLength); + ByteBuffer metricsBuf = ByteBuffer.allocate(maxLength); + ColumnarStoreMetricsDesc cStoreMetricsDesc = getColumnarStoreMetricsDesc(metricsEncoding); + ColumnDataWriter metricsWriter = cStoreMetricsDesc.getMetricsWriter(metricsOut, metricValueList.size()); +// metricMeta.setStoreInFixedLength(false); + for (Object metricValue : metricValueList) { + metricsBuf.clear(); + serializer.serialize(metricValue, metricsBuf); + byte[] metricBytes = Arrays.copyOf(metricsBuf.array(), metricsBuf.position()); + metricsWriter.write(metricBytes); + } + metricsWriter.flush(); + metricMeta.setMetricLength(metricsOut.size()); + metricMeta.setCompression(cStoreMetricsDesc.getCompression().name()); + stopwatch.stop(); + if (logger.isDebugEnabled()) { + logger.debug("cuboid-{} saved measure:{}, took: {}ms", cuboidId, measureName, stopwatch.elapsedMillis()); + } + } + + private ColumnarStoreMetricsDesc getColumnarStoreMetricsDesc(ColumnarMetricsEncoding metricsEncoding) { + return ColumnarStoreMetricsDesc.getDefaultCStoreMetricsDesc(metricsEncoding); + } + + private Dictionary buildDictionary(TblColRef dim, List inputValues) throws IOException { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + final Collection values = Collections2.transform(Sets.newHashSet(inputValues), + new Function() { + @Nullable + @Override + public String apply(Object input) { + String value = (String) input; + return value; + } + }); + final Dictionary dict = DictionaryGenerator.buildDictionary(dim.getType(), + new IterableDictionaryValueEnumerator(values)); + stopwatch.stop(); + if (logger.isDebugEnabled()) { + logger.debug("BuildDictionary for column : " + dim.getName() + " took : " + stopwatch.elapsedMillis() + + " ms "); + } + return dict; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncoding.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncoding.java new file mode 100644 index 00000000000..2ffe5dd8248 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncoding.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; + +public abstract class ColumnarMetricsEncoding { + protected DataType dataType; + + public ColumnarMetricsEncoding(DataType dataType) { + this.dataType = dataType; + } + + public abstract boolean isFixLength(); + + /** return the fixed length of encoded bytes */ + public abstract int getFixLength(); + + public abstract DataTypeSerializer asDataTypeSerializer(); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncodingFactory.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncodingFactory.java new file mode 100644 index 00000000000..c1137a5090e --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarMetricsEncodingFactory.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.nio.ByteBuffer; +import java.util.Set; + +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.apache.kylin.metadata.datatype.DoubleSerializer; +import org.apache.kylin.metadata.datatype.Long8Serializer; + +import com.google.common.collect.Sets; + +public class ColumnarMetricsEncodingFactory { + private static Set longEncodingTypes = Sets.newHashSet(); + private static Set doubleEncodingTypes = Sets.newHashSet(); + static { + longEncodingTypes.add("bigint"); + longEncodingTypes.add("long"); + longEncodingTypes.add("integer"); + longEncodingTypes.add("int"); + longEncodingTypes.add("tinyint"); + longEncodingTypes.add("smallint"); + + doubleEncodingTypes.add("double"); + doubleEncodingTypes.add("float"); + } + + public static ColumnarMetricsEncoding create(DataType dataType) { + if (longEncodingTypes.contains(dataType.getName())) { + return new ColumnarLongMetricsEncoding(dataType); + } + if (doubleEncodingTypes.contains(dataType.getName())) { + return new ColumnarDoubleMetricsEncoding(dataType); + } + return new ColumnarComplexMetricsEncoding(dataType); + } + + public static class ColumnarLongMetricsEncoding extends ColumnarMetricsEncoding { + public ColumnarLongMetricsEncoding(DataType dataType) { + super(dataType); + } + + @Override + public boolean isFixLength() { + return true; + } + + @Override + public int getFixLength() { + return 8; + } + + @Override + public DataTypeSerializer asDataTypeSerializer() { + return new Long8Serializer(dataType); + } + } + + public static class ColumnarFixLenLongMetricsEncoding extends ColumnarMetricsEncoding { + private int fixLength; + + public ColumnarFixLenLongMetricsEncoding(DataType dataType, int fixLength) { + super(dataType); + this.fixLength = fixLength; + } + + @Override + public boolean isFixLength() { + return true; + } + + @Override + public int getFixLength() { + return fixLength; + } + + @Override + public DataTypeSerializer asDataTypeSerializer() { + return new DataTypeSerializer() { + @Override + public int peekLength(ByteBuffer in) { + return fixLength; + } + + @Override + public int maxLength() { + return fixLength; + } + + @Override + public int getStorageBytesEstimate() { + return fixLength; + } + + @Override + public void serialize(Long value, ByteBuffer out) { + long longV = value; + for (int i = 0; i < fixLength; i++) { + out.put((byte) longV); + longV >>>= 8; + } + } + + @Override + public Long deserialize(ByteBuffer in) { + long integer = 0; + int mask = 0xff; + int shift = 0; + for (int i = 0; i < fixLength; i++) { + integer |= (in.get() << shift) & mask; + mask = mask << 8; + shift += 8; + } + return integer; + } + }; + } + } + + public static class ColumnarDoubleMetricsEncoding extends ColumnarMetricsEncoding { + + public ColumnarDoubleMetricsEncoding(DataType dataType) { + super(dataType); + } + + @Override + public boolean isFixLength() { + return true; + } + + @Override + public int getFixLength() { + return 8; + } + + @Override + public DataTypeSerializer asDataTypeSerializer() { + return new DoubleSerializer(dataType); + } + } + + public static class ColumnarComplexMetricsEncoding extends ColumnarMetricsEncoding { + + public ColumnarComplexMetricsEncoding(DataType dataType) { + super(dataType); + } + + @Override + public boolean isFixLength() { + return false; + } + + @Override + public int getFixLength() { + return -1; + } + + @Override + public DataTypeSerializer asDataTypeSerializer() { + return DataTypeSerializer.create(dataType); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarRecordCodec.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarRecordCodec.java new file mode 100644 index 00000000000..c99219e8acc --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarRecordCodec.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.nio.ByteBuffer; +import java.nio.charset.Charset; + +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.dimension.DictionaryDimEnc; +import org.apache.kylin.dimension.DictionaryDimEnc.DictionarySerializer; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; + +public class ColumnarRecordCodec { + private DimensionEncoding[] dimEncodings; + private DataTypeSerializer[] dimSerializers; + private DataTypeSerializer[] metricsSerializers; + + public ColumnarRecordCodec(DimensionEncoding[] dimEncodings, ColumnarMetricsEncoding[] metricsEncodings) { + this.dimEncodings = dimEncodings; + this.dimSerializers = new DataTypeSerializer[dimEncodings.length]; + for (int i = 0; i < dimEncodings.length; i++) { + dimSerializers[i] = dimEncodings[i].asDataTypeSerializer(); + } + this.metricsSerializers = new DataTypeSerializer[metricsEncodings.length]; + for (int i = 0; i < metricsEncodings.length; i++) { + metricsSerializers[i] = metricsEncodings[i].asDataTypeSerializer(); + } + } + + public Object decodeMetrics(int i, byte[] metricsVal) { + return metricsSerializers[i].deserialize(ByteBuffer.wrap(metricsVal)); + } + + public String decodeDimension(int i, byte[] dimVal) { + return dimEncodings[i].decode(dimVal, 0, dimVal.length); + } + + public void encodeDimension(int col, Object value, int roundingFlag, ByteBuffer buf) { + DataTypeSerializer serializer = dimSerializers[col]; + if (serializer instanceof DictionarySerializer) { + DictionaryDimEnc dictEnc = ((DictionaryDimEnc) dimEncodings[col]); + if (dictEnc.getRoundingFlag() != roundingFlag) { + serializer = dictEnc.copy(roundingFlag).asDataTypeSerializer(); + } + try { + serializer.serialize(value, buf); + } catch (IllegalArgumentException ex) { + IllegalArgumentException rewordEx = new IllegalArgumentException("Column " + col + " value '" + toStringBinary(value) + "' met dictionary error: " + ex.getMessage()); + rewordEx.setStackTrace(ex.getStackTrace()); + throw rewordEx; + } + } else { + if (value instanceof String) { + // for dimensions; measures are converted by MeasureIngestor before reaching this point + value = serializer.valueOf((String) value); + } + serializer.serialize(value, buf); + } + } + + public int getMaxDimLength() { + int max = 0; + for (int i = 0; i < dimSerializers.length; i++) { + max = Math.max(max, dimSerializers[i].maxLength()); + } + return max; + } + + public int getMaxMetricsLength() { + int max = 0; + for (int i = 0; i < metricsSerializers.length; i++) { + max = Math.max(max, metricsSerializers[i].maxLength()); + } + return max; + } + + public DimensionEncoding[] getDimensionEncodings() { + return dimEncodings; + } + + private String toStringBinary(Object value) { + if (value == null) + return "Null"; + byte[] bytes; + bytes = value.toString().getBytes(Charset.forName("UTF-8")); + return Bytes.toStringBinary(bytes); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStore.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStore.java new file mode 100644 index 00000000000..c7080b5568c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStore.java @@ -0,0 +1,465 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.File; +import java.io.FileFilter; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.stream.core.exception.IllegalStorageException; +import org.apache.kylin.stream.core.metrics.StreamingMetrics; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.model.stats.SegmentStoreStats; +import org.apache.kylin.stream.core.storage.IStreamingSegmentStore; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.ResultCollector.CloseListener; +import org.apache.kylin.stream.core.storage.StreamingCubeSegment; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.MetricRegistry; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; + +public class ColumnarSegmentStore implements IStreamingSegmentStore { + private static final String STATE_FILE = "_STATE"; + private static Logger logger = LoggerFactory.getLogger(ColumnarSegmentStore.class); + + private static ExecutorService fragmentMergeExecutor; + { + fragmentMergeExecutor = new ThreadPoolExecutor(0, 10, 60L, TimeUnit.SECONDS, + new LinkedBlockingQueue(), new NamedThreadFactory("fragments-merge")); + } + + private volatile SegmentMemoryStore activeMemoryStore; + private volatile SegmentMemoryStore persistingMemoryStore; + private ReentrantReadWriteLock persistLock = new ReentrantReadWriteLock(); + private ReadLock persistReadLock = persistLock.readLock(); + private WriteLock persistWriteLock = persistLock.writeLock(); + + private ReentrantReadWriteLock mergeLock = new ReentrantReadWriteLock(); + private ReadLock mergeReadLock = mergeLock.readLock(); + private WriteLock mergeWriteLock = mergeLock.writeLock(); + + private volatile boolean persisting = false; + private volatile boolean inMerging = false; + + private ColumnarMemoryStorePersister memoryStorePersister; + private String baseStorePath; + private File dataSegmentFolder; + + private int maxRowsInMemory; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private String cubeName; + private String segmentName; + private boolean autoMergeEnabled; + + private List fragments = Lists.newCopyOnWriteArrayList(); + protected int latestCheckpointFragment = 0; + + public ColumnarSegmentStore(String baseStorePath, CubeInstance cubeInstance, String segmentName) { + this.maxRowsInMemory = cubeInstance.getConfig().getStreamingIndexMaxRows(); + this.baseStorePath = baseStorePath; + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + this.cubeName = cubeInstance.getName(); + this.segmentName = segmentName; + + this.dataSegmentFolder = new File(baseStorePath + File.separator + cubeName + File.separator + segmentName); + if (!dataSegmentFolder.exists()) { + dataSegmentFolder.mkdirs(); + } + this.activeMemoryStore = new SegmentMemoryStore(parsedStreamingCubeInfo, segmentName); + this.memoryStorePersister = new ColumnarMemoryStorePersister(parsedStreamingCubeInfo, segmentName); + this.autoMergeEnabled = cubeInstance.getConfig().isStreamingFragmentsAutoMergeEnabled(); + try { + StreamingMetrics + .getInstance() + .getMetrics() + .register(MetricRegistry.name("streaming.inMem.row.cnt", cubeInstance.getName(), segmentName), + new Gauge() { + @Override + public Integer getValue() { + return activeMemoryStore.getRowCount(); + } + }); + } catch (Exception e) { + logger.warn("metrics register failed", e); + } + } + + @Override + public void init() { + fragments.addAll(getFragmentsFromFileSystem()); + } + + public int addEvent(StreamingMessage event) { + if (activeMemoryStore == null) { + throw new IllegalStateException("the segment has not opened:" + segmentName); + } + int rowsIndexed = activeMemoryStore.index(event); + if (rowsIndexed >= maxRowsInMemory) { + persist(); + } + return rowsIndexed; + } + + @Override + public File getStorePath() { + return dataSegmentFolder; + } + + @Override + public Object checkpoint() { + persist(); + latestCheckpointFragment = getLargestFragmentID(); + return String.valueOf(latestCheckpointFragment); + } + + @Override + public void persist() { + if (activeMemoryStore.getRowCount() <= 0) { + logger.info("no data in the memory store, skip persist."); + return; + } + DataSegmentFragment newFragment; + persistWriteLock.lock(); + try { + persisting = true; + newFragment = createNewFragment(); + persistingMemoryStore = activeMemoryStore; + activeMemoryStore = new SegmentMemoryStore(parsedStreamingCubeInfo, segmentName); + } finally { + persistWriteLock.unlock(); + } + + memoryStorePersister.persist(persistingMemoryStore, newFragment); + + persistWriteLock.lock(); + try { + persistingMemoryStore = null; + fragments.add(newFragment); + persisting = false; + } finally { + persistWriteLock.unlock(); + } + checkRequireMerge(); + } + + private void checkRequireMerge() { + if (!autoMergeEnabled || inMerging) { + return; + } + KylinConfig config = parsedStreamingCubeInfo.cubeDesc.getConfig(); + int maxFragmentNum = config.getStreamingMaxFragmentsInSegment(); + if (fragments.size() <= maxFragmentNum) { + return; + } + final List fragmentsToMerge = chooseFragmentsToMerge(config, Lists.newArrayList(fragments)); + if (fragmentsToMerge.size() <= 1) { + return; + } + logger.info("found some fragments need to merge:{}", fragmentsToMerge); + inMerging = true; + fragmentMergeExecutor.submit(new Runnable() { + @Override + public void run() { + try { + doMergeFragments(fragmentsToMerge); + } catch (Exception e) { + logger.error("error happens when merge fragments:" + fragmentsToMerge, e); + } + } + }); + } + + protected void doMergeFragments(final List fragmentsToMerge) throws IOException { + logger.info("start to merge fragments:{}", fragmentsToMerge); + FragmentFilesMerger fragmentsMerger = new FragmentFilesMerger(parsedStreamingCubeInfo, dataSegmentFolder); + FragmentsMergeResult mergeResult = fragmentsMerger.merge(fragmentsToMerge); + logger.info("finish to merge fragments, try to commit the merge result"); + commitFragmentsMerge(mergeResult); + fragmentsMerger.cleanMergeDirectory(); + inMerging = false; + } + + protected List chooseFragmentsToMerge(KylinConfig config, + List allFragments) { + Collections.sort(allFragments); + List result = doChooseFragments(config, allFragments, true); + // if (result.size() > 1) { + // return result; + // } else { + // logger.info("recheck existing merged fragments"); + // result = doChooseFragments(config, allFragments, false); + // } + return result; + } + + protected List doChooseFragments(KylinConfig config, List allFragments, + boolean ignoreMergedFragments) { + List result = Lists.newArrayList(); + int originFragmentsNum = allFragments.size(); + int minFragments = config.getStreamingMinFragmentsInSegment(); + long maxFragmentSize = config.getStreamingMaxFragmentSizeInMb() * 1024 * 1024; + long toMergeDataSize = 0; + for (int i = 0; i < originFragmentsNum; i++) { + DataSegmentFragment fragment = allFragments.get(i); + if (originFragmentsNum - result.size() <= minFragments - 1) { + return result; + } + if (fragment.getFragmentId().getEndId() > latestCheckpointFragment) { + return result; + } + if (ignoreMergedFragments && fragment.isMergedFragment()) { + if (result.size() > 1) { + return result; + } else if (result.size() == 1) { + toMergeDataSize = 0; + result.clear(); + } + continue; + } + long fragmentDataSize = fragment.getDataFileSize(); + if (toMergeDataSize + fragmentDataSize <= maxFragmentSize) { + toMergeDataSize += fragmentDataSize; + result.add(fragment); + } else if (result.size() > 1) { + return result; + } else if (result.size() == 1) { + toMergeDataSize = 0; + result.clear(); + } + } + return result; + } + + private void commitFragmentsMerge(FragmentsMergeResult mergeResult) throws IOException { + mergeWriteLock.lock(); + try { + mergeResult.getOrigFragments(); + removeFragments(mergeResult.getOrigFragments()); + DataSegmentFragment fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, + mergeResult.getMergedFragmentId()); + FileUtils.moveFileToDirectory(mergeResult.getMergedFragmentDataFile(), fragment.getFragmentFolder(), true); + FileUtils.moveFileToDirectory(mergeResult.getMergedFragmentMetaFile(), fragment.getFragmentFolder(), true); + fragments.add(fragment); + } finally { + mergeWriteLock.unlock(); + } + } + + @Override + public void purge() { + try { + FileUtils.deleteDirectory(dataSegmentFolder); + logger.info("removed segment data, cube-{} segment-{}", cubeName, segmentName); + ColumnarStoreCache.getInstance().removeFragmentsCache(fragments); + fragments = Lists.newCopyOnWriteArrayList(); + logger.info("removed segment cache, cube-{} segment-{}", cubeName, segmentName); + } catch (IOException e) { + logger.error("error happens when purge segment", e); + } + } + + @Override + public void restoreFromCheckpoint(Object checkpoint) { + String checkpointFragmentIDString = (String) checkpoint; + FragmentId checkpointFragmentID = FragmentId.parse(checkpointFragmentIDString); + List fragments = getFragmentsFromFileSystem(); + for (DataSegmentFragment fragment : fragments) { + if (fragment.getFragmentId().compareTo(checkpointFragmentID) > 0) { + fragment.purge(); + } + } + } + + @Override + public String getSegmentName() { + return segmentName; + } + + @Override + public StreamingCubeSegment.State getSegmentState() { + File stateFile = new File(dataSegmentFolder, STATE_FILE); + if (stateFile.exists()) { + StreamingCubeSegment.State state = parseStateFile(stateFile); + return state; + } + return StreamingCubeSegment.State.ACTIVE; + } + + @Override + public void setSegmentState(StreamingCubeSegment.State state) { + File stateFile = new File(dataSegmentFolder, STATE_FILE); + FileOutputStream outPut = null; + try { + if (!stateFile.exists()) { + stateFile.createNewFile(); + } + outPut = new FileOutputStream(stateFile); + outPut.write(Bytes.toBytes(state.name())); + outPut.flush(); + } catch (IOException e) { + throw new IllegalStorageException(e); + } finally { + if (outPut != null) { + try { + outPut.close(); + } catch (IOException e) { + logger.error("error when close", e); + } + } + } + } + + public DataSegmentFragment createNewFragment() { + int largestFragID = getLargestFragmentID(); + DataSegmentFragment newFragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, new FragmentId( + ++largestFragID)); + return newFragment; + } + + public List getAllFragments() { + return fragments; + } + + private void removeFragments(List fragmentsToRemove) { + fragments.removeAll(Sets.newHashSet(fragmentsToRemove)); + for (DataSegmentFragment fragment : fragmentsToRemove) { + ColumnarStoreCache.getInstance().removeFragmentCache(fragment); + fragment.purge(); + } + } + + private int getLargestFragmentID() { + List existingFragments = getFragmentsFromFileSystem(); + int largestFragId = 0; + for (DataSegmentFragment existingFragment : existingFragments) { + int id = existingFragment.getFragmentId().getEndId(); + if (id > largestFragId) { + largestFragId = id; + } + } + return largestFragId; + } + + private List getFragmentsFromFileSystem() { + List fragments = Lists.newArrayList(); + File dataSegmentFolder = getStorePath(); + File[] fragmentFolders = dataSegmentFolder.listFiles(new FileFilter() { + @Override + public boolean accept(File file) { + if (!file.isDirectory()) { + return false; + } + if (file.getName().equalsIgnoreCase("_SUCCESS")) { + return false; + } + try { + FragmentId.parse(file.getName()); + } catch (Exception e) { + return false; + } + return true; + } + }); + if (fragmentFolders != null) { + for (File fragmentFolder : fragmentFolders) { + fragments.add(new DataSegmentFragment(baseStorePath, cubeName, segmentName, FragmentId + .parse(fragmentFolder.getName()))); + } + } + return fragments; + } + + @Override + public SegmentStoreStats getStoreStats() { + SegmentStoreStats storeStats = new SegmentStoreStats(); + storeStats.setNumRowsInMem(activeMemoryStore.getRowCount()); + storeStats.setNumFragments(getFragmentsFromFileSystem().size()); + return storeStats; + } + + public SegmentMemoryStore getActiveMemoryStore() { + return activeMemoryStore; + } + + @Override + public void search(final StreamingSearchContext searchContext, ResultCollector collector) throws IOException { + SegmentMemoryStore searchMemoryStore; + List searchFragments; + mergeReadLock.lock(); + collector.addCloseListener(new CloseListener() { + @Override + public void onClose() { + mergeReadLock.unlock(); + } + }); + persistReadLock.lock(); + try { + searchFragments = getAllFragments(); + if (persisting) { + searchMemoryStore = persistingMemoryStore; + } else { + searchMemoryStore = activeMemoryStore; + } + } finally { + persistReadLock.unlock(); + } + new ColumnarSegmentStoreFilesSearcher(segmentName, searchFragments).search(searchContext, collector); + searchMemoryStore.search(searchContext, collector); + } + + public void close() throws IOException { + logger.warn("closing the streaming cube segment, cube {}, segment {}.", cubeName, segmentName); + StreamingMetrics.getInstance().getMetrics() + .remove(MetricRegistry.name("streaming.inMem.row.cnt", cubeName, segmentName)); + } + + private StreamingCubeSegment.State parseStateFile(File stateFile) { + StreamingCubeSegment.State result = StreamingCubeSegment.State.ACTIVE; + try { + String stateName = Files.toString(stateFile, Charsets.UTF_8); + result = StreamingCubeSegment.State.valueOf(stateName.trim()); + } catch (IOException e) { + logger.error("error when parse state file", e); + } + return result; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreFilesSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreFilesSearcher.java new file mode 100644 index 00000000000..308c9cb3e1b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreFilesSearcher.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Locale; + +import org.apache.kylin.stream.core.query.IStreamingGTSearcher; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.ResultCollector.CloseListener; +import org.apache.kylin.stream.core.query.StreamingDataQueryPlanner; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("restriction") +public class ColumnarSegmentStoreFilesSearcher implements IStreamingGTSearcher { + private static Logger logger = LoggerFactory.getLogger(ColumnarSegmentStoreFilesSearcher.class); + + private String segmentName; + private List fragments; + private StreamingQueryProfile queryProfile; + + public ColumnarSegmentStoreFilesSearcher(String segmentName, List fragments) { + this.segmentName = segmentName; + this.fragments = fragments; + this.queryProfile = StreamingQueryProfile.get(); + } + + @Override + public void search(final StreamingSearchContext searchContext, ResultCollector collector) throws IOException { + logger.info("query-{}: scan segment {}, fragment files num:{}", queryProfile.getQueryId(), + segmentName, fragments.size()); + for (DataSegmentFragment fragment : fragments) { + File metaFile = fragment.getMetaFile(); + if (!metaFile.exists()) { + if (queryProfile.isDetailProfileEnable()) { + logger.info("query-{}: for segment {} skip fragment {}, no meta file exists", + queryProfile.getQueryId(), segmentName, fragment.getFragmentId()); + } + continue; + } + + FragmentData fragmentData = loadFragmentData(fragment); + FragmentMetaInfo fragmentMetaInfo = fragmentData.getFragmentMetaInfo(); + StreamingDataQueryPlanner queryPlanner = searchContext.getQueryPlanner(); + if (fragmentMetaInfo.hasValidEventTimeRange() + && queryPlanner.canSkip(fragmentMetaInfo.getMinEventTime(), fragmentMetaInfo.getMaxEventTime())) { + continue; + } + queryProfile.incScanFile(fragmentData.getSize()); + + FragmentFileSearcher fragmentFileSearcher = new FragmentFileSearcher(fragment, fragmentData); + fragmentFileSearcher.search(searchContext, collector); + } + + collector.addCloseListener(new CloseListener() { + @Override + public void onClose() { + for (DataSegmentFragment fragment : fragments) { + ColumnarStoreCache.getInstance().finishReadFragmentData(fragment); + } + } + }); + } + + private FragmentData loadFragmentData(DataSegmentFragment fragment) throws IOException { + if (queryProfile.isDetailProfileEnable()) { + queryProfile.startStep(getLoadFragmentDataStep(fragment)); + } + FragmentData fragmentData = ColumnarStoreCache.getInstance().startReadFragmentData(fragment); + if (queryProfile.isDetailProfileEnable()) { + queryProfile.finishStep(getLoadFragmentDataStep(fragment)); + } + return fragmentData; + } + + private String getLoadFragmentDataStep(DataSegmentFragment fragment) { + return String.format(Locale.ROOT, "segment-%s_fragment-%s_load_data", segmentName, + fragment.getFragmentId()); + } + + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCache.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCache.java new file mode 100644 index 00000000000..44f40addff6 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCache.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.CacheStats; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; +import com.google.common.collect.Maps; + +public class ColumnarStoreCache { + private static Logger logger = LoggerFactory.getLogger(ColumnarStoreCache.class); + private static ColumnarStoreCache instance = new ColumnarStoreCache(); + + private static final int INIT_CACHE_SIZE = 100; + private static final int CACHE_SIZE = 10000; + + // temporary set it to Long.MAX_VALUE, just leave os to manage the buffer + private static final long MAX_BUFFERED_SIZE = Long.MAX_VALUE; + private AtomicLong currentBufferedSize = new AtomicLong(0); + + private ConcurrentMap refCounters = Maps.newConcurrentMap(); + public LoadingCache fragmentDataCache = CacheBuilder.newBuilder() + .initialCapacity(INIT_CACHE_SIZE).concurrencyLevel(8).maximumSize(CACHE_SIZE) + .expireAfterAccess(6, TimeUnit.HOURS) + .removalListener(new RemovalListener() { + @Override + public void onRemoval(RemovalNotification notification) { + DataSegmentFragment fragment = notification.getKey(); + logger.debug("Data fragment " + fragment + " is unloaded from Cache due to " + + notification.getCause()); + FragmentData fragmentData = notification.getValue(); + AtomicLong refCounter = refCounters.get(fragment); + if (refCounter != null) { + synchronized (refCounter) { + if (refCounter.get() <= 0) { + int bufferSize = fragmentData.getBufferCapacity(); + currentBufferedSize.addAndGet(-bufferSize); + fragmentData.tryForceUnMapBuffer(); + refCounters.remove(fragment); + } else { + logger.debug("Fragment mapped buffer " + fragment + + " cannot be cleaned, because it has reference " + refCounter.get()); + } + } + } else { + logger.debug("no ref counter found for fragment: " + fragment); + } + } + }).build(new CacheLoader() { + @Override + public FragmentData load(DataSegmentFragment fragment) throws Exception { + if (currentBufferedSize.get() >= MAX_BUFFERED_SIZE) { + synchronized (fragmentDataCache) { + if (currentBufferedSize.get() >= MAX_BUFFERED_SIZE) { + long entrySize = fragmentDataCache.size(); + logger.debug("Max buffer size exceeds {}, invalidate half of the cache, cacheSize {}", + currentBufferedSize, entrySize); + long removed = 0; + for (DataSegmentFragment frag : fragmentDataCache.asMap().keySet()) { + if (removed >= entrySize / 2) { + break; + } + fragmentDataCache.invalidate(frag); + removed++; + } + } + } + } + FragmentMetaInfo fragmentMetaInfo = fragment.getMetaInfo(); + if (fragmentMetaInfo == null) { + throw new IllegalStateException("no metadata file exists for fragment:" + fragment); + } + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, fragment.getDataFile()); + int bufferSize = fragmentData.getBufferCapacity(); + currentBufferedSize.addAndGet(bufferSize); + logger.debug("Data fragment {} cached, bufferSize {}, totalBufferSize {}", fragment, bufferSize, + currentBufferedSize.get()); + return fragmentData; + } + }); + + public static ColumnarStoreCache getInstance() { + return instance; + } + + public FragmentData startReadFragmentData(DataSegmentFragment fragment) throws IOException { + try { + AtomicLong refCounter = refCounters.putIfAbsent(fragment, new AtomicLong(1)); + if (refCounter != null) { + //Additional synchronize is required for reference count check. + synchronized (refCounter) { + refCounter.incrementAndGet(); + } + } + return fragmentDataCache.get(fragment); + } catch (ExecutionException e) { + throw new IOException(e); + } + } + + public void finishReadFragmentData(DataSegmentFragment fragment) { + AtomicLong refCounter = refCounters.get(fragment); + if (refCounter != null) { + refCounter.decrementAndGet(); + } else { + logger.warn("ref counter not exist for fragment:" + fragment); + } + } + + public ColumnarStoreCacheStats getCacheStats() { + ColumnarStoreCacheStats stats = new ColumnarStoreCacheStats(); + CacheStats cacheStats = fragmentDataCache.stats(); + + stats.setHitCount(cacheStats.hitCount()); + stats.setMissCount(cacheStats.missCount()); + stats.setEvictionCount(cacheStats.evictionCount()); + stats.setLoadSuccessCount(cacheStats.loadSuccessCount()); + stats.setLoadExceptionCount(cacheStats.loadExceptionCount()); + stats.setTotalLoadTime(cacheStats.totalLoadTime()); + + stats.setCacheEntriesNum(fragmentDataCache.size()); + stats.setCachedDataBufferSize(currentBufferedSize.get()); + return stats; + } + + public void removeFragmentsCache(List fragmentList) { + if (fragmentList == null) { + return; + } + for (DataSegmentFragment fragment : fragmentList) { + fragmentDataCache.invalidate(fragment); + } + } + + public void removeFragmentCache(DataSegmentFragment fragment) { + if (fragment == null) { + return; + } + fragmentDataCache.invalidate(fragment); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCacheStats.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCacheStats.java new file mode 100644 index 00000000000..c9bffba3951 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreCacheStats.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +public class ColumnarStoreCacheStats { + private long hitCount; + private long missCount; + private long loadSuccessCount; + private long loadExceptionCount; + private long totalLoadTime; + private long evictionCount; + + private long cacheEntriesNum; + private long cachedDataBufferSize; + + public long getHitCount() { + return hitCount; + } + + public void setHitCount(long hitCount) { + this.hitCount = hitCount; + } + + public long getMissCount() { + return missCount; + } + + public void setMissCount(long missCount) { + this.missCount = missCount; + } + + public long getLoadSuccessCount() { + return loadSuccessCount; + } + + public void setLoadSuccessCount(long loadSuccessCount) { + this.loadSuccessCount = loadSuccessCount; + } + + public long getLoadExceptionCount() { + return loadExceptionCount; + } + + public void setLoadExceptionCount(long loadExceptionCount) { + this.loadExceptionCount = loadExceptionCount; + } + + public long getTotalLoadTime() { + return totalLoadTime; + } + + public void setTotalLoadTime(long totalLoadTime) { + this.totalLoadTime = totalLoadTime; + } + + public long getEvictionCount() { + return evictionCount; + } + + public void setEvictionCount(long evictionCount) { + this.evictionCount = evictionCount; + } + + public long getCacheEntriesNum() { + return cacheEntriesNum; + } + + public void setCacheEntriesNum(long cacheEntriesNum) { + this.cacheEntriesNum = cacheEntriesNum; + } + + public long getCachedDataBufferSize() { + return cachedDataBufferSize; + } + + public void setCachedDataBufferSize(long cachedDataBufferSize) { + this.cachedDataBufferSize = cachedDataBufferSize; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreDimDesc.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreDimDesc.java new file mode 100644 index 00000000000..b3d1f80f680 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreDimDesc.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.stream.core.storage.columnar.compress.Compression; +import org.apache.kylin.stream.core.storage.columnar.compress.FSInputLZ4CompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.FSInputNoCompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.FSInputRLECompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.LZ4CompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.LZ4CompressedColumnWriter; +import org.apache.kylin.stream.core.storage.columnar.compress.NoCompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.NoCompressedColumnWriter; +import org.apache.kylin.stream.core.storage.columnar.compress.RunLengthCompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.RunLengthCompressedColumnWriter; +import org.apache.kylin.stream.core.util.TimeDerivedColumnType; + +public class ColumnarStoreDimDesc { + private int fixLen; + private Compression compression; + + public static ColumnarStoreDimDesc getDefaultCStoreDimDesc(CubeDesc cubeDesc, String dimName, + DimensionEncoding encoding) { + // for time dimension and the dimension at the first using rle compression + if (TimeDerivedColumnType.isTimeDerivedColumn(dimName)) { + return new ColumnarStoreDimDesc(encoding.getLengthOfEncoding(), Compression.RUN_LENGTH); + } + if (cubeDesc.getRowkey().getRowKeyColumns()[0].getColumn().equals(dimName)) { + return new ColumnarStoreDimDesc(encoding.getLengthOfEncoding(), Compression.RUN_LENGTH); + } + return new ColumnarStoreDimDesc(encoding.getLengthOfEncoding(), Compression.LZ4); + } + + public ColumnarStoreDimDesc(int fixLen, Compression compression) { + this.fixLen = fixLen; + this.compression = compression; + } + + public Compression getCompression() { + return compression; + } + + public ColumnDataWriter getDimWriter(OutputStream output, int rowCnt) { + if (compression == Compression.LZ4) { + return new LZ4CompressedColumnWriter(fixLen, rowCnt, LZ4CompressedColumnWriter.DEF_BLOCK_SIZE, output); + } else if (compression == Compression.RUN_LENGTH) { + return new RunLengthCompressedColumnWriter(fixLen, rowCnt, LZ4CompressedColumnWriter.DEF_BLOCK_SIZE, output); + } + return new NoCompressedColumnWriter(output); + } + + public ColumnDataReader getDimReader(ByteBuffer dataBuffer, int columnDataStartOffset, int columnDataLength, + int rowCount) { + if (compression == Compression.LZ4) { + return new LZ4CompressedColumnReader(dataBuffer, columnDataStartOffset, columnDataLength, rowCount); + } else if (compression == Compression.RUN_LENGTH) { + return new RunLengthCompressedColumnReader(dataBuffer, columnDataStartOffset, columnDataLength, rowCount); + } + return new NoCompressedColumnReader(dataBuffer, columnDataStartOffset, columnDataLength, rowCount); + } + + public ColumnDataReader getDimReaderFromFSInput(FSDataInputStream inputStream, int columnDataStartOffset, + int columnDataLength, int rowCount) throws IOException { + if (compression == Compression.LZ4) { + return new FSInputLZ4CompressedColumnReader(inputStream, columnDataStartOffset, columnDataLength, rowCount); + } else if (compression == Compression.RUN_LENGTH) { + return new FSInputRLECompressedColumnReader(inputStream, columnDataStartOffset, columnDataLength, rowCount); + } + return new FSInputNoCompressedColumnReader(inputStream, columnDataStartOffset, columnDataLength, rowCount); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreMetricsDesc.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreMetricsDesc.java new file mode 100644 index 00000000000..098134d1eec --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ColumnarStoreMetricsDesc.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.stream.core.storage.columnar.compress.Compression; +import org.apache.kylin.stream.core.storage.columnar.compress.FSInputLZ4CompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.FSInputNoCompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.LZ4CompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.LZ4CompressedColumnWriter; +import org.apache.kylin.stream.core.storage.columnar.compress.NoCompressedColumnReader; +import org.apache.kylin.stream.core.storage.columnar.compress.NoCompressedColumnWriter; + +public class ColumnarStoreMetricsDesc { + private int fixLen = -1; + private Compression compression; + + public static ColumnarStoreMetricsDesc getDefaultCStoreMetricsDesc(ColumnarMetricsEncoding metricsEncoding) { + return new ColumnarStoreMetricsDesc(metricsEncoding, Compression.LZ4); + } + + public ColumnarStoreMetricsDesc(ColumnarMetricsEncoding metricsEncoding, Compression compression) { + if (metricsEncoding.isFixLength()) { + this.fixLen = metricsEncoding.getFixLength(); + } + this.compression = compression; + } + + public Compression getCompression() { + return compression; + } + + public ColumnDataWriter getMetricsWriter(OutputStream output, int rowCnt) { + if (compression == Compression.LZ4 && fixLen != -1) { + return new LZ4CompressedColumnWriter(fixLen, rowCnt, LZ4CompressedColumnWriter.DEF_BLOCK_SIZE, output); + } + + if (fixLen != -1) { + return new NoCompressedColumnWriter(output); + } + return new GeneralColumnDataWriter(rowCnt, new DataOutputStream(output)); + } + + public ColumnDataReader getMetricsReader(ByteBuffer dataBuffer, int columnDataStartOffset, int columnDataLength, + int rowCount) { + if (Compression.LZ4 == compression && fixLen != -1) { + return new LZ4CompressedColumnReader(dataBuffer, columnDataStartOffset, columnDataLength, rowCount); + } + if (fixLen != -1) { + return new NoCompressedColumnReader(dataBuffer, columnDataStartOffset, columnDataLength, rowCount); + } + return new GeneralColumnDataReader(dataBuffer, columnDataStartOffset, columnDataLength); + } + + public ColumnDataReader getMetricsReaderFromFSInput(FSDataInputStream inputStream, int columnDataStartOffset, + int columnDataLength, int rowCount) throws IOException { + if (Compression.LZ4 == compression && fixLen != -1) { + return new FSInputLZ4CompressedColumnReader(inputStream, columnDataStartOffset, columnDataLength, rowCount); + } + if (fixLen != -1) { + return new FSInputNoCompressedColumnReader(inputStream, columnDataStartOffset, columnDataLength, rowCount); + } + return new FSInputGeneralColumnDataReader(inputStream, columnDataStartOffset, columnDataLength); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/DataSegmentFragment.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/DataSegmentFragment.java new file mode 100644 index 00000000000..5fcbdbb5a7d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/DataSegmentFragment.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.exception.IllegalStorageException; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.kylin.stream.core.util.Constants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DataSegmentFragment implements Comparable { + private static Logger logger = LoggerFactory.getLogger(DataSegmentFragment.class); + private String baseStorePath; + private String cubeName; + private String segmentName; + private FragmentId fragmentId; + + public DataSegmentFragment(String baseStorePath, String cubeName, String segmentName, FragmentId fragmentId) { + this.baseStorePath = baseStorePath; + this.cubeName = cubeName; + this.segmentName = segmentName; + this.fragmentId = fragmentId; + createIfNotExist(); + } + + public void createIfNotExist() { + File fragmentFolder = getFragmentFolder(); + if (!fragmentFolder.exists()) { + fragmentFolder.mkdirs(); + } + } + + public FragmentId getFragmentId() { + return fragmentId; + } + + public boolean isMergedFragment() { + return fragmentId.getEndId() != fragmentId.getStartId(); + } + + public void purge() { + File fragmentFolder = getFragmentFolder(); + try { + FileUtils.deleteDirectory(fragmentFolder); + } catch (IOException e) { + logger.error("error happens when purge fragment", e); + } + } + + public File getDataFile() { + File parentFolder = getFragmentFolder(); + File dataFile = new File(parentFolder, fragmentId + Constants.DATA_FILE_SUFFIX); + if (!dataFile.exists()) { + try { + dataFile.createNewFile(); + } catch (IOException e) { + throw new IllegalStorageException(e); + } + } + return dataFile; + } + + public long getDataFileSize() { + return getDataFile().length(); + } + + public File getFragmentFolder() { + return new File(baseStorePath + File.separator + cubeName + File.separator + segmentName + File.separator + + fragmentId.toString()); + } + + public File getMetaFile() { + File parentFolder = getFragmentFolder(); + File metaFile = new File(parentFolder, fragmentId + Constants.META_FILE_SUFFIX); + return metaFile; + } + + public FragmentMetaInfo getMetaInfo() { + File metaFile = getMetaFile(); + if (metaFile == null || !metaFile.exists()) { + return null; + } + try { + return JsonUtil.readValue(metaFile, FragmentMetaInfo.class); + } catch (IOException e) { + throw new StreamingException("error when parse meta file"); + } + } + + public String getSegmentName() { + return segmentName; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + DataSegmentFragment that = (DataSegmentFragment) o; + + if (cubeName != null ? !cubeName.equals(that.cubeName) : that.cubeName != null) + return false; + if (segmentName != null ? !segmentName.equals(that.segmentName) : that.segmentName != null) + return false; + return fragmentId != null ? fragmentId.equals(that.fragmentId) : that.fragmentId == null; + + } + + @Override + public int hashCode() { + int result = cubeName != null ? cubeName.hashCode() : 0; + result = 31 * result + (segmentName != null ? segmentName.hashCode() : 0); + result = 31 * result + (fragmentId != null ? fragmentId.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "{" + "cubeName='" + cubeName + '\'' + ", segmentName='" + segmentName + '\'' + ", fragmentId=" + + fragmentId + '}'; + } + + @Override + public int compareTo(DataSegmentFragment o) { + if (!cubeName.equals(o.cubeName)) { + return cubeName.compareTo(o.cubeName); + } + if (!segmentName.equals(o.segmentName)) { + return segmentName.compareTo(o.segmentName); + } + + return fragmentId.compareTo(o.fragmentId); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java new file mode 100644 index 00000000000..bbe06ae97c4 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.fs.FSDataInputStream; + +public class FSInputGeneralColumnDataReader implements ColumnDataReader { + private FSDataInputStream fsInputStream; + private int numOfVals; + + public FSInputGeneralColumnDataReader(FSDataInputStream fsInputStream, int dataStartOffset, int dataLength) + throws IOException { + this.fsInputStream = fsInputStream; + fsInputStream.seek(dataStartOffset + dataLength - 4); + this.numOfVals = fsInputStream.readInt(); + fsInputStream.seek(dataStartOffset); + } + + @Override + public byte[] read(int index) { + throw new UnsupportedOperationException("not support to read row operation"); + } + + @Override + public Iterator iterator() { + return new Iterator() { + int readRowCount = 0; + + @Override + public boolean hasNext() { + return readRowCount < numOfVals; + } + + @Override + public byte[] next() { + try { + int size = fsInputStream.readInt(); + byte[] result = new byte[size]; + fsInputStream.readFully(result); + readRowCount++; + return result; + } catch (IOException e) { + throw new RuntimeException("error when read data", e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("unSupport operation"); + } + }; + } + + @Override + public void close() throws IOException { + fsInputStream.close(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReader.java new file mode 100644 index 00000000000..61053f476be --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReader.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimensionMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.MetricMetaInfo; + +import com.google.common.collect.Maps; + +public class FragmentCuboidReader implements Iterable { + private long rowCount; + private long readRowCount = 0; + private int dimCnt; + private int metricCnt; + + private CubeDesc cubeDesc; + private ColumnDataReader[] dimensionDataReaders; + private ColumnDataReader[] metricDataReaders; + + public FragmentCuboidReader(CubeDesc cubeDesc, FragmentData fragmentData, CuboidMetaInfo cuboidMetaInfo, + TblColRef[] dimensions, MeasureDesc[] measures, DimensionEncoding[] dimEncodings) { + this.cubeDesc = cubeDesc; + this.dimCnt = dimensions.length; + this.metricCnt = measures.length; + this.dimensionDataReaders = new ColumnDataReader[dimCnt]; + this.metricDataReaders = new ColumnDataReader[metricCnt]; + this.rowCount = cuboidMetaInfo.getNumberOfRows(); + Map dimensionMetaInfoMap = getDimensionMetaMap(cuboidMetaInfo); + Map metricMetaInfoMap = getMetricMetaMap(cuboidMetaInfo); + int i = 0; + for (TblColRef dimension : dimensions) { + DimensionMetaInfo dimensionMetaInfo = dimensionMetaInfoMap.get(dimension.getName()); + ColumnDataReader dimensionDataReader = getDimensionDataReader(dimensionMetaInfo, + (int) cuboidMetaInfo.getNumberOfRows(), dimension, dimEncodings[i], + fragmentData.getDataReadBuffer()); + dimensionDataReaders[i] = dimensionDataReader; + i++; + } + + i = 0; + for (MeasureDesc measure : measures) { + MetricMetaInfo metricMetaInfo = metricMetaInfoMap.get(measure.getName()); + ColumnDataReader metricDataReader = getMetricsDataReader(measure, metricMetaInfo, + (int) cuboidMetaInfo.getNumberOfRows(), fragmentData.getDataReadBuffer()); + metricDataReaders[i] = metricDataReader; + i++; + } + } + + /** + * get the specified row in the cuboid data + * @param rowNum + * @return + */ + public RawRecord read(int rowNum) { + if (rowNum > rowCount - 1) { + throw new IllegalStateException("cannot read row:" + rowNum + ", total row cnt is:" + rowCount); + } + RawRecord rawRecord = new RawRecord(dimCnt, metricCnt); + for (int i = 0; i < dimCnt; i++) { + rawRecord.setDimension(i, dimensionDataReaders[i].read(rowNum)); + } + for (int i = 0; i < metricCnt; i++) { + rawRecord.setMetric(i, metricDataReaders[i].read(rowNum)); + } + readRowCount++; + return rawRecord; + } + + private Map getDimensionMetaMap(CuboidMetaInfo cuboidMetaInfo) { + Map result = Maps.newHashMap(); + List dimensionMetaInfoList = cuboidMetaInfo.getDimensionsInfo(); + for (DimensionMetaInfo dimensionMetaInfo : dimensionMetaInfoList) { + result.put(dimensionMetaInfo.getName(), dimensionMetaInfo); + } + return result; + } + + private Map getMetricMetaMap(CuboidMetaInfo cuboidMetaInfo) { + Map result = Maps.newHashMap(); + List metricMetaInfoList = cuboidMetaInfo.getMetricsInfo(); + for (MetricMetaInfo metricMetaInfo : metricMetaInfoList) { + result.put(metricMetaInfo.getName(), metricMetaInfo); + } + return result; + } + + @Override + public Iterator iterator() { + final RawRecord oneRawRecord = new RawRecord(dimCnt, metricCnt); + final Iterator[] dimValItr = new Iterator[dimensionDataReaders.length]; + for (int i = 0; i < dimensionDataReaders.length; i++) { + dimValItr[i] = dimensionDataReaders[i].iterator(); + } + + final Iterator[] metricsValItr = new Iterator[metricDataReaders.length]; + for (int i = 0; i < metricDataReaders.length; i++) { + metricsValItr[i] = metricDataReaders[i].iterator(); + } + return new Iterator() { + @Override + public boolean hasNext() { + if (readRowCount >= rowCount) { + return false; + } + return true; + } + + @Override + public RawRecord next() { + for (int i = 0; i < dimensionDataReaders.length; i++) { + oneRawRecord.setDimension(i, dimValItr[i].next()); + } + for (int i = 0; i < metricDataReaders.length; i++) { + oneRawRecord.setMetric(i, metricsValItr[i].next()); + } + readRowCount++; + return oneRawRecord; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("unSupported"); + } + }; + } + + public long getReadRowCount() { + return readRowCount; + } + + public ColumnDataReader getDimensionDataReader(DimensionMetaInfo dimensionMetaInfo, int numberOfRows, + TblColRef dimension, DimensionEncoding dimensionEncoding, ByteBuffer dataReadBuffer) { + ColumnarStoreDimDesc cStoreDimDesc = new ColumnarStoreDimDesc(dimensionEncoding.getLengthOfEncoding(), + dimensionMetaInfo.getCompressionType()); + + return cStoreDimDesc.getDimReader(dataReadBuffer, dimensionMetaInfo.getStartOffset(), + dimensionMetaInfo.getDataLength(), numberOfRows); + } + + public ColumnDataReader getMetricsDataReader(MeasureDesc measure, MetricMetaInfo metricMetaInfo, int numberOfRows, + ByteBuffer dataReadBuffer) { + DataType type = measure.getFunction().getReturnDataType(); + + ColumnarMetricsEncoding metricsEncoding = ColumnarMetricsEncodingFactory.create(type); + ColumnarStoreMetricsDesc cStoreMetricsDesc = new ColumnarStoreMetricsDesc(metricsEncoding, + metricMetaInfo.getCompressionType()); + return cStoreMetricsDesc.getMetricsReader(dataReadBuffer, metricMetaInfo.getStartOffset(), + metricMetaInfo.getMetricLength(), numberOfRows); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentData.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentData.java new file mode 100644 index 00000000000..0edf97e5bea --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentData.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; + +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.dict.DictionarySerializer; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimDictionaryMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import sun.nio.ch.DirectBuffer; + +import com.google.common.collect.Maps; +import com.google.common.io.Files; + +public class FragmentData { + private static Logger logger = LoggerFactory.getLogger(FragmentData.class); + + private ByteBuffer dataBuffer; + private FragmentMetaInfo fragmentMetaInfo; + private File fragmentDataFile; + + private ConcurrentMap> dictionaryMap; + + public FragmentData(FragmentMetaInfo fragmentMetaInfo, File fragmentDataFile) throws IOException { + this.dictionaryMap = Maps.newConcurrentMap(); + this.fragmentMetaInfo = fragmentMetaInfo; + this.fragmentDataFile = fragmentDataFile; + this.dataBuffer = Files.map(fragmentDataFile, FileChannel.MapMode.READ_ONLY); + } + + @SuppressWarnings("unchecked") + public Map> getDimensionDictionaries(TblColRef[] dimensions) { + Map> result = Maps.newHashMap(); + ByteBuffer dictBuffer = dataBuffer.asReadOnlyBuffer(); + for (TblColRef dimension : dimensions) { + Dictionary dict = dictionaryMap.get(dimension); + if (dict == null) { + for (DimDictionaryMetaInfo dimDictMetaInfo : fragmentMetaInfo.getDimDictionaryMetaInfos()) { + if (dimDictMetaInfo.getDimName().equals(dimension.getName())) { + int dictLength = dimDictMetaInfo.getDictLength(); + dictBuffer.position(dimDictMetaInfo.getStartOffset()); + byte[] dictBytes = new byte[dictLength]; + dictBuffer.get(dictBytes, 0, dictLength); + dict = (Dictionary) DictionarySerializer.deserialize(new ByteArray(dictBytes)); + dictionaryMap.put(dimension, dict); + break; + } + } + } + result.put(dimension, dict); + } + return result; + } + + public int getBufferCapacity() { + return dataBuffer.capacity(); + } + + public ByteBuffer getDataReadBuffer() { + return dataBuffer.asReadOnlyBuffer(); + } + + public FragmentMetaInfo getFragmentMetaInfo() { + return fragmentMetaInfo; + } + + public int getSize() { + return dataBuffer.capacity(); + } + + public void tryForceUnMapBuffer() { + if (dataBuffer instanceof DirectBuffer) { + try { + sun.misc.Cleaner cleaner = ((DirectBuffer) dataBuffer).cleaner(); + if (cleaner != null) { + cleaner.clean(); + logger.debug("directBuffer cleaned for fragment data:" + fragmentDataFile.getAbsolutePath()); + } + } catch (Exception e) { + logger.error("error when clean the fragment data:" + fragmentDataFile.getAbsolutePath()); + } + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcher.java new file mode 100644 index 00000000000..20b123af92d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcher.java @@ -0,0 +1,360 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.cube.kv.CubeDimEncMap; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.dict.BuiltInFunctionTransformer; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.dimension.IDimensionEncodingMap; +import org.apache.kylin.metadata.filter.ColumnTupleFilter; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.ConstantTupleFilter; +import org.apache.kylin.metadata.filter.FilterOptimizeTransformer; +import org.apache.kylin.metadata.filter.ITupleFilterTransformer; +import org.apache.kylin.metadata.filter.LogicalTupleFilter; +import org.apache.kylin.metadata.filter.StringCodeSystem; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum; +import org.apache.kylin.metadata.filter.TupleFilterSerializer; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.query.IStreamingGTSearcher; +import org.apache.kylin.stream.core.query.IStreamingSearchResult; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.kylin.stream.core.util.CompareFilterTimeRangeChecker; +import org.apache.kylin.stream.core.util.CompareFilterTimeRangeChecker.CheckResult; +import org.apache.kylin.stream.core.util.TimeDerivedColumnType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +/** + * FragmentFileSearcher is responsible to scan the columnar based storage on disk and deal with all the bytes level details for each DataFragment and return the result as GTRecords. + * + */ +public class FragmentFileSearcher implements IStreamingGTSearcher { + private static Logger logger = LoggerFactory.getLogger(FragmentFileSearcher.class); + + private FragmentData fragmentData; + private DataSegmentFragment fragment; + + public FragmentFileSearcher(DataSegmentFragment fragment, FragmentData fragmentData) { + this.fragment = fragment; + this.fragmentData = fragmentData; + } + + @Override + public void search(StreamingSearchContext searchContext, ResultCollector collector) throws IOException { + FragmentMetaInfo fragmentMetaInfo = fragmentData.getFragmentMetaInfo(); + CuboidMetaInfo cuboidMetaInfo; + if (searchContext.hitBasicCuboid()) { + cuboidMetaInfo = fragmentMetaInfo.getBasicCuboidMetaInfo(); + } else { + cuboidMetaInfo = fragmentMetaInfo.getCuboidMetaInfo(searchContext.getHitCuboid()); + if (cuboidMetaInfo == null) { + logger.warn("the cuboid:{} is not exist in the fragment:{}, use basic cuboid instead", + searchContext.getHitCuboid(), fragment.getFragmentId()); + cuboidMetaInfo = fragmentMetaInfo.getBasicCuboidMetaInfo(); + } + } + + ResponseResultSchema responseSchema = searchContext.getRespResultSchema(); + TblColRef[] dimensions = responseSchema.getDimensions(); + FunctionDesc[] metrics = responseSchema.getMetrics(); + Map> dictMap = fragmentData.getDimensionDictionaries(dimensions); + + CubeDesc cubeDesc = responseSchema.getCubeDesc(); + List allMeasures = cubeDesc.getMeasures(); + Map funcMeasureMap = Maps.newHashMap(); + for (MeasureDesc measure : allMeasures) { + funcMeasureMap.put(measure.getFunction(), measure); + } + MeasureDesc[] measures = new MeasureDesc[metrics.length]; + for (int i = 0; i < measures.length; i++) { + measures[i] = funcMeasureMap.get(metrics[i]); + } + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings(cubeDesc, dimensions, + dictMap); + ColumnarMetricsEncoding[] metricsEncodings = ParsedStreamingCubeInfo.getMetricsEncodings(measures); + ColumnarRecordCodec recordCodec = new ColumnarRecordCodec(dimensionEncodings, metricsEncodings); + + // change the unEvaluable dimensions to groupBy + Set unEvaluateDims = Sets.newHashSet(); + TupleFilter fragmentFilter = null; + if (searchContext.getFilter() != null) { + fragmentFilter = convertFilter(fragmentMetaInfo, searchContext.getFilter(), recordCodec, + dimensions, new CubeDimEncMap(cubeDesc, dictMap), unEvaluateDims); + } + if (ConstantTupleFilter.TRUE == fragmentFilter) { + fragmentFilter = null; + } else if (ConstantTupleFilter.FALSE == fragmentFilter) { + collector.collectSearchResult(IStreamingSearchResult.EMPTY_RESULT); + } + Set groups = searchContext.getGroups(); + if (!unEvaluateDims.isEmpty()) { + searchContext.addNewGroups(unEvaluateDims); + groups = Sets.union(groups, unEvaluateDims); + } + collector.collectSearchResult(new FragmentSearchResult(fragment, fragmentData, cuboidMetaInfo, responseSchema, fragmentFilter, groups, searchContext.getHavingFilter(), + recordCodec)); + } + + private TupleFilter convertFilter(FragmentMetaInfo fragmentMetaInfo, TupleFilter rootFilter, + ColumnarRecordCodec recordCodec, final TblColRef[] dimensions, final IDimensionEncodingMap dimEncodingMap, // + final Set unEvaluableColumnCollector) { + Map colMapping = Maps.newHashMap(); + for (int i = 0; i < dimensions.length; i++) { + colMapping.put(dimensions[i], i); + } + byte[] bytes = TupleFilterSerializer.serialize(rootFilter, null, StringCodeSystem.INSTANCE); + TupleFilter filter = TupleFilterSerializer.deserialize(bytes, StringCodeSystem.INSTANCE); + + BuiltInFunctionTransformer builtInFunctionTransformer = new BuiltInFunctionTransformer(dimEncodingMap); + filter = builtInFunctionTransformer.transform(filter); + FragmentFilterConverter fragmentFilterConverter = new FragmentFilterConverter(fragmentMetaInfo, unEvaluableColumnCollector, + colMapping, recordCodec); + filter = fragmentFilterConverter.transform(filter); + + filter = new FilterOptimizeTransformer().transform(filter); + return filter; + } + + protected static class FragmentFilterConverter implements ITupleFilterTransformer { + protected final Set unEvaluableColumnCollector; + protected final Map colMapping; + private CompareFilterTimeRangeChecker filterTimeRangeChecker; + private ColumnarRecordCodec recordCodec; + transient ByteBuffer buf; + + public FragmentFilterConverter(FragmentMetaInfo fragmentMetaInfo, Set unEvaluableColumnCollector, + Map colMapping, ColumnarRecordCodec recordCodec) { + this.unEvaluableColumnCollector = unEvaluableColumnCollector; + this.recordCodec = recordCodec; + this.colMapping = colMapping; + if (fragmentMetaInfo.hasValidEventTimeRange()) { + this.filterTimeRangeChecker = new CompareFilterTimeRangeChecker(fragmentMetaInfo.getMinEventTime(), + fragmentMetaInfo.getMaxEventTime(), true); + } + buf = ByteBuffer.allocate(recordCodec.getMaxDimLength()); + } + + protected int mapCol(TblColRef col) { + Integer i = colMapping.get(col); + return i == null ? -1 : i; + } + + @Override + public TupleFilter transform(TupleFilter filter) { + if (filter.getOperator() == TupleFilter.FilterOperatorEnum.NOT + && !TupleFilter.isEvaluableRecursively(filter)) { + TupleFilter.collectColumns(filter, unEvaluableColumnCollector); + return ConstantTupleFilter.TRUE; + } + + // shortcut for unEvaluatable filter + if (!filter.isEvaluable()) { + TupleFilter.collectColumns(filter, unEvaluableColumnCollector); + return ConstantTupleFilter.TRUE; + } + + if (filter instanceof CompareTupleFilter) { + return translateCompareFilter((CompareTupleFilter) filter); + } else if (filter instanceof LogicalTupleFilter) { + @SuppressWarnings("unchecked") + ListIterator childIterator = (ListIterator) filter.getChildren().listIterator(); + while (childIterator.hasNext()) { + TupleFilter transformed = transform(childIterator.next()); + if (transformed != null) { + childIterator.set(transformed); + } else { + throw new IllegalStateException("Should not be null"); + } + } + } + return filter; + } + + + @SuppressWarnings({ "rawtypes", "unchecked" }) + protected TupleFilter translateCompareFilter(CompareTupleFilter oldCompareFilter) { + // extract ColumnFilter & ConstantFilter + TblColRef externalCol = oldCompareFilter.getColumn(); + + if (externalCol == null) { + return oldCompareFilter; + } + + Collection constValues = oldCompareFilter.getValues(); + if (constValues == null || constValues.isEmpty()) { + return oldCompareFilter; + } + + if (TimeDerivedColumnType.isTimeDerivedColumn(externalCol.getName()) && filterTimeRangeChecker != null) { + CheckResult checkResult = filterTimeRangeChecker.check(oldCompareFilter, + TimeDerivedColumnType.getTimeDerivedColumnType(externalCol.getName())); + if (checkResult == CheckResult.INCLUDED) { + return ConstantTupleFilter.TRUE; + } else if (checkResult == CheckResult.EXCLUDED) { + return ConstantTupleFilter.FALSE; + } + } + + //CompareTupleFilter containing BuiltInFunctionTupleFilter will not reach here caz it will be transformed by BuiltInFunctionTransformer + CompareTupleFilter newCompareFilter = new CompareTupleFilter(oldCompareFilter.getOperator()); + newCompareFilter.addChild(new ColumnTupleFilter(externalCol)); + + //for CompareTupleFilter containing dynamicVariables, the below codes will actually replace dynamicVariables + //with normal ConstantTupleFilter + + Object firstValue = constValues.iterator().next(); + int col = mapCol(externalCol); + + TupleFilter result; + ByteArray code; + + // translate constant into code + switch (newCompareFilter.getOperator()) { + case EQ: + case IN: + Set newValues = Sets.newHashSet(); + for (Object value : constValues) { + code = translate(col, value, 0); + if (code != null) + newValues.add(code); + } + if (newValues.isEmpty()) { + result = ConstantTupleFilter.FALSE; + } else { + newCompareFilter.addChild(new ConstantTupleFilter(newValues)); + result = newCompareFilter; + } + break; + case NOTIN: + Set notInValues = Sets.newHashSet(); + for (Object value : constValues) { + code = translate(col, value, 0); + if (code != null) + notInValues.add(code); + } + if (notInValues.isEmpty()) { + result = ConstantTupleFilter.TRUE; + } else { + newCompareFilter.addChild(new ConstantTupleFilter(notInValues)); + result = newCompareFilter; + } + break; + case NEQ: + code = translate(col, firstValue, 0); + if (code == null) { + result = ConstantTupleFilter.TRUE; + } else { + newCompareFilter.addChild(new ConstantTupleFilter(code)); + result = newCompareFilter; + } + break; + case LT: + code = translate(col, firstValue, 0); + if (code == null) { + code = translate(col, firstValue, -1); + if (code == null) + result = ConstantTupleFilter.FALSE; + else + result = newCompareFilter(FilterOperatorEnum.LTE, externalCol, code); + } else { + newCompareFilter.addChild(new ConstantTupleFilter(code)); + result = newCompareFilter; + } + break; + case LTE: + code = translate(col, firstValue, -1); + if (code == null) { + result = ConstantTupleFilter.FALSE; + } else { + newCompareFilter.addChild(new ConstantTupleFilter(code)); + result = newCompareFilter; + } + break; + case GT: + code = translate(col, firstValue, 0); + if (code == null) { + code = translate(col, firstValue, 1); + if (code == null) + result = ConstantTupleFilter.FALSE; + else + result = newCompareFilter(FilterOperatorEnum.GTE, externalCol, code); + } else { + newCompareFilter.addChild(new ConstantTupleFilter(code)); + result = newCompareFilter; + } + break; + case GTE: + code = translate(col, firstValue, 1); + if (code == null) { + result = ConstantTupleFilter.FALSE; + } else { + newCompareFilter.addChild(new ConstantTupleFilter(code)); + result = newCompareFilter; + } + break; + default: + throw new IllegalStateException("Cannot handle operator " + newCompareFilter.getOperator()); + } + return result; + } + + private TupleFilter newCompareFilter(FilterOperatorEnum op, TblColRef col, ByteArray code) { + CompareTupleFilter r = new CompareTupleFilter(op); + r.addChild(new ColumnTupleFilter(col)); + r.addChild(new ConstantTupleFilter(code)); + return r; + } + + protected ByteArray translate(int col, Object value, int roundingFlag) { + try { + buf.clear(); + recordCodec.encodeDimension(col, value, roundingFlag, buf); + int length = buf.position(); + return ByteArray.copyOf(buf.array(), 0, length); + } catch (IllegalArgumentException ex) { + return null; + } + } + + + } + +} \ No newline at end of file diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMerger.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMerger.java new file mode 100644 index 00000000000..c35bfe9a479 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMerger.java @@ -0,0 +1,624 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.PriorityQueue; + +import javax.annotation.Nullable; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.dict.DictionaryGenerator; +import org.apache.kylin.dict.DictionarySerializer; +import org.apache.kylin.dict.MultipleDictionaryValueEnumerator; +import org.apache.kylin.dict.TrieDictionary; +import org.apache.kylin.dimension.DictionaryDimEnc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.measure.MeasureAggregators; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.columnar.ParsedStreamingCubeInfo.CuboidInfo; +import org.apache.kylin.stream.core.storage.columnar.invertindex.FixLenIIColumnDescriptor; +import org.apache.kylin.stream.core.storage.columnar.invertindex.IIColumnDescriptor; +import org.apache.kylin.stream.core.storage.columnar.invertindex.SeqIIColumnDescriptor; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimDictionaryMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimensionMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.MetricMetaInfo; +import org.apache.kylin.stream.core.util.Constants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; + +public class FragmentFilesMerger { + private static Logger logger = LoggerFactory.getLogger(FragmentFilesMerger.class); + + private ParsedStreamingCubeInfo parsedCubeInfo; + private File segmentFolder; + + private File mergeWorkingDirectory; + + public FragmentFilesMerger(ParsedStreamingCubeInfo parsedCubeInfo, File segmentFolder) { + this.parsedCubeInfo = parsedCubeInfo; + this.segmentFolder = segmentFolder; + this.mergeWorkingDirectory = new File(segmentFolder, ".merge-" + System.currentTimeMillis()); + } + + public FragmentsMergeResult merge(List fragmentList) throws IOException { + if (!mergeWorkingDirectory.exists()) { + mergeWorkingDirectory.mkdirs(); + } else { + logger.info("clean the merge working dir:{}", mergeWorkingDirectory.getAbsolutePath()); + FileUtils.cleanDirectory(mergeWorkingDirectory); + } + + Collections.sort(fragmentList); + FragmentId mergedFragmentId = new FragmentId(fragmentList.get(0).getFragmentId().getStartId(), fragmentList + .get(fragmentList.size() - 1).getFragmentId().getEndId()); + List fragmentDataList = Lists.newArrayList(); + Map>> dimDictListMap = Maps.newHashMap(); + Map>> fragmentDictionaryMaps = Maps.newHashMap(); + List additionalCuboidsToMerge = null; + long minMergedFragmentEventTime = Long.MAX_VALUE; + long maxMergedFragmentEventTime = 0; + long originNumOfRows = 0; + for (DataSegmentFragment fragment : fragmentList) { + FragmentData fragmentData = ColumnarStoreCache.getInstance().startReadFragmentData(fragment); + FragmentMetaInfo fragmentMetaInfo = fragmentData.getFragmentMetaInfo(); + long fragmentMinTime = fragmentMetaInfo.getMinEventTime(); + long fragmentMaxTime = fragmentMetaInfo.getMaxEventTime(); + originNumOfRows += fragmentMetaInfo.getOriginNumOfRows(); + if (fragmentMinTime < minMergedFragmentEventTime) { + minMergedFragmentEventTime = fragmentMinTime; + } + if (fragmentMaxTime > maxMergedFragmentEventTime) { + maxMergedFragmentEventTime = fragmentMaxTime; + } + if (additionalCuboidsToMerge == null) { + Map cuboidMetaInfoMap = fragmentMetaInfo.getCuboidMetaInfoMap(); + if (cuboidMetaInfoMap != null) { + additionalCuboidsToMerge = Lists.transform(Lists.newArrayList(cuboidMetaInfoMap.keySet()), + new Function() { + @Nullable + @Override + public Long apply(@Nullable String input) { + return Long.valueOf(input); + } + }); + } else { + additionalCuboidsToMerge = Lists.newArrayList(); + } + } + fragmentDataList.add(fragmentData); + Map> dictionaryMap = fragmentData + .getDimensionDictionaries(parsedCubeInfo.dimensionsUseDictEncoding); + fragmentDictionaryMaps.put(fragment.getFragmentId(), dictionaryMap); + for (Entry> dimDictEntry : dictionaryMap.entrySet()) { + List> dictionaryList = dimDictListMap.get(dimDictEntry.getKey()); + if (dictionaryList == null) { + dictionaryList = Lists.newArrayList(); + dimDictListMap.put(dimDictEntry.getKey(), dictionaryList); + } + dictionaryList.add(dimDictEntry.getValue()); + } + } + File mergedFragmentDataFile = new File(mergeWorkingDirectory, mergedFragmentId + Constants.DATA_FILE_SUFFIX); + File mergedFragmentMetaFile = new File(mergeWorkingDirectory, mergedFragmentId + Constants.META_FILE_SUFFIX); + try { + FragmentMetaInfo mergedFragmentMeta = new FragmentMetaInfo(); + CountingOutputStream fragmentDataOutput = new CountingOutputStream(new BufferedOutputStream( + FileUtils.openOutputStream(mergedFragmentDataFile))); + // merge dictionaries + Map> mergedDictMap = mergeAndPersistDictionaries(mergedFragmentMeta, + dimDictListMap, fragmentDataOutput); + + // merge basicCuboid + logger.info("merge basic cuboid"); + CuboidMetaInfo basicCuboidMeta = mergeAndPersistCuboidData(fragmentDataList, fragmentDictionaryMaps, + mergedDictMap, fragmentDataOutput, parsedCubeInfo.basicCuboid.getId()); + mergedFragmentMeta.setBasicCuboidMetaInfo(basicCuboidMeta); + long totalRowCnt = basicCuboidMeta.getNumberOfRows(); + // merge additional cuboids + Map cuboidMetaInfoMap = Maps.newHashMap(); + for (Long cuboidId : additionalCuboidsToMerge) { + logger.info("merge cuboid:{}", cuboidId); + CuboidMetaInfo cuboidMeta = mergeAndPersistCuboidData(fragmentDataList, fragmentDictionaryMaps, + mergedDictMap, fragmentDataOutput, cuboidId); + cuboidMetaInfoMap.put(String.valueOf(cuboidId), cuboidMeta); + totalRowCnt += cuboidMeta.getNumberOfRows(); + } + mergedFragmentMeta.setMaxEventTime(maxMergedFragmentEventTime); + mergedFragmentMeta.setMinEventTime(minMergedFragmentEventTime); + mergedFragmentMeta.setCuboidMetaInfoMap(cuboidMetaInfoMap); + mergedFragmentMeta.setFragmentId(mergedFragmentId.toString()); + mergedFragmentMeta.setNumberOfRows(totalRowCnt); + mergedFragmentMeta.setOriginNumOfRows(originNumOfRows); + fragmentDataOutput.flush(); + fragmentDataOutput.close(); + + FileOutputStream metaOutputStream = FileUtils.openOutputStream(mergedFragmentMetaFile); + JsonUtil.writeValueIndent(metaOutputStream, mergedFragmentMeta); + metaOutputStream.flush(); + metaOutputStream.close(); + } finally { + for (DataSegmentFragment fragment : fragmentList) { + ColumnarStoreCache.getInstance().finishReadFragmentData(fragment); + } + } + FragmentsMergeResult result = new FragmentsMergeResult(fragmentList, mergedFragmentId, mergedFragmentMetaFile, + mergedFragmentDataFile); + return result; + } + + public void cleanMergeDirectory() { + FileUtils.deleteQuietly(mergeWorkingDirectory); + } + + private Map> mergeAndPersistDictionaries(FragmentMetaInfo fragmentMetaInfo, + Map>> dimDictListMap, CountingOutputStream fragmentOut) + throws IOException { + logger.info("merge dimension dictionaries"); + Map> mergedDictMap = Maps.newHashMap(); + List dimDictionaryMetaInfos = Lists.newArrayList(); + for (TblColRef dimension : parsedCubeInfo.dimensionsUseDictEncoding) { + List> dicts = dimDictListMap.get(dimension); + MultipleDictionaryValueEnumerator multipleDictionaryValueEnumerator = new MultipleDictionaryValueEnumerator(dimension.getType(), + dicts); + Dictionary mergedDict = DictionaryGenerator.buildDictionary(dimension.getType(), + multipleDictionaryValueEnumerator); + mergedDictMap.put(dimension, mergedDict); + + DimDictionaryMetaInfo dimDictionaryMetaInfo = new DimDictionaryMetaInfo(); + dimDictionaryMetaInfo.setDimName(dimension.getName()); + dimDictionaryMetaInfo.setDictType(mergedDict.getClass().getName()); + dimDictionaryMetaInfo.setStartOffset((int) fragmentOut.getCount()); + + DictionarySerializer.serialize(mergedDict, fragmentOut); + dimDictionaryMetaInfo.setDictLength((int) fragmentOut.getCount() - dimDictionaryMetaInfo.getStartOffset()); + dimDictionaryMetaInfos.add(dimDictionaryMetaInfo); + } + fragmentMetaInfo.setDimDictionaryMetaInfos(dimDictionaryMetaInfos); + return mergedDictMap; + } + + private CuboidMetaInfo mergeAndPersistCuboidData(List fragmentDataList, + Map>> fragmentDictionaryMaps, + Map> mergedDictMap, CountingOutputStream fragmentDataOutput, long cuboidId) + throws IOException { + List fragmentCuboidReaders = Lists.newArrayList(); + List fragmentsDimensionEncodings = Lists.newArrayList(); + + CuboidInfo cuboidInfo = parsedCubeInfo.getCuboidInfo(cuboidId); + TblColRef[] dimensions = cuboidInfo.getDimensions(); + int dimCount = dimensions.length; + + for (FragmentData fragmentData : fragmentDataList) { + FragmentMetaInfo fragmentMetaInfo = fragmentData.getFragmentMetaInfo(); + CuboidMetaInfo cuboidMetaInfo; + if (cuboidId == parsedCubeInfo.basicCuboid.getId()) { + cuboidMetaInfo = fragmentMetaInfo.getBasicCuboidMetaInfo(); + } else { + cuboidMetaInfo = fragmentMetaInfo.getCuboidMetaInfo(cuboidId); + } + Map> dictMap = fragmentDictionaryMaps.get(FragmentId.parse(fragmentMetaInfo + .getFragmentId())); + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings( + parsedCubeInfo.cubeDesc, dimensions, dictMap); + FragmentCuboidReader fragmentCuboidReader = new FragmentCuboidReader(parsedCubeInfo.cubeDesc, fragmentData, + cuboidMetaInfo, cuboidInfo.getDimensions(), parsedCubeInfo.measureDescs, dimensionEncodings); + fragmentCuboidReaders.add(fragmentCuboidReader); + fragmentsDimensionEncodings.add(dimensionEncodings); + } + MeasureAggregators measureAggregators = new MeasureAggregators(parsedCubeInfo.measureDescs); + + DimensionEncoding[] mergedDimEncodings = ParsedStreamingCubeInfo.getDimensionEncodings(parsedCubeInfo.cubeDesc, + cuboidInfo.getDimensions(), mergedDictMap); + + IIColumnDescriptor[] invertIndexColDescs = new IIColumnDescriptor[dimCount]; + for (int i = 0; i < mergedDimEncodings.length; i++) { + TblColRef dim = dimensions[i]; + DimensionEncoding encoding = mergedDimEncodings[i]; + if (encoding instanceof DictionaryDimEnc) { + DictionaryDimEnc dictDimEnc = (DictionaryDimEnc) encoding; + Dictionary dict = dictDimEnc.getDictionary(); + if (dict instanceof TrieDictionary) { + invertIndexColDescs[i] = new SeqIIColumnDescriptor(dim.getName(), dict.getMinId(), dict.getMaxId()); + } else { + invertIndexColDescs[i] = new FixLenIIColumnDescriptor(dim.getName(), encoding.getLengthOfEncoding()); + } + } else { + invertIndexColDescs[i] = new FixLenIIColumnDescriptor(dim.getName(), encoding.getLengthOfEncoding()); + } + } + + CuboidColumnDataWriter[] dimDataWriters = new CuboidColumnDataWriter[dimCount]; + CuboidMetricDataWriter[] metricDataWriters = new CuboidMetricDataWriter[parsedCubeInfo.measureCount]; + ColumnarMetricsEncoding[] metricsEncodings = new ColumnarMetricsEncoding[parsedCubeInfo.measureCount]; + + for (int i = 0; i < dimDataWriters.length; i++) { + dimDataWriters[i] = new CuboidColumnDataWriter(cuboidId, dimensions[i].getName()); + } + + for (int i = 0; i < metricDataWriters.length; i++) { + metricDataWriters[i] = new CuboidMetricDataWriter(cuboidId, parsedCubeInfo.measureDescs[i].getName(), + parsedCubeInfo.getMeasureTypeSerializer(i).maxLength()); + metricsEncodings[i] = ColumnarMetricsEncodingFactory.create(parsedCubeInfo.measureDescs[i].getFunction() + .getReturnDataType()); + } + + FragmentCuboidDataMerger fragmentCuboidDataMerger = new FragmentCuboidDataMerger(cuboidInfo, + fragmentCuboidReaders, fragmentsDimensionEncodings, mergedDimEncodings, measureAggregators, + metricsEncodings); + + logger.info("start to merge and write dimension data"); + int rowCnt = 0; + while (fragmentCuboidDataMerger.hasNext()) { + RawRecord rawRecord = fragmentCuboidDataMerger.next(); + for (int i = 0; i < rawRecord.getDimensions().length; i++) { + byte[] bytes = rawRecord.getDimensions()[i]; + dimDataWriters[i].write(bytes); + } + for (int i = 0; i < rawRecord.getMetrics().length; i++) { + metricDataWriters[i].write(rawRecord.getMetrics()[i]); + } + rowCnt++; + } + for (int i = 0; i < dimDataWriters.length; i++) { + dimDataWriters[i].close(); + } + for (int i = 0; i < metricDataWriters.length; i++) { + metricDataWriters[i].close(); + } + logger.info("all dimensions data wrote to separate file"); + logger.info("start to merge dimension data and build invert index"); + + CuboidMetaInfo cuboidMeta = new CuboidMetaInfo(); + cuboidMeta.setNumberOfRows(rowCnt); + cuboidMeta.setNumberOfDim(dimCount); + cuboidMeta.setNumberOfMetrics(parsedCubeInfo.measureCount); + List dimensionMetaList = Lists.newArrayList(); + List metricMetaList = Lists.newArrayList(); + cuboidMeta.setDimensionsInfo(dimensionMetaList); + cuboidMeta.setMetricsInfo(metricMetaList); + + for (int i = 0; i < dimDataWriters.length; i++) { + DimensionEncoding encoding = mergedDimEncodings[i]; + int dimFixLen = encoding.getLengthOfEncoding(); + InputStream dimInput = new BufferedInputStream(FileUtils.openInputStream(dimDataWriters[i].getOutputFile())); + DimensionMetaInfo dimensionMeta = new DimensionMetaInfo(); + dimensionMeta.setName(dimensions[i].getName()); + int startOffset = (int) fragmentDataOutput.getCount(); + dimensionMeta.setStartOffset(startOffset); + + ColumnarStoreDimDesc cStoreDimDesc = ColumnarStoreDimDesc.getDefaultCStoreDimDesc(parsedCubeInfo.cubeDesc, + dimensions[i].getName(), encoding); + ColumnDataWriter columnDataWriter = cStoreDimDesc.getDimWriter(fragmentDataOutput, rowCnt); + for (int j = 0; j < rowCnt; j++) { + byte[] dimValue = new byte[dimFixLen]; + dimInput.read(dimValue); + if (DimensionEncoding.isNull(dimValue, 0, dimValue.length)) { + dimensionMeta.setHasNull(true); + } + invertIndexColDescs[i].getWriter().addValue(dimValue); + columnDataWriter.write(dimValue); + } + columnDataWriter.flush(); + int dimLen = (int) fragmentDataOutput.getCount() - startOffset; + dimensionMeta.setDataLength(dimLen); + invertIndexColDescs[i].getWriter().write(fragmentDataOutput); + dimensionMeta.setIndexLength((int) fragmentDataOutput.getCount() - startOffset - dimLen); + dimensionMeta.setCompression(cStoreDimDesc.getCompression().name()); + dimensionMetaList.add(dimensionMeta); + } + + for (int i = 0; i < metricDataWriters.length; i++) { + DataInputStream metricInput = new DataInputStream(new BufferedInputStream( + FileUtils.openInputStream(metricDataWriters[i].getOutputFile()))); + ColumnarMetricsEncoding metricsEncoding = ColumnarMetricsEncodingFactory + .create(parsedCubeInfo.measureDescs[i].getFunction().getReturnDataType()); + ColumnarStoreMetricsDesc cStoreMetricsDesc = ColumnarStoreMetricsDesc + .getDefaultCStoreMetricsDesc(metricsEncoding); + ColumnDataWriter columnDataWriter = cStoreMetricsDesc.getMetricsWriter(fragmentDataOutput, rowCnt); + MetricMetaInfo metricMeta = new MetricMetaInfo(); + metricMeta.setName(parsedCubeInfo.measureDescs[i].getName()); + int startOffset = (int) fragmentDataOutput.getCount(); + metricMeta.setStartOffset(startOffset); + for (int j = 0; j < rowCnt; j++) { + int metricLen = metricInput.readInt(); + byte[] metricValue = new byte[metricLen]; + metricInput.read(metricValue); + columnDataWriter.write(metricValue); + } + columnDataWriter.flush(); + int metricsLen = (int) fragmentDataOutput.getCount() - startOffset; + metricMeta.setMetricLength(metricsLen); + metricMeta.setMaxSerializeLength(metricDataWriters[i].getMaxValueLen()); + metricMeta.setCompression(cStoreMetricsDesc.getCompression().name()); + metricMetaList.add(metricMeta); + + ByteStreams.copy(metricInput, fragmentDataOutput); + } + return cuboidMeta; + } + + public class CuboidColumnDataWriter { + private long cuboidId; + private String colName; + private File tmpColDataFile; + private CountingOutputStream output; + + public CuboidColumnDataWriter(long cuboidId, String colName) throws IOException { + this.cuboidId = cuboidId; + this.colName = colName; + + this.tmpColDataFile = new File(mergeWorkingDirectory, cuboidId + "-" + colName + ".data"); + this.output = new CountingOutputStream(new BufferedOutputStream(FileUtils.openOutputStream(tmpColDataFile))); + } + + public void write(byte[] value) throws IOException { + output.write(value); + } + + public void close() throws IOException { + output.close(); + } + + public long getLength() { + return output.getCount(); + } + + public File getOutputFile() { + return tmpColDataFile; + } + } + + public class CuboidMetricDataWriter { + private long cuboidId; + private String metricName; + private File tmpMetricDataFile; + private DataOutputStream output; + private CountingOutputStream countingOutput; + private int maxValLen; + + public CuboidMetricDataWriter(long cuboidId, String metricName, int maxValLen) throws IOException { + this.cuboidId = cuboidId; + this.metricName = metricName; + this.maxValLen = maxValLen; + this.tmpMetricDataFile = new File(mergeWorkingDirectory, cuboidId + "-" + metricName + ".data"); + this.countingOutput = new CountingOutputStream(new BufferedOutputStream( + FileUtils.openOutputStream(tmpMetricDataFile))); + this.output = new DataOutputStream(countingOutput); + } + + public void write(byte[] value) throws IOException { + output.writeInt(value.length); + output.write(value); + } + + public void close() throws IOException { + output.close(); + } + + public int getMaxValueLen() { + return maxValLen; + } + + public long getLength() { + return countingOutput.getCount(); + } + + public File getOutputFile() { + return tmpMetricDataFile; + } + } + + public class FragmentCuboidDataMerger implements Iterator { + private List fragmentsDimensionEncodings; + private DimensionEncoding[] mergedDimensionEncodings; + private List fragmentsRecordDecoders; + private List> fragmentsCuboidRecords; + private PriorityQueue> minHeap; + private MeasureAggregators resultAggrs; + private DataTypeSerializer[] metricsSerializers; + private RawRecord oneRawRecord; + private ByteBuffer metricsBuf; + + public FragmentCuboidDataMerger(CuboidInfo cuboidInfo, List fragmentCuboidReaders, + List fragmentsDimensionEncodings, DimensionEncoding[] mergedDimEncodings, + MeasureAggregators resultAggrs, ColumnarMetricsEncoding[] metricsEncodings) { + this.mergedDimensionEncodings = mergedDimEncodings; + this.fragmentsDimensionEncodings = fragmentsDimensionEncodings; + this.fragmentsRecordDecoders = Lists.newArrayList(); + for (DimensionEncoding[] fragmentDimensionEncodings : fragmentsDimensionEncodings) { + fragmentsRecordDecoders.add(new RecordDecoder(fragmentDimensionEncodings)); + } + + this.fragmentsCuboidRecords = Lists.newArrayListWithCapacity(fragmentCuboidReaders.size()); + for (FragmentCuboidReader reader : fragmentCuboidReaders) { + fragmentsCuboidRecords.add(reader.iterator()); + } + this.resultAggrs = resultAggrs; + this.metricsSerializers = new DataTypeSerializer[metricsEncodings.length]; + for (int i = 0; i < metricsEncodings.length; i++) { + metricsSerializers[i] = metricsEncodings[i].asDataTypeSerializer(); + } + this.minHeap = new PriorityQueue<>(fragmentCuboidReaders.size(), + new Comparator>() { + @Override + public int compare(Pair o1, Pair o2) { + return StringArrayComparator.INSTANCE.compare(o1.getFirst().dimensions, + o2.getFirst().dimensions); + } + }); + + this.oneRawRecord = new RawRecord(cuboidInfo.getDimCount(), parsedCubeInfo.measureCount); + for (int i = 0; i < fragmentCuboidReaders.size(); i++) { + enqueueFromFragment(i); + } + metricsBuf = ByteBuffer.allocate(getMaxMetricsLength()); + } + + public int getMaxMetricsLength() { + int result = -1; + for (int i = 0; i < metricsSerializers.length; i++) { + int maxLength = metricsSerializers[i].maxLength(); + if (result < maxLength) { + result = maxLength; + } + } + return result; + } + + @Override + public boolean hasNext() { + return !minHeap.isEmpty(); + } + + @Override + public RawRecord next() { + Pair currRecordEntry = minHeap.poll(); + DecodedRecord currRecord = currRecordEntry.getFirst(); + + enqueueFromFragment(currRecordEntry.getSecond()); + boolean needAggregate = false; + boolean first = true; + while ((!minHeap.isEmpty()) + && StringArrayComparator.INSTANCE.compare(currRecord.dimensions, + minHeap.peek().getFirst().dimensions) == 0) { + if (first) { + doAggregate(currRecord); + first = false; + needAggregate = true; + } + Pair nextRecord = minHeap.poll(); + doAggregate(nextRecord.getFirst()); + enqueueFromFragment(nextRecord.getSecond()); + } + byte[][] newEncodedDimVals = encodeToNewDimValues(currRecord.dimensions); + if (!needAggregate) { + return new RawRecord(newEncodedDimVals, currRecord.metrics); + } + + for (int i = 0; i < oneRawRecord.getDimensions().length; i++) { + oneRawRecord.setDimension(i, newEncodedDimVals[i]); + } + Object[] metricValues = new Object[parsedCubeInfo.measureCount]; + resultAggrs.collectStates(metricValues); + for (int i = 0; i < metricValues.length; i++) { + metricsBuf.clear(); + metricsSerializers[i].serialize(metricValues[i], metricsBuf); + byte[] metricBytes = Arrays.copyOf(metricsBuf.array(), metricsBuf.position()); + oneRawRecord.setMetric(i, metricBytes); + } + resultAggrs.reset(); + return oneRawRecord; + } + + private byte[][] encodeToNewDimValues(String[] dimensionValues) { + byte[][] result = new byte[dimensionValues.length][]; + for (int i = 0; i < dimensionValues.length; i++) { + DimensionEncoding dimensionEncoding = mergedDimensionEncodings[i]; + byte[] bytes = new byte[dimensionEncoding.getLengthOfEncoding()]; + dimensionEncoding.encode(dimensionValues[i], bytes, 0); + result[i] = bytes; + } + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("unSupport operation"); + } + + private void doAggregate(DecodedRecord record) { + Object[] metricValues = new Object[parsedCubeInfo.measureCount]; + decode(record.metrics, metricValues); + resultAggrs.aggregate(metricValues); + } + + public void decode(byte[][] metricsBytes, Object[] result) { + for (int i = 0; i < metricsSerializers.length; i++) { + result[i] = metricsSerializers[i].deserialize(ByteBuffer.wrap(metricsBytes[i])); + } + } + + private void enqueueFromFragment(int index) { + Iterator fragmentCuboidRecords = fragmentsCuboidRecords.get(index); + RecordDecoder recordDecoder = fragmentsRecordDecoders.get(index); + if (fragmentCuboidRecords.hasNext()) { + RawRecord rawRecord = fragmentCuboidRecords.next(); + minHeap.offer(new Pair<>(recordDecoder.decode(rawRecord), index)); + } + } + } + + private static class RecordDecoder { + private DimensionEncoding[] dimEncodings; + + public RecordDecoder(DimensionEncoding[] dimEncodings) { + this.dimEncodings = dimEncodings; + } + + public DecodedRecord decode(RawRecord rawRecord) { + byte[][] rawDimValues = rawRecord.getDimensions(); + String[] dimValues = new String[rawDimValues.length]; + for (int i = 0; i < dimValues.length; i++) { + byte[] dimVal = rawDimValues[i]; + dimValues[i] = dimEncodings[i].decode(dimVal, 0, dimVal.length); + } + byte[][] metricsValues = rawRecord.getMetrics(); + return new DecodedRecord(dimValues, Arrays.copyOf(metricsValues, metricsValues.length)); + } + } + + private static class DecodedRecord { + String[] dimensions; + byte[][] metrics; + + DecodedRecord(String[] dimensions, byte[][] metrics) { + this.dimensions = dimensions; + this.metrics = metrics; + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentId.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentId.java new file mode 100644 index 00000000000..1e68a7deb71 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentId.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +public class FragmentId implements Comparable { + private static final String SEP = "-"; + private int startId; + private int endId; + + public static FragmentId parse(String idString) { + String[] splits = idString.split(SEP); + if (splits.length == 1) { + return new FragmentId(Integer.parseInt(splits[0])); + } else if (splits.length == 2) { + return new FragmentId(Integer.parseInt(splits[0]), Integer.parseInt(splits[1])); + } else { + throw new IllegalArgumentException("illegal fragment id format:" + idString); + } + } + + public FragmentId(int id) { + this.startId = id; + this.endId = id; + } + + public FragmentId(int startId, int endId) { + this.startId = startId; + this.endId = endId; + } + + public int getStartId() { + return startId; + } + + public int getEndId() { + return endId; + } + + @Override + public int compareTo(FragmentId other) { + return endId - other.endId; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + FragmentId that = (FragmentId) o; + + if (startId != that.startId) + return false; + return endId == that.endId; + + } + + @Override + public int hashCode() { + int result = startId; + result = 31 * result + endId; + return result; + } + + @Override + public String toString() { + if (startId == endId) { + return String.valueOf(endId); + } + return startId + SEP + endId; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentSearchResult.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentSearchResult.java new file mode 100644 index 00000000000..920926e2077 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentSearchResult.java @@ -0,0 +1,483 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Locale; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedMap; + +import com.google.common.collect.Maps; +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.measure.MeasureAggregator; +import org.apache.kylin.metadata.filter.IFilterCodeSystem; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.metadata.tuple.IEvaluatableTuple; +import org.apache.kylin.stream.core.query.HavingFilterChecker; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.query.IStreamingSearchResult; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.invertindex.IndexSearchResult; +import org.apache.kylin.stream.core.storage.columnar.invertindex.InvertIndexSearcher; +import org.apache.kylin.stream.core.util.StreamFilterUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Iterators; + +public class FragmentSearchResult implements IStreamingSearchResult { + private static Logger logger = LoggerFactory.getLogger(FragmentSearchResult.class); + private TupleFilter filter; + private InvertIndexSearcher iiSearcher; + private DataSegmentFragment fragment; + private ResponseResultSchema responseSchema; + private ColumnarRecordCodec recordCodec; + + private Set groups; + private TupleFilter havingFilter; + + private FragmentCuboidReader fragmentCuboidReader; + + private StreamingQueryProfile queryProfile; + private int filterRowCnt = 0; + private int finalRowCnt = 0; + + /** + * + * @param fragment + * @param fragmentData + * @param cuboidMetaInfo + * @param filter + * @param recordCodec + */ + public FragmentSearchResult(final DataSegmentFragment fragment, final FragmentData fragmentData, + final CuboidMetaInfo cuboidMetaInfo, ResponseResultSchema responseSchema, + final TupleFilter filter, final Set groups, final TupleFilter havingFilter, ColumnarRecordCodec recordCodec) throws IOException { + this.fragment = fragment; + this.filter = filter; + this.responseSchema = responseSchema; + this.groups = groups; + this.havingFilter = havingFilter; + + TblColRef[] dimensions = responseSchema.getDimensions(); + + ByteBuffer readBuffer = fragmentData.getDataReadBuffer(); + this.iiSearcher = new InvertIndexSearcher(cuboidMetaInfo, dimensions, readBuffer.asReadOnlyBuffer()); + + CubeDesc cubeDesc = responseSchema.getCubeDesc(); + + this.recordCodec = recordCodec; + this.fragmentCuboidReader = new FragmentCuboidReader(cubeDesc, fragmentData, cuboidMetaInfo, dimensions, + responseSchema.getMeasureDescs(), recordCodec.getDimensionEncodings()); + this.queryProfile = StreamingQueryProfile.get(); + } + + public Iterator iterator() { + final Iterator sourceRecords = searchFragment(); + FilteredAndAggregatedRecords filterAggrRecords = new FilteredAndAggregatedRecords(sourceRecords, + responseSchema, recordCodec, filter, groups, havingFilter); + return filterAggrRecords.iterator(); + } + + private Iterator searchFragment() { + IndexSearchResult indexSearchResult = searchFromIndex(); + Iterator result; + // Full table scan + if (indexSearchResult == null || indexSearchResult.needFullScan()) { + result = fragmentCuboidReader.iterator(); + queryProfile.addStepInfo(getFragmentDataScanStep(), "use_index", "false"); + } else { + queryProfile.addStepInfo(getFragmentDataScanStep(), "use_index", "true"); + if (indexSearchResult.rows == null) { + if (queryProfile.isDetailProfileEnable()) { + logger.info("query-{}: no data match the query in the file segment-{}_fragment-{}", + queryProfile.getQueryId(), fragment.getSegmentName(), fragment.getFragmentId()); + } + return Iterators.emptyIterator(); + } + final Iterator rows = indexSearchResult.rows; + result = new Iterator() { + @Override + public boolean hasNext() { + return rows.hasNext(); + } + + @Override + public RawRecord next() { + return fragmentCuboidReader.read(rows.next() - 1); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + return result; + } + + private IndexSearchResult searchFromIndex() { + if (queryProfile.isDetailProfileEnable()) { + queryProfile.startStep(getFragmentIdxSearchStep()); + } + IndexSearchResult result = iiSearcher.search(filter); + if (queryProfile.isDetailProfileEnable()) { + queryProfile.finishStep(getFragmentIdxSearchStep()); + } + return result; + } + + private String getFragmentIdxSearchStep() { + return String.format(Locale.ROOT, "segment-%s_fragment-%s_idx_search", fragment.getSegmentName(), fragment.getFragmentId()); + } + + private String getFragmentDataScanStep() { + return String.format(Locale.ROOT, "segment-%s_fragment-%s_data_scan", fragment.getSegmentName(), fragment.getFragmentId()); + } + + @Override + public void close() throws IOException { + // TODO Auto-generated method stub + } + + @Override + public void startRead() { + if (queryProfile.isDetailProfileEnable()) { + String stepName = getFragmentDataScanStep(); + queryProfile.startStep(stepName); + logger.info("query-{}: start to search segment-{}_fragment-{} file", + queryProfile.getQueryId(), fragment.getSegmentName(), fragment.getFragmentId()); + } + } + + @Override + public void endRead() { + long scanRowCnt = fragmentCuboidReader.getReadRowCount(); + queryProfile.incScanRows(scanRowCnt); + queryProfile.incFilterRows(filterRowCnt); + if (queryProfile.isDetailProfileEnable()) { + String stepName = getFragmentDataScanStep(); + StreamingQueryProfile.ProfileStep profileStep = queryProfile.finishStep(stepName).stepInfo( + "row_count", String.valueOf(fragmentCuboidReader.getReadRowCount())); + logger.info("query-{}: segment-{}_fragment-{} scan finished, scan {} rows, filter {} rows, return {} rows, take {} ms", + queryProfile.getQueryId(), fragment.getSegmentName(), fragment.getFragmentId(), scanRowCnt, filterRowCnt, finalRowCnt, + profileStep.getDuration()); + } + } + + public class FilteredAndAggregatedRecords implements Iterable { + private TupleFilter filter; + private TupleFilter havingFilter; + private Iterator sourceRecords; + private AggregationCache aggrCache; + private ResponseResultSchema schema; + private int[] groupIndexes; + + private ColumnarRecordCodec recordDecoder; + private int pushDownLimit = Integer.MAX_VALUE; + + RawRecord next; + final IEvaluatableTuple oneTuple = new IEvaluatableTuple() { + @Override + public Object getValue(TblColRef col) { + return new ByteArray(next.getDimensions()[schema.getIndexOfDimension(col)]); + } + }; + final IFilterCodeSystem filterCodeSystem = StreamFilterUtil.getStreamingFilterCodeSystem(); + + public FilteredAndAggregatedRecords(Iterator sourceRecords, ResponseResultSchema schema, ColumnarRecordCodec recordDecoder, + TupleFilter filter, Set groups, TupleFilter havingFilter) { + this.sourceRecords = sourceRecords; + this.schema = schema; + this.recordDecoder = recordDecoder; + this.filter = filter; + this.havingFilter = havingFilter; + this.groupIndexes = new int[groups.size()]; + int i = 0; + for (TblColRef group : groups) { + groupIndexes[i] = schema.getIndexOfDimension(group); + i++; + } + if (groupIndexes.length == 0) { + this.aggrCache = new OneValueAggregationCache(); + } else { + this.aggrCache = new TreeMapAggregationCache(); + } + } + + @Override + public Iterator iterator() { + if (hasAggregation()) { + while (sourceRecords.hasNext()) { + RawRecord rawRecord = sourceRecords.next(); + if (filter != null && !satisfyFilter(rawRecord)) { + filterRowCnt ++; + } else { + aggrCache.aggregate(rawRecord); + } + } + return aggrCache.iterator(); + } else { + return transformAndFilterRecords(); + } + } + + private Iterator transformAndFilterRecords() { + return new Iterator() { + Record oneRecord = new Record(schema.getDimensionCount(), schema.getMetricsCount()); + @Override + public boolean hasNext() { + if (next != null) + return true; + + while (sourceRecords.hasNext()) { + next = sourceRecords.next(); + if (filter != null && !evaluateFilter()) { + filterRowCnt ++; + continue; + } + return true; + } + next = null; + return false; + } + + private boolean evaluateFilter() { + return filter.evaluate(oneTuple, filterCodeSystem); + } + + @Override + public Record next() { + // fetch next record + if (next == null) { + hasNext(); + if (next == null) + throw new NoSuchElementException(); + } + byte[][] rawDimVals = next.getDimensions(); + for (int i = 0; i < rawDimVals.length; i++) { + oneRecord.setDimension(i, recordDecoder.decodeDimension(i, rawDimVals[i])); + } + // no metrics here, will go to aggregate result if there are metrics. + next = null; + return oneRecord; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + }; + } + + private boolean hasAggregation() { + return groupIndexes.length > 0 || schema.getMetricsCount() > 0; + } + + private boolean satisfyFilter(RawRecord rawRecord) { + next = rawRecord; + return filter.evaluate(oneTuple, filterCodeSystem); + } + + private MeasureAggregator[] newAggregators() { + String[] aggrFuncs = schema.getAggrFuncs(); + MeasureAggregator[] result = new MeasureAggregator[aggrFuncs.length]; + for (int i = 0; i < result.length; i++) { + result[i] = MeasureAggregator.create(aggrFuncs[i], schema.getMetricsDataType(i)); + } + return result; + } + + class TreeMapAggregationCache implements AggregationCache { + + final Comparator bytesComparator = new Comparator() { + @Override + public int compare(byte[][] o1, byte[][] o2) { + for (int i = 0; i < groupIndexes.length; i++) { + int groupIdx = groupIndexes[i]; + int result = Bytes.compareTo(o1[groupIdx], o2[groupIdx]); + if (result != 0) { + return result; + } + } + return 0; + } + }; + + SortedMap aggBufMap; + + public TreeMapAggregationCache() { + aggBufMap = createBuffMap(); + } + + private SortedMap createBuffMap() { + return Maps.newTreeMap(bytesComparator); + } + + public boolean aggregate(RawRecord r) { + byte[][] dimVals = r.getDimensions(); +// for (int i = 0; i < dimVals.length; i++) { +// copyDimVals[i] = new byte[dimVals[i].length]; +// System.arraycopy(dimVals[i], 0, copyDimVals[i], 0, dimVals[i].length); +// } + byte[][] metricsVals = r.getMetrics(); + MeasureAggregator[] aggrs = aggBufMap.get(dimVals); + if (aggrs == null) { + //for storage push down limit + if (aggBufMap.size() >= pushDownLimit) { + return false; + } + byte[][] copyDimVals = new byte[schema.getDimensionCount()][]; + System.arraycopy(dimVals, 0, copyDimVals, 0, dimVals.length); + aggrs = newAggregators(); + aggBufMap.put(copyDimVals, aggrs); + } + for (int i = 0; i < aggrs.length; i++) { + Object metrics = recordDecoder.decodeMetrics(i, metricsVals[i]); + aggrs[i].aggregate(metrics); + } + return true; + } + + @Override + public void close() throws RuntimeException { + + } + + public Iterator iterator() { + Iterator> it = aggBufMap.entrySet().iterator(); + + final Iterator> input = it; + + return new Iterator() { + + final Record oneRecord = new Record(schema.getDimensionCount(), schema.getMetricsCount()); + Entry returningEntry = null; + final HavingFilterChecker havingFilterChecker = (havingFilter == null) ? null + : new HavingFilterChecker(havingFilter, schema); + + @Override + public boolean hasNext() { + while (returningEntry == null && input.hasNext()) { + returningEntry = input.next(); + if (havingFilterChecker != null) { + if (!havingFilterChecker.check(returningEntry.getValue())) { + returningEntry = null; + } + } + } + return returningEntry != null; + } + + @Override + public Record next() { + byte[][] rawDimVals = returningEntry.getKey(); + for (int i = 0; i < rawDimVals.length; i++) { + oneRecord.setDimension(i, recordDecoder.decodeDimension(i, rawDimVals[i])); + } + MeasureAggregator[] measures = returningEntry.getValue(); + for (int i = 0; i < measures.length; i++) { + oneRecord.setMetric(i, measures[i].getState()); + } + finalRowCnt ++; + returningEntry = null; + return oneRecord; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + } + + class OneValueAggregationCache implements AggregationCache { + MeasureAggregator[] aggrs; + byte[][] rawDimValues; + + public OneValueAggregationCache() { + aggrs = newAggregators(); + } + + public boolean aggregate(RawRecord r) { + if (rawDimValues == null) { + rawDimValues = r.getDimensions(); + } + byte[][] metricsVals = r.getMetrics(); + for (int i = 0; i < aggrs.length; i++) { + Object metrics = recordDecoder.decodeMetrics(i, metricsVals[i]); + aggrs[i].aggregate(metrics); + } + return true; + } + + @Override + public void close() throws RuntimeException { + } + + public Iterator iterator() { + if (rawDimValues == null) { + return Iterators.emptyIterator(); + } + HavingFilterChecker havingFilterChecker = (havingFilter == null) ? null + : new HavingFilterChecker(havingFilter, schema); + if (havingFilterChecker == null) { + return Iterators.singletonIterator(createRecord()); + } + if (havingFilterChecker.check(aggrs)) { + return Iterators.singletonIterator(createRecord()); + } else { + return Iterators.emptyIterator(); + } + } + + Record createRecord() { + Record record = new Record(schema.getDimensionCount(), schema.getMetricsCount()); + for (int i = 0; i < rawDimValues.length; i++) { + record.setDimension(i, recordDecoder.decodeDimension(i, rawDimValues[i])); + } + for (int i = 0; i < aggrs.length; i++) { + record.setMetric(i, aggrs[i].getState()); + } + finalRowCnt ++; + return record; + } + + } + + } + interface AggregationCache extends Closeable { + boolean aggregate(RawRecord r); + Iterator iterator(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentsMergeResult.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentsMergeResult.java new file mode 100644 index 00000000000..f5c11e30004 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FragmentsMergeResult.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.File; +import java.util.List; + +public class FragmentsMergeResult { + private List origFragments; + private FragmentId mergedFragmentId; + private File mergedFragmentMetaFile; + private File mergedFragmentDataFile; + + public FragmentsMergeResult(List origFragments, FragmentId mergedFragmentId, + File mergedFragmentMetaFile, File mergedFragmentDataFile) { + this.origFragments = origFragments; + this.mergedFragmentId = mergedFragmentId; + this.mergedFragmentMetaFile = mergedFragmentMetaFile; + this.mergedFragmentDataFile = mergedFragmentDataFile; + } + + public FragmentId getMergedFragmentId() { + return mergedFragmentId; + } + + public File getMergedFragmentMetaFile() { + return mergedFragmentMetaFile; + } + + public File getMergedFragmentDataFile() { + return mergedFragmentDataFile; + } + + public List getOrigFragments() { + return origFragments; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataReader.java new file mode 100644 index 00000000000..807e136074d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataReader.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +public class GeneralColumnDataReader implements ColumnDataReader{ + private ByteBuffer dataBuffer; + private int numOfVals; + private int dataStartOffset; + private int indexStartOffset; + + public GeneralColumnDataReader(ByteBuffer dataBuffer, int dataStartOffset, int dataLength) { + this.dataBuffer = dataBuffer; + this.dataStartOffset = dataStartOffset; + dataBuffer.position(dataStartOffset + dataLength - 4); + this.numOfVals = dataBuffer.getInt(); + this.indexStartOffset = dataStartOffset + dataLength - 4 - 4 * numOfVals; + } + + @Override + public byte[] read(int index) { + int offset; + if (index == 0) { + offset = 0; + } else { + dataBuffer.position(indexStartOffset + ((index - 1) << 2)); + offset = dataBuffer.getInt(); + } + dataBuffer.position(dataStartOffset + offset); + int length = dataBuffer.getInt(); + byte[] result = new byte[length]; + dataBuffer.get(result); + return result; + } + + @Override + public Iterator iterator() { + dataBuffer.position(dataStartOffset); + return new Iterator() { + int readRowCount = 0; + @Override + public boolean hasNext() { + return readRowCount < numOfVals; + } + + @Override + public byte[] next() { + int size = dataBuffer.getInt(); + byte[] result = new byte[size]; + dataBuffer.get(result); + readRowCount ++; + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("unSupport operation"); + } + }; + } + + public ByteBuffer get(int index) { + int offset; + if (index == 0) { + offset = 0; + } else { + dataBuffer.position(indexStartOffset + ((index - 1) << 2)); + offset = dataBuffer.getInt(); + } + + ByteBuffer resultBuffer = dataBuffer.asReadOnlyBuffer(); + int startOffset = dataStartOffset + offset; + resultBuffer.position(startOffset); + int length = resultBuffer.getInt(); + resultBuffer.limit(startOffset + 4 + length); + return resultBuffer; + } + + public int getNumOfVals() { + return numOfVals; + } + + @Override + public void close() throws IOException { + // do nothing + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataWriter.java new file mode 100644 index 00000000000..9b73c961f6c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataWriter.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.io.DataOutputStream; +import java.io.IOException; + +public class GeneralColumnDataWriter implements ColumnDataWriter{ + private DataOutputStream dataOutput; + private int totalBytes; + private int[] valueOffsets; + private int valueNum; + + public GeneralColumnDataWriter(int numOfVals, DataOutputStream dataOutput) { + this.dataOutput = dataOutput; + this.valueOffsets = new int[numOfVals]; + } + + @Override + public void write(byte[] value) throws IOException { + write(value, 0, value.length); + } + + public void write(byte[] value, int offset, int len) throws IOException { + dataOutput.writeInt(len); + dataOutput.write(value, offset, len); + totalBytes += len + 4; + valueOffsets[valueNum] = totalBytes; + valueNum++; + } + + private void writeIndex() throws IOException { + for (int i = 0; i < valueOffsets.length; i++) { + dataOutput.writeInt(valueOffsets[i]); + } + dataOutput.writeInt(valueOffsets.length); + } + + @Override + public void flush() throws IOException { + writeIndex(); + dataOutput.flush(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ParsedStreamingCubeInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ParsedStreamingCubeInfo.java new file mode 100644 index 00000000000..503278a06d7 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/ParsedStreamingCubeInfo.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.cuboid.Cuboid; +import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc; +import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich; +import org.apache.kylin.cube.model.RowKeyColDesc; +import org.apache.kylin.dimension.DictionaryDimEnc; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.dimension.DimensionEncodingFactory; +import org.apache.kylin.measure.MeasureIngester; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class ParsedStreamingCubeInfo { + public final CubeInstance cubeInstance; + public final CubeDesc cubeDesc; + public final CubeJoinedFlatTableEnrich intermediateTableDesc; + public final MeasureDesc[] measureDescs; + public final MeasureIngester[] measureIngesters; + public final String[] metricsAggrFuncs; + public final int dimCount; + public final int measureCount; + public final TblColRef[] dimensions; + public final TblColRef[] dimensionsUseDictEncoding; + + public final Cuboid basicCuboid; + public List additionalCuboidsToBuild; + public CuboidToGridTableMapping basicCuboidMapping; + + private Map dimensionsMap = Maps.newHashMap(); + + public ParsedStreamingCubeInfo(CubeInstance cubeInstance) { + this.cubeInstance = cubeInstance; + this.cubeDesc = cubeInstance.getDescriptor(); + this.basicCuboid = Cuboid.getBaseCuboid(cubeDesc); + this.intermediateTableDesc = new CubeJoinedFlatTableEnrich(new CubeJoinedFlatTableDesc(cubeDesc), cubeDesc); + this.measureCount = cubeDesc.getMeasures().size(); + this.measureDescs = cubeDesc.getMeasures().toArray(new MeasureDesc[measureCount]); + this.measureIngesters = MeasureIngester.create(cubeDesc.getMeasures()); + this.dimensions = basicCuboid.getColumns().toArray(new TblColRef[basicCuboid.getColumns().size()]); + this.dimCount = dimensions.length; + this.basicCuboidMapping = new CuboidToGridTableMapping(basicCuboid); + + boolean buildAdditionalCuboids = cubeDesc.getConfig().isStreamingBuildAdditionalCuboids(); + Set mandatoryCuboids = cubeDesc.getMandatoryCuboids(); + if (buildAdditionalCuboids) { + additionalCuboidsToBuild = Lists.newArrayListWithCapacity(mandatoryCuboids.size()); + for (long cuboidID : mandatoryCuboids) { + CuboidInfo cuboidInfo = new CuboidInfo(cuboidID); + cuboidInfo.init(cubeDesc, intermediateTableDesc); + additionalCuboidsToBuild.add(cuboidInfo); + } + } + + List dimUseDictList = Lists.newArrayList(); + for (TblColRef column : dimensions) { + dimensionsMap.put(column.getName(), column); + if (cubeDesc.getRowkey().isUseDictionary(column)) { + dimUseDictList.add(column); + } + } + this.dimensionsUseDictEncoding = dimUseDictList.toArray(new TblColRef[dimUseDictList.size()]); + + List metricsAggrFuncsList = Lists.newArrayListWithCapacity(measureCount); + for (int i = 0; i < measureCount; i++) { + MeasureDesc measureDesc = measureDescs[i]; + metricsAggrFuncsList.add(measureDesc.getFunction().getExpression()); + } + this.metricsAggrFuncs = metricsAggrFuncsList.toArray(new String[metricsAggrFuncsList.size()]); + } + + public CubeInstance getCubeInstance() { + return cubeInstance; + } + + public CubeDesc getCubeDesc() { + return cubeDesc; + } + + public TblColRef[] getDimensions() { + return dimensions; + } + + public MeasureDesc[] getMeasures() { + return measureDescs; + } + + public MeasureIngester[] getMeasureIngesters() { + return measureIngesters; + } + + public DataType[] getAllDataTypes() { + return basicCuboidMapping.getDataTypes(); + } + + public DataTypeSerializer getMeasureTypeSerializer(int measureIdx) { + DataType type = measureDescs[measureIdx].getFunction().getReturnDataType(); + return DataTypeSerializer.create(type); + } + + public List getAdditionalCuboidsToBuild() { + return additionalCuboidsToBuild; + } + + public CuboidToGridTableMapping getBasicCuboidToGridTableMapping() { + return basicCuboidMapping; + } + + public CuboidInfo getCuboidInfo(long cuboidId) { + CuboidInfo cuboidInfo = new CuboidInfo(cuboidId); + cuboidInfo.init(cubeDesc, intermediateTableDesc); + + return cuboidInfo; + } + + public int getMetricIndexInAllMetrics(FunctionDesc metric) { + return basicCuboidMapping.getIndexOf(metric) - dimensions.length; + } + + public TblColRef getDimensionByName(String dimensionName) { + return dimensionsMap.get(dimensionName); + } + + public static DimensionEncoding[] getDimensionEncodings(CubeDesc cubeDesc, TblColRef[] dimensions, + Map> dimDictMap) { + DimensionEncoding[] result = new DimensionEncoding[dimensions.length]; + for (int i = 0; i < dimensions.length; i++) { + TblColRef dimension = dimensions[i]; + RowKeyColDesc colDesc = cubeDesc.getRowkey().getColDesc(dimension); + if (colDesc.isUsingDictionary()) { + @SuppressWarnings({ "unchecked" }) + Dictionary dict = dimDictMap.get(dimension); + if (dict == null) { + throw new RuntimeException("No dictionary found for dict-encoding column " + dimension); + } else { + result[i] = new DictionaryDimEnc(dict); + } + } else { + result[i] = DimensionEncodingFactory.create(colDesc.getEncodingName(), colDesc.getEncodingArgs(), + colDesc.getEncodingVersion()); + } + } + return result; + } + + public static ColumnarMetricsEncoding[] getMetricsEncodings(MeasureDesc[] measures) { + ColumnarMetricsEncoding[] result = new ColumnarMetricsEncoding[measures.length]; + for (int i = 0; i < measures.length; i++) { + result[i] = ColumnarMetricsEncodingFactory.create(measures[i].getFunction().getReturnDataType()); + } + return result; + } + + public static class CuboidInfo { + private long cuboidID; + private int[] columnsIndex; + private List dimensions; + + public CuboidInfo(long cuboidID) { + this.cuboidID = cuboidID; + } + + public void init(CubeDesc cubeDesc, CubeJoinedFlatTableEnrich intermediateTableDesc) { + dimensions = Lists.newArrayList(); + columnsIndex = new int[Long.bitCount(cuboidID)]; + + int colIdx = 0; + RowKeyColDesc[] allColumns = cubeDesc.getRowkey().getRowKeyColumns(); + for (int i = 0; i < allColumns.length; i++) { + // NOTE: the order of column in list!!! + long bitmask = 1L << allColumns[i].getBitIndex(); + if ((cuboidID & bitmask) != 0) { + TblColRef colRef = allColumns[i].getColRef(); + dimensions.add(colRef); + columnsIndex[colIdx] = intermediateTableDesc.getColumnIndex(colRef); + colIdx++; + } + } + } + + public long getCuboidID() { + return cuboidID; + } + + public int[] getColumnsIndex() { + if (columnsIndex == null) { + throw new IllegalStateException("it is not initialized"); + } + return columnsIndex; + } + + public TblColRef[] getDimensions() { + if (dimensions == null) { + throw new IllegalStateException("it is not initialized"); + } + return dimensions.toArray(new TblColRef[dimensions.size()]); + } + + public int getDimCount() { + return dimensions.size(); + } + + public int getIndexOf(TblColRef dimension) { + return dimensions.indexOf(dimension); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CuboidInfo that = (CuboidInfo) o; + + return cuboidID == that.cuboidID; + + } + + @Override + public int hashCode() { + return (int) (cuboidID ^ (cuboidID >>> 32)); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/RawRecord.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/RawRecord.java new file mode 100644 index 00000000000..464778de61f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/RawRecord.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.util.Arrays; + +public class RawRecord { + private byte[][] dimensions; + private byte[][] metrics; + + public RawRecord(int dimNum, int metricNum) { + this.dimensions = new byte[dimNum][]; + this.metrics = new byte[metricNum][]; + } + + public RawRecord(byte[][] dimensions, byte[][] metrics) { + this.dimensions = dimensions; + this.metrics = metrics; + } + + public void copy(RawRecord another) { + if (another.getDimensions().length != dimensions.length || another.getMetrics().length != metrics.length) { + throw new IllegalStateException("cannot copy record with different schema"); + } + for (int i = 0; i < another.dimensions.length; i++) { + this.dimensions[i] = another.dimensions[i]; + } + for (int i = 0; i < another.metrics.length; i++) { + this.metrics[i] = another.metrics[i]; + } + } + + public RawRecord clone() { + RawRecord rawRecord = new RawRecord(dimensions.length, metrics.length); + System.arraycopy(dimensions, 0, rawRecord.dimensions, 0, dimensions.length); + System.arraycopy(metrics, 0, rawRecord.metrics, 0, metrics.length); + return rawRecord; + } + + public byte[][] getDimensions() { + return dimensions; + } + + public void setDimension(int idx, byte[] value) { + this.dimensions[idx] = value; + } + + public byte[][] getMetrics() { + return metrics; + } + + public void setMetric(int idx, byte[] value) { + this.metrics[idx] = value; + } + + @Override + public String toString() { + return "Record{" + "dimensions=" + Arrays.toString(dimensions) + ", metrics=" + Arrays.toString(metrics) + + '}'; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/StringArrayComparator.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/StringArrayComparator.java new file mode 100644 index 00000000000..643b6e2fa75 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/StringArrayComparator.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.util.Comparator; + +public class StringArrayComparator implements Comparator { + public final static StringArrayComparator INSTANCE = new StringArrayComparator(); + + @Override + public int compare(String[] o1, String[] o2) { + if (o1.length != o2.length) { + return o1.length - o2.length; + } + int result = 0; + for (int i = 0; i < o1.length; ++i) { + if (o1[i] == null && o2[i] == null) { + continue; + } else if (o1[i] != null && o2[i] == null) { + return 1; + } else if (o1[i] == null && o2[i] != null) { + return -1; + } else { + result = o1[i].compareTo(o2[i]); + if (result == 0) { + continue; + } else { + return result; + } + } + } + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/TimeDerivedColumnEncoding.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/TimeDerivedColumnEncoding.java new file mode 100644 index 00000000000..c342c044d33 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/TimeDerivedColumnEncoding.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.stream.core.util.TimeDerivedColumnType; + +public class TimeDerivedColumnEncoding { + private String columnName; + private TimeDerivedColumnType timeColumnType; + private long baseTime; + private long normalizedBaseTime; + + public TimeDerivedColumnEncoding(String columnName, long baseTime) { + this.columnName = columnName; + this.baseTime = baseTime; + this.timeColumnType = TimeDerivedColumnType.getTimeDerivedColumnType(columnName); + this.normalizedBaseTime = timeColumnType.normalize(baseTime); + } + + public void encode(String value, byte[] output, int outputOffset) { + long timeVal = DateFormat.stringToMillis(value); + timeColumnType.calculateTimeUnitRange(timeVal); + + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/Compression.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/Compression.java new file mode 100644 index 00000000000..65d907e8883 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/Compression.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +public enum Compression { + NONE, LZ4, RUN_LENGTH, SNAPPY +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java new file mode 100644 index 00000000000..f9e4560469c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; + +public class FSInputLZ4CompressedColumnReader implements ColumnDataReader { + private int rowCount; + + private int valLen; + private int numValInBlock; + private int maxDecompressedLength; + + private int currBlockNum; + private LZ4SafeDecompressor deCompressor; + private FSDataInputStream fsInputStream; + + public FSInputLZ4CompressedColumnReader(FSDataInputStream fsInputStream, int columnDataStartOffset, + int columnDataLength, int rowCount) throws IOException { + this.rowCount = rowCount; + this.fsInputStream = fsInputStream; + int footStartOffset = columnDataStartOffset + columnDataLength - 8; + fsInputStream.seek(footStartOffset); + this.numValInBlock = fsInputStream.readInt(); + this.valLen = fsInputStream.readInt(); + + fsInputStream.seek(columnDataStartOffset); + this.currBlockNum = -1; + + this.deCompressor = LZ4Factory.fastestInstance().safeDecompressor(); + this.maxDecompressedLength = numValInBlock * valLen; + } + + public Iterator iterator() { + return new LZ4CompressedColumnDataItr(); + } + + @Override + public byte[] read(int rowNum) { + throw new UnsupportedOperationException("not support to read row operation"); + } + + @Override + public void close() throws IOException { + fsInputStream.close(); + } + + private class LZ4CompressedColumnDataItr implements Iterator { + private int readRowCount = 0; + private ByteBuffer decompressedBuffer; + private byte[] decompressedBytes; + + public LZ4CompressedColumnDataItr() { + this.decompressedBytes = new byte[maxDecompressedLength]; + } + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + if (currBlockNum == -1 || !decompressedBuffer.hasRemaining()) { + try { + loadNextBuffer(); + } catch (IOException e) { + throw new RuntimeException("error when read data", e); + } + } + byte[] readBuffer = new byte[valLen]; + decompressedBuffer.get(readBuffer); + readRowCount++; + return readBuffer; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + + private void loadNextBuffer() throws IOException { + int len = fsInputStream.readInt(); + byte[] bytes = new byte[len]; + fsInputStream.readFully(bytes); + int decompressedSize = deCompressor.decompress(bytes, decompressedBytes); + decompressedBuffer = ByteBuffer.wrap(decompressedBytes, 0, decompressedSize); + currBlockNum++; + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputNoCompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputNoCompressedColumnReader.java new file mode 100644 index 00000000000..c6255cd95b2 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputNoCompressedColumnReader.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; + +public class FSInputNoCompressedColumnReader implements ColumnDataReader { + private FSDataInputStream fsInputStream; + private byte[] readBuffer; + private int colDataStartOffset; + private int colValLength; + private int rowCount; + + public FSInputNoCompressedColumnReader(FSDataInputStream fsInputStream, int colDataStartOffset, int colValLength, + int rowCount) throws IOException { + this.fsInputStream = fsInputStream; + this.colDataStartOffset = colDataStartOffset; + fsInputStream.seek(colDataStartOffset); + this.colValLength = colValLength; + this.rowCount = rowCount; + this.readBuffer = new byte[colValLength]; + } + + public Iterator iterator() { + return new NoCompressedColumnDataItr(); + } + + @Override + public byte[] read(int rowNum) { + throw new UnsupportedOperationException("not support to read row operation"); + } + + @Override + public void close() throws IOException { + fsInputStream.close(); + } + + private class NoCompressedColumnDataItr implements Iterator { + private int readRowCount = 0; + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + try { + fsInputStream.readFully(readBuffer); + } catch (IOException e) { + throw new RuntimeException("error when read data", e); + } + readRowCount++; + return readBuffer; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputRLECompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputRLECompressedColumnReader.java new file mode 100644 index 00000000000..fb17ceb0dc7 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputRLECompressedColumnReader.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; + +public class FSInputRLECompressedColumnReader implements ColumnDataReader { + private int valLen; + private int numValInBlock; + private ByteBuffer currBlockBuffer; + private int currBlockNum; + private FSDataInputStream fsInputStream; + + private int rowCount; + + public FSInputRLECompressedColumnReader(FSDataInputStream fsInputStream, int columnDataStartOffset, + int columnDataLength, int rowCount) throws IOException { + this.rowCount = rowCount; + this.fsInputStream = fsInputStream; + int footStartOffset = columnDataStartOffset + columnDataLength - 8; + fsInputStream.seek(footStartOffset); + this.numValInBlock = fsInputStream.readInt(); + this.valLen = fsInputStream.readInt(); + this.fsInputStream.seek(columnDataStartOffset); + this.currBlockNum = -1; + } + + public void reset() { + this.currBlockNum = -1; + } + + @Override + public Iterator iterator() { + return new RunLengthCompressedColumnDataItr(); + } + + @Override + public byte[] read(int rowNum) { + throw new UnsupportedOperationException("not support to read row operation"); + } + + @Override + public void close() throws IOException { + fsInputStream.close(); + } + + private class RunLengthCompressedColumnDataItr implements Iterator { + private int currRLEntryValCnt; + private byte[] currRLEntryVal; + private int readRLEntryValCnt; + + private int readRowCount = 0; + private int blockReadRowCount = 0; + + public RunLengthCompressedColumnDataItr() { + currRLEntryVal = new byte[valLen]; + } + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + if (readRLEntryValCnt >= currRLEntryValCnt) { + loadNextEntry(); + } + readRLEntryValCnt++; + readRowCount++; + blockReadRowCount++; + return currRLEntryVal; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + + private void loadNextEntry() { + if (currBlockNum == -1 || blockReadRowCount >= numValInBlock) { + try { + loadNextBuffer(); + } catch (IOException e) { + throw new RuntimeException("error when read data", e); + } + currRLEntryVal = new byte[valLen]; + blockReadRowCount = 0; + } + currRLEntryValCnt = currBlockBuffer.getInt(); + currBlockBuffer.get(currRLEntryVal); + readRLEntryValCnt = 0; + } + + private void loadNextBuffer() throws IOException { + int len = fsInputStream.readInt(); + byte[] bytes = new byte[len]; + fsInputStream.readFully(bytes); + currBlockBuffer = ByteBuffer.wrap(bytes); + currBlockNum++; + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnReader.java new file mode 100644 index 00000000000..b923220453c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnReader.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; +import org.apache.kylin.stream.core.storage.columnar.GeneralColumnDataReader; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +public class LZ4CompressedColumnReader implements ColumnDataReader { + private int rowCount; + + private int valLen; + private int numValInBlock; + private int maxBufferLength; + + private int currBlockNum; + private LZ4SafeDecompressor deCompressor; + private ByteBuffer dataBuffer; + private ByteBuffer decompressedBuffer; + +// private byte[] readBuffer; + private GeneralColumnDataReader blockDataReader; + + public LZ4CompressedColumnReader(ByteBuffer dataBuffer, int columnDataStartOffset, int columnDataLength, + int rowCount) { + this.rowCount = rowCount; + this.dataBuffer = dataBuffer; + int footStartOffset = columnDataStartOffset + columnDataLength - 8; + dataBuffer.position(footStartOffset); + this.numValInBlock = dataBuffer.getInt(); + this.valLen = dataBuffer.getInt(); + + this.blockDataReader = new GeneralColumnDataReader(dataBuffer, columnDataStartOffset, columnDataLength - 8); + this.currBlockNum = -1; +// this.readBuffer = new byte[valLen]; + + this.deCompressor = LZ4Factory.fastestInstance().safeDecompressor(); + this.maxBufferLength = numValInBlock * valLen; + this.decompressedBuffer = ByteBuffer.allocate(maxBufferLength); + } + + private void loadBuffer(int targetBlockNum) { + ByteBuffer compressedBuffer = blockDataReader.get(targetBlockNum); + int length = compressedBuffer.limit() - compressedBuffer.position(); + deCompressor.decompress(compressedBuffer, compressedBuffer.position(), length, decompressedBuffer, 0, + maxBufferLength); + decompressedBuffer.position(0); + currBlockNum = targetBlockNum; + } + + @Override + public Iterator iterator() { + return new LZ4CompressedColumnDataItr(); + } + + @Override + public byte[] read(int rowNum) { + int targetBlockNum = rowNum / numValInBlock; + if (targetBlockNum != currBlockNum) { + loadBuffer(targetBlockNum); + } + decompressedBuffer.position((rowNum % numValInBlock) * valLen); + byte[] readBuffer = new byte[valLen]; + decompressedBuffer.get(readBuffer); + return readBuffer; + } + + @Override + public void close() throws IOException { + //do nothing + } + + private class LZ4CompressedColumnDataItr implements Iterator { + private int readRowCount = 0; + + public LZ4CompressedColumnDataItr() { + } + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + if (currBlockNum == -1 || !decompressedBuffer.hasRemaining()) { + loadNextBuffer(); + } + byte[] readBuffer = new byte[valLen]; + decompressedBuffer.get(readBuffer); + readRowCount ++; + return readBuffer; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + + private void loadNextBuffer() { + loadBuffer(currBlockNum + 1); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnWriter.java new file mode 100644 index 00000000000..a2eea8ec3eb --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressedColumnWriter.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataWriter; +import org.apache.kylin.stream.core.storage.columnar.GeneralColumnDataWriter; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +public class LZ4CompressedColumnWriter implements ColumnDataWriter { + public static final int DEF_BLOCK_SIZE = 64 * 1024; + private int valLen; + private int numValInBlock; + private int blockCnt; + private LZ4Compressor compressor; + private ByteBuffer writeBuffer; + private DataOutputStream dataOutput; + + private GeneralColumnDataWriter blockDataWriter; + + public LZ4CompressedColumnWriter(int valLen, int rowCnt, int compressBlockSize, OutputStream output) { + this.valLen = valLen; + this.numValInBlock = compressBlockSize / valLen; + this.blockCnt = rowCnt / numValInBlock; + if (rowCnt % numValInBlock != 0) { + blockCnt++; + } + this.compressor = LZ4Factory.fastestInstance().highCompressor(); + this.writeBuffer = ByteBuffer.allocate(numValInBlock * valLen); + this.dataOutput = new DataOutputStream(output); + this.blockDataWriter = new GeneralColumnDataWriter(blockCnt, dataOutput); + } + + public void write(byte[] valBytes) throws IOException { + if (!writeBuffer.hasRemaining()) { + writeBuffer.rewind(); + byte[] block = compressor.compress(writeBuffer.array(), 0, writeBuffer.limit()); + blockDataWriter.write(block); + } + writeBuffer.put(valBytes); + } + + public void flush() throws IOException { + if (writeBuffer != null) { + writeBuffer.flip(); + if (writeBuffer.hasRemaining()) { + byte[] block = compressor.compress(writeBuffer.array(), 0, writeBuffer.limit()); + blockDataWriter.write(block); + } + } + blockDataWriter.flush(); + dataOutput.writeInt(numValInBlock); + dataOutput.writeInt(valLen); + dataOutput.flush(); + writeBuffer = null; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressorTest.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressorTest.java new file mode 100644 index 00000000000..3bf281be79f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressorTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.File; + +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; +import net.jpountz.lz4.LZ4SafeDecompressor; + +import com.google.common.io.Files; + +public class LZ4CompressorTest { + public static void main(String[] args) throws Exception { + LZ4Factory factory = LZ4Factory.fastestInstance(); + + byte[] data = Files.toByteArray(new File("/Users/ganma/dev/githome/kylin/stream-core/stream_index/test_streaming_v2_cube/20180730070000_20180730080000/1/1.data")); + final int decompressedLength = data.length; + + // compress data + LZ4Compressor compressor = factory.fastCompressor(); + long start = System.currentTimeMillis(); + int maxCompressedLength = compressor.maxCompressedLength(decompressedLength); + byte[] compressed = new byte[maxCompressedLength]; + int compressedLength = compressor.compress(data, 0, decompressedLength, compressed, 0, maxCompressedLength); + System.out.println("compress take:" + (System.currentTimeMillis() - start)); + System.out.println(compressedLength); + + // decompress data + // - method 1: when the decompressed length is known + LZ4FastDecompressor decompressor = factory.fastDecompressor(); + start = System.currentTimeMillis(); + byte[] restored = new byte[decompressedLength]; + int compressedLength2 = decompressor.decompress(compressed, 0, restored, 0, decompressedLength); + System.out.println("decompress take:" + (System.currentTimeMillis() - start)); + System.out.println(decompressedLength); + // compressedLength == compressedLength2 + + // - method 2: when the compressed length is known (a little slower) + // the destination buffer needs to be over-sized + LZ4SafeDecompressor decompressor2 = factory.safeDecompressor(); + int decompressedLength2 = decompressor2.decompress(compressed, 0, compressedLength, restored, 0); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnReader.java new file mode 100644 index 00000000000..29ec3cae70a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnReader.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; + +public class NoCompressedColumnReader implements ColumnDataReader { + private ByteBuffer dataBuffer; + private byte[] readBuffer; + private int colDataStartOffset; + private int colValLength; + private int rowCount; + + public NoCompressedColumnReader(ByteBuffer dataBuffer, int colDataStartOffset, int colValLength, int rowCount) { + this.dataBuffer = dataBuffer; + this.colDataStartOffset = colDataStartOffset; + this.colValLength = colValLength; + this.rowCount = rowCount; + this.readBuffer = new byte[colValLength]; + } + + public Iterator iterator() { + return new NoCompressedColumnDataItr(); + } + + public byte[] read(int rowNum) { + dataBuffer.position(colDataStartOffset + rowNum * colValLength); + dataBuffer.get(readBuffer); + return readBuffer; + } + + @Override + public void close() throws IOException { + //do nothing + } + + private class NoCompressedColumnDataItr implements Iterator { + private int readRowCount = 0; + + public NoCompressedColumnDataItr() { + dataBuffer.position(colDataStartOffset); + } + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + dataBuffer.get(readBuffer); + readRowCount++; + return readBuffer; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnWriter.java new file mode 100644 index 00000000000..9de61c90954 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressedColumnWriter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.kylin.stream.core.storage.columnar.ColumnDataWriter; + +public class NoCompressedColumnWriter implements ColumnDataWriter { + private OutputStream dataOutput; + + public NoCompressedColumnWriter(OutputStream output) { + this.dataOutput = output; + } + + public void write(byte[] valBytes) throws IOException { + dataOutput.write(valBytes); + } + + public void flush() throws IOException { + dataOutput.flush(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnReader.java new file mode 100644 index 00000000000..a4aee1d5911 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnReader.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader; +import org.apache.kylin.stream.core.storage.columnar.GeneralColumnDataReader; + +public class RunLengthCompressedColumnReader implements ColumnDataReader { + private int valLen; + private int numValInBlock; + private ByteBuffer currBlockBuffer; + private int currBlockNum; + + private byte[] readBuffer; + private int rowCount; + private GeneralColumnDataReader blockDataReader; + + public RunLengthCompressedColumnReader(ByteBuffer dataBuffer, int columnDataStartOffset, int columnDataLength, + int rowCount) { + this.rowCount = rowCount; + int footStartOffset = columnDataStartOffset + columnDataLength - 8; + dataBuffer.position(footStartOffset); + this.numValInBlock = dataBuffer.getInt(); + this.valLen = dataBuffer.getInt(); + + this.blockDataReader = new GeneralColumnDataReader(dataBuffer, columnDataStartOffset, columnDataLength - 8); + this.currBlockNum = -1; + this.readBuffer = new byte[valLen]; + } + + private void loadBuffer(int targetBlockNum) { + currBlockBuffer = blockDataReader.get(targetBlockNum); + currBlockNum = targetBlockNum; + } + + public void reset() { + this.currBlockNum = -1; + } + + @Override + public Iterator iterator() { + return new RunLengthCompressedColumnDataItr(); + } + + @Override + public byte[] read(int rowNum) { + int targetBlockNum = rowNum / numValInBlock; + if (targetBlockNum != currBlockNum) { + loadBuffer(targetBlockNum); + } + int blockStartOffset = currBlockBuffer.position(); + int limit = currBlockBuffer.limit(); + int entryNum = currBlockBuffer.getInt(limit - 4); + int entryIndexStartOffset = limit - 4 - (entryNum << 2); + int blockRowNum = rowNum % numValInBlock; + + int targetEntry = binarySearchIndex(currBlockBuffer, entryIndexStartOffset, entryNum, blockRowNum); + currBlockBuffer.position(blockStartOffset + (valLen + 4) * targetEntry + 4); + currBlockBuffer.get(readBuffer); + currBlockBuffer.position(blockStartOffset); + return readBuffer; + } + + private int binarySearchIndex(ByteBuffer currBlockBuffer, int entryIndexStartOffset, int entryNum, int rowNum) { + int low = 0; + int high = entryNum - 1; + while (low <= high) { + int mid = (low + high) >>> 1; + + int idxVal = currBlockBuffer.getInt(entryIndexStartOffset + (mid << 2)); + if (idxVal >= rowNum) { + high = mid - 1; + } else { + low = mid + 1; + } + } + return low; + } + + @Override + public void close() throws IOException { + //do nothing + } + + private class RunLengthCompressedColumnDataItr implements Iterator { + private int currRLEntryValCnt; + private byte[] currRLEntryVal; + private int readRLEntryValCnt; + + private int readRowCount = 0; + private int blockReadRowCount = 0; + + public RunLengthCompressedColumnDataItr() { + currRLEntryVal = new byte[valLen]; + } + + @Override + public boolean hasNext() { + return readRowCount < rowCount; + } + + @Override + public byte[] next() { + if (readRLEntryValCnt >= currRLEntryValCnt) { + loadNextEntry(); + } + readRLEntryValCnt++; + readRowCount++; + blockReadRowCount++; + return currRLEntryVal; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("not supported"); + } + + private void loadNextEntry() { + if (currBlockNum == -1 || blockReadRowCount >= numValInBlock) { + loadNextBuffer(); + blockReadRowCount = 0; + } + currRLEntryVal = new byte[valLen]; + currRLEntryValCnt = currBlockBuffer.getInt(); + currBlockBuffer.get(currRLEntryVal); + readRLEntryValCnt = 0; + } + + private void loadNextBuffer() { + loadBuffer(currBlockNum + 1); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnWriter.java new file mode 100644 index 00000000000..cffe48cdfd8 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressedColumnWriter.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.stream.core.storage.columnar.ColumnDataWriter; + +import com.google.common.collect.Lists; +import org.apache.kylin.stream.core.storage.columnar.GeneralColumnDataWriter; + +public class RunLengthCompressedColumnWriter implements ColumnDataWriter { + private int valLen; + private int numValInBlock; + private int blockCnt; + + private ByteBuffer writeBuffer; + private DataOutputStream dataOutput; + private byte[] previousVal; + private int numOfVals = 1; + private int blockWriteCnt; + private int totalWriteCntBeforeTheEntry; + private int entryNum; + private List entryIndex; + private GeneralColumnDataWriter blockDataWriter; + + public RunLengthCompressedColumnWriter(int valLen, int rowCnt, int compressBlockSize, OutputStream output) { + this.valLen = valLen; + this.numValInBlock = compressBlockSize / valLen; + this.blockCnt = rowCnt / numValInBlock; + if (rowCnt % numValInBlock != 0) { + blockCnt++; + } + this.writeBuffer = ByteBuffer.allocate(numValInBlock * (valLen + 8) + 4); + this.dataOutput = new DataOutputStream(output); + this.blockDataWriter = new GeneralColumnDataWriter(blockCnt, dataOutput); + this.entryIndex = Lists.newArrayListWithCapacity(512); + } + + public void write(byte[] valBytes) throws IOException { + blockWriteCnt++; + boolean lastVal = (blockWriteCnt == numValInBlock); + if (previousVal == null) { + previousVal = valBytes; + if (lastVal) { + writeEntry(numOfVals, previousVal); + writeBlockData(); + } + } else { + boolean same = Bytes.compareTo(previousVal, valBytes) == 0; + if (same) { + numOfVals++; + } else { + writeEntry(numOfVals, previousVal); + } + previousVal = valBytes; + if (lastVal) { + if (same) { + writeEntry(numOfVals, previousVal); + } else { + writeEntry(1, valBytes); + } + + writeBlockData(); + } + } + } + + private void writeEntry(int numOfVals, byte[] val) { + writeBuffer.putInt(numOfVals); + writeBuffer.put(val); + totalWriteCntBeforeTheEntry += numOfVals; + entryIndex.add(totalWriteCntBeforeTheEntry - 1); + entryNum++; + this.numOfVals = 1; + } + + private void writeBlockData() throws IOException { + writeEntriesIndex(); + blockDataWriter.write(writeBuffer.array(), 0, writeBuffer.position()); + + writeBuffer.rewind(); + previousVal = null; + numOfVals = 1; + blockWriteCnt = 0; + totalWriteCntBeforeTheEntry = 0; + entryNum = 0; + entryIndex.clear(); + } + + private void writeEntriesIndex() { + for (int indexVal : entryIndex) { + writeBuffer.putInt(indexVal); + } + writeBuffer.putInt(entryNum); + } + + public void flush() throws IOException { + if (blockWriteCnt > 0) { + writeEntry(numOfVals, previousVal); + writeBlockData(); + } + blockDataWriter.flush(); + dataOutput.writeInt(numValInBlock); + dataOutput.writeInt(valLen); + dataOutput.flush(); + writeBuffer = null; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexSearcher.java new file mode 100644 index 00000000000..113724a9efa --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexSearcher.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.dimension.DimensionEncoding; +import org.roaringbitmap.buffer.ImmutableRoaringBitmap; + +public class ColInvertIndexSearcher { + + private ByteBuffer bitmapBuffer; + + private int bitmapStartOffset; + + private IOffsetDictionary offsetDictionary; + + public ColInvertIndexSearcher() { + } + + /** + * Load the Inverted index bitmap. + * + * @param buffer + * @return + * @throws IOException + */ + public static ColInvertIndexSearcher load(ByteBuffer buffer) throws IOException { + ColInvertIndexSearcher result = new ColInvertIndexSearcher(); + result.bitmapStartOffset = buffer.position(); + int type = buffer.getInt(buffer.limit() - 4); + if (type == ColInvertIndexWriter.II_DICT_TYPE_SEQ) { + result.offsetDictionary = new SeqOffsetDictionary(buffer); + } else { + result.offsetDictionary = new SortValueOffsetDictionary(buffer); + } + result.bitmapBuffer = buffer; + return result; + } + + public ImmutableRoaringBitmap searchValue(byte[] value) { + int offset = offsetDictionary.getBitMapOffset(value); + if (offset == -1) { + return null; + } + ByteBuffer usedBuffer = bitmapBuffer.asReadOnlyBuffer(); + usedBuffer.position(bitmapStartOffset + offset); + ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(usedBuffer); + + return bitmap; + } + + public interface IOffsetDictionary { + int getBitMapOffset(byte[] value); + } + + public static class SeqOffsetDictionary implements IOffsetDictionary { + private int minVal; + private int maxVal; + private boolean hasNullVal = false; + private ByteBuffer offsetBuffer; + private int dictStartOffset; + private int cardinality; + + SeqOffsetDictionary(ByteBuffer byteBuffer) { + this.offsetBuffer = byteBuffer; + + byteBuffer.position(byteBuffer.limit() - ColInvertIndexWriter.II_DICT_TYPE_SEQ_FOOT_LEN); + this.cardinality = byteBuffer.getInt(); + this.minVal = byteBuffer.getInt(); + this.maxVal = byteBuffer.getInt(); + this.dictStartOffset = byteBuffer.limit() - ColInvertIndexWriter.II_DICT_TYPE_SEQ_FOOT_LEN + - (cardinality << 2); + byte nullByte = byteBuffer.get(); + if (nullByte != 0) { + hasNullVal = true; + } + } + + @Override + public int getBitMapOffset(byte[] value) { + if (value == null || DimensionEncoding.isNull(value, 0, value.length)) { + if (!hasNullVal) { + return -1; + } else { + return offsetBuffer.getInt(dictStartOffset + ((cardinality - 1) << 2)); + } + } else { + int intVal = Bytes.readAsInt(value, 0, value.length); + if (intVal < minVal || intVal > maxVal) { + return -1; + } + int idx = intVal - minVal; + return offsetBuffer.getInt(dictStartOffset + (idx << 2)); + } + } + } + + public static class SortValueOffsetDictionary implements IOffsetDictionary { + private int valueLen; + private ByteBuffer offsetBuffer; + private int cardinality; + private int dictStartOffset; + + SortValueOffsetDictionary(ByteBuffer byteBuffer) { + this.offsetBuffer = byteBuffer; + byteBuffer.position(byteBuffer.limit() - ColInvertIndexWriter.II_DICT_TYPE_SORT_VAL_FOOT_LEN); + this.cardinality = byteBuffer.getInt(); + this.valueLen = byteBuffer.getInt(); + this.dictStartOffset = byteBuffer.limit() - ColInvertIndexWriter.II_DICT_TYPE_SORT_VAL_FOOT_LEN + - (cardinality * (valueLen + 4)); + } + + @Override + public int getBitMapOffset(byte[] value) { + if (value == null) { + value = getNullValue(); + } + return binarySearch(value); + } + + private int binarySearch(byte[] value) { + int low = 0; + int high = cardinality - 1; + byte[] currVal = new byte[valueLen]; + while (low <= high) { + int mid = (low + high) >>> 1; + offsetBuffer.position(dictStartOffset + mid * (valueLen + 4)); + offsetBuffer.get(currVal); + int cmp = Bytes.compareTo(value, currVal); + if (cmp == 0) { + return offsetBuffer.getInt(); + } else if (cmp < 0) { + high = mid - 1; + } else { + low = mid + 1; + } + } + return -1; + } + + private byte[] getNullValue() { + byte[] result = new byte[valueLen]; + for (int i = 0; i < valueLen; i++) { + result[i] = DimensionEncoding.NULL; + } + return result; + } + + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriter.java new file mode 100644 index 00000000000..683ec18dfb1 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.io.IOException; +import java.io.OutputStream; + +public abstract class ColInvertIndexWriter { + public static final byte II_DICT_TYPE_SEQ = 0; + public static final byte II_DICT_TYPE_SORT_VAL = 1; + public static final int II_DICT_TYPE_SEQ_FOOT_LEN = 17; + public static final int II_DICT_TYPE_SORT_VAL_FOOT_LEN = 12; + + private String colName; + + public ColInvertIndexWriter(String colName) { + this.colName = colName; + } + + public String getColName() { + return colName; + } + + public abstract void addValue(byte[] value); + + public abstract void write(OutputStream out) throws IOException; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenColInvertIndexWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenColInvertIndexWriter.java new file mode 100644 index 00000000000..8e13c3bd85d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenColInvertIndexWriter.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.util.Bytes; +import org.roaringbitmap.buffer.MutableRoaringBitmap; + +/** + * not thread safe + */ +public class FixLenColInvertIndexWriter extends ColInvertIndexWriter { + private int valueLenInBytes; + + private int rows; + private Map valueBitmaps = new TreeMap<>(new Bytes.ByteArrayComparator()); + + public FixLenColInvertIndexWriter(String columnName, int valueLenInBytes) { + super(columnName); + this.valueLenInBytes = valueLenInBytes; + } + + @Override + public void addValue(byte[] value) { + if (value.length != valueLenInBytes) { + throw new IllegalArgumentException("the value:" + Bytes.toHex(value) + " is not valid."); + } + MutableRoaringBitmap bitmap = valueBitmaps.get(value); + if (bitmap == null) { + bitmap = new MutableRoaringBitmap(); + valueBitmaps.put(value, bitmap); + } + bitmap.add(++rows); + } + + @Override + public void write(OutputStream out) throws IOException { + int cardinality = valueBitmaps.size(); + int footLen = II_DICT_TYPE_SORT_VAL_FOOT_LEN; + int dictAndFootLen = cardinality * (valueLenInBytes + 4) + footLen; + + DataOutputStream bitmapOut = new DataOutputStream(out); + ByteBuffer dictBuffer = ByteBuffer.allocate(dictAndFootLen); + int offset = 0; + for (Map.Entry bitmapEntry : valueBitmaps.entrySet()) { + byte[] colValue = bitmapEntry.getKey(); + MutableRoaringBitmap bitmap = bitmapEntry.getValue(); + bitmap.runOptimize(); + int bitmapSize = bitmap.serializedSizeInBytes(); + bitmap.serialize(bitmapOut); + + dictBuffer.put(colValue); + dictBuffer.putInt(offset); + offset += bitmapSize; + } + + dictBuffer.putInt(cardinality); + dictBuffer.putInt(valueLenInBytes); + dictBuffer.putInt(II_DICT_TYPE_SORT_VAL); // type + out.write(dictBuffer.array(), 0, dictBuffer.position()); + + out.flush(); + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenIIColumnDescriptor.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenIIColumnDescriptor.java new file mode 100644 index 00000000000..e6110926391 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/FixLenIIColumnDescriptor.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +public class FixLenIIColumnDescriptor extends IIColumnDescriptor { + private ColInvertIndexWriter writer; + + public FixLenIIColumnDescriptor(String colName, int fixedLength) { + super(colName); + this.writer = new FixLenColInvertIndexWriter(getColName(), fixedLength); + } + + @Override + public ColInvertIndexWriter getWriter() { + return writer; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IIColumnDescriptor.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IIColumnDescriptor.java new file mode 100644 index 00000000000..bb369d6ce8a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IIColumnDescriptor.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +public abstract class IIColumnDescriptor { + private String colName; + + public IIColumnDescriptor(String colName) { + this.colName = colName; + } + + public String getColName() { + return colName; + } + + abstract public ColInvertIndexWriter getWriter(); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IndexSearchResult.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IndexSearchResult.java new file mode 100644 index 00000000000..2080d3d6c61 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/IndexSearchResult.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.util.Iterator; + +public class IndexSearchResult { + public boolean allMatch = false; + public Iterator rows; + + public boolean needFullScan() { + return allMatch; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/InvertIndexSearcher.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/InvertIndexSearcher.java new file mode 100644 index 00000000000..90b69ad6b2a --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/InvertIndexSearcher.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.ConstantTupleFilter; +import org.apache.kylin.metadata.filter.LogicalTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.columnar.protocol.CuboidMetaInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.DimensionMetaInfo; +import org.roaringbitmap.buffer.ImmutableRoaringBitmap; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class InvertIndexSearcher { + Map colIndexSearchers = Maps.newHashMap(); + + public InvertIndexSearcher(CuboidMetaInfo cuboidMetaInfo, TblColRef[] cols, ByteBuffer idxBuffer) + throws IOException { + Map> columnMetas = Maps.newHashMap(); + for (DimensionMetaInfo dimensionInfo : cuboidMetaInfo.getDimensionsInfo()) { + for (TblColRef col : cols) { + if (dimensionInfo.getName().equals(col.getName())) { + columnMetas + .put(col.getName(), new Pair<>(dimensionInfo.getStartOffset() + dimensionInfo.getDataLength(), dimensionInfo.getIndexLength())); + } + } + } + for (Map.Entry> columnMeta : columnMetas.entrySet()) { + String colName = columnMeta.getKey(); + Pair positionInfo = columnMeta.getValue(); + int offset = positionInfo.getFirst(); + int length = positionInfo.getSecond(); + //start offset of this column + ByteBuffer colIdxBuf = idxBuffer.asReadOnlyBuffer(); + colIdxBuf.position(offset); + colIdxBuf.limit(offset + length); + ColInvertIndexSearcher colIndexSearcher = ColInvertIndexSearcher.load(colIdxBuf); + colIndexSearchers.put(colName, colIndexSearcher); + } + } + + /** + * + * @param tupleFilter + * @return null if the tupleFilter is null, or filter cannot be applied + */ + public IndexSearchResult search(TupleFilter tupleFilter) { + if (tupleFilter == null) + return null; + EvalResult evalResult = doEval(tupleFilter); + IndexSearchResult result = new IndexSearchResult(); + result.allMatch = evalResult.allMatch; + if (evalResult.bitmap != null) { + result.rows = evalResult.bitmap.iterator(); + } + return result; + } + + public EvalResult doEval(TupleFilter filter) { + if (filter == null) + return EvalResult.ALL_MATCH; + + if (filter instanceof LogicalTupleFilter) + return doEvalLogical((LogicalTupleFilter) filter); + + if (filter instanceof CompareTupleFilter) + return doEvalCompare((CompareTupleFilter) filter); + + if (filter instanceof ConstantTupleFilter) { + if (filter.getValues().size() == 0) { + return new EvalResult(); + } else if (filter.getValues().size() > 0) { + return EvalResult.ALL_MATCH; + } + } + + return EvalResult.ALL_MATCH; // unable to evaluate + } + + private EvalResult doEvalCompare(CompareTupleFilter filter) { + switch (filter.getOperator()) { + case EQ: + return doEvalCompareEqual(filter); + case IN: + return doEvalCompareIn(filter); + case ISNULL: + return doEvalCompareIsNull(filter); + case ISNOTNULL: + return doEvalCompareIsNotNull(filter); + case NEQ: + return doEvalCompareNotEqual(filter); + case NOTIN: + return doEvalCompareNotIn(filter); + case LT: + return doEvalCompareLT(filter); + case LTE: + return doEvalCompareLTE(filter); + case GT: + return doEvalCompareGT(filter); + case GTE: + return doEvalCompareGTE(filter); + default: + throw new IllegalStateException("Unsupported operator " + filter.getOperator()); + } + } + + private EvalResult doEvalCompareGTE(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareGT(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareLTE(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareLT(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareNotIn(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareNotEqual(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareIsNotNull(CompareTupleFilter filter) { + return EvalResult.ALL_MATCH; + } + + private EvalResult doEvalCompareIsNull(CompareTupleFilter filter) { + EvalResult result = new EvalResult(); + String column = filter.getColumn().getName(); + ColInvertIndexSearcher colSearcher = colIndexSearchers.get(column); + if (colSearcher == null) { + return EvalResult.ALL_MATCH; + } + ImmutableRoaringBitmap bitmap = colSearcher.searchValue(null); + if (bitmap != null) { + result.bitmap = bitmap; + } + return result; + } + + private EvalResult doEvalCompareEqual(CompareTupleFilter filter) { + EvalResult result = new EvalResult(); + String column = filter.getColumn().getName(); + byte[] value = null; + if (filter.getFirstValue() instanceof ByteArray) { + value = ((ByteArray)filter.getFirstValue()).array(); + } else if (filter.getFirstValue() instanceof byte[]) { + value = (byte[])filter.getFirstValue(); + } else if (filter.getFirstValue() instanceof String) { + value = Bytes.toBytes((String) filter.getFirstValue()); + } + ColInvertIndexSearcher colSearcher = colIndexSearchers.get(column); + if (colSearcher == null) { + return EvalResult.ALL_MATCH; + } + ImmutableRoaringBitmap bitmap = colSearcher.searchValue(value); + if (bitmap != null) { + result.bitmap = bitmap; + } + return result; + } + + private EvalResult doEvalCompareIn(CompareTupleFilter filter) { + EvalResult result = new EvalResult(); + String column = filter.getColumn().getName(); + ColInvertIndexSearcher colSearcher = colIndexSearchers.get(column); + if (colSearcher == null) { + return EvalResult.ALL_MATCH; + } + List bitmaps = Lists.newArrayList(); + for (Object value : filter.getValues()) { + byte[] bytes = null; + if (value instanceof ByteArray) { + bytes = ((ByteArray)value).array(); + } else if (value instanceof byte[]) { + bytes = (byte[])value; + } else if (value instanceof String) { + bytes = Bytes.toBytes((String)value); + } + ImmutableRoaringBitmap bitmap = colSearcher.searchValue(bytes); + if (bitmap != null) { + bitmaps.add(bitmap); + } + } + if (bitmaps.isEmpty()) { + return result; + } + + result.bitmap = ImmutableRoaringBitmap.or(bitmaps.toArray(new ImmutableRoaringBitmap[bitmaps.size()])); + return result; + } + + private EvalResult doEvalLogical(LogicalTupleFilter filter) { + List children = filter.getChildren(); + + switch (filter.getOperator()) { + case AND: + return doEvalLogicalAnd(children); + case OR: + return doEvalLogicalOr(children); + case NOT: + return doEvalLogicalNot(children); + default: + throw new IllegalStateException("Unsupported operator " + filter.getOperator()); + } + } + + private EvalResult doEvalLogicalAnd(List children) { + EvalResult result = new EvalResult(); + List childResults = Lists.newArrayList(); + for (TupleFilter child : children) { + EvalResult childResult = doEval(child); + childResults.add(childResult); + if (childResult.isNoneMatch()) { + break; + } + } + boolean childrenAllMatched = true; + for (EvalResult childResult : childResults) { + if (childResult.isNoneMatch()) { + return new EvalResult(); + } + if (childResult.isAllMatch()) { + continue; + } + childrenAllMatched = false; + ImmutableRoaringBitmap childBitmap = childResult.getBitmap(); + if (result.bitmap == null) { + result.bitmap = childBitmap; + } else { + result.bitmap = ImmutableRoaringBitmap.and(result.bitmap, childBitmap); + } + } + if (childrenAllMatched) { + result.setAllMatch(true); + } + return result; + } + + private EvalResult doEvalLogicalOr(List children) { + EvalResult result = new EvalResult(); + List childResults = Lists.newArrayList(); + for (TupleFilter child : children) { + EvalResult childResult = doEval(child); + childResults.add(childResult); + if (childResult.isAllMatch()) { + break; + } + } + for (EvalResult childResult : childResults) { + if (childResult.isAllMatch()) { + return EvalResult.ALL_MATCH; + } + if (childResult.isNoneMatch()) { + continue; + } + ImmutableRoaringBitmap childBitmap = childResult.getBitmap(); + if (result.bitmap == null) { + result.bitmap = childBitmap; + } else { + result.bitmap = ImmutableRoaringBitmap.or(result.bitmap, childBitmap); + } + } + return result; + } + + private EvalResult doEvalLogicalNot(List children) { + return EvalResult.ALL_MATCH; + } + + private static class EvalResult { + public static final EvalResult ALL_MATCH = new EvalResult(true, null); + private boolean allMatch = false; + private ImmutableRoaringBitmap bitmap; + + public EvalResult() { + this(false, null); + } + + public EvalResult(boolean allMatch, ImmutableRoaringBitmap bitmap) { + this.allMatch = allMatch; + this.bitmap = bitmap; + } + + public boolean isAllMatch() { + return this.allMatch; + } + + public void setAllMatch(boolean allMatch) { + this.allMatch = allMatch; + } + + public boolean isNoneMatch() { + return !this.allMatch && this.bitmap == null; + } + + public ImmutableRoaringBitmap getBitmap() { + return this.bitmap; + } + + public void setBitmap(ImmutableRoaringBitmap bitmap) { + this.bitmap = bitmap; + } + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqColInvertIndexWriter.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqColInvertIndexWriter.java new file mode 100644 index 00000000000..632c47dcf2b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqColInvertIndexWriter.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.dimension.DimensionEncoding; +import org.roaringbitmap.buffer.MutableRoaringBitmap; + +/** + * not thread safe + */ +public class SeqColInvertIndexWriter extends ColInvertIndexWriter { + private int minColVal; + private int maxColVal; + + private MutableRoaringBitmap[] valueBitmaps; + private MutableRoaringBitmap nullValueBitmap; + + private int rows; + + public SeqColInvertIndexWriter(String columnName, int minColVal, int maxColVal) { + super(columnName); + this.minColVal = minColVal; + this.maxColVal = maxColVal; + this.valueBitmaps = new MutableRoaringBitmap[maxColVal - minColVal + 1]; + } + + @Override + public void addValue(byte[] value) { + if (value == null || DimensionEncoding.isNull(value, 0, value.length)) { + if (nullValueBitmap == null) { + nullValueBitmap = new MutableRoaringBitmap(); + } + nullValueBitmap.add(++rows); + } else { + int intVal = Bytes.readAsInt(value, 0, value.length); + if (intVal < minColVal || intVal > maxColVal) { + throw new IllegalArgumentException("the value:" + intVal + " is not in (" + minColVal + "," + maxColVal + + ")"); + } + int idx = intVal - minColVal; + MutableRoaringBitmap bitmap = valueBitmaps[idx]; + if (bitmap == null) { + bitmap = new MutableRoaringBitmap(); + valueBitmaps[idx] = bitmap; + } + bitmap.add(++rows); + } + } + + @Override + public void write(OutputStream out) throws IOException { + int cardinality = valueBitmaps.length; + if (nullValueBitmap != null) { + cardinality++; + } + int dictionaryPartLen = cardinality * 4; + + DataOutputStream bitmapOut = new DataOutputStream(out); + int footAndDictLen = II_DICT_TYPE_SEQ_FOOT_LEN + dictionaryPartLen; + ByteBuffer footBuffer = ByteBuffer.allocate(footAndDictLen); + + int offset = 0; + for (MutableRoaringBitmap bitmap : valueBitmaps) { + bitmap.runOptimize(); + int bitmapSize = bitmap.serializedSizeInBytes(); + bitmap.serialize(bitmapOut); + + footBuffer.putInt(offset); + offset += bitmapSize; + } + if (nullValueBitmap != null) { + nullValueBitmap.serialize(bitmapOut); + footBuffer.putInt(offset); + } + + footBuffer.putInt(cardinality); // cardinality + footBuffer.putInt(minColVal); + footBuffer.putInt(maxColVal); + footBuffer.putInt(II_DICT_TYPE_SEQ);// type + if (nullValueBitmap != null) { + footBuffer.put((byte) 1); + } else { + footBuffer.put((byte) 0); + } + out.write(footBuffer.array(), 0, footBuffer.position()); + + out.flush(); + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqIIColumnDescriptor.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqIIColumnDescriptor.java new file mode 100644 index 00000000000..b8b5f2c4293 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SeqIIColumnDescriptor.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +public class SeqIIColumnDescriptor extends IIColumnDescriptor { + private ColInvertIndexWriter writer; + + public SeqIIColumnDescriptor(String colName, int minVal, int maxVal) { + super(colName); + this.writer = new SeqColInvertIndexWriter(getColName(), minVal, maxVal); + } + + @Override + public ColInvertIndexWriter getWriter() { + return writer; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/CuboidMetaInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/CuboidMetaInfo.java new file mode 100644 index 00000000000..cc870601bcb --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/CuboidMetaInfo.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +import java.util.List; + +/** + * the cuboid metadata in the fragment + * + */ +public class CuboidMetaInfo { + // Dimension info, data are divided per dimension. Each dimension contains + // three types of data: dictionary, raw data and inverted indexes. + private List dimensionsInfo; + + // Metric info, different metric data stores in different position of the file + private List metricsInfo; + + // Number of dimensions + private int numberOfDim; + + // Number of metrics + private int numberOfMetrics; + + // Number of rows in the fragment cuboid data + private long numberOfRows; + + public List getDimensionsInfo() { + return dimensionsInfo; + } + + public void setDimensionsInfo(List dimensionsInfo) { + this.dimensionsInfo = dimensionsInfo; + } + + public List getMetricsInfo() { + return metricsInfo; + } + + public void setMetricsInfo(List metricsInfo) { + this.metricsInfo = metricsInfo; + } + + public int getNumberOfMetrics() { + return numberOfMetrics; + } + + public void setNumberOfMetrics(int numberOfMetrics) { + this.numberOfMetrics = numberOfMetrics; + } + + public int getNumberOfDim() { + return numberOfDim; + } + + public void setNumberOfDim(int numberOfDim) { + this.numberOfDim = numberOfDim; + } + + public long getNumberOfRows() { + return numberOfRows; + } + + public void setNumberOfRows(long numberOfRows) { + this.numberOfRows = numberOfRows; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CuboidMetaInfo that = (CuboidMetaInfo) o; + + if (numberOfDim != that.numberOfDim) + return false; + if (numberOfMetrics != that.numberOfMetrics) + return false; + if (numberOfRows != that.numberOfRows) + return false; + if (dimensionsInfo != null ? !dimensionsInfo.equals(that.dimensionsInfo) : that.dimensionsInfo != null) + return false; + return metricsInfo != null ? metricsInfo.equals(that.metricsInfo) : that.metricsInfo == null; + + } + + @Override + public int hashCode() { + int result = dimensionsInfo != null ? dimensionsInfo.hashCode() : 0; + result = 31 * result + (metricsInfo != null ? metricsInfo.hashCode() : 0); + result = 31 * result + numberOfDim; + result = 31 * result + numberOfMetrics; + result = 31 * result + (int) (numberOfRows ^ (numberOfRows >>> 32)); + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimDictionaryMetaInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimDictionaryMetaInfo.java new file mode 100644 index 00000000000..4d397107938 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimDictionaryMetaInfo.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +public class DimDictionaryMetaInfo { + // Name of the dimension column + private String dimName; + + // The start of the dimension dictionary data within the file + private int startOffset; + + // Dictionary type + private String dictType; + + // Dictionary part length in bytes + private int dictLength; + + public String getDimName() { + return dimName; + } + + public void setDimName(String dimName) { + this.dimName = dimName; + } + + public int getStartOffset() { + return startOffset; + } + + public void setStartOffset(int startOffset) { + this.startOffset = startOffset; + } + + public String getDictType() { + return dictType; + } + + public void setDictType(String dictType) { + this.dictType = dictType; + } + + public int getDictLength() { + return dictLength; + } + + public void setDictLength(int dictLength) { + this.dictLength = dictLength; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + DimDictionaryMetaInfo that = (DimDictionaryMetaInfo) o; + + if (startOffset != that.startOffset) + return false; + if (dictLength != that.dictLength) + return false; + if (dimName != null ? !dimName.equals(that.dimName) : that.dimName != null) + return false; + return dictType != null ? dictType.equals(that.dictType) : that.dictType == null; + + } + + @Override + public int hashCode() { + int result = dimName != null ? dimName.hashCode() : 0; + result = 31 * result + startOffset; + result = 31 * result + (dictType != null ? dictType.hashCode() : 0); + result = 31 * result + dictLength; + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimensionMetaInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimensionMetaInfo.java new file mode 100644 index 00000000000..017bbc43b9b --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/DimensionMetaInfo.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.kylin.stream.core.storage.columnar.compress.Compression; + +public class DimensionMetaInfo { + // Name of the dimension column + private String name; + + // Compression type + private String compression; + + // include null values or not + private boolean hasNull; + + // The start of the dimension data within the file + private int startOffset; + + // Data part length in bytes + private int dataLength; + + // Index part length in bytes + private int indexLength; + + // Magic number for future use. + private int magic; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public boolean isHasNull() { + return hasNull; + } + + public void setHasNull(boolean hasNull) { + this.hasNull = hasNull; + } + + public int getStartOffset() { + return startOffset; + } + + public void setStartOffset(int startOffset) { + this.startOffset = startOffset; + } + + public int getDataLength() { + return dataLength; + } + + public void setDataLength(int dataLength) { + this.dataLength = dataLength; + } + + public int getIndexLength() { + return indexLength; + } + + public void setIndexLength(int indexLength) { + this.indexLength = indexLength; + } + + public int getMagic() { + return magic; + } + + public void setMagic(int magic) { + this.magic = magic; + } + + public String getCompression() { + return compression; + } + + @JsonIgnore + public Compression getCompressionType() { + return Compression.valueOf(compression); + } + + public void setCompression(String compression) { + this.compression = compression; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + DimensionMetaInfo that = (DimensionMetaInfo) o; + + if (hasNull != that.hasNull) + return false; + if (startOffset != that.startOffset) + return false; + if (dataLength != that.dataLength) + return false; + if (indexLength != that.indexLength) + return false; + if (magic != that.magic) + return false; + return name != null ? name.equals(that.name) : that.name == null; + + } + + @Override + public int hashCode() { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (hasNull ? 1 : 0); + result = 31 * result + startOffset; + result = 31 * result + dataLength; + result = 31 * result + indexLength; + result = 31 * result + magic; + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/Footer.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/Footer.java new file mode 100644 index 00000000000..f9561c3310c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/Footer.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +/** + * The Footer section of the file provides the necessary information to interpret the rest of the file including the version info and the length of the segment meta. + * + * + */ +public class Footer { + //Version of the storage format for the segment + private int version; + + //The start of the segmentMetaInfo within the file + private long segmentInfoOffSet; + + //The length of the segmentMetaInfo in bytes + private long segmentInfoLength; + + public int getVersion() { + return version; + } + + public void setVersion(int version) { + this.version = version; + } + + public long getSegmentInfoOffSet() { + return segmentInfoOffSet; + } + + public void setSegmentInfoOffSet(long segmentInfoOffSet) { + this.segmentInfoOffSet = segmentInfoOffSet; + } + + public long getSegmentInfoLength() { + return segmentInfoLength; + } + + public void setSegmentInfoLength(long segmentInfoLength) { + this.segmentInfoLength = segmentInfoLength; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) (segmentInfoLength ^ (segmentInfoLength >>> 32)); + result = prime * result + (int) (segmentInfoOffSet ^ (segmentInfoOffSet >>> 32)); + result = prime * result + version; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Footer other = (Footer) obj; + if (segmentInfoLength != other.segmentInfoLength) + return false; + if (segmentInfoOffSet != other.segmentInfoOffSet) + return false; + if (version != other.version) + return false; + return true; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/FragmentMetaInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/FragmentMetaInfo.java new file mode 100644 index 00000000000..aba61e47231 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/FragmentMetaInfo.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +import java.util.List; +import java.util.Map; + +/** + * The fragment metadata, it will be stored in the file in JSON format. It + * contains the layout of the body of the file. The body of the file is divided + * into different parts to organize the data in columnar format. + * + */ +public class FragmentMetaInfo { + + private List dimDictionaryMetaInfos; + + private CuboidMetaInfo basicCuboidMetaInfo; + + // Cuboids mataData info + private Map cuboidMetaInfoMap; + + // Number of rows in the fragment + private long numberOfRows; + + // Original rows number from source + private long originNumOfRows; + + // FragmentId + private String fragmentId; + + private long minEventTime; + + private long maxEventTime; + + // Version of the fragment + private int version; + + public List getDimDictionaryMetaInfos() { + return dimDictionaryMetaInfos; + } + + public void setDimDictionaryMetaInfos(List dimDictionaryMetaInfos) { + this.dimDictionaryMetaInfos = dimDictionaryMetaInfos; + } + + public Map getCuboidMetaInfoMap() { + return cuboidMetaInfoMap; + } + + public void setCuboidMetaInfoMap(Map cuboidMetaInfoMap) { + this.cuboidMetaInfoMap = cuboidMetaInfoMap; + } + + public CuboidMetaInfo getBasicCuboidMetaInfo() { + return basicCuboidMetaInfo; + } + + public void setBasicCuboidMetaInfo(CuboidMetaInfo basicCuboidMetaInfo) { + this.basicCuboidMetaInfo = basicCuboidMetaInfo; + } + + public CuboidMetaInfo getCuboidMetaInfo(long cuboidID) { + return cuboidMetaInfoMap.get(String.valueOf(cuboidID)); + } + + public long getNumberOfRows() { + return numberOfRows; + } + + public void setNumberOfRows(long numberOfRows) { + this.numberOfRows = numberOfRows; + } + + public long getOriginNumOfRows() { + return originNumOfRows; + } + + public void setOriginNumOfRows(long originNumOfRows) { + this.originNumOfRows = originNumOfRows; + } + + public String getFragmentId() { + return fragmentId; + } + + public void setFragmentId(String fragmentId) { + this.fragmentId = fragmentId; + } + + public int getVersion() { + return version; + } + + public void setVersion(int version) { + this.version = version; + } + + public long getMinEventTime() { + return minEventTime; + } + + public void setMinEventTime(long minEventTime) { + this.minEventTime = minEventTime; + } + + public long getMaxEventTime() { + return maxEventTime; + } + + public void setMaxEventTime(long maxEventTime) { + this.maxEventTime = maxEventTime; + } + + public boolean hasValidEventTimeRange() { + return minEventTime != 0 && maxEventTime != 0; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + FragmentMetaInfo that = (FragmentMetaInfo) o; + + if (numberOfRows != that.numberOfRows) + return false; + if (version != that.version) + return false; + if (dimDictionaryMetaInfos != null ? !dimDictionaryMetaInfos.equals(that.dimDictionaryMetaInfos) + : that.dimDictionaryMetaInfos != null) + return false; + if (basicCuboidMetaInfo != null ? !basicCuboidMetaInfo.equals(that.basicCuboidMetaInfo) + : that.basicCuboidMetaInfo != null) + return false; + if (cuboidMetaInfoMap != null ? !cuboidMetaInfoMap.equals(that.cuboidMetaInfoMap) + : that.cuboidMetaInfoMap != null) + return false; + return fragmentId != null ? fragmentId.equals(that.fragmentId) : that.fragmentId == null; + + } + + @Override + public int hashCode() { + int result = dimDictionaryMetaInfos != null ? dimDictionaryMetaInfos.hashCode() : 0; + result = 31 * result + (basicCuboidMetaInfo != null ? basicCuboidMetaInfo.hashCode() : 0); + result = 31 * result + (cuboidMetaInfoMap != null ? cuboidMetaInfoMap.hashCode() : 0); + result = 31 * result + (int) (numberOfRows ^ (numberOfRows >>> 32)); + result = 31 * result + (fragmentId != null ? fragmentId.hashCode() : 0); + result = 31 * result + version; + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/MetricMetaInfo.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/MetricMetaInfo.java new file mode 100644 index 00000000000..3c13a92d838 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/protocol/MetricMetaInfo.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.protocol; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.kylin.stream.core.storage.columnar.compress.Compression; + +public class MetricMetaInfo { + // Name of the measure column + private String name; + + // Compression type + private String compression; + + // Integer of the column index + private int col; + + // include null values or not + private boolean hasNull; + + // The start of the measure data within the file + private int startOffset; + + private int metricLength; + + //the max number of bytes to the longest possible serialization of this metric data type + private int maxSerializeLength; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getCol() { + return col; + } + + public void setCol(int col) { + this.col = col; + } + + public boolean isHasNull() { + return hasNull; + } + + public void setHasNull(boolean hasNull) { + this.hasNull = hasNull; + } + + public int getStartOffset() { + return startOffset; + } + + public void setStartOffset(int startOffset) { + this.startOffset = startOffset; + } + + public int getMetricLength() { + return metricLength; + } + + public void setMetricLength(int metricLength) { + this.metricLength = metricLength; + } + + public int getMaxSerializeLength() { + return this.maxSerializeLength; + } + + public void setMaxSerializeLength(int length) { + this.maxSerializeLength = length; + } + + public String getCompression() { + return compression; + } + + @JsonIgnore + public Compression getCompressionType() { + return Compression.valueOf(compression); + } + + public void setCompression(String compression) { + this.compression = compression; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + MetricMetaInfo that = (MetricMetaInfo) o; + + if (col != that.col) + return false; + if (hasNull != that.hasNull) + return false; + if (startOffset != that.startOffset) + return false; + if (metricLength != that.metricLength) + return false; + if (maxSerializeLength != that.maxSerializeLength) + return false; + if (name != null ? !name.equals(that.name) : that.name != null) + return false; + return compression != null ? compression.equals(that.compression) : that.compression == null; + + } + + @Override + public int hashCode() { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (compression != null ? compression.hashCode() : 0); + result = 31 * result + col; + result = 31 * result + (hasNull ? 1 : 0); + result = 31 * result + startOffset; + result = 31 * result + metricLength; + result = 31 * result + maxSerializeLength; + return result; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/rocksdb/RocksDBSegmentStore.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/rocksdb/RocksDBSegmentStore.java new file mode 100644 index 00000000000..1fe107d4170 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/rocksdb/RocksDBSegmentStore.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.rocksdb; + +import java.io.File; +import java.io.IOException; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.model.stats.SegmentStoreStats; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.IStreamingSegmentStore; +import org.apache.kylin.stream.core.storage.StreamingCubeSegment; +import org.rocksdb.FlushOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RocksDBSegmentStore implements IStreamingSegmentStore { + private static Logger logger = LoggerFactory.getLogger(RocksDBSegmentStore.class); + + private File dataSegmentFolder; + + private CubeInstance cubeInstance; + private String cubeName; + private String segmentName; + private RocksDB db; + + public RocksDBSegmentStore(String baseStorePath, CubeInstance cubeInstance, String segmentName) { + this.cubeInstance = cubeInstance; + this.cubeName = cubeInstance.getName(); + this.segmentName = segmentName; + + this.dataSegmentFolder = new File(baseStorePath + File.separator + cubeName + File.separator + segmentName); + if (!dataSegmentFolder.exists()) { + dataSegmentFolder.mkdirs(); + } + } + + @Override + public void init() { + Options options = getOptions().setCreateIfMissing(true); + try { + String dataPath = dataSegmentFolder.getAbsolutePath() + "/data"; + db = RocksDB.open(options, dataPath); + } catch (RocksDBException e) { + logger.error("init rocks db fail"); + } + } + + private Options getOptions() { + return new Options(); + } + + @Override + public int addEvent(StreamingMessage event) { + return 0; + } + + @Override + public File getStorePath() { + return null; + } + + @Override + public void persist() { + try { + db.flush(new FlushOptions()); + } catch (RocksDBException e) { + e.printStackTrace(); + } + } + + @Override + public Object checkpoint() { + return null; + } + + @Override + public void purge() { + + } + + @Override + public void restoreFromCheckpoint(Object checkpoint) { + + } + + @Override + public String getSegmentName() { + return null; + } + + @Override + public StreamingCubeSegment.State getSegmentState() { + return null; + } + + @Override + public void setSegmentState(StreamingCubeSegment.State state) { + + } + + @Override + public void search(final StreamingSearchContext searchContext, ResultCollector collector) throws IOException { + } + + @Override + public SegmentStoreStats getStoreStats() { + return null; + } + + @Override + public void close() throws IOException { + + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/CompareFilterTimeRangeChecker.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/CompareFilterTimeRangeChecker.java new file mode 100644 index 00000000000..5be5fd19acf --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/CompareFilterTimeRangeChecker.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.util; + +import java.util.Collection; +import java.util.Set; + +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.metadata.filter.CompareTupleFilter; + +/** + * check the time related filter is in a specify time range or not + */ +public class CompareFilterTimeRangeChecker { + public enum CheckResult { + INCLUDED, EXCLUDED, OVERLAP; + + public CheckResult or(CheckResult other) { + if (this == INCLUDED || other == INCLUDED) { + return INCLUDED; + } + if (this == EXCLUDED && other == EXCLUDED) { + return EXCLUDED; + } + return OVERLAP; + } + } + + protected long timeStart; + protected long timeEnd; + private boolean endClose; + + public CompareFilterTimeRangeChecker(long timeStart, long timeEnd) { + this(timeStart, timeEnd, false); + } + + public CompareFilterTimeRangeChecker(long timeStart, long timeEnd, boolean endClose) { + this.timeStart = timeStart; + this.timeEnd = timeEnd; + this.endClose = endClose; + } + + public CheckResult check(CompareTupleFilter compFilter, TimeDerivedColumnType timeDerivedColumnType) { + Object timestampValue = compFilter.getFirstValue(); + Set conditionValues = compFilter.getValues(); + Pair timeUnitRange = timeDerivedColumnType.getTimeUnitRange(timestampValue); + switch (compFilter.getOperator()) { + case EQ: + return checkForEqValue(timeUnitRange); + case NEQ: + if (timeUnitRange.getFirst() <= timeStart && timeUnitRange.getSecond() >= timeEnd) { + return CheckResult.EXCLUDED; + } + if (timeUnitRange.getSecond() <= timeStart || timeUnitRange.getFirst() >= timeEnd) { + return CheckResult.INCLUDED; + } + return CheckResult.OVERLAP; + case LT: + if ((!endClose && timeUnitRange.getFirst() >= timeEnd) || (endClose && timeUnitRange.getFirst() > timeEnd)) { + return CheckResult.INCLUDED; + } + if (timeUnitRange.getFirst() <= timeStart) { + return CheckResult.EXCLUDED; + } + return CheckResult.OVERLAP; + case LTE: + if (timeUnitRange.getFirst() >= timeEnd) { + return CheckResult.INCLUDED; + } + if (timeUnitRange.getSecond() < timeStart) { + return CheckResult.EXCLUDED; + } + return CheckResult.OVERLAP; + case GT: + if (timeUnitRange.getSecond() < timeStart) { + return CheckResult.INCLUDED; + } + if (timeUnitRange.getFirst() >= timeEnd) { + return CheckResult.EXCLUDED; + } + return CheckResult.OVERLAP; + case GTE: + if (timeUnitRange.getFirst() <= timeStart) { + return CheckResult.INCLUDED; + } + if ((!endClose && timeUnitRange.getFirst() >= timeEnd) || (endClose && timeUnitRange.getFirst() > timeEnd)) { + return CheckResult.EXCLUDED; + } + return CheckResult.OVERLAP; + case IN: + return checkForInValues(timeDerivedColumnType, conditionValues); + default: + return CheckResult.OVERLAP; + } + } + + private CheckResult checkForEqValue(Pair timeUnitRange) { + if (timeUnitRange.getFirst() <= timeStart && timeUnitRange.getSecond() >= timeEnd) { + return CheckResult.INCLUDED; + } + if (timeUnitRange.getSecond() <= timeStart || timeUnitRange.getFirst() >= timeEnd) { + return CheckResult.EXCLUDED; + } + return CheckResult.OVERLAP; + } + + private CheckResult checkForInValues(TimeDerivedColumnType timeDerivedColumnType, Collection values) { + CheckResult result = null; + for (Object timestampValue : values) { + Pair timeUnitRange = timeDerivedColumnType.getTimeUnitRange(timestampValue); + CheckResult checkResult = checkForEqValue(timeUnitRange); + if (result == null) { + result = checkResult; + } else { + result = result.or(checkResult); + } + } + return result; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/Constants.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/Constants.java new file mode 100644 index 00000000000..5ec8f9abf4e --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/Constants.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +public interface Constants { + String META_FILE_SUFFIX = ".meta"; + String DATA_FILE_SUFFIX = ".data"; +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/HDFSUtil.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/HDFSUtil.java new file mode 100644 index 00000000000..4912ac27faa --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/HDFSUtil.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.kylin.common.KylinConfig; + +public class HDFSUtil { + private static final ThreadLocal hadoopConfig = new ThreadLocal<>(); + + public static FileSystem getFileSystemForPath(String path) throws IOException { + return FileSystem.get(makeURI(path), getCurrentConfiguration()); + } + + private static URI makeURI(String filePath) { + try { + return new URI(fixWindowsPath(filePath)); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("Cannot create FileSystem from URI: " + filePath, e); + } + } + + private static String fixWindowsPath(String path) { + // fix windows path + if (path.startsWith("file://") && !path.startsWith("file:///") && path.contains(":\\")) { + path = path.replace("file://", "file:///"); + } + if (path.startsWith("file:///")) { + path = path.replace('\\', '/'); + } + return path; + } + + public static Configuration getCurrentConfiguration() { + if (hadoopConfig.get() == null) { + Configuration conf = healSickConfig(new Configuration()); + return conf; + } + return hadoopConfig.get(); + } + + private static Configuration healSickConfig(Configuration conf) { + // why we have this hard code? + conf.set(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, "8"); + + // https://issues.apache.org/jira/browse/KYLIN-953 + if (StringUtils.isBlank(conf.get("hadoop.tmp.dir"))) { + conf.set("hadoop.tmp.dir", "/tmp"); + } + if (StringUtils.isBlank(conf.get("hbase.fs.tmp.dir"))) { + conf.set("hbase.fs.tmp.dir", "/tmp"); + } + return conf; + } + + public static String getStreamingSegmentFilePath(String cubeName, String segmentName) { + return getStreamingCubeFilePath(cubeName) + "/" + segmentName; + } + + public static String getStreamingCubeFilePath(String cubeName) { + return KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "stream/" + cubeName; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/NamedThreadFactory.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/NamedThreadFactory.java new file mode 100644 index 00000000000..4631fd7ce4f --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/NamedThreadFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +public class NamedThreadFactory implements ThreadFactory { + private static AtomicInteger threadNumber = new AtomicInteger(1); + private final String namePrefix; + private final boolean daemon; + + public NamedThreadFactory(String namePrefix, boolean daemon) { + this.namePrefix = namePrefix; + this.daemon = daemon; + + } + + public NamedThreadFactory(String namePrefix) { + this(namePrefix, true); + } + + public Thread newThread(Runnable runnable) { + final Thread thread = new Thread(runnable, namePrefix + "-thread-" + threadNumber.getAndIncrement()); + thread.setDaemon(daemon); + return thread; + } + +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/NodeUtil.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/NodeUtil.java new file mode 100644 index 00000000000..ae3c3904182 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/NodeUtil.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.stream.core.model.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Map; + +public class NodeUtil { + private static final Logger logger = LoggerFactory.getLogger(NodeUtil.class); + + public static Node getCurrentNode(int defaultPort) { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + String configNodeStr = kylinConfig.getStreamingNode(); + Node result; + if (configNodeStr != null) { + result = Node.from(configNodeStr); + } else { + result = new Node(getLocalhostName(), defaultPort); + } + Map nodeProperties = kylinConfig.getStreamingNodeProperties(); + result.setProperties(nodeProperties); + return result; + } + + private static String getLocalhostName() { + String ip; + try { + InetAddress addr = InetAddress.getLocalHost(); + ip = addr.getHostAddress(); + } catch (UnknownHostException e) { + logger.error("Fail to get local ip address", e); + ip = "UNKNOWN"; + } + return ip; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/RecordsSerializer.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RecordsSerializer.java new file mode 100644 index 00000000000..640862be8bb --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RecordsSerializer.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.zip.DataFormatException; + +import org.apache.kylin.common.util.BytesUtil; +import org.apache.kylin.common.util.CompressionUtils; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.storage.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RecordsSerializer { + private static final Logger logger = LoggerFactory.getLogger(RecordsSerializer.class); + private static final int ROWVALUE_BUFFER_SIZE = 1024 * 1024; + + private ResponseResultSchema schema; + private DataTypeSerializer[] metricsSerializers; + + public RecordsSerializer(ResponseResultSchema schema) { + this.schema = schema; + DataType[] metricsDataTypes = schema.getMetricsDataTypes(); + this.metricsSerializers = new DataTypeSerializer[metricsDataTypes.length]; + for (int i = 0; i < metricsSerializers.length; i++) { + metricsSerializers[i] = DataTypeSerializer.create(metricsDataTypes[i]); + } + } + + public Pair serialize(Iterator records, int storagePushDownLimit) throws IOException { + ByteBuffer buffer = ByteBuffer.allocate(ROWVALUE_BUFFER_SIZE); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(ROWVALUE_BUFFER_SIZE); + long finalRowCnt = 0; + while (records.hasNext()) { + Record record = records.next(); + buffer.clear(); + serializeRecord(record, buffer); + buffer.flip(); + outputStream.write(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + finalRowCnt++; + if (finalRowCnt >= storagePushDownLimit) { + //read one more rexcord than limit + logger.info( + "The finalScanner aborted because storagePushDownLimit is satisfied, storagePushDownLimit is:{}", + storagePushDownLimit); + break; + } + } + + byte[] compressedAllRows = CompressionUtils.compress(outputStream.toByteArray()); + return new Pair<>(compressedAllRows, finalRowCnt); + } + + @SuppressWarnings("unchecked") + private void serializeRecord(Record record, ByteBuffer out) { + String[] dimValues = record.getDimensions(); + Object[] metricValues = record.getMetrics(); + for (int i = 0; i < dimValues.length; i++) { + BytesUtil.writeUTFString(dimValues[i], out); + } + for (int i = 0; i < metricValues.length; i++) { + metricsSerializers[i].serialize(metricValues[i], out); + } + } + + private void deserializeRecord(Record resultRecord, ByteBuffer in) { + for (int i = 0; i < schema.getDimensionCount(); i++) { + resultRecord.setDimension(i, BytesUtil.readUTFString(in)); + } + for (int i = 0; i < schema.getMetricsCount(); i++) { + resultRecord.setMetric(i, metricsSerializers[i].deserialize(in)); + } + } + + public Iterator deserialize(final byte[] recordsBytes) throws IOException, + DataFormatException { + final byte[] decompressedData = CompressionUtils.decompress(recordsBytes); + return new Iterator() { + private ByteBuffer inputBuffer = null; + private Record oneRecord = new Record(schema.getDimensionCount(), schema.getMetricsCount()); + + @Override + public boolean hasNext() { + if (inputBuffer == null) { + inputBuffer = ByteBuffer.wrap(decompressedData); + } + + return inputBuffer.position() < inputBuffer.limit(); + } + + @Override + public Record next() { + deserializeRecord(oneRecord, inputBuffer); + return oneRecord; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("UnSupport Operation"); + } + }; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/RestService.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RestService.java new file mode 100644 index 00000000000..1c75460ed4d --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RestService.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.io.IOException; + +import org.apache.http.HttpResponse; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpDelete; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.DefaultHttpClient; +import org.apache.http.params.BasicHttpParams; +import org.apache.http.params.HttpConnectionParams; +import org.apache.http.params.HttpParams; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RestService { + private static final Logger logger = LoggerFactory.getLogger(RestService.class); + + private int connectionTimeout; + private int readTimeout; + + public RestService(int connectionTimeout, int readTimeout) { + this.connectionTimeout = connectionTimeout; + this.readTimeout = readTimeout; + + } + + public String postRequest(String url, String postContent) throws IOException { + return postRequest(url, postContent, connectionTimeout, readTimeout); + } + + public String putRequest(String url) throws IOException { + return putRequest(url, null); + } + + public String putRequest(String url, String putContent) throws IOException { + return putRequest(url, putContent, connectionTimeout, readTimeout); + } + + public String getRequest(String url) throws IOException { + return getRequest(url, connectionTimeout, readTimeout); + } + + public String deleteRequest(String url) throws IOException { + return deleteRequest(url, connectionTimeout, readTimeout); + } + + public String postRequest(String url, String postContent, int connTimeout, int readTimeout) throws IOException { + HttpPost request = new HttpPost(url); + StringEntity requestEntity = new StringEntity(postContent, ContentType.APPLICATION_JSON); + request.setEntity(requestEntity); + return execRequest(request, connTimeout, readTimeout); + } + + public String putRequest(String url, String putContent, int connTimeout, int readTimeout) throws IOException { + HttpPut request = new HttpPut(url); + if (putContent != null) { + StringEntity requestEntity = new StringEntity(putContent, ContentType.APPLICATION_JSON); + request.setEntity(requestEntity); + } + return execRequest(request, connTimeout, readTimeout); + } + + public String getRequest(String url, int connTimeout, int readTimeout) throws IOException { + HttpGet request = new HttpGet(url); + return execRequest(request, connTimeout, readTimeout); + } + + public String deleteRequest(String url, int connTimeout, int readTimeout) throws IOException { + HttpDelete request = new HttpDelete(url); + return execRequest(request, connTimeout, readTimeout); + } + + private HttpClient getHttpClient(int connectionTimeout, int readTimeout) { + final HttpParams httpParams = new BasicHttpParams(); + HttpConnectionParams.setSoTimeout(httpParams, readTimeout); + HttpConnectionParams.setConnectionTimeout(httpParams, connectionTimeout); + + return new DefaultHttpClient(httpParams); + } + + public String execRequest(HttpRequestBase request, int connectionTimeout, int readTimeout) throws IOException { + HttpClient httpClient = getHttpClient(connectionTimeout, readTimeout); + try { + HttpResponse response = httpClient.execute(request); + String msg = EntityUtils.toString(response.getEntity()); + int code = response.getStatusLine().getStatusCode(); + if (code != 200) + throw new IOException("Invalid http response " + code + " when send request: " + + request.getURI().toString() + "\n" + msg); + return msg; + } catch (IOException e) { + logger.error("error when send http request:" + request.getURI().toString(), e); + throw e; + } finally { + request.releaseConnection(); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCallable.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCallable.java new file mode 100644 index 00000000000..b80010d8aed --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCallable.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +public interface RetryCallable { + T call() throws Exception; + + /** + * if result is not expected, will do retry + * @param result + * @return true if the result is expected + */ + boolean isResultExpected(T result); + + /** + * when retry happens, this method will be called + */ + void update(); +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCaller.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCaller.java new file mode 100644 index 00000000000..88a343e6099 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/RetryCaller.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RetryCaller { + public static final int[] RETRY_BACKOFF = { 1, 2, 3, 5, 7 }; + private static final Logger logger = LoggerFactory.getLogger(RetryCaller.class); + private int retries; + private int pauseTimeInMs; + + public RetryCaller(int retries, int pauseTimeInMs) { + this.retries = retries; + this.pauseTimeInMs = pauseTimeInMs; + } + + public T call(RetryCallable callable) throws IOException { + int tries = 0; + while (true) { + try { + if (tries > 0) { + callable.update(); + } + T result = callable.call(); + if (callable.isResultExpected(result)) { + return result; + } else { + throw new UnExpectResultException("unexpected result:" + result.toString()); + } + } catch (Throwable t) { + tries++; + if (tries > retries) { + throw new IOException("Fail after " + retries + " retries", t); + } + logger.info("call fail because of {}, {} retry", t.getMessage(), tries); + sleep(tries); + } + } + } + + private void sleep(int retryCnt) { + int sleepTime = getSleepTime(retryCnt); + try { + Thread.sleep(sleepTime); + } catch (InterruptedException ie) { + logger.error("interrupted", ie); + } + } + + private int getSleepTime(int retryCnt) { + int tries = retryCnt; + if (retryCnt >= RETRY_BACKOFF.length) { + tries = RETRY_BACKOFF.length - 1; + } + return pauseTimeInMs * RETRY_BACKOFF[tries]; + } + + private static class UnExpectResultException extends RuntimeException { + public UnExpectResultException(String msg) { + super(msg); + } + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/StreamFilterUtil.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/StreamFilterUtil.java new file mode 100644 index 00000000000..e1d96531125 --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/StreamFilterUtil.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.core.util; + +import java.nio.ByteBuffer; + +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.BytesUtil; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.metadata.filter.IFilterCodeSystem; + +public class StreamFilterUtil { + + public static IFilterCodeSystem getStreamingFilterCodeSystem() { + return new IFilterCodeSystem() { + @Override + public int compare(ByteArray o1, ByteArray o2) { + return o1.compareTo(o2); + } + + @Override + public boolean isNull(ByteArray code) { + return DimensionEncoding.isNull(code.array(), code.offset(), code.length()); + } + + @Override + public void serialize(ByteArray code, ByteBuffer buffer) { + if (code == null) + BytesUtil.writeByteArray(null, 0, 0, buffer); + else + BytesUtil.writeByteArray(code.array(), code.offset(), code.length(), buffer); + } + + @Override + public ByteArray deserialize(ByteBuffer buffer) { + return new ByteArray(BytesUtil.readByteArray(buffer)); + } + }; + } +} diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/util/TimeDerivedColumnType.java b/stream-core/src/main/java/org/apache/kylin/stream/core/util/TimeDerivedColumnType.java new file mode 100644 index 00000000000..902caf43f0c --- /dev/null +++ b/stream-core/src/main/java/org/apache/kylin/stream/core/util/TimeDerivedColumnType.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.util.Locale; +import java.util.Map; + +import org.apache.kylin.common.util.ByteArray; +import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.common.util.TimeUtil; + +import com.google.common.collect.Maps; + +public enum TimeDerivedColumnType { + MINUTE_START("minute_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + long calTimeStart = TimeUtil.getMinuteStart(time); + long calTimeEnd = calTimeStart + TimeUtil.ONE_MINUTE_TS; + return new Pair<>(calTimeStart, calTimeEnd); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToTimeWithoutMilliStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getMinuteStart(time); + } + }, + HOUR_START("hour_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + long calTimeStart = TimeUtil.getHourStart(time); + long calTimeEnd = calTimeStart + TimeUtil.ONE_HOUR_TS; + return new Pair<>(calTimeStart, calTimeEnd); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToTimeWithoutMilliStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getHourStart(time); + } + }, + DAY_START("day_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + long calTimeStart = TimeUtil.getDayStart(time); + long calTimeEnd = calTimeStart + TimeUtil.ONE_DAY_TS; + return new Pair<>(calTimeStart, calTimeEnd); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToDateStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getDayStart(time); + } + }, + WEEK_START("week_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + return new Pair<>(TimeUtil.getWeekStart(time), TimeUtil.getWeekEnd(time)); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToDateStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getWeekStart(time); + } + }, + MONTH_START("month_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + return new Pair<>(TimeUtil.getMonthStart(time), TimeUtil.getMonthEnd(time)); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToDateStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getMonthStart(time); + } + }, + QUARTER_START("quarter_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + return new Pair<>(TimeUtil.getQuarterStart(time), TimeUtil.getQuarterEnd(time)); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToDateStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getQuarterStart(time); + } + }, + YEAR_START("year_start") { + @Override + public Pair calculateTimeUnitRange(long time) { + return new Pair<>(TimeUtil.getYearStart(time), TimeUtil.getYearEnd(time)); + } + + @Override + public String normalizeTimeFormat(long time) { + return DateFormat.formatToDateStr(normalize(time)); + } + + @Override + public long normalize(long time) { + return TimeUtil.getYearStart(time); + } + }; + + private static final String MINUTE_START_NAME = "minute_start"; + private static final String HOUR_START_NAME = "hour_start"; + private static final String DAY_START_NAME = "day_start"; + private static final String WEEK_START_NAME = "week_start"; + private static final String MONTH_START_NAME = "month_start"; + private static final String QUARTER_START_NAME = "quarter_start"; + private static final String YEAR_START_NAME = "year_start"; + private static Map nameColumnsMap = Maps.newHashMap(); + + static { + nameColumnsMap.put(MINUTE_START_NAME, MINUTE_START); + nameColumnsMap.put(HOUR_START_NAME, HOUR_START); + nameColumnsMap.put(DAY_START_NAME, DAY_START); + nameColumnsMap.put(WEEK_START_NAME, WEEK_START); + nameColumnsMap.put(MONTH_START_NAME, MONTH_START); + nameColumnsMap.put(QUARTER_START_NAME, QUARTER_START); + nameColumnsMap.put(YEAR_START_NAME, YEAR_START); + } + + private String name; + + TimeDerivedColumnType(String name) { + this.name = name; + } + + public static boolean isTimeDerivedColumn(String columnName) { + return nameColumnsMap.containsKey(columnName.toLowerCase(Locale.ROOT)); + } + + public static TimeDerivedColumnType getTimeDerivedColumnType(String columnName) { + return nameColumnsMap.get(columnName.toLowerCase(Locale.ROOT)); + } + + public static long parseTimeValue(Object value) { + if (value instanceof Long) { + return (Long) value; + } + String dateStr; + if (value instanceof String) { + dateStr = (String) value; + } else if (value instanceof ByteArray) { + dateStr = Bytes.toString(((ByteArray) value).array()); + } else { + throw new IllegalArgumentException("unknown type of value:" + value.getClass()); + } + return DateFormat.stringToMillis(dateStr); + } + + public boolean hasTimeRangeOverlap(long timeStart, long timeEnd, Object timeValue) { + long time = parseTimeValue(timeValue); + Pair calUnitTimeRange = calculateTimeUnitRange(time); + if (calUnitTimeRange.getSecond() <= timeStart || calUnitTimeRange.getFirst() >= timeEnd) { + return false; + } + return true; + } + + public Pair getTimeUnitRange(Object timeValue) { + long time = parseTimeValue(timeValue); + return calculateTimeUnitRange(time); + } + + abstract public Pair calculateTimeUnitRange(long time); + + abstract public String normalizeTimeFormat(long time); + + abstract public long normalize(long time); +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/CheckPointStoreTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/CheckPointStoreTest.java new file mode 100644 index 00000000000..73fd4b4b02e --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/CheckPointStoreTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.model.stats.LongLatencyInfo; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Maps; + +public class CheckPointStoreTest { + private CheckPointStore cpStore; + + @Before + public void setup() { + String cubeName = "testCube"; + File checkPointFolder = new File(cubeName); + if (!checkPointFolder.exists()) { + checkPointFolder.mkdirs(); + } else { + Assert.assertTrue(checkPointFolder.isDirectory()); + } + cpStore = new CheckPointStore(cubeName, checkPointFolder); + } + + @Test + public void getLatestCheckPoint() throws IOException { + CheckPoint cp1 = createCheckPoint1(); + CheckPoint cp2 = createCheckPoint2(); + cpStore.saveCheckPoint(cp1); + cpStore.saveCheckPoint(cp2); + + CheckPoint cp3 = cpStore.getLatestCheckPoint(); + System.out.println(cp3); + } + + @Test + public void getLatestCheckPoint2() { + CheckPoint cp3 = cpStore.getLatestCheckPoint(); + System.out.println(cp3); + } + + @Test + public void deleteOldCheckPointFiles() throws IOException { + long DAY_TIMESTAMP_BASE = 24 * 3600 * 1000L; + + CheckPoint cp1 = createCheckPoint1(); + long startTime = cp1.getCheckPointTime(); + for (int i = 0; i < 10; i++) { + cp1.setCheckPointTime(startTime + i * DAY_TIMESTAMP_BASE); + cpStore.saveCheckPoint(cp1); + } + cpStore.deleteOldCPFiles(); + Assert.assertEquals(cpStore.getCheckPointFiles().length, CheckPointStore.CP_FILE_MAX_NUM); + } + + public CheckPoint createCheckPoint1() throws IOException{ + CheckPoint cp = new CheckPoint(); + Map consumingStats = Maps.newHashMap(); + Map persistedIndexes = Maps.newHashMap(); + long currTime = System.currentTimeMillis(); + long currHour = (currTime / 3600000) * 3600000; + consumingStats.put(1, 10000L); + consumingStats.put(2, 20000L); + consumingStats.put(3, 30000L); + + + persistedIndexes.put(currHour, "1"); + persistedIndexes.put(currHour - 1, "2"); + persistedIndexes.put(currHour - 2, "3"); + + cp.setCheckPointTime(System.currentTimeMillis()); + + cp.setSourceConsumePosition(JsonUtil.writeValueAsString(consumingStats)); + cp.setPersistedIndexes(persistedIndexes); + cp.setTotalCount(1000000); + cp.setCheckPointCount(50000); + cp.setLongLatencyInfo(new LongLatencyInfo()); + Map segmentSourceStartOffsets = Maps.newHashMap(); + Map sourceStartOffsets = Maps.newHashMap(); + sourceStartOffsets.put(1, 10000L); + sourceStartOffsets.put(2, 20000L); + sourceStartOffsets.put(3, 30000L); + segmentSourceStartOffsets.put(currHour, JsonUtil.writeValueAsString(sourceStartOffsets)); + cp.setSegmentSourceStartPosition(segmentSourceStartOffsets); + return cp; + } + + public CheckPoint createCheckPoint2() throws IOException { + CheckPoint cp = new CheckPoint(); + Map consumingStats = Maps.newHashMap(); + Map persistedIndexes = Maps.newHashMap(); + long currTime = System.currentTimeMillis() + 60000; + long currHour = (currTime / 3600000) * 3600000; + consumingStats.put(1, 30000L); + consumingStats.put(2, 40000L); + consumingStats.put(3, 50000L); + + persistedIndexes.put(currHour, "1"); + persistedIndexes.put(currHour - 1, "2"); + persistedIndexes.put(currHour - 2, "3"); + + cp.setCheckPointTime(System.currentTimeMillis()); + + cp.setSourceConsumePosition(JsonUtil.writeValueAsString(consumingStats)); + cp.setPersistedIndexes(persistedIndexes); + cp.setTotalCount(1000000); + cp.setCheckPointCount(50000); + cp.setLongLatencyInfo(new LongLatencyInfo()); + Map segmentSourceStartOffsets = Maps.newHashMap(); + Map sourceStartOffsets = Maps.newHashMap(); + sourceStartOffsets.put(1, 10000L); + sourceStartOffsets.put(2, 20000L); + sourceStartOffsets.put(3, 30000L); + segmentSourceStartOffsets.put(currHour, JsonUtil.writeValueAsString(sourceStartOffsets)); + cp.setSegmentSourceStartPosition(segmentSourceStartOffsets); + return cp; + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/MockPositionHandler.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/MockPositionHandler.java new file mode 100644 index 00000000000..f7c6eb05033 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/MockPositionHandler.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import org.apache.kylin.stream.core.source.ISourcePosition; +import org.apache.kylin.stream.core.source.ISourcePositionHandler; + +import java.util.Collection; +import java.util.Map; + +public class MockPositionHandler implements ISourcePositionHandler{ + @Override + public ISourcePosition mergePositions(Collection positions, MergeStrategy mergeStrategy) { + return null; + } + + @Override + public ISourcePosition createEmptyPosition() { + return new ISourcePosition() { + @Override + public void update(IPartitionPosition point) { + + } + + @Override + public void updateWhenPartitionNotExist(IPartitionPosition partitionPosition) { + + } + + @Override + public ISourcePosition advance() { + return null; + } + + @Override + public Map getPartitionPositions() { + return null; + } + + @Override + public void copy(ISourcePosition other) { + + } + }; + } + + @Override + public ISourcePosition parsePosition(String positionStr) { + return null; + } + + @Override + public String serializePosition(ISourcePosition position) { + return null; + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/RecordsSerDeTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/RecordsSerDeTest.java new file mode 100644 index 00000000000..9a26a778042 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/RecordsSerDeTest.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.zip.DataFormatException; + +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.util.RecordsSerializer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Sets; + +public class RecordsSerDeTest extends LocalFileMetadataTestCase { + + private static final String cubeName = "test_streaming_v2_cube"; + + private CubeDesc cubeDesc; + private CubeInstance cubeInstance; + private TestHelper testHelper; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).getCube(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + this.testHelper = new TestHelper(cubeInstance); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + } + + @Test + public void serDeTest() throws IOException, DataFormatException { + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE", + "STREAMING_V2_TABLE.MINUTE_START"); + Set metrics = Sets.newHashSet(); + + metrics.add(testHelper.simulateCountMetric()); + + ResponseResultSchema schema = new ResponseResultSchema(cubeDesc, dimensions, metrics); + RecordsSerializer serializer = new RecordsSerializer(schema); + List records = new ArrayList<>(); + int rowsNum = 10; + for (int i = 0; i < rowsNum; i++) { + Record record = new Record(dimensions.size(), metrics.size()); + record.setDimension(0, "site" + i); + record.setDimension(1, "" + i); + record.setMetric(0, 10000L + i); + records.add(record); + } + Pair serializedData = serializer.serialize(records.iterator(), Integer.MAX_VALUE); + + Iterator desRecords = serializer.deserialize(serializedData.getFirst()); + while (desRecords.hasNext()) { + Record record = desRecords.next(); + System.out.println(record); + } + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/StreamingSegmentManagerTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/StreamingSegmentManagerTest.java new file mode 100644 index 00000000000..9db3bdace2a --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/StreamingSegmentManagerTest.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.IStreamingSearchResult; +import org.apache.kylin.stream.core.query.StreamingCubeDataSearcher; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.columnar.StreamingDataSimulator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Sets; + +public class StreamingSegmentManagerTest extends LocalFileMetadataTestCase { + + private static final String cubeName = "test_streaming_v2_cube"; + private static Logger logger = LoggerFactory.getLogger(StreamingSegmentManagerTest.class); + + private CubeInstance cubeInstance; + private CubeDesc cubeDesc; + private String baseStorePath; + private TestHelper testHelper; + private StreamingSegmentManager streamingSegmentManager; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + setUpTestKylinCube(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); + this.streamingSegmentManager = new StreamingSegmentManager(baseStorePath, cubeInstance, + new MockPositionHandler(), null); + this.cleanupSegments(); + this.testHelper = new TestHelper(cubeInstance); + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + } + + protected void setUpTestKylinCube() { + this.cubeInstance = getCubeManager().reloadCubeQuietly(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getTestConfig()); + } + + @Test + public void testAddEventAndScan() { + int eventNum = 80000; + long time = DateFormat.stringToMillis("2018-07-30 20:00:00"); + Stopwatch sw = new Stopwatch(); + sw.start(); + StreamingDataSimulator simulator = new StreamingDataSimulator(); + Iterator messageIterator = simulator.simulate(eventNum, time); + while (messageIterator.hasNext()) { + StreamingMessage message = messageIterator.next(); + streamingSegmentManager.addEvent(message); + } + StreamingCubeDataSearcher searcher = streamingSegmentManager.getSearcher(); + + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + Set groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + Set metrics = Sets.newHashSet(testHelper.simulateMetric("STREAMING_V2_TABLE.GMV", "SUM", "decimal(19,6)")); + + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, null, + null); + IStreamingSearchResult segmentResults1 = searcher.doSearch(searchRequest, -1, true); + int recordNum = 0; + for (Record record : segmentResults1) { + recordNum++; + } + System.out.println("record cnt is:" + recordNum); + assertEquals(10, recordNum); + + dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.ITM"); + groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.ITM"); + + searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, null, + null); + segmentResults1 = searcher.doSearch(searchRequest, -1, true); + recordNum = 0; + for (Record record : segmentResults1) { + recordNum++; + } + System.out.println("record cnt is:" + recordNum); + assertEquals(eventNum, recordNum); + + dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START", "STREAMING_V2_TABLE.SITE"); + groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + CompareTupleFilter filter1 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.MINUTE_START", + FilterOperatorEnum.GTE, "2018-07-30 20:00:00"); + CompareTupleFilter filter2 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.MINUTE_START", + FilterOperatorEnum.LT, "2018-07-30 20:04:00"); + TupleFilter filter = testHelper.buildAndFilter(filter1, filter2); + metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, filter, + null); + segmentResults1 = searcher.doSearch(searchRequest, -1, true); + recordNum = 0; + for (Record record : segmentResults1) { + recordNum++; + long cnt = (Long)record.getMetrics()[0]; + assertEquals(4000, cnt); + } + System.out.println("record cnt is:" + recordNum); + assertEquals(10, recordNum); + } + + private TupleFilter simulateAndFilter() { + CompareTupleFilter filter1 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.DAY_START", + FilterOperatorEnum.EQ, "2016-10-20"); + CompareTupleFilter filter2 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.SITE", FilterOperatorEnum.EQ, + "site2"); + TupleFilter andFilter = testHelper.buildAndFilter(filter1, filter2); + return andFilter; + } + + private TupleFilter simulateOrFilter(List columns) { + CompareTupleFilter filter1 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.DAY_START", + FilterOperatorEnum.EQ, "2016-10-20"); + CompareTupleFilter filter2 = testHelper.buildCompareFilter("STREAMING_V2_TABLE.SITE", FilterOperatorEnum.EQ, + "site2"); + TupleFilter orFilter = testHelper.buildOrFilter(filter1, filter2); + return orFilter; + } + + private Set simulateMetrics() { + return Sets.newHashSet(testHelper.simulateMetric("STREAMING_V2_TABLE.GMV", "SUM", "decimal(19,6)")); + } + + private void cleanupSegments() { + FileUtils.deleteQuietly(new File(baseStorePath)); + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/TestHelper.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/TestHelper.java new file mode 100644 index 00000000000..2285bbe4af7 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/TestHelper.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage; + +import java.util.List; +import java.util.Set; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.measure.percentile.PercentileMeasureType; +import org.apache.kylin.metadata.filter.BuiltInFunctionTupleFilter; +import org.apache.kylin.metadata.filter.ColumnTupleFilter; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.ConstantTupleFilter; +import org.apache.kylin.metadata.filter.LogicalTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.ParameterDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class TestHelper { + private CubeInstance cubeInstance; + + public TestHelper(CubeInstance cubeInstance) { + this.cubeInstance = cubeInstance; + } + + public Set simulateDimensions(String... dimName) { + Set dimensions = Sets.newHashSet(); + for (String dim : dimName) { + TblColRef cf1 = getColumnRef(dim); + dimensions.add(cf1); + } + + return dimensions; + } + + public Set simulateMetrics() { + List functions = Lists.newArrayList(); + TblColRef gmvCol = getColumnRef("STREAMING_V2_TABLE.GMV"); + +// FunctionDesc f1 = new FunctionDesc(); +// f1.setExpression("SUM"); +// ParameterDesc p1 = ParameterDesc.newInstance(gmvCol); +// f1.setParameter(p1); +// f1.setReturnType("decimal(19,6)"); +// functions.add(f1); + + FunctionDesc f2 = new FunctionDesc(); + f2.setExpression(PercentileMeasureType.FUNC_PERCENTILE_APPROX); + ParameterDesc p2 = ParameterDesc.newInstance(gmvCol); + f2.setParameter(p2); + f2.setReturnType("percentile(100)"); + functions.add(f2); + + return Sets.newHashSet(functions); + } + + public FunctionDesc simulateMetric(String columnName, String funcName, String returnType) { + TblColRef gmvCol = getColumnRef(columnName); + + FunctionDesc f1 = new FunctionDesc(); + f1.setExpression(funcName); + ParameterDesc p1 = ParameterDesc.newInstance(gmvCol); + f1.setParameter(p1); + f1.setReturnType(returnType); + return f1; + } + + public FunctionDesc simulateCountMetric() { + FunctionDesc f1 = new FunctionDesc(); + f1.setExpression("COUNT"); + ParameterDesc p1 = ParameterDesc.newInstance("1"); + f1.setParameter(p1); + f1.setReturnType("bigint"); + return f1; + } + + public TblColRef getColumnRef(String col) { + return cubeInstance.getModel().findColumn(col); + } + + public CompareTupleFilter buildEQFilter(String columnName, String value) { + return buildCompareFilter(columnName, FilterOperatorEnum.EQ, value); + } + + public TupleFilter buildLikeFilter(String columnName, String value) { + BuiltInFunctionTupleFilter likeFilter = new BuiltInFunctionTupleFilter("like"); + likeFilter.addChild(buildColumnFilter(columnName)); + likeFilter.addChild(new ConstantTupleFilter(value)); + return likeFilter; + } + + public TupleFilter buildLowerFilter(String columnName, FilterOperatorEnum op, String value) { + CompareTupleFilter compareFilter = new CompareTupleFilter(op); + ColumnTupleFilter columnFilter1 = buildColumnFilter(columnName); + BuiltInFunctionTupleFilter lowerFilter = new BuiltInFunctionTupleFilter("lower"); + lowerFilter.addChild(columnFilter1); + compareFilter.addChild(lowerFilter); + compareFilter.addChild(new ConstantTupleFilter(value)); + return compareFilter; + } + + public CompareTupleFilter buildCompareFilter(String columnName, FilterOperatorEnum op, Object value) { + CompareTupleFilter compareFilter = new CompareTupleFilter(op); + ColumnTupleFilter columnFilter1 = buildColumnFilter(columnName); + compareFilter.addChild(columnFilter1); + ConstantTupleFilter constantFilter1 = new ConstantTupleFilter(value); + compareFilter.addChild(constantFilter1); + return compareFilter; + } + + public ColumnTupleFilter buildColumnFilter(String columnName) { + TblColRef column = getColumnRef(columnName); + return new ColumnTupleFilter(column); + } + + public TupleFilter buildAndFilter(TupleFilter filter1, TupleFilter filter2) { + return buildLogicalFilter(filter1, filter2, FilterOperatorEnum.AND); + } + + public TupleFilter buildOrFilter(TupleFilter filter1, TupleFilter filter2) { + return buildLogicalFilter(filter1, filter2, FilterOperatorEnum.OR); + } + + public TupleFilter buildLogicalFilter(TupleFilter filter1, TupleFilter filter2, FilterOperatorEnum op) { + LogicalTupleFilter andFilter = new LogicalTupleFilter(op); + andFilter.addChild(filter1); + andFilter.addChild(filter2); + return andFilter; + } + + public TupleFilter buildTimeRangeFilter(String timeColumn, Object startTime, Object endTime) { + CompareTupleFilter leftFilter = buildCompareFilter(timeColumn, FilterOperatorEnum.GTE, startTime); + CompareTupleFilter rightFilter = buildCompareFilter(timeColumn, FilterOperatorEnum.LT, endTime); + return buildAndFilter(leftFilter, rightFilter); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreTest.java new file mode 100644 index 00000000000..2918b564bd7 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/ColumnarSegmentStoreTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.SingleThreadResultCollector; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +public class ColumnarSegmentStoreTest extends LocalFileMetadataTestCase { + + private static final String cubeName = "test_streaming_v2_cube"; + private static Logger logger = LoggerFactory.getLogger(ColumnarSegmentStoreTest.class); + + private CubeInstance cubeInstance; + private CubeDesc cubeDesc; + private String baseStorePath; + private String segmentName; + private ColumnarSegmentStore segmentStore; + private TestHelper testHelper; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + cleanupData(); + this.cubeInstance = getCubeManager().getCube(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + this.segmentName = "20171218100000_20171218110000"; + this.segmentStore = new ColumnarSegmentStore(baseStorePath, cubeInstance, segmentName); + this.testHelper = new TestHelper(cubeInstance); + } + + private void prepareTestData() { + Iterator messages = new StreamingDataSimulator().simulate(200000, System.currentTimeMillis()); + while (messages.hasNext()) { + segmentStore.addEvent(messages.next()); + } + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getTestConfig()); + } + + @Test + public void testSearch() throws Exception { + kickSearchRequests(2); + prepareTestData(); + segmentStore.doMergeFragments(Lists.newArrayList(segmentStore.getAllFragments())); + Thread.sleep(5000); + cleanupData(); + } + + private void kickSearchRequests(int parallel) { + for (int i = 0; i < parallel; i++) { + new SearchClient(i).start(); + } + } + + private void scanStore() throws IOException { + Set dimensions = testHelper.simulateDimensions(new String[] { "STREAMING_V2_TABLE.SITE" }); + Set groups = testHelper.simulateDimensions(new String[] { "STREAMING_V2_TABLE.SITE" }); + Set metrics = testHelper.simulateMetrics(); + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + null, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + segmentStore.search(searchRequest, resultCollector); + int count = 0; + for (Record record : resultCollector) { + count++; + } + resultCollector.close(); + } + + @Test + public void testFindFragmentsToMerge() { + List allFragments = Lists.newArrayList(); + KylinConfig config = getTestConfig(); + config.setProperty("kylin.stream.segment-min-fragments", "5"); + config.setProperty("kylin.stream.max-fragment-size-mb", "100"); + segmentStore.latestCheckpointFragment = 1000; + List result = segmentStore.chooseFragmentsToMerge(config, allFragments); + assertTrue(result.isEmpty()); + for (int i = 0; i < 10; i++) { + allFragments.add(new MockFragment(new FragmentId(i), 15)); + } + result = segmentStore.chooseFragmentsToMerge(config, allFragments); + assertEquals(6, result.size()); + + allFragments.clear(); + for (int i = 0; i < 50; i++) { + allFragments.add(new MockFragment(new FragmentId(i), 1)); + } + result = segmentStore.chooseFragmentsToMerge(config, allFragments); + assertEquals(46, result.size()); + + allFragments.clear(); + allFragments.add(new MockFragment(new FragmentId(0), 100)); + for (int i = 1; i < 10; i++) { + allFragments.add(new MockFragment(new FragmentId(i), 15)); + } + result = segmentStore.chooseFragmentsToMerge(config, allFragments); + assertEquals(6, result.size()); + + allFragments.clear(); + allFragments.add(new MockFragment(new FragmentId(0, 5), 50)); + for (int i = 6; i < 20; i++) { + allFragments.add(new MockFragment(new FragmentId(i), 15)); + } + result = segmentStore.chooseFragmentsToMerge(config, allFragments); + assertTrue(result.get(0).getFragmentId().equals(new FragmentId(6))); + assertEquals(6, result.size()); + } + + private void cleanupData() throws IOException { + FileUtils.deleteQuietly(new File(baseStorePath)); + } + + private class SearchClient extends Thread { + private int id; + + public SearchClient(int id) { + super("thread-" + id); + this.id = id; + this.setDaemon(true); + } + + @Override + public void run() { + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + System.out.println("search client " + id + " start"); + int round = 1; + while (true) { + try { + scanStore(); + System.out.println("client:" + id + " round" + round + " scan finished"); + round++; + Thread.sleep(100); + } catch (Exception e) { + logger.error("error", e); + } + } + } + } + + private class MockFragment extends DataSegmentFragment { + private long mockedDataFileSize; + + public MockFragment(FragmentId fragmentId, int mockedDataFileSizeMb) { + super(baseStorePath, cubeName, segmentName, fragmentId); + this.mockedDataFileSize = mockedDataFileSizeMb * 1024 * 1024; + } + + public long getDataFileSize() { + return mockedDataFileSize; + } + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReaderTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReaderTest.java new file mode 100644 index 00000000000..9c0bb9681bc --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentCuboidReaderTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.storage.columnar.ParsedStreamingCubeInfo.CuboidInfo; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.log4j.PropertyConfigurator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class FragmentCuboidReaderTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private String segmentName; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private DataSegmentFragment fragment; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).reloadCubeQuietly(cubeName); + this.segmentName = "20171018100000_20171018110000"; + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + this.fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, new FragmentId(0)); + PropertyConfigurator.configure("../build/conf/kylin-tools-log4j.properties"); + prepareData(); + } + + @Test + public void testIterateAndRead() throws Exception { + FragmentMetaInfo fragmentMetaInfo = fragment.getMetaInfo(); + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, fragment.getDataFile()); + CuboidInfo cuboidInfo = parsedStreamingCubeInfo.getCuboidInfo(parsedStreamingCubeInfo.basicCuboid.getId()); + Map> dictionaryMap = fragmentData + .getDimensionDictionaries(parsedStreamingCubeInfo.dimensionsUseDictEncoding); + TblColRef[] dimensions = cuboidInfo.getDimensions(); + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings( + parsedStreamingCubeInfo.cubeDesc, dimensions, dictionaryMap); + FragmentCuboidReader fragmentCuboidReader = new FragmentCuboidReader(parsedStreamingCubeInfo.cubeDesc, + fragmentData, fragmentMetaInfo.getBasicCuboidMetaInfo(), cuboidInfo.getDimensions(), + parsedStreamingCubeInfo.measureDescs, dimensionEncodings); + + int i = 0; + for (RawRecord rawRecord : fragmentCuboidReader) { + System.out.println(rawRecord); + if (i > 10) { + break; + } + i++; + } + RawRecord rawRecord = fragmentCuboidReader.read(9999); + byte[] itmVal = rawRecord.getDimensions()[0]; + assertEquals("ITM0000009999", dimensionEncodings[0].decode(itmVal, 0, itmVal.length)); + + cuboidInfo = parsedStreamingCubeInfo.getCuboidInfo(17); + dictionaryMap = fragmentData.getDimensionDictionaries(parsedStreamingCubeInfo.dimensionsUseDictEncoding); + dimensions = cuboidInfo.getDimensions(); + dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings(parsedStreamingCubeInfo.cubeDesc, + dimensions, dictionaryMap); + fragmentCuboidReader = new FragmentCuboidReader(parsedStreamingCubeInfo.cubeDesc, fragmentData, + fragmentMetaInfo.getBasicCuboidMetaInfo(), cuboidInfo.getDimensions(), + parsedStreamingCubeInfo.measureDescs, dimensionEncodings); + + int j = 0; + for (RawRecord newRawRecord : fragmentCuboidReader) { + System.out.println(newRawRecord); + if (j >= 10) { + break; + } + j++; + } + } + + private void prepareData() { + ColumnarMemoryStorePersister memStorePersister = new ColumnarMemoryStorePersister(parsedStreamingCubeInfo, + segmentName); + StreamingDataSimulator simulator = new StreamingDataSimulator( + StreamingDataSimulator.getDefaultCardinalityMap(), 100000); + Iterator streamingMessages = simulator.simulate(50000, System.currentTimeMillis()); + SegmentMemoryStore memoryStore = new SegmentMemoryStore(new ParsedStreamingCubeInfo(cubeInstance), segmentName); + while (streamingMessages.hasNext()) { + memoryStore.index(streamingMessages.next()); + } + + memStorePersister.persist(memoryStore, fragment); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + cleanupData(); + } + + private void cleanupData() throws IOException { + FileUtils.deleteQuietly(new File(baseStorePath)); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcherTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcherTest.java new file mode 100644 index 00000000000..f6bcd035f01 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFileSearcherTest.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.KylinConfigExt; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.measure.percentile.PercentileCounter; +import org.apache.kylin.metadata.filter.CompareTupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.ResponseResultSchema; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.SingleThreadResultCollector; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.apache.log4j.PropertyConfigurator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class FragmentFileSearcherTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private String segmentName; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private DataSegmentFragment fragment; + private FragmentFileSearcher fragmentFileSearcher; + private TestHelper testHelper; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).reloadCubeQuietly(cubeName); + this.segmentName = "20171018100000_20171018110000"; + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + this.fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, new FragmentId(0)); + PropertyConfigurator.configure("../build/conf/kylin-tools-log4j.properties"); + prepareData(); + fragmentFileSearcher = new FragmentFileSearcher(fragment, new FragmentData(fragment.getMetaInfo(), + fragment.getDataFile())); + this.testHelper = new TestHelper(cubeInstance); + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + cleanupData(); + } + + @Test + public void testIterator() throws Exception { + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START", "STREAMING_V2_TABLE.SITE"); + Set groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + Set metrics = testHelper.simulateMetrics(); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, + dimensions, groups, metrics, null, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + for (Record record : resultCollector) { + Object[] values = record.getMetrics(); + PercentileCounter counter = (PercentileCounter) values[values.length - 1]; + values[values.length - 1] = estPercentileValue(counter); + System.out.println(record); + } + + dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE", "STREAMING_V2_TABLE.ITM", + "STREAMING_V2_TABLE.MINUTE_START"); + + groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dimensions, groups, metrics, + null, null); + resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + int count = 0; + for (Record record : resultCollector) { + Object[] values = record.getMetrics(); + if (count < 5) { + PercentileCounter counter = (PercentileCounter) values[values.length - 1]; + values[values.length - 1] = estPercentileValue(counter); + System.out.println(record); + } + count++; + } + assertEquals(10, count); + } + + @Test + public void testInvertIndexSearch() throws Exception { + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE", + "STREAMING_V2_TABLE.ITM", "STREAMING_V2_TABLE.MINUTE_START"); + Set groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + Set metrics = testHelper.simulateMetrics(); + String itmValue = "ITM0000009000"; + CompareTupleFilter itemFilter = testHelper.buildEQFilter("STREAMING_V2_TABLE.ITM", itmValue); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, + dimensions, groups, metrics, itemFilter, null); + ResponseResultSchema schema = searchRequest.getRespResultSchema(); + int itmDimIdx = schema.getIndexOfDimension(itemFilter.getColumn()); + ResultCollector resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + for (Record record : resultCollector) { + assertEquals(itmValue, record.getDimensions()[itmDimIdx]); + Object[] values = record.getMetrics(); + PercentileCounter counter = (PercentileCounter) values[values.length - 1]; + values[values.length - 1] = estPercentileValue(counter); + System.out.println(Lists.newArrayList(values)); + } + + String siteValue = "SITE0"; + CompareTupleFilter siteFilter = testHelper.buildEQFilter("STREAMING_V2_TABLE.SITE", siteValue); + int siteDimIdx = schema.getIndexOfDimension(siteFilter.getColumn()); + TupleFilter andFilter = testHelper.buildAndFilter(itemFilter, siteFilter); + searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dimensions, groups, metrics, + andFilter, null); + resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + for (Record record : resultCollector) { + assertEquals(itmValue, record.getDimensions()[itmDimIdx]); + assertEquals(siteValue, record.getDimensions()[siteDimIdx]); + Object[] values = record.getMetrics(); + PercentileCounter counter = (PercentileCounter) values[values.length - 1]; + values[values.length - 1] = estPercentileValue(counter); + System.out.println(Lists.newArrayList(values)); + } + + TupleFilter likeFilter = testHelper.buildLikeFilter("STREAMING_V2_TABLE.ITM", "ITM000001%"); + metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, + dimensions, groups, metrics, likeFilter, null); + resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + long count = 0; + int rowsNum = 0; + for (Record record : resultCollector) { + count += (Long)record.getMetrics()[0]; + rowsNum ++; + } + assertEquals(10000, count); + assertEquals(10, rowsNum); + } + + private double estPercentileValue(PercentileCounter counter) { + PercentileCounter counter1 = new PercentileCounter(100, 0.5); + counter1.merge(counter); + return counter1.getResultEstimate(); + } + + protected void prepareData() { + // build additional cuboids + KylinConfigExt configExt = (KylinConfigExt) cubeInstance.getDescriptor().getConfig(); + configExt.getExtendedOverrides().put("kylin.stream.build.additional.cuboids", "true"); + + ColumnarMemoryStorePersister memStorePersister = new ColumnarMemoryStorePersister(parsedStreamingCubeInfo, + segmentName); + StreamingDataSimulator simulator = new StreamingDataSimulator( + StreamingDataSimulator.getDefaultCardinalityMap(), 10000); + Iterator streamingMessages = simulator.simulate(50000, System.currentTimeMillis()); + SegmentMemoryStore memoryStore = new SegmentMemoryStore(new ParsedStreamingCubeInfo(cubeInstance), segmentName); + while (streamingMessages.hasNext()) { + memoryStore.index(streamingMessages.next()); + } + + memStorePersister.persist(memoryStore, fragment); + } + + private void cleanupData() throws IOException { + FileUtils.deleteQuietly(new File(baseStorePath)); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMergerTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMergerTest.java new file mode 100644 index 00000000000..411ab3a7ea6 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/FragmentFilesMergerTest.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.KylinConfigExt; +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.SingleThreadResultCollector; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; +import org.apache.log4j.PropertyConfigurator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class FragmentFilesMergerTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private String segmentName; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private FragmentFilesMerger fragmentFilesMerger; + private TestHelper testHelper; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = getCubeManager().reloadCubeQuietly(cubeName); + this.segmentName = "20171218100000_20171218110000"; + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + this.fragmentFilesMerger = new FragmentFilesMerger(parsedStreamingCubeInfo, new File(new File(baseStorePath, + cubeName), segmentName)); + this.testHelper = new TestHelper(cubeInstance); + cleanupData(); + PropertyConfigurator.configure("../build/conf/kylin-tools-log4j.properties"); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + cleanupData(); + } + + @Test + public void testMerge() throws Exception { + List fragments = createFragmentFiles(0, 5, new StreamingDataSimulator()); + FragmentsMergeResult mergeResult = fragmentFilesMerger.merge(fragments); + File mergedFragmentMetaFile = mergeResult.getMergedFragmentMetaFile(); + FragmentMetaInfo fragmentMetaInfo = JsonUtil.readValue(mergedFragmentMetaFile, FragmentMetaInfo.class); + assertEquals(5 * 50000, fragmentMetaInfo.getNumberOfRows()); + assertEquals(5 * 50000, fragmentMetaInfo.getOriginNumOfRows()); + + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, mergeResult.getMergedFragmentDataFile()); + Set dims = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + TupleFilter siteFilter = testHelper.buildEQFilter("SITE", "SITE0"); + Set metrics = Sets.newHashSet(); + + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + DataSegmentFragment fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, new FragmentId(0)); + FragmentFileSearcher fragmentFileSearcher = new FragmentFileSearcher(fragment, fragmentData); + // InvertIndexSearcher iiSearcher = new InvertIndexSearcher(cuboidMetaInfo, Lists.newArrayList(dims), fragmentData.getDataReadBuffer()); + // IndexSearchResult searchResult = iiSearcher.search(scanFilter); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dims, + Sets. newHashSet(), metrics, siteFilter, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + int rowCnt = 0; + for (Record record : resultCollector) { + rowCnt++; + } + assertEquals(5 * 5000, rowCnt); + + dims = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE", "STREAMING_V2_TABLE.MINUTE_START"); + Set groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dims, groups, metrics, null, + null); + resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + int totalOriginCnt = 0; + for (Record record : resultCollector) { + System.out.println(record); + totalOriginCnt += (Long) record.getMetrics()[0]; + } + assertEquals(5 * 50000, totalOriginCnt); + } + + @Test + public void testMerge2() throws Exception { + int fragmentNum = 4; + int eventCntPerMin = 100000; + StreamingDataSimulator simulator = new StreamingDataSimulator(getTestCardinalityMap(), eventCntPerMin); + List fragments = createFragmentFiles(5, fragmentNum, simulator); + int originRowCnt = fragmentNum * 50000; + FragmentsMergeResult mergeResult = fragmentFilesMerger.merge(fragments); + File mergedFragmentMetaFile = mergeResult.getMergedFragmentMetaFile(); + File mergedFragmentDataFile = mergeResult.getMergedFragmentDataFile(); + FragmentMetaInfo fragmentMetaInfo = JsonUtil.readValue(mergedFragmentMetaFile, FragmentMetaInfo.class); + assertEquals(160000, fragmentMetaInfo.getNumberOfRows()); + assertEquals(originRowCnt, fragmentMetaInfo.getOriginNumOfRows()); + + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, mergedFragmentDataFile); + Set dims = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + TupleFilter siteFilter = null; + Set groups = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + DataSegmentFragment fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, new FragmentId(0)); + FragmentFileSearcher fragmentFileSearcher = new FragmentFileSearcher(fragment, fragmentData); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dims, + groups, metrics, siteFilter, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + fragmentFileSearcher.search(searchRequest, resultCollector); + int rowCnt = 0; + int totalOriginCnt = 0; + for (Record record : resultCollector) { + rowCnt++; + long everyMinuteCnt = (Long) record.getMetrics()[0]; + assertEquals(eventCntPerMin, (int) everyMinuteCnt); + System.out.println(record); + totalOriginCnt += everyMinuteCnt; + } + assertEquals(2, rowCnt); + assertEquals(originRowCnt, totalOriginCnt); + } + + @Test + public void testMergeWithAdditionalCuboids() throws Exception { + setBuildAdditionalCuboids(); + StreamingDataSimulator simulator = new StreamingDataSimulator(getTestCardinalityMap(), 200000); + List fragments = createFragmentFiles(9, 3, simulator); + FragmentsMergeResult mergeResult = fragmentFilesMerger.merge(fragments); + File mergedFragmentMetaFile = mergeResult.getMergedFragmentMetaFile(); + File mergedFragmentDataFile = mergeResult.getMergedFragmentDataFile(); + FragmentMetaInfo fragmentMetaInfo = JsonUtil.readValue(mergedFragmentMetaFile, FragmentMetaInfo.class); + assertEquals(160010, fragmentMetaInfo.getNumberOfRows()); + } + + private void setBuildAdditionalCuboids() { + KylinConfigExt configExt = (KylinConfigExt) cubeInstance.getDescriptor().getConfig(); + configExt.getExtendedOverrides().put("kylin.stream.build.additional.cuboids", "true"); + } + + private Map getTestCardinalityMap() { + Map result = new HashMap(); + + result.put("SITE", 10); + result.put("ITM", 80000); + return result; + } + + private List createFragmentFiles(int startFragmentId, int fragmentNum, + StreamingDataSimulator simulator) { + List result = Lists.newArrayListWithCapacity(fragmentNum); + ColumnarMemoryStorePersister memStorePersister = new ColumnarMemoryStorePersister(parsedStreamingCubeInfo, + segmentName); + int eventCntInMem = 50000; + long startTime = DateFormat.stringToMillis("2018-07-30 20:00:00"); + Iterator streamingMessages = simulator.simulate(eventCntInMem * fragmentNum, startTime); + for (int i = 0; i < fragmentNum; i++) { + DataSegmentFragment fragment = new DataSegmentFragment(baseStorePath, cubeName, segmentName, + new FragmentId(startFragmentId + i)); + int eventCnt = 0; + SegmentMemoryStore memoryStore = new SegmentMemoryStore(new ParsedStreamingCubeInfo(cubeInstance), + segmentName); + while (streamingMessages.hasNext() && eventCnt < eventCntInMem) { + memoryStore.index(streamingMessages.next()); + eventCnt++; + } + memStorePersister.persist(memoryStore, fragment); + result.add(fragment); + } + return result; + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getTestConfig()); + } + + private void cleanupData() throws IOException { + fragmentFilesMerger.cleanMergeDirectory(); + FileUtils.deleteQuietly(new File(baseStorePath)); + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataTest.java new file mode 100644 index 00000000000..e682eee1d8a --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/GeneralColumnDataTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.Arrays; +import java.util.Random; + +import org.apache.kylin.metadata.datatype.DataType; +import org.apache.kylin.metadata.datatype.DataTypeSerializer; +import org.junit.Test; + +import com.google.common.io.CountingOutputStream; +import com.google.common.io.Files; + +public class GeneralColumnDataTest { + @Test + public void testWriteRead() throws Exception { + File tmpColFile = File.createTempFile("testCol", ".general"); + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + tmpColFile.deleteOnExit(); + + DataType dataType = DataType.getType("decimal"); + DataTypeSerializer serializer = DataTypeSerializer.create(dataType); + CountingOutputStream countingOutputStream = new CountingOutputStream(new FileOutputStream(tmpColFile)); + + int rowCnt = 1000; + int maxLength = serializer.maxLength(); + ByteBuffer writeBuffer = ByteBuffer.allocate(maxLength); + + GeneralColumnDataWriter writer = new GeneralColumnDataWriter(rowCnt, new DataOutputStream(countingOutputStream)); + for (int i = 0; i < rowCnt; i++) { + writeBuffer.rewind(); + serializer.serialize(new BigDecimal(i), writeBuffer); + byte[] bytes = Arrays.copyOf(writeBuffer.array(), writeBuffer.position()); + writer.write(bytes); + } + writer.flush(); + ByteBuffer dataBuffer = Files.map(tmpColFile, MapMode.READ_ONLY); + try (GeneralColumnDataReader reader = new GeneralColumnDataReader(dataBuffer, 0, + (int) countingOutputStream.getCount())) { + + int k = 0; + for (byte[] val : reader) { + assertEquals(new BigDecimal(k), serializer.deserialize(ByteBuffer.wrap(val))); + k++; + } + assertEquals(k, rowCnt); + + Random random = new Random(); + for (int i = 0; i < 50; i++) { + int rowNum = random.nextInt(rowCnt); + byte[] val = reader.read(rowNum); + assertEquals(new BigDecimal(rowNum), serializer.deserialize(ByteBuffer.wrap(val))); + } + } + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/SegmentMemoryStoreTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/SegmentMemoryStoreTest.java new file mode 100644 index 00000000000..ad4d8ec8cfa --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/SegmentMemoryStoreTest.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import static org.junit.Assert.assertEquals; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.common.KylinConfigExt; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeDescManager; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.ParameterDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.SingleThreadResultCollector; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class SegmentMemoryStoreTest extends LocalFileMetadataTestCase { + + private static final String cubeName = "test_streaming_v2_cube"; + + private CubeInstance cubeInstance; + private String segmentName; + private CubeDesc cubeDesc; + private SegmentMemoryStore memoryStore; + private TestHelper testHelper; + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + setUpTestKylinCube(); + this.segmentName = "20171218100000_20171218110000"; + this.memoryStore = new SegmentMemoryStore(new ParsedStreamingCubeInfo(cubeInstance), segmentName); + this.testHelper = new TestHelper(cubeInstance); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + } + + protected void setUpTestKylinCube() { + this.cubeInstance = getCubeManager().reloadCubeQuietly(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + } + + public CubeDescManager getCubeDescManager() { + return CubeDescManager.getInstance(getTestConfig()); + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getTestConfig()); + } + + @Test + public void testIndexEvent() { + prepareDataToMemoryStore(50000); + assertEquals(50000, memoryStore.getRowCount()); + } + + @Test + public void testIndexEventForMultipleCuboids() { + setBuildAdditionalCuboids(); + int eventCnt = 50000; + prepareDataToMemoryStore(eventCnt); + int expectedCnt = eventCnt + eventCnt + 10; + assertEquals(expectedCnt, memoryStore.getRowCount()); + } + + @Test + public void testSearchBasicCuboid() throws Exception { + StreamingQueryProfile profile = new StreamingQueryProfile("test-query-id", System.currentTimeMillis()); + StreamingQueryProfile.set(profile); + + int eventCnt = 50000; + prepareDataToMemoryStore(eventCnt); + Set dimensions = testHelper.simulateDimensions("DAY_START", "SITE"); + Set groups = testHelper.simulateDimensions("DAY_START", "SITE"); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + null, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + memoryStore.search(searchRequest, resultCollector); + int returnRecordCnt = 0; + int returnColNum = 0; + for (Record record : resultCollector) { + returnRecordCnt++; + returnColNum = record.getDimensions().length + record.getMetrics().length; + } + assertEquals(eventCnt, returnRecordCnt); + assertEquals(3, returnColNum); + } + + @Test + public void testSearchWithFilter() throws Exception { + StreamingQueryProfile profile = new StreamingQueryProfile("test-query-id", System.currentTimeMillis()); + StreamingQueryProfile.set(profile); + + int eventCnt = 50000; + prepareDataToMemoryStore(eventCnt); + Set dimensions = testHelper.simulateDimensions("DAY_START", "SITE"); + Set groups = testHelper.simulateDimensions("DAY_START", "SITE"); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + TupleFilter filter = testHelper.buildEQFilter("SITE", "SITE0"); + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + filter, null); + ResultCollector resultCollector = new SingleThreadResultCollector(); + memoryStore.search(searchRequest, resultCollector); + int returnRecordCnt = 0; + int returnColNum = 0; + for (Record record : resultCollector) { + returnRecordCnt++; + returnColNum = record.getDimensions().length + record.getMetrics().length; + } + assertEquals(eventCnt/10, returnRecordCnt); + assertEquals(3, returnColNum); + + dimensions = testHelper.simulateDimensions("DAY_START", "SITE", "ITM"); + filter = testHelper.buildLikeFilter("ITM", "ITM000001%"); + searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + filter, null); + resultCollector = new SingleThreadResultCollector(); + memoryStore.search(searchRequest, resultCollector); + returnRecordCnt = 0; + for (Record record : resultCollector) { + returnRecordCnt++; + } + assertEquals(10000, returnRecordCnt); + + filter = testHelper.buildLowerFilter("ITM", FilterOperatorEnum.EQ, "itm0000010000"); + searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + filter, null); + resultCollector = new SingleThreadResultCollector(); + memoryStore.search(searchRequest, resultCollector); + returnRecordCnt = 0; + for (Record record : resultCollector) { + returnRecordCnt++; + } + assertEquals(1, returnRecordCnt); + } + + @Test + public void testSearchSpecificCuboid() throws Exception { + StreamingQueryProfile profile = new StreamingQueryProfile("test-query-id", System.currentTimeMillis()); + StreamingQueryProfile.set(profile); + + setBuildAdditionalCuboids(); + int eventCnt = 50000; + prepareDataToMemoryStore(eventCnt); + Set dimensions = simulateColumns("SITE"); + Set groups = Sets.newHashSet(); + Set metrics = simulateMetrics(); + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + null, null); + assertEquals(1L, searchRequest.getHitCuboid()); + ResultCollector resultCollector = new SingleThreadResultCollector(); + memoryStore.search(searchRequest, resultCollector); + int returnRecordCnt = 0; + int returnColNum = 0; + for (Record record : resultCollector) { + returnRecordCnt++; + returnColNum = record.getDimensions().length + record.getMetrics().length; + } + assertEquals(10, returnRecordCnt); + assertEquals(2, returnColNum); + } + + private void setBuildAdditionalCuboids() { + KylinConfigExt configExt = (KylinConfigExt) cubeInstance.getDescriptor().getConfig(); + configExt.getExtendedOverrides().put("kylin.stream.build.additional.cuboids", "true"); + this.memoryStore = new SegmentMemoryStore(new ParsedStreamingCubeInfo(cubeInstance), segmentName); + } + + private void prepareDataToMemoryStore(int eventCnt) { + Iterator streamingMessages = new StreamingDataSimulator().simulate(eventCnt, + System.currentTimeMillis()); + while (streamingMessages.hasNext()) { + memoryStore.index(streamingMessages.next()); + } + } + + private Map getColCardMap() { + Map result = new HashMap(); + + result.put("SITE", 10); + result.put("ITM", Integer.MAX_VALUE); + return result; + } + + private Set simulateColumns(String... columnNames) { + Set columns = Sets.newHashSet(); + for (String columnName : columnNames) { + if (!columnName.contains(".")) { + columnName = "STREAMING_V2_TABLE." + columnName; + } + TblColRef cf1 = cubeDesc.getModel().findColumn(columnName); + columns.add(cf1); + } + return columns; + } + + private Set simulateMetrics() { + List functions = Lists.newArrayList(); + + TblColRef gmvCol = cubeDesc.getModel().findColumn("STREAMING_V2_TABLE.GMV"); + FunctionDesc f1 = new FunctionDesc(); + f1.setExpression("SUM"); + ParameterDesc p1 = ParameterDesc.newInstance(gmvCol); + f1.setParameter(p1); + f1.setReturnType("decimal(19,6)"); + functions.add(f1); + + return Sets.newHashSet(functions); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/StreamingDataSimulator.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/StreamingDataSimulator.java new file mode 100644 index 00000000000..381c3601f2b --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/StreamingDataSimulator.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import org.apache.kylin.common.util.TimeUtil; +import org.apache.kylin.stream.core.model.StreamingMessage; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.kylin.stream.core.source.ISourcePosition.IPartitionPosition; + +public class StreamingDataSimulator { + private Map cardinalityMap; + private int eventCntPerMin = 10000; + private Random random = new Random(); + + public StreamingDataSimulator() { + this.cardinalityMap = getDefaultCardinalityMap(); + } + public StreamingDataSimulator(Map cardinalityMap) { + this.cardinalityMap = cardinalityMap; + } + public StreamingDataSimulator(Map cardinalityMap, int eventCntPerMin) { + this.cardinalityMap = cardinalityMap; + this.eventCntPerMin = eventCntPerMin; + } + + public Iterator simulate(final int eventCnt, final long startTime) { + return new Iterator() { + int currIdx = 0; + long time = startTime; + Map params = Maps.newHashMap(); + + @Override + public boolean hasNext() { + return currIdx < eventCnt; + } + + @Override + public StreamingMessage next() { + List data; + int id = currIdx; + data = simulateData(id, time, cardinalityMap); + StreamingMessage message = new StreamingMessage(data, new IPartitionPosition() { + @Override + public int getPartition() { + return 0; + } + + @Override + public int compareTo(IPartitionPosition o) { + return 0; + } + }, time, params); + currIdx++; + if (currIdx % eventCntPerMin == 0) { + time = time + 60000; + } + return message; + } + + @Override + public void remove() { + + } + }; + } + + private List simulateData(int i, long time, Map cardinalityMap) { + List data = Lists.newArrayList(); + data.add("SITE" + i % cardinalityMap.get("SITE"));//site + data.add("ITM" + getFixLenID(10, i % cardinalityMap.get("ITM")));//item + data.add(String.valueOf(TimeUtil.getDayStart(time))); + data.add(String.valueOf(TimeUtil.getHourStart(time)));// hour start + data.add(String.valueOf(TimeUtil.getMinuteStart(time)));// minute start + data.add(String.valueOf(random.nextDouble() * 1000.0));//gmv + //itm cnt + if (i % 2 == 0) { + data.add("2"); + } else { + data.add("1"); + } + + return data; + } + + private String getFixLenID(int fixLen, int id) { + StringBuilder result = new StringBuilder(); + String idStr = String.valueOf(id); + int more = fixLen - idStr.length(); + if (more > 0) { + for (int i = 0; i < more; i++) { + result.append("0"); + } + } + result.append(idStr); + return result.toString(); + } + + public static Map getDefaultCardinalityMap() { + Map result = new HashMap(); + + result.put("SITE", 10); + result.put("ITM", Integer.MAX_VALUE); + return result; + } + + public static void main(String[] args) { + StreamingDataSimulator simulator = new StreamingDataSimulator(); + System.out.println(simulator.getFixLenID(10, 100)); + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressColumnTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressColumnTest.java new file mode 100644 index 00000000000..af3601c7077 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/LZ4CompressColumnTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kylin.common.util.Bytes; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.io.CountingOutputStream; +import com.google.common.io.Files; + +public class LZ4CompressColumnTest { + private File tmpColFile; + + @Before + public void setUp() throws Exception { + this.tmpColFile = File.createTempFile("testCol", ".lz4"); + tmpColFile.deleteOnExit(); + } + + @Test + public void testWriteRead() throws Exception { + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + int rowCnt = 100000; + int compressedSize = writeCompressedData(rowCnt); + System.out.println("compressed data size:" + compressedSize); + ByteBuffer byteBuffer = Files.map(tmpColFile, MapMode.READ_ONLY); + try (LZ4CompressedColumnReader reader = new LZ4CompressedColumnReader(byteBuffer, 0, compressedSize, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + Random random = new Random(); + for (int i = 0; i < 50; i++) { + int rowNum = random.nextInt(rowCnt); + byte[] val = reader.read(rowNum); + assertEquals(rowNum, Bytes.toInt(val)); + } + } + } + + @Test + public void testReadInputStream() throws Exception { + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + int rowCnt = 100000; + int compressedSize = writeCompressedData(rowCnt); + System.out.println("compressed data size:" + compressedSize); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataInputStream fsInputStream = fs.open(new Path(tmpColFile.getAbsolutePath())); + try (FSInputLZ4CompressedColumnReader reader = new FSInputLZ4CompressedColumnReader(fsInputStream, 0, + compressedSize, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + } + } + + public int writeCompressedData(int rowCnt) throws IOException { + int compressBolckSize = 64 * 1024; + CountingOutputStream countingOutputStream = new CountingOutputStream(new FileOutputStream(tmpColFile)); + LZ4CompressedColumnWriter writer = new LZ4CompressedColumnWriter(4, rowCnt, compressBolckSize, + countingOutputStream); + int[] colValues = new int[rowCnt]; + for (int i = 0; i < rowCnt; i++) { + colValues[i] = i; + } + for (int i = 0; i < rowCnt; i++) { + writer.write(Bytes.toBytes(colValues[i])); + } + writer.flush(); + return (int) countingOutputStream.getCount(); + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressColumnTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressColumnTest.java new file mode 100644 index 00000000000..cdd306921aa --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/NoCompressColumnTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kylin.common.util.Bytes; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.io.Files; + +public class NoCompressColumnTest { + private File tmpColFile; + + @Before + public void setUp() throws Exception { + this.tmpColFile = File.createTempFile("testCol", ".nocompress"); + tmpColFile.deleteOnExit(); + } + + @Test + public void testWriteRead() throws Exception { + int rowCnt = 10000; + writeNoCompressedData(rowCnt); + + ByteBuffer byteBuffer = Files.map(tmpColFile, MapMode.READ_ONLY); + try (NoCompressedColumnReader reader = new NoCompressedColumnReader(byteBuffer, 0, 4, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + Random random = new Random(); + for (int i = 0; i < 50; i++) { + int rowNum = random.nextInt(rowCnt); + byte[] val = reader.read(rowNum); + assertEquals(rowNum, Bytes.toInt(val)); + } + } + } + + @Test + public void testFSInputRead() throws Exception { + int rowCnt = 10000; + writeNoCompressedData(rowCnt); + + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataInputStream fsInputStream = fs.open(new Path(tmpColFile.getAbsolutePath())); + try (FSInputNoCompressedColumnReader reader = new FSInputNoCompressedColumnReader(fsInputStream, 0, 4, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + } + } + + public void writeNoCompressedData(int rowCnt) throws IOException { + NoCompressedColumnWriter writer = new NoCompressedColumnWriter(new FileOutputStream(tmpColFile)); + int[] colValues = new int[rowCnt]; + for (int i = 0; i < rowCnt; i++) { + colValues[i] = i; + } + for (int i = 0; i < rowCnt; i++) { + writer.write(Bytes.toBytes(colValues[i])); + } + writer.flush(); + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressColumnTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressColumnTest.java new file mode 100644 index 00000000000..eef47fca8fa --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/compress/RunLengthCompressColumnTest.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.compress; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel.MapMode; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kylin.common.util.Bytes; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.io.CountingOutputStream; +import com.google.common.io.Files; + +public class RunLengthCompressColumnTest { + private File tmpColFile; + + @Before + public void setUp() throws Exception { + this.tmpColFile = File.createTempFile("testCol", ".rle"); + tmpColFile.deleteOnExit(); + } + + @Test + public void testWriteRead() throws Exception { + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + int rowCnt = 100000; + + int size = writeCompressData1(rowCnt); + System.out.println("compressed data size:" + size); + ByteBuffer byteBuffer = Files.map(tmpColFile, MapMode.READ_ONLY); + try (RunLengthCompressedColumnReader reader = new RunLengthCompressedColumnReader(byteBuffer, 0, size, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + + reader.reset(); + byte[] val = reader.read(10); + assertEquals(10, Bytes.toInt(val)); + + val = reader.read(16384); + assertEquals(16384, Bytes.toInt(val)); + + val = reader.read(99999); + assertEquals(99999, Bytes.toInt(val)); + + Random random = new Random(); + for (int i = 0; i < 50; i++) { + int rowNum = random.nextInt(rowCnt); + val = reader.read(rowNum); + assertEquals(rowNum, Bytes.toInt(val)); + } + } + } + + @Test + public void testReadInputStream() throws Exception { + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + int rowCnt = 100000; + + int size = writeCompressData1(rowCnt); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataInputStream fsInputStream = fs.open(new Path(tmpColFile.getAbsolutePath())); + try (FSInputRLECompressedColumnReader reader = new FSInputRLECompressedColumnReader(fsInputStream, 0, size, + rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + } + } + + public int writeCompressData1(int rowCnt) throws IOException { + int compressBolckSize = 64 * 1024; + CountingOutputStream countingOutputStream = new CountingOutputStream(new FileOutputStream(tmpColFile)); + RunLengthCompressedColumnWriter writer = new RunLengthCompressedColumnWriter(4, rowCnt, compressBolckSize, + countingOutputStream); + int[] colValues = new int[rowCnt]; + for (int i = 0; i < rowCnt; i++) { + colValues[i] = i; + } + for (int i = 0; i < rowCnt; i++) { + writer.write(Bytes.toBytes(colValues[i])); + } + writer.flush(); + return (int) countingOutputStream.getCount(); + } + + @Test + public void testWriteRead2() throws Exception { + File tmpColFile = File.createTempFile("testCol", ".rle"); + System.out.println("file path:" + tmpColFile.getAbsolutePath()); + tmpColFile.deleteOnExit(); + int rowCnt = 100000; + int compressBolckSize = 64 * 1024; + CountingOutputStream countingOutputStream = new CountingOutputStream(new FileOutputStream(tmpColFile)); + RunLengthCompressedColumnWriter writer = new RunLengthCompressedColumnWriter(4, rowCnt, compressBolckSize, + countingOutputStream); + int batchCnt = 10; + int batch = rowCnt / batchCnt; + for (int i = 0; i < batchCnt; i++) { + for (int j = 0; j < batch; j++) { + writer.write(Bytes.toBytes(i)); + } + } + + writer.flush(); + int size = (int) countingOutputStream.getCount(); + System.out.println("compressed data size:" + size); + ByteBuffer byteBuffer = Files.map(tmpColFile, MapMode.READ_ONLY); + try (RunLengthCompressedColumnReader reader = new RunLengthCompressedColumnReader(byteBuffer, 0, size, rowCnt)) { + int k = 0; + for (byte[] val : reader) { + assertEquals(k / batch, Bytes.toInt(val)); + k++; + } + assertEquals(k, rowCnt); + + reader.reset(); + byte[] val = reader.read(10); + assertEquals(10 / batch, Bytes.toInt(val)); + + val = reader.read(16384); + assertEquals(16384 / batch, Bytes.toInt(val)); + + val = reader.read(99999); + assertEquals(99999 / batch, Bytes.toInt(val)); + + Random random = new Random(); + for (int i = 0; i < 50; i++) { + int rowNum = random.nextInt(rowCnt); + val = reader.read(rowNum); + assertEquals(rowNum / batch, Bytes.toInt(val)); + } + } + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriterWriterTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriterWriterTest.java new file mode 100644 index 00000000000..37fc83cce8d --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/ColInvertIndexWriterWriterTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex;//package org.apache.kylin.stream.invertindex; +// +//import java.util.List; +//import java.util.Random; +// +//import org.apache.kylin.common.util.Bytes; +//import org.junit.Before; +//import org.junit.Test; +// +///** +// * +// */ +//public class ColInvertIndexWriterWriterTest { +// FixLenColInvertIndexWriter cii; +// String indexFolder; +// +// @Before +// public void setup(){ +// cii = new FixLenColInvertIndexWriter("column1",4); +// indexFolder = "."; +// } +// +// @Test +// public void testSearch() throws Exception{ +// int size = 100; +// Random r = new Random(); +// for (int i =0;i rows = cii.searchValue(10); +//// System.out.println(rows); +// } +// +// @Test +// public void testFlush() throws Exception{ +// int size = 5000000; +// Random r = new Random(); +// long addStart = System.currentTimeMillis(); +// for (int i =0;i result = loadedIdx.searchValue(17); +// System.out.println("search takes:" + (System.currentTimeMillis()-loadEnd)); +// System.out.println("result size:" + result.size()); +// System.out.println(result); +// } +// +//} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SearchableColInvertIndexWriterTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SearchableColInvertIndexWriterTest.java new file mode 100644 index 00000000000..696915a25e8 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SearchableColInvertIndexWriterTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex; + +import org.junit.Test; + +/** + * + */ +public class SearchableColInvertIndexWriterTest { + + @Test + public void testLoadAndSearch() throws Exception { +// long loadStart = System.currentTimeMillis(); +// ColInvertIndexSearcher idx = ColInvertIndexSearcher.load(null); +// long loadEnd = System.currentTimeMillis(); +// System.out.println("load takes:" + (loadEnd - loadStart)); +// ImmutableRoaringBitmap result = idx.searchValue(Bytes.toBytes(17)); +// System.out.println("search takes:" + (System.currentTimeMillis() - loadEnd)); +// System.out.println("result size:" + result.getLongCardinality()); +// System.out.println(result); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SimpleColInvertIndexWriterTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SimpleColInvertIndexWriterTest.java new file mode 100644 index 00000000000..5f9bbb24a82 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/invertindex/SimpleColInvertIndexWriterTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.invertindex;//package org.apache.kylin.stream.invertindex; +// +//import java.util.List; +//import java.util.Random; +// +//import org.junit.Before; +//import org.junit.Test; +// +///** +// * +// */ +//public class SimpleColInvertIndexWriterTest { +// SimpleColInvertIndex cii; +// String idxFolder; +// +// @Before +// public void setup(){ +// cii = new SimpleColInvertIndex("column1",4); +// idxFolder = "."; +// } +// +// @Test +// public void testSearch() throws Exception{ +// int size = 100; +// Random r = new Random(); +// for (int i =0;i rows = cii.searchValue(10); +// System.out.println(rows); +// } +// +// @Test +// public void testFlushAndLoad() throws Exception{ +// int size = 100; +// Random r = new Random(); +// long addStart = System.currentTimeMillis(); +// for (int i =0;i rows = loadedIdx.searchValue(17); +// System.out.println("search takes:" + (System.currentTimeMillis()-loadEnd)); +// System.out.println(rows); +// } +// +//} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/FragmentCuboidReaderPerfTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/FragmentCuboidReaderPerfTest.java new file mode 100644 index 00000000000..04c78449a3a --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/FragmentCuboidReaderPerfTest.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.performance; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.Dictionary; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.dimension.DimensionEncoding; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.apache.kylin.stream.core.storage.columnar.ColumnarSegmentStore; +import org.apache.kylin.stream.core.storage.columnar.DataSegmentFragment; +import org.apache.kylin.stream.core.storage.columnar.FragmentCuboidReader; +import org.apache.kylin.stream.core.storage.columnar.FragmentData; +import org.apache.kylin.stream.core.storage.columnar.ParsedStreamingCubeInfo; +import org.apache.kylin.stream.core.storage.columnar.RawRecord; +import org.apache.kylin.stream.core.storage.columnar.protocol.FragmentMetaInfo; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Lists; + +public class FragmentCuboidReaderPerfTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private TestHelper testHelper; + private DataSegmentFragment[] fragments; + + public FragmentCuboidReaderPerfTest() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).reloadCubeQuietly(cubeName); + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + ColumnarSegmentStore segmentStore1 = new ColumnarSegmentStore(baseStorePath, cubeInstance, + "20180730070000_20180730080000"); + ColumnarSegmentStore segmentStore2 = new ColumnarSegmentStore(baseStorePath, cubeInstance, + "20180730080000_20180730090000"); + segmentStore1.init(); + segmentStore2.init(); + List allFragments = Lists.newArrayList(); + allFragments.addAll(segmentStore1.getAllFragments()); + allFragments.addAll(segmentStore2.getAllFragments()); + this.fragments = allFragments.toArray(new DataSegmentFragment[allFragments.size()]); + + this.testHelper = new TestHelper(cubeInstance); + } + + public static void main(String[] args) throws Exception { + FragmentCuboidReaderPerfTest test = new FragmentCuboidReaderPerfTest(); + test.scanPerformance(); + test.readRowPerformance(); + test.cleanData(); + } + + public void scanPerformance() throws Exception { + Pair, List> readDimAndMetrics = getReadDimensionsAndMetrics(); + + for (int i = 1; i < 5; i++) { + scan(i, readDimAndMetrics.getFirst(), readDimAndMetrics.getSecond().toArray(new MeasureDesc[0])); + } + } + + private void scan(int time, List dimensions, MeasureDesc[] metrics) throws IOException { + System.out.println("start " + time + " scan, " + dimensions.size() + " dimensions," + metrics.length + + " measures"); + TblColRef[] dimArray = dimensions.toArray(new TblColRef[dimensions.size()]); + List fragmentDatas = Lists.newArrayList(); + List fragmentCuboidReaders = Lists.newArrayList(); + for (int i = 0; i < fragments.length; i++) { + FragmentMetaInfo fragmentMetaInfo = fragments[i].getMetaInfo(); + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, fragments[i].getDataFile()); + Map> dictionaryMap = fragmentData + .getDimensionDictionaries(parsedStreamingCubeInfo.dimensionsUseDictEncoding); + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings( + parsedStreamingCubeInfo.cubeDesc, dimArray, dictionaryMap); + FragmentCuboidReader fragmentCuboidReader = new FragmentCuboidReader(parsedStreamingCubeInfo.cubeDesc, + fragmentData, fragmentMetaInfo.getBasicCuboidMetaInfo(), dimArray, metrics, dimensionEncodings); + fragmentDatas.add(fragmentData); + fragmentCuboidReaders.add(fragmentCuboidReader); + } + + Stopwatch sw = new Stopwatch(); + sw.start(); + int rowNum = 0; + long scanTime = 0; + for (int i = 0; i < fragments.length; i++) { + FragmentData fragmentData = fragmentDatas.get(i); + FragmentMetaInfo fragmentMetaInfo = fragmentData.getFragmentMetaInfo(); + Map> dictionaryMap = fragmentData + .getDimensionDictionaries(parsedStreamingCubeInfo.dimensionsUseDictEncoding); + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings( + parsedStreamingCubeInfo.cubeDesc, dimArray, dictionaryMap); + FragmentCuboidReader fragmentCuboidReader = new FragmentCuboidReader(parsedStreamingCubeInfo.cubeDesc, + fragmentData, fragmentMetaInfo.getBasicCuboidMetaInfo(), dimArray, metrics, dimensionEncodings); + long scanStartTime = System.currentTimeMillis(); + for (RawRecord rawRecord : fragmentCuboidReader) { + rowNum++; + } + scanTime += System.currentTimeMillis() - scanStartTime; + } + sw.stop(); + long takeTime = sw.elapsedMillis(); + System.out.println(time + " scan finished, total rows:" + rowNum); + System.out.println(time + " scan took:" + takeTime + ", scan time: " + scanTime + ", rowsPerSec:" + + (rowNum / takeTime) * 1000); + } + + public void readRowPerformance() throws Exception { + Pair, List> readDimAndMetrics = getReadDimensionsAndMetrics(); + for (int i = 1; i < 21; i++) { + readRow(i, readDimAndMetrics.getFirst(), readDimAndMetrics.getSecond().toArray(new MeasureDesc[0])); + } + } + + private void readRow(int time, List dimensions, MeasureDesc[] metrics) throws IOException { + System.out.println("start " + time + " read, " + dimensions.size() + " dimensions," + metrics.length + + " measures"); + TblColRef[] dimArray = dimensions.toArray(new TblColRef[dimensions.size()]); + Random rand = new Random(); + int randReadNum = 100; + int[][] readRows = new int[fragments.length][randReadNum]; + List fragmentCuboidReaders = Lists.newArrayList(); + for (int i = 0; i < fragments.length; i++) { + FragmentMetaInfo fragmentMetaInfo = fragments[i].getMetaInfo(); + FragmentData fragmentData = new FragmentData(fragmentMetaInfo, fragments[i].getDataFile()); + Map> dictionaryMap = fragmentData + .getDimensionDictionaries(parsedStreamingCubeInfo.dimensionsUseDictEncoding); + DimensionEncoding[] dimensionEncodings = ParsedStreamingCubeInfo.getDimensionEncodings( + parsedStreamingCubeInfo.cubeDesc, dimArray, dictionaryMap); + FragmentCuboidReader fragmentCuboidReader = new FragmentCuboidReader(parsedStreamingCubeInfo.cubeDesc, + fragmentData, fragmentMetaInfo.getBasicCuboidMetaInfo(), dimArray, metrics, dimensionEncodings); + fragmentCuboidReaders.add(fragmentCuboidReader); + for (int j = 0; j < randReadNum; j++) { + readRows[i][j] = rand.nextInt((int) fragmentMetaInfo.getNumberOfRows()); + } + } + Stopwatch sw = new Stopwatch(); + sw.start(); + int rowNum = 0; + + for (int i = 0; i < fragments.length; i++) { + for (int j = 0; j < readRows.length; j++) { + fragmentCuboidReaders.get(i).read(readRows[i][j]); + } + } + sw.stop(); + long takeTime = sw.elapsedMillis(); + System.out.println(time + " scan finished, total rows:" + rowNum); + System.out.println(time + " scan took:" + takeTime + ",rowsPerSec:" + (rowNum / takeTime) * 1000); + } + + public Pair, List> getReadDimensionsAndMetrics() { + Set dimensionsSet = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + List dimensions = new ArrayList<>(dimensionsSet); + FunctionDesc metricFunc = testHelper.simulateMetric("STREAMING_V2_TABLE.GMV", "SUM", "decimal(19,6)"); + List allMetrics = cubeInstance.getDescriptor().getMeasures(); + List targetMetrics = Lists.newArrayList(); + for (MeasureDesc metric : allMetrics) { + if (metric.getFunction().equals(metricFunc)) { + targetMetrics.add(metric); + } + if (metric.getName().equalsIgnoreCase("_COUNT_")) { +// targetMetrics.add(metric); + } + } + return new Pair<>(dimensions, targetMetrics); + } + + public void cleanData() throws Exception { + this.cleanupTestMetadata(); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerfDataPrepare.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerfDataPrepare.java new file mode 100644 index 00000000000..7cc4e5c031b --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerfDataPrepare.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.performance; + +import java.io.File; +import java.util.Iterator; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.storage.MockPositionHandler; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.columnar.StreamingDataSimulator; + +public class PerfDataPrepare extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private int totalRows; + private StreamingSegmentManager streamingSegmentManager; + + public PerfDataPrepare(int totalRows) throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).reloadCubeQuietly(cubeName); + this.totalRows = totalRows; + setupCubeConfig(); + this.streamingSegmentManager = new StreamingSegmentManager(baseStorePath, cubeInstance, new MockPositionHandler(), null); + } + + private void setupCubeConfig() { + KylinConfig kylinConfig = cubeInstance.getConfig(); + kylinConfig.setProperty("kylin.stream.index.maxrows", "1000000"); + } + + public static void main(String[] args) throws Exception { + PerfDataPrepare test = new PerfDataPrepare(72000000); + test.prepareData(); + test.cleanTmpMeta(); + } + + public void prepareData() { + System.out.println("start preparing data..."); + long startTime = System.currentTimeMillis(); + StreamingDataSimulator simulator = new StreamingDataSimulator( + StreamingDataSimulator.getDefaultCardinalityMap(), 600000); + long eventStartTime = DateFormat.stringToMillis("2018-07-30 07:00:00"); + Iterator streamingMessages = simulator.simulate(totalRows, eventStartTime); + + while (streamingMessages.hasNext()) { + streamingSegmentManager.addEvent(streamingMessages.next()); + } + streamingSegmentManager.checkpoint(); + System.out.println("data is prepared. take time:" + (System.currentTimeMillis() - startTime)); + } + + public void cleanData() throws Exception { + FileUtils.deleteQuietly(new File(baseStorePath)); + } + + public void cleanTmpMeta() { + cleanupTestMetadata(); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerformanceTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerformanceTest.java new file mode 100644 index 00000000000..b1b9d2f8bd0 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/PerformanceTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.performance; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.query.ResultCollector; +import org.apache.kylin.stream.core.query.SingleThreadResultCollector; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.MockPositionHandler; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.StreamingCubeSegment; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.apache.kylin.stream.core.storage.columnar.ColumnarSegmentStore; +import org.apache.kylin.stream.core.storage.columnar.StreamingDataSimulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Stopwatch; + +public class PerformanceTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + private static Logger logger = LoggerFactory.getLogger(PerformanceTest.class); + + private CubeInstance cubeInstance; + private CubeDesc cubeDesc; + private String baseStorePath; + private StreamingSegmentManager cubeDataStore; + private TestHelper testHelper; + + public PerformanceTest() throws Exception { + this.createTestMetadata(); + setUpTestKylinCube(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeDataStore = new StreamingSegmentManager(baseStorePath, cubeInstance, new MockPositionHandler(), null); + testHelper = new TestHelper(cubeInstance); + } + + public static void main(String[] args) throws Exception { + PerformanceTest test = new PerformanceTest(); + test.run(); + } + + public void run() { + System.out.println("Start generating data"); + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + StreamingDataSimulator simulator = new StreamingDataSimulator(); + long time = System.currentTimeMillis(); + int rowCnt = 10000000; + Iterator messageItr = simulator.simulate(rowCnt, time); + Stopwatch sw = new Stopwatch(); + sw.start(); + while (messageItr.hasNext()) { + StreamingMessage message = messageItr.next(); + cubeDataStore.addEvent(message); + } + long takeTime = sw.elapsedMillis(); + System.out.println("Index took:" + takeTime + ",qps:" + rowCnt / (takeTime / 1000)); + sw.reset(); + + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + Set groups = dimensions; + Set metrics = testHelper.simulateMetrics(); + System.out.println("Scan with no filter:"); + try { + System.out.println("start first scan"); + sw.start(); + final Iterator firstScan = scan(cubeDataStore, null, dimensions, groups, metrics); + int rowNum = 0; + while (firstScan.hasNext()) { + Record record = firstScan.next(); + rowNum++; + } + + takeTime = sw.elapsedMillis(); + System.out.println("scan finished, total rows:" + rowNum); + System.out.println("first scan took:" + takeTime + ",rowsPerSec:" + (rowNum / takeTime) * 1000); + + System.out.println("start second scan"); + sw.reset(); + sw.start(); + final Iterator secondScan = scan(cubeDataStore, null, dimensions, groups, metrics); + rowNum = 0; + while (secondScan.hasNext()) { + Record record = secondScan.next(); + rowNum++; + } + + takeTime = sw.elapsedMillis(); + System.out.println("total rows:" + rowNum); + System.out.println("second scan took:" + takeTime + ",rowsPerSec:" + (rowNum / takeTime) * 1000); + } catch (IOException e) { + e.printStackTrace(); + } + cleanupData(); + } + + private Iterator scan(StreamingSegmentManager cubeDataStore, TupleFilter tupleFilter, + Set dimensions, Set groups, Set metrics) throws IOException { + StreamingSearchContext searchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, metrics, + tupleFilter, null); + ResultCollector resultCollector = getResultCollector(); + for (StreamingCubeSegment queryableSegment : cubeDataStore.getAllSegments()) { + ColumnarSegmentStore segmentStore = new ColumnarSegmentStore(baseStorePath, cubeInstance, + queryableSegment.getSegmentName()); + segmentStore.init(); + segmentStore.search(searchRequest, resultCollector); + + } + + return resultCollector.iterator(); + } + + private ResultCollector getResultCollector() { + return new SingleThreadResultCollector(); + } + + protected void setUpTestKylinCube() { + this.cubeInstance = getCubeManager().reloadCubeQuietly(cubeName); + this.cubeDesc = cubeInstance.getDescriptor(); + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getTestConfig()); + } + + private void cleanupData() { + cleanupTestMetadata(); + FileUtils.deleteQuietly(new File(baseStorePath)); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/StreamingCubeDataSearcherPerfTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/StreamingCubeDataSearcherPerfTest.java new file mode 100644 index 00000000000..bc4d638b636 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/storage/columnar/performance/StreamingCubeDataSearcherPerfTest.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.storage.columnar.performance; + +import java.io.File; +import java.io.IOException; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.DateFormat; +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.gridtable.StorageSideBehavior; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.query.IStreamingSearchResult; +import org.apache.kylin.stream.core.query.StreamingCubeDataSearcher; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.storage.MockPositionHandler; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.TestHelper; +import org.apache.kylin.stream.core.storage.columnar.ParsedStreamingCubeInfo; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Sets; + +public class StreamingCubeDataSearcherPerfTest extends LocalFileMetadataTestCase { + private static final String cubeName = "test_streaming_v2_cube"; + + private String baseStorePath; + private CubeInstance cubeInstance; + private ParsedStreamingCubeInfo parsedStreamingCubeInfo; + private TestHelper testHelper; + private int totalRows; + private StreamingSegmentManager streamingSegmentManager; + private StreamingCubeDataSearcher searcher; + + public StreamingCubeDataSearcherPerfTest() throws Exception { + this.createTestMetadata(); + this.baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + this.cubeInstance = CubeManager.getInstance(getTestConfig()).reloadCubeQuietly(cubeName); + this.parsedStreamingCubeInfo = new ParsedStreamingCubeInfo(cubeInstance); + setupCubeConfig(); + this.testHelper = new TestHelper(cubeInstance); + this.streamingSegmentManager = new StreamingSegmentManager(baseStorePath, cubeInstance, + new MockPositionHandler(), null); + this.searcher = streamingSegmentManager.getSearcher(); + } + + private void setupCubeConfig() { + KylinConfig kylinConfig = cubeInstance.getConfig(); + kylinConfig.setProperty("kylin.stream.index.maxrows", "1000000"); + } + + public static void main(String[] args) throws Exception { + StreamingCubeDataSearcherPerfTest test = new StreamingCubeDataSearcherPerfTest(); + test.prepareData(); + test.searchPerformance(); + // test.scanPerformance(); + } + + public void searchPerformance() throws Exception { + StreamingQueryProfile.set(new StreamingQueryProfile("test-query-id", System.currentTimeMillis())); + KylinConfig.getInstanceFromEnv().setProperty("kylin.stream.receiver.use-threads-per-query", "1"); + int times = Integer.MAX_VALUE; + for (int i = 1; i < times; i++) { + search(i); + } + + for (int i = 1; i < times; i++) { + iiSearch(i); + } + } + + private void search(int time) throws IOException { + System.out.println("start " + time + " search"); + Stopwatch sw = new Stopwatch(); + sw.start(); + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START"); + Set groups = testHelper.simulateDimensions(); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + long startTime = DateFormat.stringToMillis("2018-07-30 07:00:00"); + long endTime = DateFormat.stringToMillis("2018-07-30 08:00:00"); + TupleFilter filter = testHelper.buildTimeRangeFilter("STREAMING_V2_TABLE.MINUTE_START", + String.valueOf(startTime), String.valueOf(endTime)); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dimensions, + groups, metrics, filter, null); + IStreamingSearchResult searchResult = searcher.doSearch(searchRequest, 0L, true); + for (Record record : searchResult) { + System.out.println(record); + } + sw.stop(); + long takeTime = sw.elapsedMillis(); + System.out.println(time + " search finished, took:" + takeTime); + } + + private void iiSearch(int time) throws IOException { + System.out.println("start " + time + " invertIndex search"); + Stopwatch sw = new Stopwatch(); + sw.start(); + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.MINUTE_START", + "STREAMING_V2_TABLE.ITM"); + Set groups = testHelper.simulateDimensions(); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + long startTime = DateFormat.stringToMillis("2018-07-30 07:00:00"); + long endTime = DateFormat.stringToMillis("2018-07-30 09:00:00"); + TupleFilter timeFilter = testHelper.buildTimeRangeFilter("STREAMING_V2_TABLE.MINUTE_START", + String.valueOf(startTime), String.valueOf(endTime)); + TupleFilter itemFilter = testHelper.buildEQFilter("STREAMING_V2_TABLE.ITM", "ITM0000000000"); + TupleFilter filter = testHelper.buildAndFilter(timeFilter, itemFilter); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dimensions, + groups, metrics, filter, null); + IStreamingSearchResult searchResult = searcher.doSearch(searchRequest, 0L, true); + for (Record record : searchResult) { + System.out.println(record); + } + sw.stop(); + long takeTime = sw.elapsedMillis(); + System.out.println(time + " search finished, took:" + takeTime); + } + + public void scanPerformance() throws Exception { + StreamingQueryProfile queryProfile = new StreamingQueryProfile("test-query-id", System.currentTimeMillis()); + queryProfile.setStorageBehavior(StorageSideBehavior.RAW_SCAN); + StreamingQueryProfile.set(queryProfile); + for (int i = 1; i < 500; i++) { + scan(i); + } + } + + private void scan(int time) throws IOException { + System.out.println("start " + time + " scan"); + Stopwatch sw = new Stopwatch(); + sw.start(); + Set dimensions = testHelper.simulateDimensions("STREAMING_V2_TABLE.SITE"); + Set groups = testHelper.simulateDimensions(); + Set metrics = Sets.newHashSet(testHelper.simulateCountMetric()); + long startTime = DateFormat.stringToMillis("2018-07-30 07:00:00"); + long endTime = DateFormat.stringToMillis("2018-07-30 08:00:00"); + TupleFilter filter = testHelper.buildTimeRangeFilter("STREAMING_V2_TABLE.MINUTE_START", + String.valueOf(startTime), String.valueOf(endTime)); + StreamingSearchContext searchRequest = new StreamingSearchContext(parsedStreamingCubeInfo.cubeDesc, dimensions, + groups, metrics, null, null); + IStreamingSearchResult searchResult = searcher.doSearch(searchRequest, 0L, true); + long scanRowCnt = 0; + for (Record record : searchResult) { + scanRowCnt++; + } + sw.stop(); + long takeTime = sw.elapsedMillis(); + System.out.println(time + " search finished, scan row cnt:" + scanRowCnt + ", took:" + takeTime + + ",numRowsPerSec:" + scanRowCnt * 1000 / takeTime); + } + + public void prepareData() { + streamingSegmentManager.restoreSegmentsFromLocal(); + } + + public void cleanData() throws Exception { + this.cleanupTestMetadata(); + FileUtils.deleteQuietly(new File(baseStorePath)); + } + +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/util/DataGenerator.java b/stream-core/src/test/java/org/apache/kylin/stream/core/util/DataGenerator.java new file mode 100644 index 00000000000..175c2641758 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/util/DataGenerator.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.kylin.metadata.datatype.DataType; + +/** + * Created by gwang3 on 2017/3/6. + */ +public class DataGenerator { + + public static List generateDataForTypes(String[] types, int rowCount) { + int len = types.length; + List rows = new ArrayList<>(); + for (int r = 0; r < rowCount; r++) { + rows.add(new String[len]); + } + for (int i = 0; i < len; i++) { + String[] colunmValues = generateDataForType(types[i], rowCount); + for (int m = 0; m < rowCount; m++) { + rows.get(m)[i] = colunmValues[m]; + } + } + return rows; + } + + public static String[] generateDataForType(String type, int count) { + String[] data = new String[count]; + + if (DataType.INTEGER_FAMILY.contains(type)) { + for (int j = 0; j < count; j++) { + data[j] = "1"; + } + } else if (DataType.DATETIME_FAMILY.contains(type)) { + for (int j = 0; j < count; j++) { + data[j] = "1"; + } + } else if (DataType.NUMBER_FAMILY.contains(type)) { + for (int j = 0; j < count; j++) { + data[j] = "1"; + } + } else if (DataType.STRING_FAMILY.contains(type)) { + if (type.equalsIgnoreCase("char")) { + for (int j = 0; j < count; j++) { + data[j] = "i"; + } + } + if (type.equalsIgnoreCase("varchar")) { + for (int j = 0; j < count; j++) { + data[j] = "1"; + } + } else { + throw new IllegalArgumentException("Illegal data type: " + type); + } + } else { + throw new IllegalArgumentException("Illegal data type: " + type); + } + return data; + } +} diff --git a/stream-core/src/test/java/org/apache/kylin/stream/core/util/TimeDerivedColumnTypeTest.java b/stream-core/src/test/java/org/apache/kylin/stream/core/util/TimeDerivedColumnTypeTest.java new file mode 100644 index 00000000000..b136d585bf0 --- /dev/null +++ b/stream-core/src/test/java/org/apache/kylin/stream/core/util/TimeDerivedColumnTypeTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.core.util; + +import org.apache.kylin.common.util.DateFormat; +import org.junit.Assert; +import org.junit.Test; + +public class TimeDerivedColumnTypeTest { + + @Test + public void testMinuteStartTimeRangeOverlap() { + String segmentStartStr = "2017-11-01 08:00:00"; + String segmentEndStr = "2017-11-01 09:00:00"; + + long segmentStart = DateFormat.stringToMillis(segmentStartStr); + long segmentEnd = DateFormat.stringToMillis(segmentEndStr); + + String minStartTime = "2017-11-01 08:05:00"; + boolean overlap = false; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, minStartTime); + Assert.assertTrue(overlap); + + minStartTime = "2017-11-01 08:00:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, minStartTime); + Assert.assertTrue(overlap); + + minStartTime = "2017-11-01 09:00:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, minStartTime); + Assert.assertFalse(overlap); + } + + @Test + public void testHourStartTimeRangeOverlap() { + String segmentStartStr = "2017-11-01 08:00:00"; + String segmentEndStr = "2017-11-01 09:00:00"; + + long segmentStart = DateFormat.stringToMillis(segmentStartStr); + long segmentEnd = DateFormat.stringToMillis(segmentEndStr); + + String hourStartTime = "2017-11-01 08:00:00"; + boolean overlap = false; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, hourStartTime); + Assert.assertTrue(overlap); + + hourStartTime = "2017-11-01 08:05:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, hourStartTime); + Assert.assertTrue(overlap); + + hourStartTime = "2017-11-01 07:00:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, hourStartTime); + Assert.assertFalse(overlap); + + hourStartTime = "2017-11-01 09:00:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, hourStartTime); + Assert.assertFalse(overlap); + + hourStartTime = "2017-11-01 10:00:00"; + overlap = TimeDerivedColumnType.MINUTE_START.hasTimeRangeOverlap(segmentStart, segmentEnd, hourStartTime); + Assert.assertFalse(overlap); + } + + @Test + public void testDayStartTimeRangeOverlap() { + String segmentStartStr = "2017-11-01 08:00:00"; + String segmentEndStr = "2017-11-01 09:00:00"; + + long segmentStart = DateFormat.stringToMillis(segmentStartStr); + long segmentEnd = DateFormat.stringToMillis(segmentEndStr); + + String dayStartTime = "2017-11-01"; + boolean overlap = false; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertTrue(overlap); + + dayStartTime = "2017-10-29"; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertFalse(overlap); + + dayStartTime = "2017-11-02"; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertFalse(overlap); + + segmentStartStr = "2017-11-01 23:00:00"; + segmentEndStr = "2017-11-02 02:00:00"; + + segmentStart = DateFormat.stringToMillis(segmentStartStr); + segmentEnd = DateFormat.stringToMillis(segmentEndStr); + + dayStartTime = "2017-11-02"; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertTrue(overlap); + + dayStartTime = "2017-11-01"; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertTrue(overlap); + + dayStartTime = "2017-10-30"; + overlap = TimeDerivedColumnType.DAY_START.hasTimeRangeOverlap(segmentStart, segmentEnd, dayStartTime); + Assert.assertFalse(overlap); + } +} diff --git a/stream-receiver/pom.xml b/stream-receiver/pom.xml new file mode 100644 index 00000000000..ed21feb52a1 --- /dev/null +++ b/stream-receiver/pom.xml @@ -0,0 +1,308 @@ + + + + + + kylin + org.apache.kylin + 2.6.0-SNAPSHOT + + 4.0.0 + kylin-stream-receiver + jar + Apache Kylin - Stream Receiver + + + + com.fasterxml.jackson.core + jackson-databind + + + + + org.apache.kylin + kylin-core-common + + + org.apache.kylin + kylin-core-metadata + + + javax.servlet + servlet-api + + + org.apache.kafka + kafka-clients + + + org.apache.kafka + kafka_2.11 + + + + + org.apache.kylin + kylin-core-dictionary + + + org.apache.kylin + kylin-stream-source-kafka + + + org.apache.kylin + kylin-stream-core + + + org.apache.kylin + kylin-stream-coordinator + + + + org.apache.kylin + kylin-storage-hbase + + + + org.apache.kylin + kylin-query + + + + log4j + log4j + + + org.slf4j + jcl-over-slf4j + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + com.google.guava + guava + compile + + + + + org.springframework + spring-webmvc + + + org.springframework + spring-jdbc + + + org.springframework + spring-aop + + + org.springframework + spring-context-support + + + org.springframework + spring-test + + + + cglib + cglib + 2.2.2 + + + + + org.aspectj + aspectjrt + + + org.aspectj + aspectjweaver + + + + + org.apache.hadoop + hadoop-common + provided + + + javax.servlet + servlet-api + + + javax.servlet.jsp + jsp-api + + + avro + org.apache.avro + + + + + + org.apache.hadoop + hadoop-hdfs + provided + + + javax.servlet + servlet-api + + + javax.servlet.jsp + jsp-api + + + + + org.apache.hbase + hbase-client + provided + + + javax.servlet + servlet-api + + + javax.servlet.jsp + jsp-api + + + avro + org.apache.avro + + + + + + + + + + + + + + + + + + + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + + junit + junit + test + + + org.springframework.security + spring-security-core + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.2 + + + package + + shade + + + true + all + + + + META-INF/spring.handlers + + + META-INF/spring.schemas + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/eclipse.inf + + + + + + + + + maven-compiler-plugin + + + + + + + + spring-snapshots + http://repo.spring.io/libs-snapshot + + true + + + + + + spring-snapshots + http://repo.spring.io/libs-snapshot + + true + + + + diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/ReplicaSetLeaderSelector.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/ReplicaSetLeaderSelector.java new file mode 100644 index 00000000000..80a6adcb93c --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/ReplicaSetLeaderSelector.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.leader.LeaderSelector; +import org.apache.curator.framework.recipes.leader.LeaderSelectorListenerAdapter; +import org.apache.kylin.stream.coordinator.ZKUtils; +import org.apache.kylin.stream.core.model.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +public class ReplicaSetLeaderSelector extends LeaderSelectorListenerAdapter implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(ReplicaSetLeaderSelector.class); + private LeaderSelector leaderSelector; + private int replicaSetID; + private Node node; + private List leaderChangeListeners; + + public ReplicaSetLeaderSelector(CuratorFramework client, Node currNode, int replicaSetID) { + this.node = currNode; + this.replicaSetID = replicaSetID; + String path = ZKUtils.REPLICASETS_LEADER_ELECT + "/" + replicaSetID; + leaderSelector = new LeaderSelector(client, path, this); + leaderSelector.autoRequeue(); + leaderChangeListeners = Lists.newArrayList(); + } + + @Override + public void close() throws IOException { + leaderSelector.close(); + } + + public void start() { + leaderSelector.start(); + } + + public void addLeaderChangeListener(LeaderChangeListener listener) { + this.leaderChangeListeners.add(listener); + } + + @Override + public void takeLeadership(CuratorFramework client) throws Exception { + logger.info("become the leader of the replicaSet:{}", replicaSetID); + for (LeaderChangeListener leaderChangeListener : leaderChangeListeners) { + try { + leaderChangeListener.becomeLeader(); + } catch (Exception e) { + logger.error("error when call listener", e); + } + } + while (true) { + try { + Thread.sleep(5 * 60 * 1000); + } catch (InterruptedException exception) { + Thread.interrupted(); + break; + } + if (!leaderSelector.hasLeadership()) { + break; + } + } + logger.info("become the follower of the replicaSet:{}", replicaSetID); + for (LeaderChangeListener leaderChangeListener : leaderChangeListeners) { + try { + leaderChangeListener.becomeFollower(); + } catch (Exception e) { + logger.error("error when call listener", e); + } + } + } + + public interface LeaderChangeListener { + void becomeLeader(); + + void becomeFollower(); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/ServerContext.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/ServerContext.java new file mode 100644 index 00000000000..ba950c99873 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/ServerContext.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server; + +import org.apache.kylin.stream.core.model.Node; + +public class ServerContext { + private Node receiver; + private int replicaSetID; + + public Node getReceiver() { + return receiver; + } + + public void setReceiver(Node receiver) { + this.receiver = receiver; + } + + public int getReplicaSetID() { + return replicaSetID; + } + + public void setReplicaSetID(int replicaSetID) { + this.replicaSetID = replicaSetID; + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingReceiver.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingReceiver.java new file mode 100644 index 00000000000..8faed312938 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingReceiver.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server; + +import java.io.File; +import java.io.FileInputStream; +import java.util.Properties; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.util.NodeUtil; +import org.apache.log4j.PropertyConfigurator; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.handler.ContextHandler; +import org.eclipse.jetty.server.handler.ContextHandlerCollection; +import org.eclipse.jetty.server.handler.ResourceHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.thread.QueuedThreadPool; +import org.eclipse.jetty.util.thread.ThreadPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.web.context.support.XmlWebApplicationContext; +import org.springframework.web.servlet.DispatcherServlet; + +public class StreamingReceiver { + private static final Logger logger = LoggerFactory.getLogger(StreamingReceiver.class); + private Server httpServer; + + public static void main(String[] args) { + try { + StreamingReceiver receiver = new StreamingReceiver(); + receiver.start(); + } catch (Exception e) { + logger.error("streaming receiver start fail", e); + } + } + + private void start() throws Exception { + if (System.getProperty("debug") != null) { + setupDebugEnv(); + } + startStreamingServer(); + startRpcServer(); + startHttpServer(); + } + + private void startStreamingServer() throws Exception { + StreamingServer.getInstance().start(); + } + + private void startHttpServer() throws Exception { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + createAndConfigHttpServer(kylinConfig); + + ContextHandlerCollection contexts = new ContextHandlerCollection(); + + ServletContextHandler context = new ServletContextHandler(); + context.setContextPath("/kylin"); + + XmlWebApplicationContext ctx = new XmlWebApplicationContext(); + ctx.setConfigLocation("classpath:applicationContext.xml"); + ctx.refresh(); + DispatcherServlet dispatcher = new DispatcherServlet(ctx); + context.addServlet(new ServletHolder(dispatcher), "/api/*"); + + ContextHandler logContext = new ContextHandler("/kylin/logs"); + String logDir = getLogDir(kylinConfig); + ResourceHandler logHandler = new ResourceHandler(); + logHandler.setResourceBase(logDir); + logHandler.setDirectoriesListed(true); + logContext.setHandler(logHandler); + + contexts.setHandlers(new Handler[] { context, logContext }); + httpServer.setHandler(contexts); + httpServer.start(); + httpServer.join(); + } + + private void startRpcServer() throws Exception { + // currently use http server as rpc server + } + + private String getLogDir(KylinConfig kylinConfig) { + String kylinHome = kylinConfig.getKylinHome(); + if (kylinHome == null) { + kylinHome = System.getProperty("KYLIN_HOME"); + } + return kylinHome + File.separator + "logs"; + } + + private void createAndConfigHttpServer(KylinConfig kylinConfig) { + httpServer = new Server(createThreadPool(kylinConfig)); + ServerConnector httpConnector = getHttpConnector(); + httpConnector.setPort(getHttpPort()); + httpConnector.setIdleTimeout(30000); + httpServer.addConnector(httpConnector); + } + + private ServerConnector getHttpConnector() { + return new ServerConnector(httpServer); + } + + private ThreadPool createThreadPool(KylinConfig kylinConfig) { + QueuedThreadPool threadPool = new QueuedThreadPool(); + threadPool.setMinThreads(kylinConfig.getStreamingReceiverHttpMinThreads()); + threadPool.setMaxThreads(kylinConfig.getStreamingReceiverHttpMaxThreads()); + return threadPool; + } + + private void setupDebugEnv() { + try { + Properties props = new Properties(); + props.load(new FileInputStream("../build/conf/kylin-tools-log4j.properties")); + PropertyConfigurator.configure(props); + KylinConfig.setSandboxEnvIfPossible(); + KylinConfig config = KylinConfig.getInstanceFromEnv(); + config.setProperty("kylin.stream.settled.storage", + "org.apache.kylin.stream.server.storage.LocalStreamStorage"); + } catch (Exception e) { + logger.error("debug env setup fail", e); + } + } + + private int getHttpPort() { + Node currentNode = NodeUtil.getCurrentNode(StreamingServer.DEFAULT_PORT); + return currentNode.getPort(); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingServer.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingServer.java new file mode 100644 index 00000000000..481b74670ff --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/StreamingServer.java @@ -0,0 +1,731 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server; + +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.FileUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.persistence.JsonSerializer; +import org.apache.kylin.common.persistence.ResourceStore; +import org.apache.kylin.common.util.HadoopUtil; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeDescManager; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.CubeSegment; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.metadata.TableMetadataManager; +import org.apache.kylin.metadata.model.DataModelDesc; +import org.apache.kylin.metadata.model.DataModelManager; +import org.apache.kylin.metadata.project.ProjectInstance; +import org.apache.kylin.metadata.project.ProjectManager; +import org.apache.kylin.stream.coordinator.StreamMetadataStore; +import org.apache.kylin.stream.coordinator.StreamMetadataStoreFactory; +import org.apache.kylin.stream.coordinator.ZKUtils; +import org.apache.kylin.stream.coordinator.client.CoordinatorClient; +import org.apache.kylin.stream.coordinator.client.HttpCoordinatorClient; +import org.apache.kylin.stream.core.consumer.ConsumerStartProtocol; +import org.apache.kylin.stream.core.consumer.EndPositionStopCondition; +import org.apache.kylin.stream.core.consumer.IConsumerProvider; +import org.apache.kylin.stream.core.consumer.IStopConsumptionCondition; +import org.apache.kylin.stream.core.consumer.IStreamingConnector; +import org.apache.kylin.stream.core.consumer.StreamingConsumerChannel; +import org.apache.kylin.stream.core.metrics.StreamingMetrics; +import org.apache.kylin.stream.core.model.ConsumerStatsResponse; +import org.apache.kylin.stream.core.model.Node; +import org.apache.kylin.stream.core.model.ReplicaSet; +import org.apache.kylin.stream.core.model.StreamingCubeConsumeState; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; +import org.apache.kylin.stream.core.model.stats.SegmentStats; +import org.apache.kylin.stream.core.source.ISourcePosition; +import org.apache.kylin.stream.core.source.ISourcePositionHandler; +import org.apache.kylin.stream.core.source.IStreamingSource; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingSourceConfigManager; +import org.apache.kylin.stream.core.source.StreamingSourceFactory; +import org.apache.kylin.stream.core.storage.StreamingCubeSegment; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.columnar.ColumnarStoreCache; +import org.apache.kylin.stream.core.util.HDFSUtil; +import org.apache.kylin.stream.core.util.NamedThreadFactory; +import org.apache.kylin.stream.core.util.NodeUtil; +import org.apache.kylin.stream.server.retention.PolicyInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class StreamingServer implements ReplicaSetLeaderSelector.LeaderChangeListener, IConsumerProvider { + private static final Logger logger = LoggerFactory.getLogger(StreamingServer.class); + public static final int DEFAULT_PORT = 9090; + private static final int CONSUMER_STOP_WAIT_TIMEOUT = 10000; + private static StreamingServer instance = new StreamingServer(); + + private Map cubeConsumerMap = Maps.newHashMap(); + private Map> assignments = Maps.newHashMap(); + private Map streamingSegmentManagerMap = new ConcurrentHashMap<>(); + + private CuratorFramework streamZKClient; + private ReplicaSetLeaderSelector leaderSelector; + private CoordinatorClient coordinatorClient; + private StreamMetadataStore streamMetadataStore; + private Node currentNode; + private int replicaSetID = -1; + private volatile boolean isLeader = false; + + private ScheduledExecutorService segmentStateCheckerExecutor; + private ExecutorService segmentFlushExecutor; + + private String baseStorePath; + + public StreamingServer() { + streamZKClient = ZKUtils.getZookeeperClient(); + streamMetadataStore = StreamMetadataStoreFactory.getStreamMetaDataStore(); + coordinatorClient = new HttpCoordinatorClient(streamMetadataStore); + currentNode = NodeUtil.getCurrentNode(DEFAULT_PORT); + baseStorePath = KylinConfig.getInstanceFromEnv().getStreamingIndexPath(); + + segmentStateCheckerExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory( + "segment_state_check")); + segmentFlushExecutor = Executors.newFixedThreadPool(5, new NamedThreadFactory("segment_flush")); + } + + public static StreamingServer getInstance() { + return instance; + } + + public void start() throws Exception { + registerReceiver(); + ReplicaSet rs = findBelongReplicaSet(); + if (rs != null) { + addToReplicaSet(rs.getReplicaSetID()); + } + startMetrics(); + startSegmentStateChecker(); + addShutdownHook(); + } + + private void startSegmentStateChecker() { + segmentStateCheckerExecutor.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + Collection segmentManagers = getAllCubeSegmentManagers(); + ServerContext serverContext = new ServerContext(); + serverContext.setReplicaSetID(replicaSetID); + serverContext.setReceiver(currentNode); + for (StreamingSegmentManager segmentManager : segmentManagers) { + CubeInstance cubeInstance = segmentManager.getCubeInstance(); + String cubeName = cubeInstance.getName(); + try { + PolicyInfo policyInfo = new PolicyInfo(); + String policyName = cubeInstance.getConfig().getStreamingSegmentRetentionPolicy(); + Map policyProps = cubeInstance.getConfig() + .getStreamingSegmentRetentionPolicyProperties(policyName); + policyInfo.setName(policyName); + policyInfo.setProperties(policyProps); + //The returned segments that require remote persisted are already sorted in ascending order by the segment start time + Collection segments = segmentManager.getRequireRemotePersistSegments(); + if (!segments.isEmpty()) { + logger.info("found cube {} segments:{} are immutable, retention policy is: {}", cubeName, + segments, policyInfo.getName()); + } else { + continue; + } + handleImmutableCubeSegments(cubeName, segmentManager, segments, policyInfo); + } catch (Exception e) { + logger.error("error when handle cube:" + cubeName, e); + } + } + } + }, 60, 60, TimeUnit.SECONDS); + } + + private void handleImmutableCubeSegments(String cubeName, StreamingSegmentManager segmentManager, + Collection segments, PolicyInfo policyInfo) throws Exception { + if (PolicyInfo.FULL_BUILD_POLICY.equalsIgnoreCase(policyInfo.getName())) { + if (isLeader) { + sendSegmentsToFullBuild(cubeName, segmentManager, segments); + } + } else { + purgeSegments(cubeName, segments, policyInfo.getProperties()); + } + } + + private void sendSegmentsToFullBuild(String cubeName, StreamingSegmentManager segmentManager, + Collection segments) throws Exception { + List> futureList = Lists.newArrayList(); + for (StreamingCubeSegment segment : segments) { + String segmentHDFSPath = HDFSUtil.getStreamingSegmentFilePath(cubeName, segment.getSegmentName()) + "/" + + replicaSetID; + SegmentHDFSFlusher flusher = new SegmentHDFSFlusher(segment, segmentHDFSPath); + futureList.add(segmentFlushExecutor.submit(flusher)); + } + + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).getCube(cubeName); + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cubeInstance); + int i = 0; + for (StreamingCubeSegment segment : segments) { + futureList.get(i).get(); + logger.info("save remote store state to metadata store."); + streamMetadataStore.addCompleteReplicaSetForSegmentBuild(segment.getCubeName(), segment.getSegmentName(), + replicaSetID); + + logger.info("save remote checkpoint to metadata store"); + ISourcePosition smallestSourcePosition = segmentManager.getSmallestSourcePosition(segment); + String smallestSourcePosStr = streamingSource.getSourcePositionHandler().serializePosition(smallestSourcePosition); + streamMetadataStore.saveSourceCheckpoint(segment.getCubeName(), segment.getSegmentName(), replicaSetID, + smallestSourcePosStr); + + logger.info("send notification to coordinator for cube {} segment {}.", cubeName, segment.getSegmentName()); + coordinatorClient.segmentRemoteStoreComplete(currentNode, segment.getCubeName(), + new Pair<>(segment.getDateRangeStart(), segment.getDateRangeEnd())); + logger.info("send notification success."); + segment.saveState(StreamingCubeSegment.State.REMOTE_PERSISTED); + logger.info("cube {} segment {} status converted to {}", segment.getCubeName(), segment.getSegmentName(), + StreamingCubeSegment.State.REMOTE_PERSISTED.name()); + i++; + } + + } + + private void purgeSegments(String cubeName, Collection segments, + Map properties) { + long retentionTimeInSec = Long.valueOf(properties.get("retentionTimeInSec")); + boolean hasPurgedSegment = false; + for (StreamingCubeSegment segment : segments) { + long liveTime = System.currentTimeMillis() - segment.getCreateTime(); + if (retentionTimeInSec * 1000 < liveTime) { + logger.info("purge segment:{}", segment); + getStreamingSegmentManager(cubeName).purgeSegment(segment.getSegmentName()); + hasPurgedSegment = true; + } + } + if (hasPurgedSegment) { + resumeConsumerIfPaused(cubeName); + } + } + + private void addShutdownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + logger.info("start to shut down streaming receiver"); + for (Map.Entry consumerEntry : cubeConsumerMap.entrySet()) { + logger.info("start to stop consumer for cube:{}", consumerEntry.getKey()); + StreamingConsumerChannel consumer = consumerEntry.getValue(); + consumer.stop(CONSUMER_STOP_WAIT_TIMEOUT); + logger.info("finish to stop consumer for cube:{}", consumerEntry.getKey()); + } + logger.info("streaming receiver shut down successfully"); + } + }); + } + + private void startMetrics() { + StreamingMetrics.getInstance().start(); + } + + private ReplicaSet findBelongReplicaSet() { + List replicaSets = streamMetadataStore.getReplicaSets(); + for (ReplicaSet rs : replicaSets) { + if (rs.containPhysicalNode(currentNode)) { + return rs; + } + } + return null; + } + + private void registerReceiver() throws Exception { + logger.info("register receiver:" + currentNode); + streamMetadataStore.addReceiver(currentNode); + } + + private void joinReplicaSetLeaderElection(int replicaSetID) { + leaderSelector = new ReplicaSetLeaderSelector(streamZKClient, currentNode, replicaSetID); + leaderSelector.addLeaderChangeListener(this); + leaderSelector.start(); + } + + public synchronized void assign(Map> cubeAssignment) { + this.assignments.putAll(cubeAssignment); + } + + public synchronized void assign(String cubeName, List partitions) { + this.assignments.put(cubeName, partitions); + } + + public synchronized void unAssign(String cubeName) { + stopConsumer(cubeName); + this.assignments.remove(cubeName); + removeCubeData(cubeName); + } + + public synchronized void startConsumers(List cubes) { + for (String cube : cubes) { + startConsumer(cube, null); + } + } + + public synchronized void startConsumer(String cubeName, ConsumerStartProtocol startProtocol) { + List partitions = assignments.get(cubeName); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cubeName); + if (consumer != null) { + List consumingPartitions = consumer.getConsumePartitions(); + Collections.sort(partitions); + Collections.sort(consumingPartitions); + if (partitions.equals(consumingPartitions)) { + logger.info("The consumer for cube:{} is already running, skip starting", cubeName); + } else { + String msg = String + .format(Locale.ROOT, "The running consumer for cube:%s partition:%s is conflict with assign partitions:%s, should stop the consumer first.", + cubeName, consumingPartitions, partitions); + throw new IllegalStateException(msg); + } + } else { + if (partitions == null || partitions.isEmpty()) { + logger.info("partitions is empty for cube:{}", cubeName); + return; + } + logger.info("create and start new consumer for cube:{}", cubeName); + try { + reloadCubeMetadata(cubeName); + StreamingConsumerChannel newConsumer = createNewConsumer(cubeName, partitions, startProtocol); + newConsumer.start(); + } catch (Exception e) { + logger.error("consumer start fail for cube:" + cubeName, e); + } + } + } + + public synchronized ConsumerStatsResponse stopConsumer(String cube) { + logger.info("stop consumers for cube: " + cube); + ConsumerStatsResponse response = new ConsumerStatsResponse(); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cube); + if (consumer != null) { + consumer.stop(CONSUMER_STOP_WAIT_TIMEOUT); + cubeConsumerMap.remove(cube); + response.setCubeName(cube); + response.setConsumePosition(consumer.getSourceConsumeInfo()); + } + return response; + } + + public synchronized void stopAllConsumers() { + List cubes = Lists.newArrayList(cubeConsumerMap.keySet()); + for (String cube : cubes) { + stopConsumer(cube); + } + } + + public synchronized ConsumerStatsResponse pauseConsumer(String cubeName) { + logger.info("pause consumers for cube: " + cubeName); + ConsumerStatsResponse response = new ConsumerStatsResponse(); + response.setCubeName(cubeName); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cubeName); + if (consumer != null) { + consumer.pause(true); + response.setConsumePosition(consumer.getSourceConsumeInfo()); + } else { + logger.warn("the consumer for cube:{} does not exist " + cubeName); + } + return response; + } + + public synchronized ConsumerStatsResponse resumeConsumer(String cubeName, String resumeToPosition) { + logger.info("resume consumers for cube: " + cubeName); + ConsumerStatsResponse response = new ConsumerStatsResponse(); + response.setCubeName(cubeName); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cubeName); + if (consumer == null) { + logger.warn("the consumer for cube:{} does not exist " + cubeName); + return response; + } + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + CubeInstance cube = CubeManager.getInstance(kylinConfig).getCube(cubeName); + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cube); + if (resumeToPosition != null && !resumeToPosition.isEmpty()) { + IStopConsumptionCondition stopCondition = new EndPositionStopCondition(streamingSource.getSourcePositionHandler().parsePosition(resumeToPosition)); + consumer.resumeToStopCondition(stopCondition); + cubeConsumerMap.remove(cubeName); + } else { + consumer.resume(); + } + response.setConsumePosition(consumer.getSourceConsumeInfo()); + + return response; + } + + public void addToReplicaSet(int replicaSetID) { + logger.info("add the node to the replicaSet:{}, join the group leader election.", replicaSetID); + if (this.replicaSetID == replicaSetID) { + logger.info("the receiver already in the replica set:{}, return", replicaSetID); + return; + } + if (this.replicaSetID != -1) { + throw new IllegalStateException("the receiver is in replica set:" + this.replicaSetID + + ", please remove first"); + } + this.replicaSetID = replicaSetID; + joinReplicaSetLeaderElection(replicaSetID); + Map> nodeAssignments = streamMetadataStore.getAssignmentsByReplicaSet(replicaSetID); + if (nodeAssignments != null) { + assign(nodeAssignments); + List assignedCubes = Lists.newArrayList(nodeAssignments.keySet()); + initLocalSegmentManager(assignedCubes); + startConsumers(assignedCubes); + } else { + initLocalSegmentManager(Lists. newArrayList()); + } + } + + public void removeFromReplicaSet() { + if (leaderSelector != null) { + try { + leaderSelector.close(); + } catch (Exception e) { + logger.error("error happens when close leader selector", e); + } + } + this.replicaSetID = -1; + this.isLeader = false; + assignments.clear(); + stopAllConsumers(); + List cubes = Lists.newArrayList(streamingSegmentManagerMap.keySet()); + for (String cube : cubes) { + removeCubeData(cube); + } + } + + public ReceiverStats getReceiverStats() { + ReceiverStats stats = new ReceiverStats(); + stats.setAssignments(assignments); + stats.setLead(isLeader); + + Set allCubes = Sets.newHashSet(); + allCubes.addAll(assignments.keySet()); + allCubes.addAll(cubeConsumerMap.keySet()); + allCubes.addAll(streamingSegmentManagerMap.keySet()); + for (String cube : allCubes) { + stats.addCubeStats(cube, getCubeStats(cube)); + } + stats.setCacheStats(ColumnarStoreCache.getInstance().getCacheStats()); + return stats; + } + + public ReceiverCubeStats getCubeStats(String cubeName) { + ReceiverCubeStats receiverCubeStats = new ReceiverCubeStats(); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cubeName); + if (consumer != null) { + receiverCubeStats.setConsumerStats(consumer.getConsumerStats()); + } + + StreamingSegmentManager segmentManager = streamingSegmentManagerMap.get(cubeName); + if (segmentManager != null) { + Map segmentStatsMap = segmentManager.getSegmentStats(); + receiverCubeStats.setSegmentStatsMap(segmentStatsMap); + receiverCubeStats.setTotalIngest(segmentManager.getIngestCount()); + receiverCubeStats.setLatestEventTime(segmentManager.getLatestEventTime()); + receiverCubeStats.setLatestEventIngestTime(segmentManager.getLatestEventIngestTime()); + receiverCubeStats.setLongLatencyInfo(segmentManager.getLongLatencyInfo()); + } + return receiverCubeStats; + } + + public void makeCubeImmutable(String cubeName) { + if (cubeConsumerMap.containsKey(cubeName)) { + logger.info("before make cube immutable, stop consumer for cube:{}", cubeName); + StreamingConsumerChannel consumer = cubeConsumerMap.get(cubeName); + consumer.stop(CONSUMER_STOP_WAIT_TIMEOUT); + cubeConsumerMap.remove(cubeName); + } + + StreamingSegmentManager segmentManager = streamingSegmentManagerMap.get(cubeName); + if (segmentManager == null) { + return; + } + segmentManager.makeAllSegmentsImmutable(); + } + + public void makeCubeSegmentImmutable(String cubeName, String segmentName) { + StreamingSegmentManager cubeStore = streamingSegmentManagerMap.get(cubeName); + if (cubeStore == null) { + return; + } + cubeStore.makeSegmentImmutable(segmentName); + } + + public void remoteSegmentBuildComplete(String cubeName, String segmentName) { + StreamingSegmentManager segmentManager = getStreamingSegmentManager(cubeName); + List removedSegments = segmentManager.remoteSegmentBuildComplete(segmentName); + if (removedSegments.size() > 0) { + resumeConsumerIfPaused(cubeName); + } + } + + /** + * resume cube consumer, if it is paused by too many segments + * @param cubeName + */ + private void resumeConsumerIfPaused(String cubeName) { + StreamingConsumerChannel consumer = getConsumer(cubeName); + if (consumer == null || !consumer.isPaused()) { + return; + } + StreamingCubeConsumeState consumeState = streamMetadataStore.getStreamingCubeConsumeState(cubeName); + if (consumeState == null || consumeState == StreamingCubeConsumeState.RUNNING) { + logger.info("resume the cube consumer:{} after remove some local immutable segments", cubeName); + consumer.resume(); + } + } + + private StreamingConsumerChannel createNewConsumer(String cubeName, List partitions, ConsumerStartProtocol startProtocol) + throws IOException { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + CubeInstance cube = CubeManager.getInstance(kylinConfig).getCube(cubeName); + StreamingSegmentManager segmentManager = getStreamingSegmentManager(cubeName); + + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cube); + IStreamingConnector streamingConnector = streamingSource.createStreamingConnector(cubeName, partitions, + startProtocol, segmentManager); + StreamingConsumerChannel consumer = new StreamingConsumerChannel(cubeName, streamingConnector, segmentManager, + IStopConsumptionCondition.NEVER_STOP); + long minAcceptEventTime = cube.getDescriptor().getPartitionDateStart(); + CubeSegment latestRemoteSegment = cube.getLatestReadySegment(); + if (latestRemoteSegment != null) { + minAcceptEventTime = latestRemoteSegment.getTSRange().end.v; + } + if (minAcceptEventTime > 0) { + consumer.setMinAcceptEventTime(minAcceptEventTime); + } + StreamingCubeConsumeState consumeState = streamMetadataStore.getStreamingCubeConsumeState(cubeName); + if (consumeState != null && consumeState == StreamingCubeConsumeState.PAUSED) { + consumer.pause(false); + } + cubeConsumerMap.put(cubeName, consumer); + return consumer; + } + + @Override + public void becomeLeader() { + if (replicaSetID != -1) { + logger.info("become leader of the replicaSet:{}", replicaSetID); + try { + ReplicaSet rs = streamMetadataStore.getReplicaSet(replicaSetID); + rs.setLeader(currentNode); + streamMetadataStore.updateReplicaSet(rs); + coordinatorClient.replicaSetLeaderChange(replicaSetID, currentNode); + } catch (Exception e) { + logger.error("error when send lead change notification to coordinator", e); + } + } + isLeader = true; + } + + @Override + public void becomeFollower() { + isLeader = false; + if (replicaSetID != -1) { + logger.info("become follower of the replicaSet:{}", replicaSetID); + } + } + + @Override + public StreamingConsumerChannel getConsumer(String cubeName) { + return cubeConsumerMap.get(cubeName); + } + + public StreamingSegmentManager getStreamingSegmentManager(String cubeName) { + if (streamingSegmentManagerMap.get(cubeName) == null) { + synchronized (streamingSegmentManagerMap) { + if (streamingSegmentManagerMap.get(cubeName) == null) { + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); + ISourcePositionHandler sourcePositionHandler = StreamingSourceFactory.getStreamingSource(cubeInstance).getSourcePositionHandler(); + StreamingSegmentManager segmentManager = new StreamingSegmentManager(baseStorePath, cubeInstance, sourcePositionHandler, this); + streamingSegmentManagerMap.put(cubeName, segmentManager); + } + } + } + return streamingSegmentManagerMap.get(cubeName); + } + + public void removeCubeData(String cubeName) { + logger.info("remove cube data: {}", cubeName); + StreamingSegmentManager segmentManager = getStreamingSegmentManager(cubeName); + if (segmentManager != null) { + streamingSegmentManagerMap.remove(cubeName); + segmentManager.purgeAllSegments(); + } + } + + public void reSubmitCubeSegment(String cubeName, String segmentName) { + StreamingSegmentManager segmentManager = getStreamingSegmentManager(cubeName); + StreamingCubeSegment segment = segmentManager.getSegmentByName(segmentName); + if (segment == null) { + throw new IllegalStateException("cannot find segment:" + segmentName); + } + if (segment.isActive()) { + throw new IllegalStateException("the segment must be immutable:" + segment); + } + String segmentHDFSPath = HDFSUtil.getStreamingSegmentFilePath(cubeName, segmentName) + "/" + replicaSetID; + SegmentHDFSFlusher flusher = new SegmentHDFSFlusher(segment, segmentHDFSPath); + try { + flusher.flushToHDFS(); + } catch (IOException e) { + throw new RuntimeException("fail to copy segment to hdfs:" + segment, e); + } + } + + public Collection getAllCubeSegmentManagers() { + return streamingSegmentManagerMap.values(); + } + + private void initLocalSegmentManager(List assignedCubes) { + File baseFolder = new File(baseStorePath); + if (!baseFolder.exists()) { + baseFolder.mkdirs(); + } + + File[] subFolders = baseFolder.listFiles(new FileFilter() { + @Override + public boolean accept(File file) { + return file.isDirectory(); + } + }); + + for (File cubeFolder : subFolders) { + String cubeName = cubeFolder.getName(); + if (!assignedCubes.contains(cubeName)) { + logger.info("remove the cube:{} data, because it is not assigned to this node", cubeName); + try { + FileUtils.deleteDirectory(cubeFolder); + } catch (IOException e) { + logger.error("error happens when remove cube folder", e); + } + continue; + } + try { + StreamingSegmentManager segmentManager = getStreamingSegmentManager(cubeName); + segmentManager.restoreSegmentsFromLocal(); + } catch (Exception e) { + logger.error("local cube store init fail", e); + } + } + } + + private void reloadCubeMetadata(String cubeName) throws IOException { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + ResourceStore resourceStore = ResourceStore.getStore(kylinConfig); + CubeInstance rawCubeInstance = resourceStore.getResource(CubeInstance.concatResourcePath(cubeName), + CubeManager.CUBE_SERIALIZER); + CubeDesc rawCubeDesc = resourceStore.getResource(CubeDesc.concatResourcePath(rawCubeInstance.getDescName()), + CubeDescManager.CUBE_DESC_SERIALIZER); + DataModelDesc rawModel = resourceStore.getResource( + DataModelDesc.concatResourcePath(rawCubeDesc.getModelName()), + new JsonSerializer<>(DataModelDesc.class)); + ProjectManager projectManager = ProjectManager.getInstance(kylinConfig); + List projects = projectManager.findProjectsByModel(rawModel.getName()); + if (projects.isEmpty()) { + projectManager.reloadAll(); + projects = projectManager.findProjectsByModel(rawModel.getName()); + } + if (projects.size() != 1) { + throw new IllegalArgumentException("the cube:" + cubeName + " is not in any project"); + } + + TableMetadataManager.getInstance(kylinConfig).reloadSourceTableQuietly(rawModel.getRootFactTableName(), + projects.get(0).getName()); + DataModelManager.getInstance(kylinConfig).reloadDataModel(rawModel.getName()); + CubeDescManager.getInstance(kylinConfig).reloadCubeDescLocal(cubeName); + CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).reloadCubeQuietly(cubeName); + StreamingSourceConfigManager.getInstance(kylinConfig).reloadStreamingConfigLocal( + cubeInstance.getRootFactTable()); + } + + private static class SegmentHDFSFlusher implements Runnable { + private final Logger logger = LoggerFactory.getLogger(SegmentHDFSFlusher.class); + private StreamingCubeSegment segment; + private String hdfsPath; + + public SegmentHDFSFlusher(StreamingCubeSegment segment, String hdfsPath) { + this.segment = segment; + this.hdfsPath = hdfsPath; + } + + public void flushToHDFS() throws IOException { + logger.info("start to flush cube:{} segment:{} to hdfs:{}", segment.getCubeName(), + segment.getSegmentName(), hdfsPath); + final FileSystem fs = HadoopUtil.getFileSystem(hdfsPath); + final String localPath = segment.getDataSegmentFolder().getPath(); + final Path remotePath = new Path(hdfsPath); + if (fs.exists(remotePath)) { + logger.info("the remote path:{} is already exist, skip copy data to remote", remotePath); + return; + } + final Path remoteTempPath = new Path(hdfsPath + ".tmp"); + if (fs.exists(remoteTempPath)) { + FileStatus sdst = fs.getFileStatus(remoteTempPath); + if (sdst.isDirectory()) { + logger.warn("target temp path:" + remoteTempPath + " is an existed directory, try to delete it."); + fs.delete(remoteTempPath, true); + logger.warn("target temp path:" + remoteTempPath + " is deleted."); + } + } + fs.copyFromLocalFile(new Path(localPath), remoteTempPath); + logger.info("data copy to remote temp path:{}", remoteTempPath); + boolean renamed = fs.rename(remoteTempPath, remotePath); + if (renamed) { + logger.info("successfully rename the temp path to:{}", remotePath); + } + } + + @Override + public void run() { + try { + flushToHDFS(); + } catch (Exception e) { + logger.error("error when flush segment data to hdfs", e); + throw new IllegalStateException(e); + } + } + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/AdminController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/AdminController.java new file mode 100644 index 00000000000..b78142081c5 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/AdminController.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.controller; + +import org.apache.kylin.stream.core.model.AssignRequest; +import org.apache.kylin.stream.core.model.ConsumerStatsResponse; +import org.apache.kylin.stream.core.model.PauseConsumersRequest; +import org.apache.kylin.stream.core.model.ResumeConsumerRequest; +import org.apache.kylin.stream.core.model.StartConsumersRequest; +import org.apache.kylin.stream.core.model.StopConsumersRequest; +import org.apache.kylin.stream.core.model.UnAssignRequest; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.server.StreamingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; + +import com.google.common.collect.Lists; + +@Controller +@RequestMapping(value = "/admin") +public class AdminController extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(AdminController.class); + + private StreamingServer streamingServer; + + public AdminController() { + streamingServer = StreamingServer.getInstance(); + } + + @RequestMapping(value = "/assign", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public void assign(@RequestBody AssignRequest assignRequest) { + logger.info("receive assign request:{}", assignRequest); + streamingServer.assign(assignRequest.getCubeName(), assignRequest.getPartitions()); + if (assignRequest.isStartConsumers()) { + streamingServer.startConsumers(Lists.newArrayList(assignRequest.getCubeName())); + } + } + + @RequestMapping(value = "/unAssign", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public void unAssign(@RequestBody UnAssignRequest unAssignRequest) { + logger.info("receive unassign request:{}", unAssignRequest); + streamingServer.unAssign(unAssignRequest.getCube()); + } + + @RequestMapping(value = "/consumers/start", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public void startConsumers(@RequestBody StartConsumersRequest startRequest) { + logger.info("receive start consumer request:{}", startRequest); + streamingServer.startConsumer(startRequest.getCube(), startRequest.getStartProtocol()); + } + + @RequestMapping(value = "/consumers/stop", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public ConsumerStatsResponse stopConsumers(@RequestBody StopConsumersRequest request) { + logger.info("receive stop consumer request:{}", request); + ConsumerStatsResponse response = streamingServer.stopConsumer(request.getCube()); + if (request.isRemoveData()) { + streamingServer.removeCubeData(request.getCube()); + } + return response; + } + + @RequestMapping(value = "/consumers/pause", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public ConsumerStatsResponse pauseConsumers(@RequestBody PauseConsumersRequest request) { + logger.info("receive pause consumer request:{}", request); + return streamingServer.pauseConsumer(request.getCube()); + } + + @RequestMapping(value = "/consumers/resume", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public ConsumerStatsResponse resumeConsumers(@RequestBody ResumeConsumerRequest request) { + logger.info("receive resume consumer request:{}", request); + return streamingServer.resumeConsumer(request.getCube(), request.getResumeToPosition()); + } + + @RequestMapping(value = "/segment_build_complete/{cubeName}/{segmentName}", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void segmentBuildComplete(@PathVariable(value = "cubeName") String cubeName, + @PathVariable(value = "segmentName") String segmentName) { + logger.info("receive segment build complete, cube:{}, segment:{}", cubeName, segmentName); + streamingServer.remoteSegmentBuildComplete(cubeName, segmentName); + } + + @RequestMapping(value = "/data/{cubeName}/{segmentName}", method = RequestMethod.DELETE, produces = { "application/json" }) + @ResponseBody + public void removeSegment(@PathVariable(value = "cubeName") String cubeName, + @PathVariable(value = "segmentName") String segmentName) { + logger.info("receive remove segment request, cube:{}, segment:{}", cubeName, segmentName); + StreamingSegmentManager segmentManager = streamingServer.getStreamingSegmentManager(cubeName); + segmentManager.purgeSegment(segmentName); + } + + @RequestMapping(value = "/data/{cubeName}", method = RequestMethod.DELETE, produces = { "application/json" }) + @ResponseBody + public void removeCubeData(@PathVariable(value = "cubeName") String cubeName) { + logger.info("receive remove cube request, cube:{}", cubeName); + streamingServer.removeCubeData(cubeName); + } + + @RequestMapping(value = "/data/{cubeName}/immutable", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void immuteCube(@PathVariable(value = "cubeName") String cubeName) { + logger.info("receive make cube immutable request, cube:{}", cubeName); + streamingServer.makeCubeImmutable(cubeName); + } + + @RequestMapping(value = "/data/{cubeName}/{segmentName}/immutable", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void immuteCubeSegment(@PathVariable(value = "cubeName") String cubeName, + @PathVariable(value = "segmentName") String segmentName) { + logger.info("receive make cube segment immutable request, cube:{} segment:{}", cubeName, segmentName); + streamingServer.makeCubeSegmentImmutable(cubeName, segmentName); + } + + /** + * re submit segment to hadoop + * @param cubeName + */ + @RequestMapping(value = "/data/{cubeName}/{segmentName}/reSubmit", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void reSubmitCubeSegment(@PathVariable(value = "cubeName") String cubeName, + @PathVariable(value = "segmentName") String segmentName) { + logger.info("receive reSubmit segment request, cube:{} segment:{}", cubeName, segmentName); + streamingServer.reSubmitCubeSegment(cubeName, segmentName); + } + + @RequestMapping(value = "/replica_set/{rsID}/add", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void addToReplicaSet(@PathVariable(value = "rsID") int replicaSetID) { + logger.info("receive add to replica set request, rsID:{}", replicaSetID); + streamingServer.addToReplicaSet(replicaSetID); + } + + @RequestMapping(value = "/replica_set/remove", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void removeFromReplicaSet() { + logger.info("receive remove from replica set request"); + streamingServer.removeFromReplicaSet(); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/BasicController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/BasicController.java new file mode 100644 index 00000000000..999a4b7747f --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/BasicController.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.controller; + +import javax.servlet.http.HttpServletRequest; + +import org.apache.kylin.stream.server.rest.exception.BadRequestException; +import org.apache.kylin.stream.server.rest.exception.ForbiddenException; +import org.apache.kylin.stream.server.rest.exception.NotFoundException; +import org.apache.kylin.stream.server.rest.model.ErrorResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.http.HttpStatus; +import org.springframework.web.bind.annotation.ExceptionHandler; +import org.springframework.web.bind.annotation.ResponseBody; +import org.springframework.web.bind.annotation.ResponseStatus; + +/** + */ +public class BasicController { + + private static final Logger logger = LoggerFactory.getLogger(BasicController.class); + + @ResponseStatus(HttpStatus.INTERNAL_SERVER_ERROR) + @ExceptionHandler(Exception.class) + @ResponseBody + ErrorResponse handleError(HttpServletRequest req, Exception ex) { + logger.error("", ex); + return new ErrorResponse(req.getRequestURL().toString(), ex); + } + + @ResponseStatus(HttpStatus.FORBIDDEN) + @ExceptionHandler(ForbiddenException.class) + @ResponseBody + ErrorResponse handleForbidden(HttpServletRequest req, Exception ex) { + return new ErrorResponse(req.getRequestURL().toString(), ex); + } + + @ResponseStatus(HttpStatus.NOT_FOUND) + @ExceptionHandler(NotFoundException.class) + @ResponseBody + ErrorResponse handleNotFound(HttpServletRequest req, Exception ex) { + return new ErrorResponse(req.getRequestURL().toString(), ex); + } + + @ResponseStatus(HttpStatus.BAD_REQUEST) + @ExceptionHandler(BadRequestException.class) + @ResponseBody + ErrorResponse handleBadRequest(HttpServletRequest req, Exception ex) { + logger.error("", ex); + return new ErrorResponse(req.getRequestURL().toString(), ex); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/DataController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/DataController.java new file mode 100644 index 00000000000..2fdc218ae2b --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/DataController.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.controller; + +import java.util.Set; + +import org.apache.commons.codec.binary.Base64; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.util.Pair; +import org.apache.kylin.cube.CubeDescManager; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.gridtable.StorageSideBehavior; +import org.apache.kylin.metadata.filter.StringCodeSystem; +import org.apache.kylin.metadata.filter.TupleFilter; +import org.apache.kylin.metadata.filter.TupleFilterSerializer; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.model.DataRequest; +import org.apache.kylin.stream.core.model.DataResponse; +import org.apache.kylin.stream.core.query.IStreamingSearchResult; +import org.apache.kylin.stream.core.query.StreamingCubeDataSearcher; +import org.apache.kylin.stream.core.query.StreamingSearchContext; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.core.storage.Record; +import org.apache.kylin.stream.core.util.RecordsSerializer; +import org.apache.kylin.stream.server.StreamingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Sets; + +@Controller +@RequestMapping(value = "/data") +public class DataController extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(DataController.class); + private StreamingServer streamingServer; + + public DataController() { + streamingServer = StreamingServer.getInstance(); + } + + @RequestMapping(value = "/query", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public DataResponse query(@RequestBody DataRequest dataRequest) { + IStreamingSearchResult searchResult = null; + String queryId = dataRequest.getQueryId(); + StreamingQueryProfile queryProfile = new StreamingQueryProfile(queryId, dataRequest.getRequestSendTime()); + if (dataRequest.isEnableDetailProfile()) { + queryProfile.enableDetailProfile(); + } + if (dataRequest.getStorageBehavior() != null) { + queryProfile.setStorageBehavior(StorageSideBehavior.valueOf(dataRequest.getStorageBehavior())); + } + StreamingQueryProfile.set(queryProfile); + logger.info("receive query request queryId:{}", queryId); + try { + final Stopwatch sw = new Stopwatch(); + sw.start(); + String cubeName = dataRequest.getCubeName(); + long minSegmentTime = dataRequest.getMinSegmentTime(); + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + CubeDesc cubeDesc = CubeDescManager.getInstance(kylinConfig).getCubeDesc(cubeName); + + Set metrics = convertMetrics(cubeDesc, dataRequest.getMetrics()); + byte[] tupleFilterBytes = Base64.decodeBase64(dataRequest.getTupleFilter()); + TupleFilter tupleFilter = TupleFilterSerializer.deserialize(tupleFilterBytes, StringCodeSystem.INSTANCE); + + TupleFilter havingFilter = null; + if (dataRequest.getHavingFilter() != null) { + byte[] havingFilterBytes = Base64.decodeBase64(dataRequest.getHavingFilter()); + havingFilter = TupleFilterSerializer.deserialize(havingFilterBytes, StringCodeSystem.INSTANCE); + } + + Set dimensions = convertToTblColRef(dataRequest.getDimensions(), cubeDesc); + Set groups = convertToTblColRef(dataRequest.getGroups(), cubeDesc); + + StreamingSegmentManager segmentManager = streamingServer.getStreamingSegmentManager(cubeName); + StreamingCubeDataSearcher dataSearcher = segmentManager.getSearcher(); + + StreamingSearchContext gtSearchRequest = new StreamingSearchContext(cubeDesc, dimensions, groups, + metrics, tupleFilter, havingFilter); + searchResult = dataSearcher.doSearch(gtSearchRequest, minSegmentTime, + dataRequest.isAllowStorageAggregation()); + + if (StorageSideBehavior.RAW_SCAN == queryProfile.getStorageBehavior()) { + long counter = 0; + for (Record record : searchResult) { + counter ++; + } + logger.info("query-{}: scan {} rows", queryId, counter); + } + RecordsSerializer serializer = new RecordsSerializer(gtSearchRequest.getRespResultSchema()); + Pair serializedRowsInfo = serializer.serialize(searchResult.iterator(), + dataRequest.getStoragePushDownLimit()); + DataResponse dataResponse = new DataResponse(); + dataResponse.setData(Base64.encodeBase64String(serializedRowsInfo.getFirst())); + sw.stop(); + logger.info("query-{}: return response, took {} ms", queryId, sw.elapsedMillis()); + long finalCnt = serializedRowsInfo.getSecond(); + queryProfile.setFinalRows(finalCnt); + String profileInfo = queryProfile.toString(); + dataResponse.setProfile(profileInfo); + logger.info("query-{}: profile: {}", queryId, profileInfo); + return dataResponse; + } catch (Exception e) { + throw new StreamingException(e); + } finally { + if (searchResult != null) { + try { + searchResult.close(); + } catch (Exception e) { + logger.error("Fail to close result scanner, query id:" + queryId); + } + } + } + } + + private Set convertMetrics(CubeDesc cubeDesc, Set metrics) { + Set result = Sets.newHashSet(); + for (FunctionDesc metric : metrics) { + result.add(findAggrFuncFromCubeDesc(cubeDesc, metric)); + } + return result; + } + + private FunctionDesc findAggrFuncFromCubeDesc(CubeDesc cubeDesc, FunctionDesc aggrFunc) { + aggrFunc.init(cubeDesc.getModel()); + for (MeasureDesc measure : cubeDesc.getMeasures()) { + if (measure.getFunction().equals(aggrFunc)) + return measure.getFunction(); + } + return aggrFunc; + } + + private Set convertToTblColRef(Set columnSet, CubeDesc cubeDesc) { + Set result = Sets.newHashSet(); + for (String columnName : columnSet) { + TblColRef tblColRef = cubeDesc.getModel().findColumn(columnName); + result.add(tblColRef); + } + return result; + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/QueryController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/QueryController.java new file mode 100644 index 00000000000..187fc2556a6 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/QueryController.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.server.rest.controller; + +import org.apache.kylin.common.debug.BackdoorToggles; +import org.apache.kylin.stream.server.rest.exception.InternalErrorException; +import org.apache.kylin.stream.server.rest.model.SQLRequest; +import org.apache.kylin.stream.server.rest.model.SQLResponse; +import org.apache.kylin.stream.server.rest.service.QueryService; +import org.apache.kylin.stream.server.rest.util.QueryUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; + +import java.util.Locale; + +/** + * Handle query requests. + * + */ +@Controller +public class QueryController extends BasicController { + + public static final String SUCCESS_QUERY_CACHE = "StorageCache"; + public static final String EXCEPTION_QUERY_CACHE = "ExceptionQueryCache"; + private static final Logger logger = LoggerFactory.getLogger(QueryController.class); + @Autowired + private QueryService queryService; + + @RequestMapping(value = "/query", method = RequestMethod.POST, produces = { "application/json" }) + @ResponseBody + public SQLResponse query(@RequestBody SQLRequest sqlRequest) { + return doQueryWithCache(sqlRequest); + } + + private SQLResponse doQueryWithCache(SQLRequest sqlRequest) { + try { + BackdoorToggles.setToggles(sqlRequest.getBackdoorToggles()); + + String sql = sqlRequest.getSql(); + String project = sqlRequest.getProject(); + logger.info("Using project: " + project); + logger.info("The original query: " + sql); + + if (!sql.toLowerCase(Locale.ROOT).contains("select")) { + logger.debug("Directly return exception as not supported"); + throw new InternalErrorException("Not Supported SQL."); + } + + long startTime = System.currentTimeMillis(); + SQLResponse sqlResponse; + try { + sqlResponse = queryService.query(sqlRequest); + sqlResponse.setDuration(System.currentTimeMillis() - startTime); + logger.info( + "Stats of SQL response: isException: {}, duration: {}, total scan count {}", // + new String[] { String.valueOf(sqlResponse.getIsException()), + String.valueOf(sqlResponse.getDuration()), + String.valueOf(sqlResponse.getTotalScanCount()) }); + + } catch (Throwable e) { // calcite may throw AssertError + logger.error("Exception when execute sql", e); + String errMsg = QueryUtil.makeErrorMsgUserFriendly(e); + + sqlResponse = new SQLResponse(null, null, 0, true, errMsg); + + } + + if (sqlResponse.getIsException()) + throw new InternalErrorException(sqlResponse.getExceptionMessage()); + + return sqlResponse; + + } finally { + BackdoorToggles.cleanToggles(); + } + } + + public void setQueryService(QueryService queryService) { + this.queryService = queryService; + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/StatsController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/StatsController.java new file mode 100644 index 00000000000..cb9760ea4f1 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/StatsController.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.server.rest.controller; + +import org.apache.kylin.stream.core.model.HealthCheckInfo; +import org.apache.kylin.stream.core.model.stats.ReceiverCubeStats; +import org.apache.kylin.stream.core.model.stats.ReceiverStats; +import org.apache.kylin.stream.server.StreamingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; + +/** + * Handle statistics requests. + * + */ +@Controller +@RequestMapping(value = "/stats") +public class StatsController extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(StatsController.class); + + private StreamingServer streamingServer; + + public StatsController() { + streamingServer = StreamingServer.getInstance(); + } + + @RequestMapping(method = RequestMethod.GET, produces = { "application/json" }) + @ResponseBody + public ReceiverStats getAllStats() { + return streamingServer.getReceiverStats(); + } + + @RequestMapping(value = "/cubes/{cubeName}", method = RequestMethod.GET, produces = { "application/json" }) + @ResponseBody + public ReceiverCubeStats getCubeStats(@PathVariable String cubeName) { + return streamingServer.getCubeStats(cubeName); + } + + @RequestMapping(value = "/healthCheck", method = RequestMethod.GET, produces = { "application/json" }) + @ResponseBody + public HealthCheckInfo healthCheck() { + HealthCheckInfo result = new HealthCheckInfo(); + result.setStatus(HealthCheckInfo.Status.GOOD); + return result; + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/SystemController.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/SystemController.java new file mode 100644 index 00000000000..c6fcf1d460c --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/controller/SystemController.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.server.rest.controller; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; + +import javax.servlet.http.HttpServletResponse; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.kylin.stream.server.StreamingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Controller; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.ResponseBody; + +/** + * Handle system requests. + */ +@Controller +@RequestMapping(value = "/system") +public class SystemController extends BasicController { + private static final Logger logger = LoggerFactory.getLogger(SystemController.class); + + private StreamingServer streamingServer; + + public SystemController() { + streamingServer = StreamingServer.getInstance(); + } + + @RequestMapping(value = "/logLevel/{loggerName}/{logLevel}", method = RequestMethod.PUT, produces = { "application/json" }) + @ResponseBody + public void setLogLevel(@PathVariable(value = "loggerName") String loggerName, + @PathVariable(value = "logLevel") String logLevel) { + // we know it use log4j + org.apache.log4j.Logger logger = org.apache.log4j.LogManager.getLogger(loggerName); + org.apache.log4j.Level level = org.apache.log4j.Level.toLevel(logLevel); + logger.setLevel(level); + } + + @RequestMapping(value = "/logLevel/{loggerName}", method = RequestMethod.GET, produces = { "application/json" }) + @ResponseBody + public String getLogLevel(@PathVariable(value = "loggerName") String loggerName) { + org.apache.log4j.Logger logger = org.apache.log4j.LogManager.getLogger(loggerName); + org.apache.log4j.Level level = logger.getEffectiveLevel(); + if (level != null) { + return level.toString(); + } + return null; + } + + @RequestMapping(value = "/threadDump", method = RequestMethod.GET, produces = { "application/json" }) + @ResponseBody + public void threadDump(HttpServletResponse response) { + response.setContentType("text/plain;charset=utf-8"); + OutputStream outputStream = null; + try { + outputStream = response.getOutputStream(); + ReflectionUtils.printThreadInfo(new PrintStream(outputStream, false, "UTF-8"), "Thread Dump"); + } catch (IOException e) { + logger.error("exception when get stack trace", e); + IOUtils.closeQuietly(outputStream); + } + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/BadRequestException.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/BadRequestException.java new file mode 100644 index 00000000000..f4178fbb3f1 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/BadRequestException.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.exception; + +import org.springframework.http.HttpStatus; +import org.springframework.web.bind.annotation.ResponseStatus; + +/** + * @author xduo + * + */ +@ResponseStatus(value = HttpStatus.BAD_REQUEST) +public class BadRequestException extends RuntimeException { + /** + * + */ + private static final long serialVersionUID = -6798154278095441848L; + + public BadRequestException(String s) { + super(s); + } + + /** + * + */ + public BadRequestException() { + super(); + } + + /** + * @param arg0 + * @param arg1 + */ + public BadRequestException(String arg0, Throwable arg1) { + super(arg0, arg1); + } + + /** + * @param arg0 + */ + public BadRequestException(Throwable arg0) { + super(arg0); + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/ForbiddenException.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/ForbiddenException.java new file mode 100644 index 00000000000..94b76f777a6 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/ForbiddenException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.exception; + +import org.springframework.http.HttpStatus; +import org.springframework.web.bind.annotation.ResponseStatus; + +/** + * @author xduo + * + */ +@ResponseStatus(value = HttpStatus.FORBIDDEN) +public class ForbiddenException extends RuntimeException { + + private static final long serialVersionUID = 2741885728370162194L; + + public ForbiddenException() { + super(); + } + + public ForbiddenException(String message) { + super(message); + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/InternalErrorException.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/InternalErrorException.java new file mode 100644 index 00000000000..c88b5ce5f76 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/InternalErrorException.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.exception; + +import org.springframework.http.HttpStatus; +import org.springframework.web.bind.annotation.ResponseStatus; + +/** + * Class to wrap backend exception + * + * @author jianliu + * + */ +@ResponseStatus(value = HttpStatus.INTERNAL_SERVER_ERROR) +public class InternalErrorException extends RuntimeException { + /** + * + */ + private static final long serialVersionUID = -6798154278095441848L; + + public InternalErrorException(String s) { + super(s); + } + + /** + * + */ + public InternalErrorException() { + super(); + } + + /** + * @param arg0 + * @param arg1 + */ + public InternalErrorException(String arg0, Throwable arg1) { + super(arg0, arg1); + } + + /** + * @param arg0 + */ + public InternalErrorException(Throwable arg0) { + super(arg0); + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/NotFoundException.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/NotFoundException.java new file mode 100644 index 00000000000..12e69be392f --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/exception/NotFoundException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.exception; + +import org.springframework.http.HttpStatus; +import org.springframework.web.bind.annotation.ResponseStatus; + +/** + * @author xduo + * + */ +@ResponseStatus(value = HttpStatus.NOT_FOUND) +public class NotFoundException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public NotFoundException(String s) { + super(s); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/ErrorResponse.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/ErrorResponse.java new file mode 100644 index 00000000000..a889313cafd --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/ErrorResponse.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.model; + +/** + * @author xduo + * + */ +public class ErrorResponse { + + public String url; + public String exception; + + /** + * @param exception + */ + public ErrorResponse(String url, Exception exception) { + this.url = url; + this.exception = exception.getLocalizedMessage(); + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/PrepareSqlRequest.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/PrepareSqlRequest.java new file mode 100644 index 00000000000..e909b023a12 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/PrepareSqlRequest.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.model; + +import java.util.Arrays; + +/** + * @author xduo + * + */ +public class PrepareSqlRequest extends SQLRequest { + + private StateParam[] params; + + public PrepareSqlRequest() { + super(); + } + + public StateParam[] getParams() { + return params; + } + + public void setParams(StateParam[] params) { + this.params = params; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((params == null) ? 0 : Arrays.hashCode(params)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + if (!super.equals(obj)) + return false; + PrepareSqlRequest other = (PrepareSqlRequest) obj; + if (!Arrays.equals(params, other.params)) + return false; + return true; + } + + public static class StateParam { + private String className; + private String value; + + public StateParam() { + } + + public String getClassName() { + return className; + } + + public void setClassName(String className) { + this.className = className; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((className == null) ? 0 : className.hashCode()); + result = prime * result + ((value == null) ? 0 : value.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + StateParam other = (StateParam) obj; + if (className == null) { + if (other.className != null) + return false; + } else if (!className.equals(other.className)) + return false; + if (value == null) { + if (other.value != null) + return false; + } else if (!value.equals(other.value)) + return false; + return true; + } + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLRequest.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLRequest.java new file mode 100644 index 00000000000..ad9d72e80e9 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLRequest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.model; + +import java.io.Serializable; +import java.util.Map; + +public class SQLRequest implements Serializable { + protected static final long serialVersionUID = 1L; + + private String sql; + private String project; + private Integer offset = 0; + private Integer limit = 0; + private boolean acceptPartial = false; + + private Map backdoorToggles; + + public SQLRequest() { + } + + public Map getBackdoorToggles() { + return backdoorToggles; + } + + public void setBackdoorToggles(Map backdoorToggles) { + this.backdoorToggles = backdoorToggles; + } + + public String getProject() { + return project; + } + + public void setProject(String project) { + this.project = project; + } + + public String getSql() { + return sql; + } + + public void setSql(String sql) { + this.sql = sql; + } + + public Integer getOffset() { + return offset; + } + + public void setOffset(Integer offset) { + this.offset = offset; + } + + public Integer getLimit() { + return limit; + } + + public void setLimit(Integer limit) { + this.limit = limit; + } + + public boolean isAcceptPartial() { + return acceptPartial; + } + + public void setAcceptPartial(boolean acceptPartial) { + this.acceptPartial = acceptPartial; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (acceptPartial ? 1231 : 1237); + result = prime * result + ((offset == null) ? 0 : offset.hashCode()); + result = prime * result + ((limit == null) ? 0 : limit.hashCode()); + result = prime * result + ((project == null) ? 0 : project.hashCode()); + result = prime * result + ((sql == null) ? 0 : sql.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + SQLRequest other = (SQLRequest) obj; + if (acceptPartial != other.acceptPartial) + return false; + if (offset == null) { + if (other.offset != null) + return false; + } else if (!offset.equals(other.offset)) + return false; + if (limit == null) { + if (other.limit != null) + return false; + } else if (!limit.equals(other.limit)) + return false; + if (project == null) { + if (other.project != null) + return false; + } else if (!project.equals(other.project)) + return false; + if (sql == null) { + if (other.sql != null) + return false; + } else if (!sql.equals(other.sql)) + return false; + return true; + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLResponse.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLResponse.java new file mode 100644 index 00000000000..43c8f717a41 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/model/SQLResponse.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.model; + +import org.apache.kylin.metadata.querymeta.SelectedColumnMeta; + +import java.io.Serializable; +import java.util.List; + +public class SQLResponse implements Serializable { + protected static final long serialVersionUID = 1L; + + // private static final Logger logger = + // LoggerFactory.getLogger(SQLResponse.class); + + // the data type for each column + protected List columnMetas; + + // the results rows, each row contains several columns + protected List> results; + + /** + * for historical reasons it is named "cube", however it might also refer to any realizations like hybrid, II or etc. + */ + protected String cube; + + // if not select query, only return affected row count + protected int affectedRowCount; + + // flag indicating whether an exception occurred + protected boolean isException; + + // if isException, the detailed exception message + protected String exceptionMessage; + + protected long duration; + + protected boolean isPartial = false; + + protected long totalScanCount; + + protected boolean hitExceptionCache = false; + + protected boolean storageCacheUsed = false; + + public SQLResponse() { + } + + public SQLResponse(List columnMetas, List> results, int affectedRowCount, + boolean isException, String exceptionMessage) { + this.columnMetas = columnMetas; + this.results = results; + this.affectedRowCount = affectedRowCount; + this.isException = isException; + this.exceptionMessage = exceptionMessage; + } + + public SQLResponse(List columnMetas, List> results, String cube, + int affectedRowCount, boolean isException, String exceptionMessage) { + this.columnMetas = columnMetas; + this.results = results; + this.cube = cube; + this.affectedRowCount = affectedRowCount; + this.isException = isException; + this.exceptionMessage = exceptionMessage; + } + + public SQLResponse(List columnMetas, List> results, String cube, + int affectedRowCount, boolean isException, String exceptionMessage, boolean isPartial) { + this.columnMetas = columnMetas; + this.results = results; + this.cube = cube; + this.affectedRowCount = affectedRowCount; + this.isException = isException; + this.exceptionMessage = exceptionMessage; + this.isPartial = isPartial; + } + + public List getColumnMetas() { + return columnMetas; + } + + public List> getResults() { + return results; + } + + public void setResults(List> results) { + this.results = results; + } + + public String getCube() { + return cube; + } + + public int getAffectedRowCount() { + return affectedRowCount; + } + + public boolean getIsException() { + return isException; + } + + public void setIsException(boolean v) { + isException = v; + } + + public String getExceptionMessage() { + return exceptionMessage; + } + + public void setExceptionMessage(String msg) { + exceptionMessage = msg; + } + + public long getDuration() { + return duration; + } + + public void setDuration(long duration) { + this.duration = duration; + } + + public boolean isPartial() { + + return isPartial; + } + + public long getTotalScanCount() { + return totalScanCount; + } + + public void setTotalScanCount(long totalScanCount) { + this.totalScanCount = totalScanCount; + } + + public boolean isHitExceptionCache() { + return hitExceptionCache; + } + + public void setHitExceptionCache(boolean hitExceptionCache) { + this.hitExceptionCache = hitExceptionCache; + } + + public boolean isStorageCacheUsed() { + return storageCacheUsed; + } + + public void setStorageCacheUsed(boolean storageCacheUsed) { + this.storageCacheUsed = storageCacheUsed; + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/security/StreamTableInterceptor.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/security/StreamTableInterceptor.java new file mode 100644 index 00000000000..1e22e778c27 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/security/StreamTableInterceptor.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.security; + +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.query.relnode.OLAPContext; +import org.apache.kylin.query.relnode.OLAPTableScan; +import org.apache.kylin.query.security.QueryInterceptor; + +import com.google.common.collect.Sets; + +public class StreamTableInterceptor extends QueryInterceptor { + + @Override + protected boolean isEnabled() { + return KylinConfig.getInstanceFromEnv().isTableACLEnabled(); + } + + @Override + public Set getQueryIdentifiers(List contexts) { + return getAllTblsWithSchema(contexts); + } + + @Override + protected Set getIdentifierBlackList(List contexts) { + return Sets.newHashSet(); + } + + @Override + protected String getIdentifierType() { + return "table"; + } + + private Set getAllTblsWithSchema(List contexts) { + // all tables with DB, Like DB.TABLE + Set tableWithSchema = new TreeSet<>(String.CASE_INSENSITIVE_ORDER); + for (OLAPContext context : contexts) { + for (OLAPTableScan tableScan : context.allTableScans) { + tableWithSchema.add(tableScan.getTableRef().getTableIdentity()); + } + } + return tableWithSchema; + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/BasicService.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/BasicService.java new file mode 100644 index 00000000000..e3d68f4c553 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/BasicService.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.service; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.cube.CubeDescManager; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.metadata.project.ProjectManager; +import org.apache.kylin.storage.hybrid.HybridManager; + +public abstract class BasicService { + + public KylinConfig getConfig() { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + + if (kylinConfig == null) { + throw new IllegalArgumentException("Failed to load kylin config instance"); + } + + return kylinConfig; + } + + public CubeManager getCubeManager() { + return CubeManager.getInstance(getConfig()); + } + + public CubeDescManager getCubeDescManager() { + return CubeDescManager.getInstance(getConfig()); + } + + public ProjectManager getProjectManager() { + return ProjectManager.getInstance(getConfig()); + } + + public HybridManager getHybridManager() { + return HybridManager.getInstance(getConfig()); + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/QueryService.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/QueryService.java new file mode 100644 index 00000000000..07109d77492 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/service/QueryService.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.service; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.calcite.avatica.ColumnMetaData.Rep; +import org.apache.kylin.common.QueryContext; +import org.apache.kylin.common.QueryContextFacade; +import org.apache.kylin.metadata.querymeta.SelectedColumnMeta; +import org.apache.kylin.query.QueryConnection; +import org.apache.kylin.query.relnode.OLAPContext; +import org.apache.kylin.stream.server.rest.model.PrepareSqlRequest; +import org.apache.kylin.stream.server.rest.model.SQLRequest; +import org.apache.kylin.stream.server.rest.model.SQLResponse; +import org.apache.kylin.stream.server.rest.util.QueryUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; + +import com.google.common.collect.Lists; + +/** + * @author xduo + */ +@Component("queryService") +public class QueryService { + + private static final Logger logger = LoggerFactory.getLogger(QueryService.class); + + public QueryService() { + } + + private static void close(ResultSet resultSet, Statement stat, Connection conn) { + OLAPContext.clearParameter(); + + if (resultSet != null) + try { + resultSet.close(); + } catch (SQLException e) { + logger.error("failed to close", e); + } + if (stat != null) + try { + stat.close(); + } catch (SQLException e) { + logger.error("failed to close", e); + } + if (conn != null) + try { + conn.close(); + } catch (SQLException e) { + logger.error("failed to close", e); + } + } + + public SQLResponse query(SQLRequest sqlRequest) throws Exception { + return queryWithSqlMassage(sqlRequest); + } + + private SQLResponse queryWithSqlMassage(SQLRequest sqlRequest) throws Exception { + SQLResponse fakeResponse = QueryUtil.tableauIntercept(sqlRequest.getSql()); + if (null != fakeResponse) { + logger.debug("Return fake response, is exception? " + fakeResponse.getIsException()); + return fakeResponse; + } + + String correctedSql = QueryUtil.massageSql(sqlRequest); + if (correctedSql.equals(sqlRequest.getSql()) == false) + logger.info("The corrected query: " + correctedSql); + + // add extra parameters into olap context, like acceptPartial + Map parameters = new HashMap(); + parameters.put(OLAPContext.PRM_ACCEPT_PARTIAL_RESULT, String.valueOf(sqlRequest.isAcceptPartial())); + OLAPContext.setParameters(parameters); + + return execute(correctedSql, sqlRequest); + } + + /** + * @param sql + * @param sqlRequest + * @return + * @throws Exception + */ + private SQLResponse execute(String sql, SQLRequest sqlRequest) throws Exception { + Connection conn = null; + Statement stat = null; + ResultSet resultSet = null; + + List> results = Lists.newArrayList(); + List columnMetas = Lists.newArrayList(); + + try { + conn = QueryConnection.getConnection(sqlRequest.getProject()); + if (sqlRequest instanceof PrepareSqlRequest) { + PreparedStatement preparedState = conn.prepareStatement(sql); + + for (int i = 0; i < ((PrepareSqlRequest) sqlRequest).getParams().length; i++) { + setParam(preparedState, i + 1, ((PrepareSqlRequest) sqlRequest).getParams()[i]); + } + + resultSet = preparedState.executeQuery(); + } else { + stat = conn.createStatement(); + resultSet = stat.executeQuery(sql); + } + + ResultSetMetaData metaData = resultSet.getMetaData(); + int columnCount = metaData.getColumnCount(); + + // Fill in selected column meta + for (int i = 1; i <= columnCount; ++i) { + columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), + metaData.isSearchable(i), metaData.isCurrency(i), metaData.isNullable(i), metaData.isSigned(i), + metaData.getColumnDisplaySize(i), metaData.getColumnLabel(i), metaData.getColumnName(i), + metaData.getSchemaName(i), metaData.getCatalogName(i), metaData.getTableName(i), metaData + .getPrecision(i), metaData.getScale(i), metaData.getColumnType(i), metaData + .getColumnTypeName(i), metaData.isReadOnly(i), metaData.isWritable(i), metaData + .isDefinitelyWritable(i))); + } + + // fill in results + while (resultSet.next()) { + List oneRow = Lists.newArrayListWithCapacity(columnCount); + for (int i = 0; i < columnCount; i++) { + oneRow.add((resultSet.getString(i + 1))); + } + + results.add(oneRow); + } + } finally { + close(resultSet, stat, conn); + } + QueryContext queryContext = QueryContextFacade.current(); + StringBuilder cubeSb = new StringBuilder(); + StringBuilder logSb = new StringBuilder("Processed rows for each storageContext: "); + boolean isPartialResult = false; + + if (OLAPContext.getThreadLocalContexts() != null) { // contexts can be null in case of 'explain plan for' + for (OLAPContext ctx : OLAPContext.getThreadLocalContexts()) { + String realizationName = "NULL"; + int realizationType = -1; + if (ctx.realization != null) { + isPartialResult |= ctx.storageContext.isPartialResultReturned(); + if (cubeSb.length() > 0) { + cubeSb.append(","); + } + cubeSb.append(ctx.realization.getCanonicalName()); + logSb.append(ctx.storageContext.getProcessedRowCount()).append(" "); + + realizationName = ctx.realization.getName(); + realizationType = ctx.realization.getStorageType(); + } + queryContext.setContextRealization(ctx.id, realizationName, realizationType); + } + } + + SQLResponse response = new SQLResponse(columnMetas, results, cubeSb.toString(), 0, false, null, isPartialResult); + response.setTotalScanCount(queryContext.getScannedRows()); + + return response; + } + + /** + * @param preparedState + * @param param + * @throws SQLException + */ + private void setParam(PreparedStatement preparedState, int index, PrepareSqlRequest.StateParam param) + throws SQLException { + boolean isNull = (null == param.getValue()); + + Class clazz; + try { + clazz = Class.forName(param.getClassName()); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e.getMessage(), e); + } + + Rep rep = Rep.of(clazz); + + switch (rep) { + case PRIMITIVE_CHAR: + case CHARACTER: + case STRING: + preparedState.setString(index, isNull ? null : String.valueOf(param.getValue())); + break; + case PRIMITIVE_INT: + case INTEGER: + preparedState.setInt(index, isNull ? 0 : Integer.valueOf(param.getValue())); + break; + case PRIMITIVE_SHORT: + case SHORT: + preparedState.setShort(index, isNull ? 0 : Short.valueOf(param.getValue())); + break; + case PRIMITIVE_LONG: + case LONG: + preparedState.setLong(index, isNull ? 0 : Long.valueOf(param.getValue())); + break; + case PRIMITIVE_FLOAT: + case FLOAT: + preparedState.setFloat(index, isNull ? 0 : Float.valueOf(param.getValue())); + break; + case PRIMITIVE_DOUBLE: + case DOUBLE: + preparedState.setDouble(index, isNull ? 0 : Double.valueOf(param.getValue())); + break; + case PRIMITIVE_BOOLEAN: + case BOOLEAN: + preparedState.setBoolean(index, !isNull && Boolean.parseBoolean(param.getValue())); + break; + case PRIMITIVE_BYTE: + case BYTE: + preparedState.setByte(index, isNull ? 0 : Byte.valueOf(param.getValue())); + break; + case JAVA_UTIL_DATE: + case JAVA_SQL_DATE: + preparedState.setDate(index, isNull ? null : java.sql.Date.valueOf(param.getValue())); + break; + case JAVA_SQL_TIME: + preparedState.setTime(index, isNull ? null : Time.valueOf(param.getValue())); + break; + case JAVA_SQL_TIMESTAMP: + preparedState.setTimestamp(index, isNull ? null : Timestamp.valueOf(param.getValue())); + break; + default: + preparedState.setObject(index, isNull ? null : param.getValue()); + } + } + + private int getInt(String content) { + try { + return Integer.parseInt(content); + } catch (Exception e) { + return -1; + } + } + + private short getShort(String content) { + try { + return Short.parseShort(content); + } catch (Exception e) { + return -1; + } + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/util/QueryUtil.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/util/QueryUtil.java new file mode 100644 index 00000000000..200323cd31a --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/rest/util/QueryUtil.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.rest.util; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.kylin.metadata.querymeta.SelectedColumnMeta; +import org.apache.kylin.stream.server.rest.model.SQLRequest; +import org.apache.kylin.stream.server.rest.model.SQLResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + */ +public class QueryUtil { + + protected static final Logger logger = LoggerFactory.getLogger(QueryUtil.class); + + private static final String S0 = "\\s*"; + private static final String S1 = "\\s"; + private static final String SM = "\\s+"; + private static final Pattern PTN_GROUP_BY = Pattern + .compile(S1 + "GROUP" + SM + "BY" + S1, Pattern.CASE_INSENSITIVE); + private static final Pattern PTN_HAVING_COUNT_GREATER_THAN_ZERO = Pattern.compile(S1 + "HAVING" + SM + "[(]?" + S0 + + "COUNT" + S0 + "[(]" + S0 + "1" + S0 + "[)]" + S0 + ">" + S0 + "0" + S0 + "[)]?", + Pattern.CASE_INSENSITIVE); + private static final Pattern PTN_SUM_1 = Pattern.compile(S1 + "SUM" + S0 + "[(]" + S0 + "[1]" + S0 + "[)]" + S1, + Pattern.CASE_INSENSITIVE); + + // private static final Pattern PTN_HAVING_ESCAPE_FUNCTION = + // Pattern.compile("\\{fn" + "(" + S0 + ")" + "\\}", + // Pattern.CASE_INSENSITIVE); + private static final Pattern PTN_HAVING_ESCAPE_FUNCTION = Pattern.compile("\\{fn" + "(.*?)" + "\\}", + Pattern.CASE_INSENSITIVE); + + private static String[] tableauTestQueries = new String[] { + "SELECT 1", + "CREATE LOCAL TEMPORARY TABLE \"XTableau_B_Connect\" ( \"COL\" INTEGER ) ON COMMIT PRESERVE ROWS", + "DROP TABLE \"XTableau_B_Connect\"", + "SELECT \"COL\" FROM (SELECT 1 AS \"COL\") AS \"SUBQUERY\"", + "SELECT TOP 1 \"COL\" FROM (SELECT 1 AS \"COL\") AS \"CHECKTOP\"", + "SELECT \"COL\" FROM (SELECT 1 AS \"COL\") AS \"CHECKTOP\" LIMIT 1", + "SELECT \"SUBCOL\" AS \"COL\" FROM ( SELECT 1 AS \"SUBCOL\" ) \"SUBQUERY\" GROUP BY 1", + "SELECT \"SUBCOL\" AS \"COL\" FROM ( SELECT 1 AS \"SUBCOL\" ) \"SUBQUERY\" GROUP BY 2", + "INSERT INTO \"XTableau_C_Connect\" SELECT * FROM (SELECT 1 AS COL) AS CHECKTEMP LIMIT 1", + "DROP TABLE \"XTableau_C_Connect\"", + "INSERT INTO \"XTableau_B_Connect\" SELECT * FROM (SELECT 1 AS COL) AS CHECKTEMP LIMIT 1" }; + + private static SQLResponse temp = new SQLResponse(new LinkedList() { + private static final long serialVersionUID = -8086728462624901359L; + + { + add(new SelectedColumnMeta(false, false, true, false, 2, true, 11, "COL", "COL", "", "", "", 10, 0, 4, + "int4", false, true, false)); + } + }, new LinkedList>() { + private static final long serialVersionUID = -470083340592928073L; + + { + add(new LinkedList() { + private static final long serialVersionUID = -3673192785838230054L; + + { + add("1"); + } + }); + } + }, 0, false, null); + + private static SQLResponse[] fakeResponses = new SQLResponse[] { temp, + new SQLResponse(null, null, 0, false, null), // + new SQLResponse(null, null, 0, false, null), // + temp, // + new SQLResponse(null, null, 0, true, "near 1 syntax error"), // + temp, // + new SQLResponse(null, null, 0, true, "group by 1????"), // + new SQLResponse(null, null, 0, true, "group by 2????"), // + new SQLResponse(null, null, 0, true, "XTableau_C_Connect not exist"), // + new SQLResponse(null, null, 0, true, "XTableau_C_Connect not exist"), + new SQLResponse(null, null, 0, true, "XTableau_B_Connect not exist"), }; + + private static ArrayList> tableauTestQueriesInToken = new ArrayList>(); + + static { + for (String q : tableauTestQueries) { + HashSet temp = new HashSet(); + for (String token : q.split("[\r\n\t \\(\\)]")) { + temp.add(token); + } + temp.add(""); + tableauTestQueriesInToken.add(temp); + } + } + + public static String massageSql(SQLRequest sqlRequest) { + String sql = sqlRequest.getSql(); + sql = sql.trim(); + + while (sql.endsWith(";")) + sql = sql.substring(0, sql.length() - 1); + + int limit = sqlRequest.getLimit(); + if (limit > 0 && !sql.toLowerCase(Locale.ROOT).contains("limit")) { + sql += ("\nLIMIT " + limit); + } + + int offset = sqlRequest.getOffset(); + if (offset > 0 && !sql.toLowerCase(Locale.ROOT).contains("offset")) { + sql += ("\nOFFSET " + offset); + } + + return healSickSql(sql); + } + + // correct sick / invalid SQL + private static String healSickSql(String sql) { + Matcher m; + + // Case fn{ EXTRACT(...) } + // Use non-greedy regrex matching to remove escape functions + while (true) { + m = PTN_HAVING_ESCAPE_FUNCTION.matcher(sql); + if (!m.find()) + break; + sql = sql.substring(0, m.start()) + m.group(1) + sql.substring(m.end()); + } + + // Case: HAVING COUNT(1)>0 without Group By + // Tableau generates: SELECT SUM(1) AS "COL" FROM "VAC_SW" HAVING + // COUNT(1)>0 + m = PTN_HAVING_COUNT_GREATER_THAN_ZERO.matcher(sql); + if (m.find() && PTN_GROUP_BY.matcher(sql).find() == false) { + sql = sql.substring(0, m.start()) + " " + sql.substring(m.end()); + } + + // Case: SUM(1) + // Replace it with COUNT(1) + while (true) { + m = PTN_SUM_1.matcher(sql); + if (!m.find()) + break; + sql = sql.substring(0, m.start()) + " COUNT(1) " + sql.substring(m.end()); + } + + return sql; + } + + public static SQLResponse tableauIntercept(String sql) { + + String[] tokens = sql.split("[\r\n\t \\(\\)]"); + for (int i = 0; i < tableauTestQueries.length; ++i) { + if (isTokenWiseEqual(tokens, tableauTestQueriesInToken.get(i))) { + logger.info("Hit fake response " + i); + return fakeResponses[i]; + } + } + + return null; + } + + public static String makeErrorMsgUserFriendly(Throwable e) { + String msg = e.getMessage(); + + // pick ParseException error message if possible + Throwable cause = e; + while (cause != null) { + if (cause.getClass().getName().contains("ParseException")) { + msg = cause.getMessage(); + break; + } + cause = cause.getCause(); + } + + return makeErrorMsgUserFriendly(msg); + } + + public static String makeErrorMsgUserFriendly(String errorMsg) { + try { + // make one line + errorMsg = errorMsg.replaceAll("\\s", " "); + + // move cause to be ahead of sql, calcite creates the message pattern below + Pattern pattern = Pattern.compile("error while executing SQL \"(.*)\":(.*)"); + Matcher matcher = pattern.matcher(errorMsg); + if (matcher.find()) { + return matcher.group(2).trim() + "\n" + "while executing SQL: \"" + matcher.group(1).trim() + "\""; + } else + return errorMsg; + } catch (Exception e) { + return errorMsg; + } + } + + private static boolean isTokenWiseEqual(String[] tokens, HashSet tokenSet) { + for (String token : tokens) { + if (!tokenSet.contains(token)) { + return false; + } + } + return true; + } + +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/retention/PolicyInfo.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/retention/PolicyInfo.java new file mode 100644 index 00000000000..08b3a6c3fc0 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/retention/PolicyInfo.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.retention; + +import java.util.Map; + +import com.google.common.collect.Maps; + +public class PolicyInfo { + public static final String PURGE_POLICY = "purge"; + public static final String FULL_BUILD_POLICY = "fullBuild"; + + private String name; + private Map properties = Maps.newHashMap(); + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + public void addProperty(String name, String value) { + properties.put(name, value); + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorage.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorage.java new file mode 100644 index 00000000000..bee6390cdf8 --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorage.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.server.storage; + +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.metadata.realization.IRealization; +import org.apache.kylin.storage.IStorage; +import org.apache.kylin.storage.IStorageQuery; + +@SuppressWarnings("unused") +public class LocalStreamStorage implements IStorage { + + @Override + public IStorageQuery createQuery(IRealization realization) { + return new LocalStreamStorageQuery((CubeInstance) realization); + } + + @Override + public I adaptToBuildEngine(Class engineInterface) { + return null; + } +} diff --git a/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorageQuery.java b/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorageQuery.java new file mode 100644 index 00000000000..25e9383a01c --- /dev/null +++ b/stream-receiver/src/main/java/org/apache/kylin/stream/server/storage/LocalStreamStorageQuery.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.stream.server.storage; + +import org.apache.kylin.common.QueryContextFacade; +import org.apache.kylin.common.debug.BackdoorToggles; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.metadata.realization.SQLDigest; +import org.apache.kylin.metadata.tuple.ITupleIterator; +import org.apache.kylin.metadata.tuple.TupleInfo; +import org.apache.kylin.storage.StorageContext; +import org.apache.kylin.storage.gtrecord.GTCubeStorageQueryBase; +import org.apache.kylin.storage.gtrecord.GTCubeStorageQueryRequest; +import org.apache.kylin.stream.core.query.StreamingQueryProfile; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.server.StreamingServer; + +/** + * local storage query for streaming + */ +public class LocalStreamStorageQuery extends GTCubeStorageQueryBase { + + public LocalStreamStorageQuery(CubeInstance cube) { + super(cube); + } + + @Override + public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) { + StreamingSegmentManager cubeDataStore = StreamingServer.getInstance().getStreamingSegmentManager( + cubeInstance.getName()); + boolean enableStreamProfile = BackdoorToggles.isStreamingProfileEnable(); + StreamingQueryProfile queryProfile = new StreamingQueryProfile(QueryContextFacade.current().getQueryId(), + System.currentTimeMillis()); + if (enableStreamProfile) { + queryProfile.enableDetailProfile(); + } + StreamingQueryProfile.set(queryProfile); + GTCubeStorageQueryRequest request = getStorageQueryRequest(context, sqlDigest, returnTupleInfo); + return cubeDataStore.getSearcher().search(returnTupleInfo, request.getFilter(), request.getHavingFilter(), + request.getDimensions(), request.getGroups(), request.getMetrics(), context.isNeedStorageAggregation()); + } + + @Override + protected String getGTStorage() { + return null; + } + +} diff --git a/stream-receiver/src/main/resources/applicationContext.xml b/stream-receiver/src/main/resources/applicationContext.xml new file mode 100644 index 00000000000..2479c71569c --- /dev/null +++ b/stream-receiver/src/main/resources/applicationContext.xml @@ -0,0 +1,77 @@ + + + + + Kylin Rest Service + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/stream-receiver/src/main/resources/ehcache-test.xml b/stream-receiver/src/main/resources/ehcache-test.xml new file mode 100644 index 00000000000..74612af42fe --- /dev/null +++ b/stream-receiver/src/main/resources/ehcache-test.xml @@ -0,0 +1,37 @@ + + +> + + + + + + + + + + \ No newline at end of file diff --git a/stream-receiver/src/main/resources/ehcache.xml b/stream-receiver/src/main/resources/ehcache.xml new file mode 100644 index 00000000000..af80e3397c3 --- /dev/null +++ b/stream-receiver/src/main/resources/ehcache.xml @@ -0,0 +1,37 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/stream-receiver/src/main/resources/stream-receiver-log4j.properties b/stream-receiver/src/main/resources/stream-receiver-log4j.properties new file mode 100644 index 00000000000..a48b996281a --- /dev/null +++ b/stream-receiver/src/main/resources/stream-receiver-log4j.properties @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +#define appenders +log4j.appender.file=org.apache.log4j.DailyRollingFileAppender +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.File=${KYLIN_HOME}/logs/kylin_streaming_receiver.log +log4j.appender.file.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}:%L : %m%n +log4j.appender.file.Append=true + + +log4j.appender.statechange=org.apache.log4j.DailyRollingFileAppender +log4j.appender.statechange.layout=org.apache.log4j.PatternLayout +log4j.appender.statechange.File=${KYLIN_HOME}/logs/kylin_streaming_statechange.log +log4j.appender.statechange.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}:%L : %m%n +log4j.appender.statechange.Append=true + +#overall config +log4j.rootLogger=INFO,file +log4j.logger.org.apache.kylin=DEBUG +log4j.logger.io.ebay.rheos.kafka.security.iaf=DEBUG +log4j.logger.org.springframework=WARN +log4j.logger.org.springframework.security=INFO + +#statechange config +log4j.logger.org.apache.kylin.stream.server.StreamingServer=DEBUG, statechange +log4j.logger.org.apache.kylin.stream.server.ReplicaSetLeaderSelector=DEBUG, statechange +log4j.logger.org.apache.kylin.stream.server.rest.controller.AdminController=DEBUG, statechange +log4j.logger.org.apache.kylin.stream.core.storage.StreamingSegmentManager=DEBUG, statechange \ No newline at end of file diff --git a/stream-receiver/src/main/webapp/WEB-INF/kylin-servlet.xml b/stream-receiver/src/main/webapp/WEB-INF/kylin-servlet.xml new file mode 100644 index 00000000000..c3f6ded6575 --- /dev/null +++ b/stream-receiver/src/main/webapp/WEB-INF/kylin-servlet.xml @@ -0,0 +1,19 @@ + + + + \ No newline at end of file diff --git a/stream-receiver/src/main/webapp/index.html b/stream-receiver/src/main/webapp/index.html new file mode 100644 index 00000000000..fa1c5b67613 --- /dev/null +++ b/stream-receiver/src/main/webapp/index.html @@ -0,0 +1,23 @@ + + + + + Welcome to Kylin + + + +

Welcome to Kylin!

+ + \ No newline at end of file diff --git a/stream-source-kafka/pom.xml b/stream-source-kafka/pom.xml new file mode 100644 index 00000000000..949cb2f8a9d --- /dev/null +++ b/stream-source-kafka/pom.xml @@ -0,0 +1,79 @@ + + + + + + kylin + org.apache.kylin + 2.6.0-SNAPSHOT + + 4.0.0 + kylin-stream-source-kafka + jar + Apache Kylin - Stream Source Kafka + + + + + + org.apache.kylin + kylin-core-common + ${project.parent.version} + + + org.apache.kylin + kylin-core-metadata + ${project.parent.version} + + + org.apache.kylin + kylin-core-dictionary + ${project.parent.version} + + + org.apache.kylin + kylin-stream-core + ${project.parent.version} + + + org.apache.kylin + kylin-stream-coordinator + ${project.parent.version} + + + org.apache.kylin + kylin-source-hive + ${project.parent.version} + + + + org.apache.kafka + kafka_2.11 + + + + + + junit + junit + test + + + + diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/BootstrapServerConfig.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/BootstrapServerConfig.java new file mode 100644 index 00000000000..0a61f9ee958 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/BootstrapServerConfig.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) +public class BootstrapServerConfig { + + @JsonProperty("host") + private String host; + + @JsonProperty("port") + private int port; + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + public String toString() { + return host + ":" + port; + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaBatchSourceAdaptor.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaBatchSourceAdaptor.java new file mode 100644 index 00000000000..2dc2a9cfdc8 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaBatchSourceAdaptor.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.io.IOException; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.source.hive.HiveSource; +import org.apache.kylin.stream.core.source.StreamingSourceConfig; +import org.apache.kylin.stream.core.source.StreamingSourceConfigManager; + +public class KafkaBatchSourceAdaptor extends HiveSource { + public KafkaBatchSourceAdaptor(KylinConfig config) { + super(config); + } + + @Override + public void unloadTable(String tableName, String project) throws IOException { + KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); + StreamingSourceConfigManager sourceConfigManager = StreamingSourceConfigManager.getInstance(kylinConfig); + StreamingSourceConfig config = sourceConfigManager.getConfig(tableName); + if (config == null) { + return; + } + sourceConfigManager.removeStreamingConfig(config); + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaConsumerStartInfo.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaConsumerStartInfo.java new file mode 100644 index 00000000000..270c9d5f68d --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaConsumerStartInfo.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.util.List; +import java.util.Map; + +public class KafkaConsumerStartInfo { + private List partitions; + private Map partitionOffsets; + + public List getPartitions() { + return partitions; + } + + public void setPartitions(List partitions) { + this.partitions = partitions; + } + + public Map getPartitionOffsets() { + return partitionOffsets; + } + + public void setPartitionOffsets(Map partitionOffsets) { + this.partitionOffsets = partitionOffsets; + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPosition.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPosition.java new file mode 100644 index 00000000000..869e0464aa1 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPosition.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import javax.annotation.Nullable; + +import org.apache.kylin.stream.core.source.ISourcePosition; + +import com.google.common.collect.Maps; +import com.google.common.collect.Maps.EntryTransformer; + +public class KafkaPosition implements ISourcePosition { + private Map partitionOffsetMap = Maps.newHashMap(); + + public KafkaPosition() { + } + + public KafkaPosition(Map partitionOffsetMap) { + this.partitionOffsetMap = partitionOffsetMap; + } + + @Override + public void update(IPartitionPosition point) { + KafkaPartitionPosition kafkaPartitionPosition = (KafkaPartitionPosition) point; + partitionOffsetMap.put(kafkaPartitionPosition.partition, kafkaPartitionPosition.offset); + } + + @Override + public void updateWhenPartitionNotExist(IPartitionPosition partPosition) { + KafkaPartitionPosition kafkaPartitionPosition = (KafkaPartitionPosition) partPosition; + if (!partitionOffsetMap.containsKey(kafkaPartitionPosition.partition)) { + partitionOffsetMap.put(kafkaPartitionPosition.partition, kafkaPartitionPosition.offset); + } + } + + @Override + public ISourcePosition advance() { + Map newOffsetMap = Maps.newHashMap(); + for (Entry partitionOffsetEntry : partitionOffsetMap.entrySet()) { + newOffsetMap.put(partitionOffsetEntry.getKey(), partitionOffsetEntry.getValue() + 1L); + } + return new KafkaPosition(newOffsetMap); + } + + @Override + public Map getPartitionPositions() { + return Maps.transformEntries(partitionOffsetMap, new EntryTransformer() { + @Override + public IPartitionPosition transformEntry(@Nullable Integer key, @Nullable Long value) { + return new KafkaPartitionPosition(key, value); + } + }); + } + + public Map getPartitionOffsets() { + return partitionOffsetMap; + } + + @Override + public String toString() { + return "KafkaPosition{" + + "partitionOffsetMap=" + partitionOffsetMap + + '}'; + } + + @Override + public void copy(ISourcePosition other) { + this.partitionOffsetMap = new HashMap<>(((KafkaPosition) other).partitionOffsetMap); + } + + public static class KafkaPartitionPosition implements IPartitionPosition { + public int partition; + public long offset; + + public KafkaPartitionPosition(int partition, long offset) { + this.partition = partition; + this.offset = offset; + } + + @Override + public int getPartition() { + return partition; + } + + @Override + public int compareTo(IPartitionPosition o) { + KafkaPartitionPosition kafkaPartPos = (KafkaPartitionPosition) o; + return Long.compare(offset, kafkaPartPos.offset); + } + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPositionHandler.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPositionHandler.java new file mode 100644 index 00000000000..6584acca263 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaPositionHandler.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.source.ISourcePosition; +import org.apache.kylin.stream.core.source.ISourcePositionHandler; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; + +public class KafkaPositionHandler implements ISourcePositionHandler { + + @Override + public ISourcePosition mergePositions(Collection positions, MergeStrategy mergeStrategy) { + KafkaPosition result = new KafkaPosition(); + for (ISourcePosition position : positions) { + KafkaPosition kafkaPosition = (KafkaPosition) position; + Map partitionOffsetMap = kafkaPosition.getPartitionOffsets(); + for (Entry partOffsetEntry : partitionOffsetMap.entrySet()) { + Long existPartOffset = result.getPartitionOffsets().get(partOffsetEntry.getKey()); + if (existPartOffset == null) { + result.getPartitionOffsets().put(partOffsetEntry.getKey(), partOffsetEntry.getValue()); + } else { + int compResult = partOffsetEntry.getValue().compareTo(existPartOffset); + if ((mergeStrategy == MergeStrategy.KEEP_LARGE && compResult > 0) + || (mergeStrategy == MergeStrategy.KEEP_SMALL && compResult < 0) + || (mergeStrategy == MergeStrategy.KEEP_LATEST)) { + result.getPartitionOffsets().put(partOffsetEntry.getKey(), partOffsetEntry.getValue()); + } + } + } + } + return result; + } + + @Override + public ISourcePosition createEmptyPosition() { + return new KafkaPosition(); + } + + @Override + public ISourcePosition parsePosition(String positionStr) { + try { + Map partitionOffsetMap = JsonUtil.readValue(positionStr, + new TypeReference>() { + }); + return new KafkaPosition(partitionOffsetMap); + } catch (IOException e) { + throw new StreamingException(e); + } + } + + @Override + public String serializePosition(ISourcePosition position) { + try { + KafkaPosition kafkaPosition = (KafkaPosition) position; + return JsonUtil.writeValueAsString(kafkaPosition.getPartitionOffsets()); + } catch (JsonProcessingException e) { + throw new StreamingException(e); + } + } + +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaSource.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaSource.java new file mode 100644 index 00000000000..8731824a793 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaSource.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.lang.reflect.Constructor; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.annotation.Nullable; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.cube.CubeInstance; +import org.apache.kylin.cube.CubeManager; +import org.apache.kylin.cube.CubeSegment; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.stream.core.consumer.ConsumerStartMode; +import org.apache.kylin.stream.core.consumer.ConsumerStartProtocol; +import org.apache.kylin.stream.core.consumer.IStreamingConnector; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.source.ISourcePositionHandler; +import org.apache.kylin.stream.core.source.IStreamingMessageParser; +import org.apache.kylin.stream.core.source.IStreamingSource; +import org.apache.kylin.stream.core.source.MessageParserInfo; +import org.apache.kylin.stream.core.source.Partition; +import org.apache.kylin.stream.core.source.StreamingSourceConfig; +import org.apache.kylin.stream.core.source.StreamingSourceConfigManager; +import org.apache.kylin.stream.core.source.StreamingSourceFactory; +import org.apache.kylin.stream.core.source.StreamingTableSourceInfo; +import org.apache.kylin.stream.core.storage.StreamingSegmentManager; +import org.apache.kylin.stream.source.kafka.consumer.KafkaConnector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; +import com.google.common.collect.MapDifference; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class KafkaSource implements IStreamingSource { + public static final String PROP_TOPIC = "topic"; + public static final String PROP_BOOTSTRAP_SERVERS = "bootstrap.servers"; + public static final String PROP_MESSAGE_PARSER = "message.parser"; + private static final Logger logger = LoggerFactory.getLogger(KafkaSource.class); + private static final String DEF_MSSAGE_PARSER_CLAZZ = "org.apache.kylin.stream.source.kafka.TimedJsonStreamParser"; + + + @Override + public StreamingTableSourceInfo load(String cubeName) { + KylinConfig kylinConf = KylinConfig.getInstanceFromEnv(); + CubeInstance cube = CubeManager.getInstance(kylinConf).getCube(cubeName); + String streamingTableName = cube.getRootFactTable(); + StreamingSourceConfig streamingSourceConfig = StreamingSourceConfigManager.getInstance(kylinConf).getConfig( + streamingTableName); + + String topicName = getTopicName(streamingSourceConfig.getProperties()); + Map conf = getKafkaConf(streamingSourceConfig.getProperties(), cube.getConfig()); + + KafkaConsumer kafkaConsumer = new KafkaConsumer(conf); + try { + List partitionInfos = kafkaConsumer.partitionsFor(topicName); + List kafkaPartitions = Lists.transform(partitionInfos, new Function() { + @Nullable + @Override + public Partition apply(@Nullable PartitionInfo input) { + return new Partition(input.partition()); + } + }); + return new StreamingTableSourceInfo(kafkaPartitions); + } finally { + kafkaConsumer.close(); + } + } + + @Override + public String getMessageTemplate(StreamingSourceConfig streamingSourceConfig) { + String template = null; + KafkaConsumer consumer = null; + try { + String topicName = getTopicName(streamingSourceConfig.getProperties()); + Map config = getKafkaConf(streamingSourceConfig.getProperties()); + consumer = new KafkaConsumer<>(config); + Set partitions = Sets.newHashSet(FluentIterable.from(consumer.partitionsFor(topicName)) + .transform(new Function() { + @Override + public TopicPartition apply(PartitionInfo input) { + return new TopicPartition(input.topic(), input.partition()); + } + })); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + ConsumerRecords records = consumer.poll(500); + if (records == null) { + return null; + } + Iterator> iterator = records.iterator(); + if (iterator == null || !iterator.hasNext()) { + return null; + } + ConsumerRecord record = iterator.next(); + template = new String(record.value(), "UTF8"); + } catch (Exception e) { + logger.error("error when fetch one record from kafka, stream:" + streamingSourceConfig.getName(), e); + } finally { + if (consumer != null) { + consumer.close(); + } + } + return template; + } + + @Override + public IStreamingConnector createStreamingConnector(String cubeName, List assignedPartitions, + ConsumerStartProtocol startProtocol, StreamingSegmentManager streamingSegmentManager) { + logger.info("Create StreamingConnector for Cube {}, assignedPartitions {}, startProtocol {}", cubeName, assignedPartitions, startProtocol); + try { + KylinConfig kylinConf = KylinConfig.getInstanceFromEnv(); + CubeInstance cubeInstance = CubeManager.getInstance(kylinConf).getCube(cubeName); + IStreamingSource streamingSource = StreamingSourceFactory.getStreamingSource(cubeInstance); + String streamingName = cubeInstance.getRootFactTable(); + StreamingSourceConfig streamingSourceConfig = StreamingSourceConfigManager.getInstance(kylinConf) + .getConfig(streamingName); + String topic = getTopicName(streamingSourceConfig.getProperties()); + Map conf = getKafkaConf(streamingSourceConfig.getProperties(), cubeInstance.getConfig()); + + Class clazz = getStreamingMessageParserClass(streamingSourceConfig.getProperties()); + Constructor constructor = clazz.getConstructor(CubeDesc.class, MessageParserInfo.class); + IStreamingMessageParser parser = (IStreamingMessageParser) constructor.newInstance( + cubeInstance.getDescriptor(), streamingSourceConfig.getParserInfo()); + KafkaConnector connector = new KafkaConnector(conf, topic, parser); + if (startProtocol != null) { + if (startProtocol.getStartPosition() != null && startProtocol.getStartPosition().length() > 0) { + KafkaPosition position = (KafkaPosition) streamingSource.getSourcePositionHandler().parsePosition(startProtocol.getStartPosition()); + connector.setStartPartition(assignedPartitions, startProtocol.getStartMode(), + position.getPartitionOffsets()); + streamingSegmentManager.restoreConsumerStates(position); + } else { + connector.setStartPartition(assignedPartitions, startProtocol.getStartMode(), + null); + } + streamingSegmentManager.checkpoint(); + } else if (streamingSegmentManager != null) { + setupConnectorFromCheckpoint(connector, assignedPartitions, streamingSource, streamingSegmentManager); + } + + return connector; + } catch (Exception e) { + throw new StreamingException("streaming connector create fail, cube:" + cubeName, e); + } + } + + @Override + public ISourcePositionHandler getSourcePositionHandler() { + return new KafkaPositionHandler(); + } + + private void setupConnectorFromCheckpoint(KafkaConnector connector, List assignedPartitions, IStreamingSource streamingSource, StreamingSegmentManager cubeDataStore) { + CubeInstance cubeInstance = cubeDataStore.getCubeInstance(); + CubeSegment latestReadySegment = cubeInstance.getLatestReadySegment(); + String localCheckpointConsumePos = cubeDataStore.getCheckPointSourcePosition(); + String remoteCheckpointConsumePos = null; + if (latestReadySegment != null) { + remoteCheckpointConsumePos = latestReadySegment.getStreamSourceCheckpoint(); + } + logger.info("localConsumeStats from local checkpoint {}, remoteConsumeStats from remote checkpoint {} ", + localCheckpointConsumePos, remoteCheckpointConsumePos); + KafkaPosition localCPPosition = null; + KafkaPosition remoteCPPosition = null; + if (localCheckpointConsumePos != null) { + localCPPosition = (KafkaPosition) streamingSource.getSourcePositionHandler().parsePosition(localCheckpointConsumePos); + } + + if (remoteCheckpointConsumePos != null) { + remoteCPPosition = (KafkaPosition) streamingSource.getSourcePositionHandler().parsePosition(remoteCheckpointConsumePos); + } + + // merge the local and remote consume stats + if (isEmptyPosition(localCPPosition) && isEmptyPosition(remoteCPPosition)) { + // no segment exists in the cube and is configured to consume from latest offset + if (cubeInstance.getSegments().isEmpty() && cubeInstance.getConfig().isStreamingConsumeFromLatestOffsets()) { + logger.info("start kafka connector from latest"); + connector.setStartPartition(assignedPartitions, ConsumerStartMode.LATEST, null); + } else { + logger.info("start kafka connector from earliest"); + connector.setStartPartition(assignedPartitions, ConsumerStartMode.EARLIEST, null); + } + return; + } + + KafkaPosition consumerStartPos; + + if (isEmptyPosition(localCPPosition) && !isEmptyPosition(remoteCPPosition)) { + consumerStartPos = remoteCPPosition; + } else if (isEmptyPosition(remoteCPPosition) && !isEmptyPosition(localCPPosition)) { + consumerStartPos = (KafkaPosition)localCPPosition.advance(); + } else { + Map mergedStartOffsets = Maps.newHashMap(); + MapDifference statsDiff = Maps.difference(localCPPosition.getPartitionOffsets(), remoteCPPosition.getPartitionOffsets()); + mergedStartOffsets.putAll(statsDiff.entriesInCommon()); + mergedStartOffsets.putAll(statsDiff.entriesOnlyOnLeft()); + mergedStartOffsets.putAll(statsDiff.entriesOnlyOnRight()); + mergedStartOffsets.putAll(Maps.transformValues(statsDiff.entriesDiffering(), + new Function, Long>() { + @Nullable + @Override + public Long apply(@Nullable MapDifference.ValueDifference input) { + return input.leftValue() > input.rightValue() ? input.leftValue() : input.rightValue(); + } + })); + consumerStartPos = new KafkaPosition(mergedStartOffsets); + } + logger.info("start kafka connector from specified position:{}", consumerStartPos); + connector.setStartPartition(assignedPartitions, ConsumerStartMode.SPECIFIC_POSITION, consumerStartPos.getPartitionOffsets()); + } + + private boolean isEmptyPosition(KafkaPosition kafkaPosition) { + return kafkaPosition == null || kafkaPosition.getPartitionOffsets().isEmpty(); + } + + protected Map getKafkaConf(Map sourceProperties, KylinConfig kylinConfig) { + return getKafkaConf(sourceProperties); + } + + protected Map getKafkaConf(Map sourceProperties) { + Map conf = Maps.newHashMap(); + String bootstrapServersString = getBootstrapServers(sourceProperties); + conf.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersString); + conf.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + conf.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, String.valueOf(20000)); + conf.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(30000)); + conf.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + conf.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + return conf; + } + + protected String getBootstrapServers(Map sourceProperties) { + return sourceProperties.get(PROP_BOOTSTRAP_SERVERS); + } + + protected String getTopicName(Map sourceProperties) { + return sourceProperties.get(PROP_TOPIC); + } + + protected Class getStreamingMessageParserClass(Map sourceProperties) + throws ClassNotFoundException { + String parserName = sourceProperties.get(PROP_MESSAGE_PARSER); + String parserClazzName = DEF_MSSAGE_PARSER_CLAZZ; + if (parserName != null) { + parserClazzName = getParserClassName(parserName); + } + return Class.forName(parserClazzName); + } + + protected String getParserClassName(String parser) { + return parser; + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaTopicAssignment.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaTopicAssignment.java new file mode 100644 index 00000000000..a16dfcdc34b --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/KafkaTopicAssignment.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.util.List; + +import org.apache.kafka.common.TopicPartition; + +public class KafkaTopicAssignment { + private Integer replicaSetID; + private List assignments; + + public KafkaTopicAssignment(Integer replicaSetID, List assignments) { + this.replicaSetID = replicaSetID; + this.assignments = assignments; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((assignments == null) ? 0 : assignments.hashCode()); + result = prime * result + ((replicaSetID == null) ? 0 : replicaSetID.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + KafkaTopicAssignment other = (KafkaTopicAssignment) obj; + if (assignments == null) { + if (other.assignments != null) + return false; + } else if (!assignments.equals(other.assignments)) + return false; + if (replicaSetID == null) { + if (other.replicaSetID != null) + return false; + } else if (!replicaSetID.equals(other.replicaSetID)) + return false; + return true; + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/TimedJsonStreamParser.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/TimedJsonStreamParser.java new file mode 100644 index 00000000000..b9e92df3845 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/TimedJsonStreamParser.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kylin.cube.model.CubeDesc; +import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc; +import org.apache.kylin.metadata.model.TblColRef; +import org.apache.kylin.stream.core.exception.StreamingException; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.source.IStreamingMessageParser; +import org.apache.kylin.stream.core.source.MessageParserInfo; +import org.apache.kylin.stream.core.util.TimeDerivedColumnType; +import org.apache.kylin.stream.source.kafka.KafkaPosition.KafkaPartitionPosition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.MapType; +import com.fasterxml.jackson.databind.type.SimpleType; +import com.google.common.collect.Lists; + +/** + * each json message with a "timestamp" field + */ +public final class TimedJsonStreamParser implements IStreamingMessageParser> { + + private static final Logger logger = LoggerFactory.getLogger(TimedJsonStreamParser.class); + private final ObjectMapper mapper = new ObjectMapper(); + private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), + SimpleType.construct(String.class)); + private List allColumns; + private boolean formatTs = false;//not used + private String tsColName = "timestamp"; + private Map columnToSourceFieldMapping; + + public TimedJsonStreamParser(CubeDesc cubeDesc, MessageParserInfo parserInfo) { + this.allColumns = new CubeJoinedFlatTableDesc(cubeDesc).getAllColumns(); + if (parserInfo != null) { + this.formatTs = parserInfo.isFormatTs(); + this.tsColName = parserInfo.getTsColName(); + this.columnToSourceFieldMapping = parserInfo.getColumnToSourceFieldMapping(); + } + + logger.info("TimedJsonStreamParser with formatTs {} tsColName {}", formatTs, tsColName); + } + + @Override + public StreamingMessage parse(ConsumerRecord record) { + try { + Map message = mapper.readValue(parseToString(record.value()), mapType); + Map root = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + root.putAll(message); + String tsStr = root.get(tsColName); + //Preconditions.checkArgument(!StringUtils.isEmpty(tsStr), "Timestamp field " + tsColName + // + //" cannot be null, the message offset is " + messageAndOffset.getOffset() + " content is " + new String(messageAndOffset.getRawData())); + long t; + if (StringUtils.isEmpty(tsStr)) { + t = 0; + } else { + t = Long.valueOf(tsStr); + } + ArrayList result = Lists.newArrayList(); + + for (TblColRef column : allColumns) { + String columnName = column.getName(); + TimeDerivedColumnType columnType = TimeDerivedColumnType.getTimeDerivedColumnType(columnName); + if (columnType != null) { + result.add(String.valueOf(columnType.normalize(t))); + } else { + String x = root.get(columnName.toLowerCase(Locale.ROOT)); + result.add(x); + } + } + + return new StreamingMessage(result, new KafkaPartitionPosition(record.partition(), record.offset()), t, + Collections. emptyMap()); + } catch (IOException e) { + logger.error("error", e); + throw new RuntimeException(e); + } + } + + private String parseToString(byte[] bytes) { + String value; + try { + value = new String(bytes, "UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new StreamingException(e); + } + return value; + } +} diff --git a/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/consumer/KafkaConnector.java b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/consumer/KafkaConnector.java new file mode 100644 index 00000000000..fa8c1d50c24 --- /dev/null +++ b/stream-source-kafka/src/main/java/org/apache/kylin/stream/source/kafka/consumer/KafkaConnector.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.source.kafka.consumer; + +import java.util.List; +import java.util.Map; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kylin.stream.core.consumer.ConsumerStartMode; +import org.apache.kylin.stream.core.consumer.IStreamingConnector; +import org.apache.kylin.stream.core.model.StreamingMessage; +import org.apache.kylin.stream.core.source.IStreamingMessageParser; +import org.apache.kylin.stream.core.source.Partition; + +import com.google.common.collect.Lists; + +public class KafkaConnector implements IStreamingConnector { + private final KafkaConsumer kafkaConsumer; + private final String topic; + private final IStreamingMessageParser parser; + private ConsumerStartMode startMode = ConsumerStartMode.EARLIEST; + + private List> buffer = Lists.newLinkedList(); + private List partitions; + private Map partitionOffsets; + + public KafkaConnector(Map conf, String topic, IStreamingMessageParser parser) { + this.kafkaConsumer = new KafkaConsumer<>(conf); + this.topic = topic; + this.parser = parser; + } + + public void setStartPartition(List partitions, ConsumerStartMode startMode, + Map partitionOffsets) { + this.partitions = partitions; + this.startMode = startMode; + this.partitionOffsets = partitionOffsets; + } + + @Override + public List getConsumePartitions() { + return partitions; + } + + @Override + public void open() { + if (partitions == null || partitions.size() <= 0) { + throw new IllegalStateException("not assign partitions"); + } + List topicPartitions = Lists.newArrayList(); + for (Partition partition : partitions) { + topicPartitions.add(new TopicPartition(topic, partition.getPartitionId())); + } + kafkaConsumer.assign(topicPartitions); + + if (startMode == ConsumerStartMode.EARLIEST) { + kafkaConsumer.seekToBeginning(topicPartitions); + } else if (startMode == ConsumerStartMode.LATEST) { + kafkaConsumer.seekToEnd(topicPartitions); + } else { + for (TopicPartition topicPartition : topicPartitions) { + Long offset = partitionOffsets.get(topicPartition.partition()); + kafkaConsumer.seek(topicPartition, offset); + } + } + } + + @Override + public void close() { + kafkaConsumer.close(); + } + + @Override + public void wakeup() { + kafkaConsumer.wakeup(); + } + + @Override + public StreamingMessage nextEvent() { + if (buffer.isEmpty()) { + fillBuffer(); + } + if (buffer.isEmpty()) { + return null; + } + ConsumerRecord record = buffer.remove(0); + return parser.parse(record); + } + + private void fillBuffer() { + ConsumerRecords records = kafkaConsumer.poll(100); + List> newBuffer = Lists.newLinkedList(); + for (TopicPartition topicPartition : records.partitions()) { + newBuffer.addAll(records.records(topicPartition)); + } + this.buffer = newBuffer; + } + +} diff --git a/stream-source-kafka/src/test/java/org/apache/kylin/stream/coordinator/assign/KafkaSourcePositionHandlerTest.java b/stream-source-kafka/src/test/java/org/apache/kylin/stream/coordinator/assign/KafkaSourcePositionHandlerTest.java new file mode 100644 index 00000000000..4c6197225fe --- /dev/null +++ b/stream-source-kafka/src/test/java/org/apache/kylin/stream/coordinator/assign/KafkaSourcePositionHandlerTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.stream.coordinator.assign; + +import static org.junit.Assert.assertEquals; + +import java.util.Map; + +import org.apache.kylin.stream.core.source.ISourcePosition; +import org.apache.kylin.stream.core.source.ISourcePosition.IPartitionPosition; +import org.apache.kylin.stream.source.kafka.KafkaPosition; +import org.apache.kylin.stream.source.kafka.KafkaPosition.KafkaPartitionPosition; +import org.apache.kylin.stream.source.kafka.KafkaPositionHandler; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Maps; + +public class KafkaSourcePositionHandlerTest { + KafkaPositionHandler positionHandler; + + @Before + public void setup() { + positionHandler = new KafkaPositionHandler(); + } + + @Test + public void testParsePosition() throws Exception { + String kafkaPosStr = "{\"0\":161400}"; + ISourcePosition sourcePosition = positionHandler.parsePosition(kafkaPosStr); + Map partitionPositionMap = sourcePosition.getPartitionPositions(); + assertEquals(((KafkaPartitionPosition)partitionPositionMap.get(0)).offset, 161400); + + ISourcePosition newPosition = positionHandler.createEmptyPosition(); + newPosition.copy(sourcePosition); + partitionPositionMap = newPosition.getPartitionPositions(); + assertEquals(((KafkaPartitionPosition)partitionPositionMap.get(0)).offset, 161400); + } + + @Test + public void testSerializePosition() throws Exception { + Map partitionOffsetMap = Maps.newHashMap(); + partitionOffsetMap.put(0, 1000L); + partitionOffsetMap.put(1, 1001L); + partitionOffsetMap.put(2, 1002L); + KafkaPosition kafkaPosition = new KafkaPosition(partitionOffsetMap); + String posStr = positionHandler.serializePosition(kafkaPosition); + System.out.println(posStr); + KafkaPosition deserializePosition = (KafkaPosition) positionHandler.parsePosition(posStr); + deserializePosition.getPartitionOffsets(); + + assertEquals(deserializePosition.getPartitionOffsets(), partitionOffsetMap); + } +} diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java index a88b049cf3d..472be425bc5 100644 --- a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java +++ b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java @@ -66,6 +66,7 @@ import org.apache.kylin.metadata.realization.RealizationStatusEnum; import org.apache.kylin.metadata.realization.RealizationType; import org.apache.kylin.storage.hbase.HBaseConnection; +import org.apache.kylin.stream.core.source.StreamingSourceConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -337,6 +338,11 @@ protected void listCubeRelatedResources(CubeInstance cube, List metaReso metaResource.add(ACL_PREFIX + cube.getUuid()); metaResource.add(ACL_PREFIX + cube.getModel().getUuid()); } + + if (cubeDesc.isStreamingCube()) { + // add streaming source config info for streaming cube + metaResource.add(StreamingSourceConfig.concatResourcePath(cubeDesc.getModel().getRootFactTableName())); + } } @Override diff --git a/tool/src/test/java/org/apache/kylin/tool/CubeMetaExtractorTest.java b/tool/src/test/java/org/apache/kylin/tool/CubeMetaExtractorTest.java index 0e9d50d5c9d..471c41e0c20 100644 --- a/tool/src/test/java/org/apache/kylin/tool/CubeMetaExtractorTest.java +++ b/tool/src/test/java/org/apache/kylin/tool/CubeMetaExtractorTest.java @@ -95,7 +95,8 @@ public void testExtractorByPrj() throws IOException { "/table/SSB.DATES.json", // "/table/SSB.PART.json", // "/table/SSB.SUPPLIER.json", // - "/table/SSB.V_LINEORDER.json" + "/table/SSB.V_LINEORDER.json", // + "/table/DEFAULT.STREAMING_V2_TABLE.json" ); Set expectTblExd = Sets.newHashSet( "/table_exd/DEFAULT.TEST_COUNTRY.json", //