You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2020/03/03 00:53:21 UTC

[incubator-pinot] branch release-0.3.0 updated (79402cf -> c0b0e6c)

This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a change to branch release-0.3.0
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


 discard 79402cf  update pinot assembly scripts
     add 7f6afdf  [TE] detection health - generate task status in monitoring task (#5065)
     add 344bade  Re-design the broker routing (#5046)
     add 2a79531  Support Text Search (#4993)
     add a257935  Support no global-dictionary columns in data anonymizer (#5071)
     add 4be7490  Use getResourceAsStream to load test resource file packaged inside (#5072)
     add 9c65deb  Reduce the non-determinism in lucene integration test (#5077)
     add 94122e8  [TE] frontend - harleyjj/alerts - update application filter for many-to-many relationship (#5062)
     add 884af43  [TE] Remove fromAddress as a compulsory field from subscription group (#5078)
     add 4779341  Disbale text query realtime integration test (#5079)
     add 3c78a8c  [TE] add initial delay upon receiving events in scheduling (#5060)
     add c613f46  [TE] Adjust dectection window using expected delay (#5059)
     add 7dfc5c1  Adding quickstart into Pinot admin command (#5080)
     add ef69b75  [TE][subscription] Fix duplicate anomaly report issue and clean up subscription pipelines (#5081)
     add 35f2117  [TE][subscription] Minor bug fix in the empty email check (#5084)
     add 21850ba  Support text index load for V1 segment format (#5085)
     add d5c8398  Support creation of inverted index on a new column in V1 segment (#5087)
     add 4124563  Support segment reload for text index (#5074)
     add e44192e  Support embedded parsing in PinotDataAndQueryAnonymizer (#5088)
     add f247e77  Plug in the new routing introduced in #5046 (#5082)
     add ef6509e  Minor improvement for RoutingManager (#5092)
     add b2608ad  [TE] Upgrade mysql version to 5.1.41 to fix compatibility issues connectin with 8.* (#5093)
     add 0e56960  Fix a typo in RoutingConfig (#5096)
     add 26ead7d  [TE] add dependencies for enabling ThirdEye https connections (#5095)
     add fdfb97e  [TE][subscription] Dimension Recipient Alerter should be agnostic to individual errors; Emit metrics to track emails and jiras (#5091)
     add 3126d56  Support non-literal expressions for right-side operand in predicate comparison (#5070)
     add 3c91a6d  Remove the support of the old Star-Tree (#5086)
     add 00fd911  [TE][subscription] destroy jira connections after use (#5097)
     add 754532e9 [TE] Change the jetty version to be compatible of loading the certificate for HTTPS (#5099)
     add b880c47  Fix the duplicate broker refresh message (#5102)
     add 9f33abe  [TE] Add additional safeguard around jira description length (#5100)
     add 16e6fbf  [Cleanup] Remove the old broker routing (#5101)
     add 9abf0a4  Adding PinotClusterConfig APIs in controller and corresponding Pinot Admin SubCommand: OperateClusterConfig (#5073)
     new 1859a6b  Update license and notice
     new c0b0e6c  update pinot assembly scripts

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (79402cf)
            \
             N -- N -- N   refs/heads/release-0.3.0 (c0b0e6c)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 LICENSE-binary                                     |   131 +-
 NOTICE                                             |     2 +-
 NOTICE-binary                                      |   448 +-
 licenses-binary/LICENSE-fusesource.txt             |    27 +
 licenses-binary/LICENSE-py4j.txt                   |    26 +
 licenses-binary/LICENSE-threeten.txt               |    29 +
 .../broker/api/resources/PinotBrokerDebug.java     |   111 +-
 .../api/resources/PinotBrokerHealthCheck.java      |    17 +-
 .../broker/api/resources/PinotBrokerRouting.java   |    76 +
 .../broker/api/resources/PinotClientRequest.java   |     2 +-
 .../broker/broker/BrokerAdminApiApplication.java   |    14 +-
 .../pinot/broker/broker/BrokerServerBuilder.java   |   144 -
 ...okerResourceOnlineOfflineStateModelFactory.java |    77 +-
 .../BrokerUserDefineMessageHandlerFactory.java     |    92 +
 .../broker/broker/helix/HelixBrokerStarter.java    |   170 +-
 .../TimeboundaryRefreshMessageHandlerFactory.java  |   145 -
 .../HelixExternalViewBasedQueryQuotaManager.java   |    10 +-
 .../requesthandler/BaseBrokerRequestHandler.java   |    46 +-
 .../SingleConnectionBrokerRequestHandler.java      |    11 +-
 .../routing/HelixExternalViewBasedRouting.java     |   625 -
 .../HelixExternalViewBasedTimeBoundaryService.java |   148 -
 .../pinot/broker/routing/RoutingManager.java       |   520 +
 .../apache/pinot/broker/routing/RoutingTable.java  |    56 -
 .../broker/routing/RoutingTableBuilderFactory.java |   143 -
 .../broker/routing/RoutingTableLookupRequest.java  |    79 -
 .../pinot/broker/routing/TimeBoundaryService.java  |    65 -
 .../builder/BalancedRandomRoutingTableBuilder.java |    62 -
 .../BasePartitionAwareRoutingTableBuilder.java     |   160 -
 .../routing/builder/BaseRoutingTableBuilder.java   |   220 -
 .../builder/DefaultOfflineRoutingTableBuilder.java |   139 -
 .../DefaultRealtimeRoutingTableBuilder.java        |   119 -
 .../builder/GeneratorBasedRoutingTableBuilder.java |   293 -
 .../HighLevelConsumerBasedRoutingTableBuilder.java |    95 -
 .../routing/builder/InstanceConfigManager.java     |    81 -
 .../builder/LargeClusterRoutingTableBuilder.java   |    63 -
 .../LowLevelConsumerRoutingTableBuilder.java       |   139 -
 .../builder/LowLevelRoutingTableBuilderUtil.java   |    81 -
 .../PartitionAwareOfflineRoutingTableBuilder.java  |   209 -
 .../PartitionAwareRealtimeRoutingTableBuilder.java |   145 -
 .../routing/builder/RoutingTableBuilder.java       |    64 -
 .../instanceselector/BalancedInstanceSelector.java |    55 +
 .../instanceselector/BaseInstanceSelector.java     |   179 +
 .../routing/instanceselector/InstanceSelector.java |    60 +
 .../instanceselector/InstanceSelectorFactory.java  |    49 +
 .../ReplicaGroupInstanceSelector.java              |    65 +
 .../segmentpruner/PartitionSegmentPruner.java      |   191 +
 .../routing/segmentpruner/SegmentPruner.java       |    57 +
 .../segmentpruner/SegmentPrunerFactory.java        |   100 +
 .../segmentselector/OfflineSegmentSelector.java    |    52 +
 .../segmentselector/RealtimeSegmentSelector.java   |   165 +
 .../routing/segmentselector/SegmentSelector.java   |    62 +
 .../segmentselector/SegmentSelectorFactory.java    |    28 +-
 .../routing/selector/MergedSegmentSelector.java    |    95 -
 .../broker/routing/selector/SegmentSelector.java   |    49 -
 .../routing/selector/SegmentSelectorProvider.java  |    47 -
 .../routing/timeboundary/TimeBoundaryInfo.java     |    29 +-
 .../routing/timeboundary/TimeBoundaryManager.java  |   193 +
 .../broker/broker/HelixBrokerStarterTest.java      |    37 +-
 ...elixExternalViewBasedQueryQuotaManagerTest.java |     4 +-
 ...ixExternalViewBasedTimeBoundaryServiceTest.java |   167 -
 .../broker/routing/RandomRoutingTableTest.java     |   107 -
 .../pinot/broker/routing/RoutingTableTest.java     |   303 -
 .../BalancedRandomRoutingTableBuilderTest.java     |   126 -
 .../HighLevelConsumerRoutingTableBuilderTest.java  |   119 -
 .../LargeClusterRoutingTableBuilderTest.java       |   275 -
 .../LowLevelConsumerRoutingTableBuilderTest.java   |   271 -
 ...rtitionAwareOfflineRoutingTableBuilderTest.java |   404 -
 ...titionAwareRealtimeRoutingTableBuilderTest.java |   374 -
 .../instanceselector/InstanceSelectorTest.java     |   400 +
 .../routing/segmentpruner/SegmentPrunerTest.java   |   222 +
 .../segmentselector/SegmentSelectorTest.java       |   126 +
 .../selector/MergedSegmentSelectorTest.java        |   149 -
 .../timeboundary/TimeBoundaryManagerTest.java      |   169 +
 .../antlr4/org/apache/pinot/pql/parsers/PQL2.g4    |     5 +
 .../apache/pinot/common/config/IndexingConfig.java |    10 -
 .../apache/pinot/common/config/RoutingConfig.java  |    26 +-
 .../SegmentsValidationAndRetentionConfig.java      |    10 -
 .../apache/pinot/common/config/TableConfig.java    |     6 +
 .../common/config/TextIndexConfigValidator.java    |    46 +
 .../pinot/common/data/StarTreeIndexSpec.java       |   128 -
 .../common/messages/SegmentRefreshMessage.java     |    46 +-
 .../messages/TimeboundaryRefreshMessage.java       |    45 -
 .../common/metadata/segment/SegmentZKMetadata.java |    22 +-
 .../apache/pinot/common/metrics/BrokerMeter.java   |     3 +
 .../pinot/common/request/FilterOperator.java       |    11 +-
 .../common/response/BrokerResponseFactory.java     |   117 -
 .../pinot/common/segment/SegmentMetadata.java      |    19 -
 .../pinot/common/segment/StarTreeMetadata.java     |    73 -
 .../apache/pinot/common/utils/CommonConstants.java |     8 -
 .../pinot/common/utils/request/RequestUtils.java   |    29 +-
 .../parsers/PinotQuery2BrokerRequestConverter.java |     2 +
 .../apache/pinot/pql/parsers/Pql2AstListener.java  |    11 +
 .../org/apache/pinot/pql/parsers/Pql2Compiler.java |     1 -
 .../pinot/pql/parsers/pql2/ast/FilterKind.java     |     3 +-
 .../pql/parsers/pql2/ast/InPredicateAstNode.java   |     4 +
 .../pql2/ast/TextMatchPredicateAstNode.java        |    93 +
 .../apache/pinot/sql/parsers/CalciteSqlParser.java |   100 +-
 .../org/apache/pinot/startree/hll/HllConfig.java   |   131 -
 .../apache/pinot/startree/hll/HllSizeUtils.java    |    60 -
 .../pinot/common/config/IndexingConfigTest.java    |    48 -
 .../pinot/common/config/TableConfigTest.java       |   156 +-
 .../apache/pinot/common/data/FieldSpecTest.java    |    19 -
 .../org/apache/pinot/common/data/SchemaTest.java   |    56 +-
 .../common/metadata/SegmentZKMetadataTest.java     |     6 +-
 .../apache/pinot/pql/parsers/Pql2CompilerTest.java |    12 +
 .../pinot/sql/parsers/CalciteSqlCompilerTest.java  |    95 +-
 pinot-common/src/thrift/request.thrift             |     3 +-
 .../pinot/controller/api/resources/Constants.java  |     1 +
 .../api/resources/PinotClusterConfigs.java         |   121 +
 .../helix/core/PinotHelixResourceManager.java      |   122 +-
 .../realtime/PinotLLCRealtimeSegmentManager.java   |     2 +-
 .../SegmentSizeBasedFlushThresholdUpdater.java     |     2 +-
 .../helix/core/util/ZKMetadataUtils.java           |     2 +-
 .../validation/OfflineSegmentIntervalChecker.java  |     2 +-
 .../RealtimeSegmentValidationManager.java          |     6 +-
 .../PinotLLCRealtimeSegmentManagerTest.java        |     4 +-
 .../segment/FlushThresholdUpdaterTest.java         |     4 +-
 .../controller/utils/SegmentMetadataMockUtils.java |     4 +-
 .../validation/ValidationManagerTest.java          |     6 +-
 pinot-core/pom.xml                                 |    15 +
 .../org/apache/pinot/core/common/Predicate.java    |     6 +-
 .../predicate/TextMatchPredicate.java}             |    28 +-
 .../core/data/manager/BaseTableDataManager.java    |     4 +-
 .../realtime/HLRealtimeSegmentDataManager.java     |     2 +-
 .../realtime/LLRealtimeSegmentDataManager.java     |    19 +-
 .../data/readers/PinotSegmentRecordReader.java     |     2 +-
 .../generator/SegmentGeneratorConfig.java          |   110 +-
 .../immutable/ImmutableSegmentLoader.java          |     6 +-
 .../indexsegment/mutable/MutableSegmentImpl.java   |   116 +-
 .../io/reader/impl/v1/SortedIndexReaderImpl.java   |     8 +
 .../pinot/core/minion/BackfillDateTimeColumn.java  |     7 -
 .../apache/pinot/core/minion/SegmentConverter.java |    10 +-
 .../apache/pinot/core/minion/SegmentPurger.java    |    16 +-
 .../core/operator/CombineGroupByOperator.java      |     2 +-
 .../operator/CombineGroupByOrderByOperator.java    |     4 +-
 .../pinot/core/operator/CombineOperator.java       |     2 +-
 .../pinot/core/operator/ExecutionStatistics.java   |    14 +-
 .../operator/blocks/IntermediateResultsBlock.java  |    43 +-
 .../operator/filter/ExpressionFilterOperator.java  |     2 +-
 .../core/operator/filter/FilterOperatorUtils.java  |    14 +-
 .../operator/filter/TextMatchFilterOperator.java   |    75 +
 .../predicate/PredicateEvaluatorProvider.java      |     8 +
 .../TextMatchPredicateEvaluatorFactory.java        |    58 +
 .../operator/query/AggregationGroupByOperator.java |    14 +-
 .../query/AggregationGroupByOrderByOperator.java   |    14 +-
 .../core/operator/query/AggregationOperator.java   |    12 +-
 .../query/DictionaryBasedAggregationOperator.java  |    14 +-
 .../operator/query/EmptySelectionOperator.java     |     2 +-
 .../query/MetadataBasedAggregationOperator.java    |    10 +-
 .../core/operator/query/SelectionOnlyOperator.java |     5 +-
 .../operator/query/SelectionOrderByOperator.java   |     9 +-
 .../plan/AggregationGroupByOrderByPlanNode.java    |     7 +-
 .../core/plan/AggregationGroupByPlanNode.java      |     6 +-
 .../pinot/core/plan/AggregationPlanNode.java       |     6 +-
 .../plan/DictionaryBasedAggregationPlanNode.java   |     2 +-
 .../org/apache/pinot/core/plan/FilterPlanNode.java |     9 +-
 .../core/plan/maker/BrokerRequestPreProcessor.java |    97 -
 .../core/plan/maker/InstancePlanMakerImplV2.java   |     7 +-
 .../function/AggregationFunctionUtils.java         |     8 -
 .../DistinctCountRawHLLAggregationFunction.java    |     2 +-
 .../function/FastHLLAggregationFunction.java       |   111 +-
 .../function/customobject/SerializedHLL.java       |    24 +-
 .../query/executor/ServerQueryExecutorV1Impl.java  |    12 +-
 .../core/query/reduce/BrokerReduceService.java     |    13 +-
 .../converter/RealtimeSegmentConverter.java        |    37 +-
 .../stats/RealtimeSegmentStatsContainer.java       |    15 +-
 .../core/realtime/impl/RealtimeSegmentConfig.java  |    52 +-
 .../invertedindex/RealtimeInvertedIndexReader.java |     8 +
 .../RealtimeLuceneIndexReaderRefreshThread.java    |   151 +
 .../RealtimeLuceneIndexRefreshState.java           |   139 +
 .../RealtimeLuceneTextIndexReader.java             |   162 +
 .../core/segment/creator/InvertedIndexCreator.java |    13 +-
 .../segment/creator/SegmentIndexCreationInfo.java  |    27 -
 .../creator/SegmentPreIndexStatsCollector.java     |     3 -
 .../creator/SegmentPreIndexStatsContainer.java     |     4 -
 .../pinot/core/segment/creator/TextIndexType.java  |     8 +-
 .../creator/impl/SegmentColumnarIndexCreator.java  |   124 +-
 .../impl/SegmentIndexCreationDriverImpl.java       |   199 +-
 .../core/segment/creator/impl/V1Constants.java     |    22 +-
 .../inv/OffHeapBitmapInvertedIndexCreator.java     |     5 +
 .../impl/inv/OnHeapBitmapInvertedIndexCreator.java |     5 +
 .../impl/inv/text/LuceneTextIndexCreator.java      |   163 +
 .../stats/AbstractColumnStatisticsCollector.java   |    12 +-
 .../stats/BytesColumnPredIndexStatsCollector.java  |    94 +-
 .../stats/DoubleColumnPreIndexStatsCollector.java  |   114 +-
 .../stats/FloatColumnPreIndexStatsCollector.java   |   114 +-
 .../stats/IntColumnPreIndexStatsCollector.java     |   111 +-
 .../stats/LongColumnPreIndexStatsCollector.java    |   114 +-
 .../stats/SegmentPreIndexStatsCollectorImpl.java   |    32 +-
 .../stats/StringColumnPreIndexStatsCollector.java  |   135 +-
 .../pinot/core/segment/index/ColumnMetadata.java   |   146 +-
 .../core/segment/index/SegmentMetadataImpl.java    |   102 +-
 .../index/column/PhysicalColumnIndexContainer.java |    16 +-
 .../converter/SegmentV1V2ToV3FormatConverter.java  |    44 +-
 .../index/data/source/ColumnDataSource.java        |     6 +-
 .../segment/index/loader/IndexLoadingConfig.java   |    53 +
 .../segment/index/loader/SegmentPreProcessor.java  |    13 +-
 .../defaultcolumn/BaseDefaultColumnHandler.java    |   111 +-
 .../loader/defaultcolumn/DefaultColumnHandler.java |     5 +-
 .../defaultcolumn/DefaultColumnHandlerFactory.java |     2 +-
 .../defaultcolumn/V1DefaultColumnHandler.java      |    19 +-
 .../defaultcolumn/V3DefaultColumnHandler.java      |    40 +-
 .../loader/invertedindex/TextIndexHandler.java     |   179 +
 .../index/readers/BitmapInvertedIndexReader.java   |     8 +
 .../segment/index/readers/InvertedIndexReader.java |     7 +
 .../index/readers/text/LuceneDocIdCollector.java   |    73 +
 .../index/readers/text/LuceneTextIndexReader.java  |   173 +
 .../core/segment/store/ColumnIndexDirectory.java   |     6 -
 .../pinot/core/segment/store/ColumnIndexType.java  |     3 +-
 .../core/segment/store/FilePerIndexDirectory.java  |     4 +
 .../pinot/core/segment/store/SegmentDirectory.java |    33 +-
 .../core/segment/store/SegmentDirectoryPaths.java  |    27 +-
 .../segment/store/SegmentLocalFSDirectory.java     |    85 +-
 .../segment/store/SingleFileIndexDirectory.java    |    22 +
 .../virtualcolumn/DocIdVirtualColumnProvider.java  |     9 +
 .../SingleStringVirtualColumnProvider.java         |     8 +
 .../pinot/core/startree/DimensionBuffer.java       |    77 -
 .../apache/pinot/core/startree/MetricBuffer.java   |   150 -
 .../core/startree/OffHeapStarTreeBuilder.java      |   855 -
 .../pinot/core/startree/StarTreeBuilder.java       |    85 -
 .../pinot/core/startree/StarTreeBuilderConfig.java |   125 -
 .../pinot/core/startree/StarTreeDataTable.java     |   216 -
 .../apache/pinot/core/startree/StarTreeNode.java   |     1 -
 .../apache/pinot/core/startree/StarTreeUtils.java  |    14 +-
 .../apache/pinot/core/startree/hll/HllUtil.java    |   157 -
 .../startree/v2/builder/BaseSingleTreeBuilder.java |     2 +-
 .../startree/v2/builder/MultipleTreesBuilder.java  |     3 -
 .../startree/v2/store/StarTreeIndexContainer.java  |    37 +-
 .../startree/v2/store/StarTreeLoaderUtils.java     |    60 -
 ...adataAndDictionaryAggregationPlanMakerTest.java |   103 +-
 .../core/segment/index/ColumnMetadataTest.java     |    43 +-
 .../segment/index/SegmentMetadataImplTest.java     |     2 -
 .../SegmentV1V2ToV3FormatConverterTest.java        |     7 -
 .../core/segment/index/loader/LoaderTest.java      |   155 +
 .../index/loader/SegmentPreProcessorTest.java      |   175 +-
 .../store/ColumnIndexDirectoryTestHelper.java      |     3 +-
 .../segment/store/SegmentLocalFSDirectoryTest.java |    29 -
 .../pinot/core/startree/BaseStarTreeIndexTest.java |   128 -
 .../core/startree/OffHeapStarTreeBuilderTest.java  |   126 -
 .../pinot/core/startree/StarTreeDataTableTest.java |    79 -
 .../startree/StarTreeIndexTestSegmentHelper.java   |   108 -
 .../pinot/core/startree/SumStarTreeIndexTest.java  |   114 -
 .../pinot/core/startree/TestStarTreeMetadata.java  |   116 -
 .../pinot/core/startree/hll/HllFieldSizeTest.java  |    53 -
 .../core/startree/hll/HllIndexCreationTest.java    |   182 -
 .../core/startree/hll/HllStarTreeIndexTest.java    |   138 -
 .../core/startree/hll/HllTypeConversionTest.java   |    72 -
 .../OffHeapStarTreeBuilderWithHllFieldTest.java    |   252 -
 .../hll/SegmentWithHllIndexCreateHelper.java       |   187 -
 .../apache/pinot/queries/FastHllQueriesTest.java   |   102 +-
 ...InnerSegmentSelectionMultiValueQueriesTest.java |    16 +-
 ...nnerSegmentSelectionSingleValueQueriesTest.java |    20 +-
 ...terSegmentAggregationMultiValueQueriesTest.java |    30 +-
 ...erSegmentAggregationSingleValueQueriesTest.java |     8 +-
 ...terSegmentResultTableMultiValueQueriesTest.java |    20 +-
 ...erSegmentResultTableSingleValueQueriesTest.java |    41 +-
 .../org/apache/pinot/queries/QueriesTestUtils.java |    18 +-
 .../pinot/queries/TestTextSearchQueries.java       |  1116 +
 .../DefaultAggregationExecutorTest.java            |    16 +-
 .../segments/v1/creator/DictionariesTest.java      |    48 +-
 .../test/java/org/apache/pinot/util/TestUtils.java |    19 +-
 .../resources/data/newColumnsWithTextSchema.json   |    76 +
 .../text_search_data/group_by_grep_results.out     |    26 +
 .../resources/data/text_search_data/pql_query1.txt | 24150 +++++++++++++++++++
 .../resources/data/text_search_data/skills.txt     |    23 +
 .../tests/BaseClusterIntegrationTest.java          |     6 +-
 .../tests/BaseClusterIntegrationTestSet.java       |   115 +-
 .../tests/ClusterIntegrationTestUtils.java         |    14 +-
 .../pinot/integration/tests/ClusterTest.java       |    14 +-
 .../ControllerPeriodicTasksIntegrationTests.java   |     6 +-
 .../tests/HybridClusterIntegrationTest.java        |    31 +-
 ...ridClusterIntegrationTestCommandLineRunner.java |     8 +-
 .../LuceneRealtimeClusterIntegrationTest.java      |   191 +
 .../tests/MapTypeClusterIntegrationTest.java       |     4 +-
 ...onaryAggregationPlanClusterIntegrationTest.java |     6 +-
 .../tests/OfflineClusterIntegrationTest.java       |     7 +-
 .../tests/StarTreeClusterIntegrationTest.java      |    89 +-
 .../tests/StarTreeV2ClusterIntegrationTest.java    |   156 -
 .../src/test/resources/OnTimeStarTreeQueries.txt   |  1000 -
 ...rformance_2014_100k_subset.test_queries_10K.sql |     7 +-
 .../apache/pinot/hadoop/io/PinotOutputFormat.java  |    97 +-
 .../pinot/hadoop/io/PinotOutputFormatTest.java     |    19 +-
 .../server/starter/helix/HelixServerStarter.java   |     6 +
 .../java/org/apache/pinot/spi/data/FieldSpec.java  |     3 -
 .../org/apache/pinot/spi/data/MetricFieldSpec.java |   115 +-
 .../java/org/apache/pinot/spi/data/Schema.java     |    19 -
 .../org/apache/pinot/spi/data/TimeFieldSpec.java   |     1 -
 .../org/apache/pinot/tools/HybridQuickstart.java   |     3 -
 .../java/org/apache/pinot/tools/Quickstart.java    |     2 -
 .../org/apache/pinot/tools/RealtimeQuickStart.java |     3 -
 .../pinot/tools/admin/PinotAdministrator.java      |     4 +
 .../admin/command/AbstractBaseAdminCommand.java    |    24 +-
 .../tools/admin/command/CreateSegmentCommand.java  |    80 +-
 .../admin/command/OperateClusterConfigCommand.java |   149 +
 .../tools/admin/command/QuickStartCommand.java     |    91 +
 .../anonymizer/PinotDataAndQueryAnonymizer.java    |   122 +-
 .../tools/query/comparison/ClusterStarter.java     |    18 +-
 .../query/comparison/StarQueryComparison.java      |   130 -
 .../query/comparison/StarTreeQueryGenerator.java   |    57 -
 .../realtime/provisioning/MemoryEstimator.java     |     4 +-
 .../converter/ColumnarToStarTreeConverter.java     |   170 -
 .../converter/DictionaryToRawIndexConverter.java   |     4 +-
 .../pinot/tools/TestDataAndQueryAnonymizer.java    |     6 +-
 pinot-tools/src/test/resources/columns.mapping     |     4 +-
 pinot-tools/src/test/resources/queries.generated   |     4 +-
 pinot-tools/src/test/resources/queries.raw         |     4 +-
 pom.xml                                            |     1 +
 thirdeye/pom.xml                                   |     4 +-
 .../app/pods/manage/alerts/index/controller.js     |    20 +-
 .../app/pods/manage/alerts/index/route.js          |    12 +-
 thirdeye/thirdeye-pinot/pom.xml                    |     5 +
 .../anomaly/ThirdEyeAnomalyApplication.java        |     3 +-
 .../trigger/DataAvailabilityTaskScheduler.java     |    29 +-
 .../DataAvailabilitySchedulingConfiguration.java   |    10 +
 .../trigger/utils/DatasetTriggerInfoRepo.java      |     2 +-
 .../anomaly/utils/ThirdeyeMetricsUtil.java         |    18 +
 .../thirdeye/datalayer/pojo/DatasetConfigBean.java |     6 +-
 .../thirdeye/detection/DetectionPipelineJob.java   |    11 -
 .../apache/pinot/thirdeye/detection/TaskUtils.java |    11 +-
 .../detection/alert/DetectionAlertTaskRunner.java  |    12 +-
 .../alert/StatefulDetectionAlertFilter.java        |     9 +-
 .../filter/DimensionDetectionAlertFilter.java      |   121 -
 .../filter/DimensionsRecipientAlertFilter.java     |     4 +-
 .../alert/filter/PerUserDimensionAlertFilter.java  |    51 +-
 .../detection/alert/filter/SubscriptionUtils.java  |    17 +-
 .../ToAllRecipientsDetectionAlertFilter.java       |    18 +-
 .../alert/scheme/DetectionAlertScheme.java         |     9 +-
 .../alert/scheme/DetectionEmailAlerter.java        |    15 +-
 .../alert/scheme/DetectionJiraAlerter.java         |    29 +-
 .../thirdeye/detection/health/DetectionHealth.java |     7 +-
 .../detection/health/DetectionTaskStatus.java      |    51 +-
 .../validators/SubscriptionConfigValidator.java    |     1 -
 .../formatter/DetectionConfigFormatter.java        |    12 +-
 .../notification/commons/ThirdEyeJiraClient.java   |    10 +-
 .../apache/pinot/thirdeye/util/ThirdEyeUtils.java  |    22 +
 .../trigger/DataAvailabilityTaskSchedulerTest.java |    35 +-
 .../detection/alert/filter/AlertFilterUtils.java   |    12 +
 .../filter/DimensionDetectionAlertFilterTest.java  |   197 -
 .../filter/PerUserDimensionAlertFilterTest.java    |    17 +-
 .../ToAllRecipientsDetectionAlertFilterTest.java   |    42 +-
 340 files changed, 33548 insertions(+), 15156 deletions(-)
 create mode 100644 licenses-binary/LICENSE-fusesource.txt
 create mode 100644 licenses-binary/LICENSE-py4j.txt
 create mode 100644 licenses-binary/LICENSE-threeten.txt
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerRouting.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerServerBuilder.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerUserDefineMessageHandlerFactory.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/TimeboundaryRefreshMessageHandlerFactory.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/HelixExternalViewBasedRouting.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/HelixExternalViewBasedTimeBoundaryService.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingManager.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingTable.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingTableBuilderFactory.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingTableLookupRequest.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/TimeBoundaryService.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/BalancedRandomRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/BasePartitionAwareRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/BaseRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/DefaultOfflineRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/DefaultRealtimeRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/GeneratorBasedRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/HighLevelConsumerBasedRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/InstanceConfigManager.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/LargeClusterRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/LowLevelConsumerRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/LowLevelRoutingTableBuilderUtil.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareRealtimeRoutingTableBuilder.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/RoutingTableBuilder.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/BalancedInstanceSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/BaseInstanceSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorFactory.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/ReplicaGroupInstanceSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/PartitionSegmentPruner.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPruner.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/OfflineSegmentSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/RealtimeSegmentSelector.java
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelector.java
 copy pinot-core/src/main/java/org/apache/pinot/core/segment/creator/SegmentPreIndexStatsCollector.java => pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorFactory.java (61%)
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/selector/MergedSegmentSelector.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/selector/SegmentSelector.java
 delete mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/selector/SegmentSelectorProvider.java
 copy pinot-core/src/main/java/org/apache/pinot/core/segment/creator/SegmentPreIndexStatsCollector.java => pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryInfo.java (66%)
 create mode 100644 pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManager.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/HelixExternalViewBasedTimeBoundaryServiceTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/RandomRoutingTableTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/RoutingTableTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/BalancedRandomRoutingTableBuilderTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/HighLevelConsumerRoutingTableBuilderTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/LargeClusterRoutingTableBuilderTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/LowLevelConsumerRoutingTableBuilderTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilderTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareRealtimeRoutingTableBuilderTest.java
 create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorTest.java
 create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java
 create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorTest.java
 delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/selector/MergedSegmentSelectorTest.java
 create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
 create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/config/TextIndexConfigValidator.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/data/StarTreeIndexSpec.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/messages/TimeboundaryRefreshMessage.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/response/BrokerResponseFactory.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/segment/StarTreeMetadata.java
 create mode 100644 pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/TextMatchPredicateAstNode.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/startree/hll/HllConfig.java
 delete mode 100644 pinot-common/src/main/java/org/apache/pinot/startree/hll/HllSizeUtils.java
 create mode 100644 pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotClusterConfigs.java
 rename pinot-core/src/main/java/org/apache/pinot/core/{plan/AggregationFunctionInitializer.java => common/predicate/TextMatchPredicate.java} (55%)
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/operator/filter/TextMatchFilterOperator.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/operator/filter/predicate/TextMatchPredicateEvaluatorFactory.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/plan/maker/BrokerRequestPreProcessor.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexReaderRefreshThread.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneIndexRefreshState.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/invertedindex/RealtimeLuceneTextIndexReader.java
 rename pinot-common/src/main/java/org/apache/pinot/startree/hll/HllConstants.java => pinot-core/src/main/java/org/apache/pinot/core/segment/creator/TextIndexType.java (81%)
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/segment/index/readers/text/LuceneDocIdCollector.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/segment/index/readers/text/LuceneTextIndexReader.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/DimensionBuffer.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/MetricBuffer.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/OffHeapStarTreeBuilder.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/StarTreeBuilder.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/StarTreeBuilderConfig.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/StarTreeDataTable.java
 delete mode 100644 pinot-core/src/main/java/org/apache/pinot/core/startree/hll/HllUtil.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/BaseStarTreeIndexTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/OffHeapStarTreeBuilderTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeDataTableTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeIndexTestSegmentHelper.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/SumStarTreeIndexTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/TestStarTreeMetadata.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/HllFieldSizeTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/HllIndexCreationTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/HllStarTreeIndexTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/HllTypeConversionTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/OffHeapStarTreeBuilderWithHllFieldTest.java
 delete mode 100644 pinot-core/src/test/java/org/apache/pinot/core/startree/hll/SegmentWithHllIndexCreateHelper.java
 create mode 100644 pinot-core/src/test/java/org/apache/pinot/queries/TestTextSearchQueries.java
 create mode 100644 pinot-core/src/test/resources/data/newColumnsWithTextSchema.json
 create mode 100644 pinot-core/src/test/resources/data/text_search_data/group_by_grep_results.out
 create mode 100644 pinot-core/src/test/resources/data/text_search_data/pql_query1.txt
 create mode 100644 pinot-core/src/test/resources/data/text_search_data/skills.txt
 create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
 delete mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StarTreeV2ClusterIntegrationTest.java
 delete mode 100644 pinot-integration-tests/src/test/resources/OnTimeStarTreeQueries.txt
 create mode 100644 pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/OperateClusterConfigCommand.java
 create mode 100644 pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickStartCommand.java
 delete mode 100644 pinot-tools/src/main/java/org/apache/pinot/tools/query/comparison/StarQueryComparison.java
 delete mode 100644 pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/ColumnarToStarTreeConverter.java
 delete mode 100644 thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/filter/DimensionDetectionAlertFilter.java
 delete mode 100644 thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/filter/DimensionDetectionAlertFilterTest.java


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 01/02: Update license and notice

Posted by xi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch release-0.3.0
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 1859a6b712d90623b4a75731d4b35f2f15eb8e62
Author: Xiang Fu <fx...@gmail.com>
AuthorDate: Mon Mar 2 03:58:37 2020 -0800

    Update license and notice
---
 LICENSE-binary                         | 131 +++++++++-
 NOTICE                                 |   2 +-
 NOTICE-binary                          | 448 ++++++++++++++++++++-------------
 licenses-binary/LICENSE-fusesource.txt |  27 ++
 licenses-binary/LICENSE-py4j.txt       |  26 ++
 licenses-binary/LICENSE-threeten.txt   |  29 +++
 6 files changed, 475 insertions(+), 188 deletions(-)

diff --git a/LICENSE-binary b/LICENSE-binary
index 49767f3..5c0e3e4 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -206,6 +206,7 @@ License Version 2.0:
 
 com.101tec:zkclient:0.7
 com.amazonaws:aws-java-sdk:1.7.4
+com.beust.jcommander:1.64
 com.clearspring.analytics:stream:2.7.0
 com.fasterxml.jackson.core:jackson-annotations:2.9.8
 com.fasterxml.jackson.core:jackson-core:2.9.8
@@ -214,40 +215,76 @@ com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.9.8
 com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.9.8
 com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.9.8
 com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.9.8
+com.fasterxml.jackson.module:jackson-module-paranamer:2.6.5
+com.fasterxml.jackson.module:jackson-module-scala_2.11:2.6.5
+com.github.jnr:jffi:1.2.9
+com.github.jnr:jnr-constants:0.8.7
+com.github.jnr:jnr-ffi:2.0.3
+com.google.api.grpc:proto-google-common-protos:1.17.0
+com.google.api.grpc:proto-google-iam-v1:0.13.0
+com.google.api:api-common:1.8.1
+com.google.api:gax:1.49.1
+com.google.api:gax-httpjson:0.66.1
+com.google.api:google-api-client:1.30.4
+com.google.api:google-api-services-storage:v1-rev20190910-1.30.3.jar
+com.google.auth:google-auth-library-credentials:0.18.0
+com.google.auth:google-auth-library-oauth2-http:0.18.0
+com.google.auto.value:auto-value-annotations:1.6.6
+com.google.cloud:google-cloud-core:1.91.3
+com.google.cloud:google-cloud-core-http:1.91.3
+com.google.cloud:google-cloud-storage:1.101.0
 com.google.code.findbugs:jsr305:3.0.0
 com.google.code.gson:gson:2.2.4
+com.google.errorprone:error_prone_annotations:2.3.2
 com.google.guava:guava:20.0
+com.google.http-client:google-http-client-appengine:1.32.1
+com.google.http-client:google-http-client-jackson2:1.32.1
+com.google.http-client:google-http-client:1.32.1
 com.google.inject.extensions:guice-servlet:3.0
 com.google.inject:guice:3.0
+com.google.j2objc:j2objc-annotations:1.3
+com.google.oauth-client:google-oauth-client:1.30.3
+com.google.protobuf:protobuf-java-util:3.10.0
+com.google.protobuf:protobuf-java:2.5.0
+com.google.protobuf:protobuf-java:3.10.0
 com.jamesmurty.utils:java-xmlbuilder:0.4
 com.lmax:disruptor:3.3.4
+com.ning:async-http-client:1.9.21
+com.ning:compress-lzf:1.0.3
+com.tdunning:t-digest:3.2
+com.twitter:chill-java:0.8.0
+com.twitter:chill_2.11:0.8.0
+com.typesafe:config:1.3.2
+com.typesafe.scala-logging:scala-logging_2.11:3.9.0
+com.yammer.metrics:metrics-core:2.2.0
 commons-beanutils:commons-beanutils:1.8.3
 commons-beanutils:commons-beanutils-core:1.8.0
 commons-cli:commons-cli:1.2
 commons-codec:commons-codec:1.6
 commons-collections:commons-collections:3.2.1
 commons-configuration:commons-configuration:1.6
+commons-daemon:commons-daemon:1.0.13
 commons-digester:commons-digester:1.8
 commons-httpclient:commons-httpclient:3.1
 commons-io:commons-io:2.4
 commons-lang:commons-lang:2.6
 commons-logging:commons-logging:1.2
 commons-net:commons-net:3.1
-com.ning:async-http-client:1.9.21
-com.tdunning:t-digest:3.2
-com.typesafe:config:1.3.2
-com.yammer.metrics:metrics-core:2.2.0
 io.airlift:aircompressor:0.10
 io.dropwizard.metrics:metrics-core:3.2.3
+io.dropwizard.metrics:metrics-graphite:3.1.2
+io.dropwizard.metrics:metrics-json:3.1.2
+io.dropwizard.metrics:metrics-jvm:3.1.2
+io.grpc:grpc-context:1.22.1
 io.netty:netty:3.9.6.Final
 io.netty:netty-all:4.1.28.Final
+io.opencensus:opencensus-api:0.24.0
+io.opencensus:opencensus-contrib-http-util:0.24.0
 io.swagger:swagger-annotations:1.5.16
 io.swagger:swagger-core:1.5.16
 io.swagger:swagger-jaxrs:1.5.16
 io.swagger:swagger-jersey2-jaxrs:1.5.16
 io.swagger:swagger-models:1.5.16
-io.vavr:vavr:0.9.2
-io.vavr:vavr-match:0.9.2
 it.unimi.dsi:fastutil:8.2.3
 javax.inject:javax.inject:1
 javax.servlet.jsp:jsp-api:2.1
@@ -255,20 +292,25 @@ javax.servlet:servlet-api:2.5
 javax.validation:validation-api:2.0.1.Final
 joda-time:joda-time:2.0
 me.lemire.integercompression:JavaFastPFOR:0.0.13
-net.java.dev.jets3t:jets3t:0.9.0
+mx4j:mx4j:3.0.2
+net.bytebuddy:byte-buddy-agent:1.7.4
+net.bytebuddy:byte-buddy:1.7.4
+net.java.dev.jets3t:jets3t:0.9.3
 net.jpountz.lz4:lz4:1.2.0
+nl.jqno.equalsverifier:equalsverifier:1.7.2
 org.apache.avro:avro:1.7.6
 org.apache.avro:avro-ipc:1.7.4
 org.apache.avro:avro-mapred:1.7.4
 org.apache.calcite.avatica:avatica-core:1.13.0
+org.apache.calcite:calcite-babel:1.19.0
 org.apache.calcite:calcite-core:1.19.0
 org.apache.calcite:calcite-linq4j:1.19.0
 org.apache.commons:commons-compress:1.9
+org.apache.commons:commons-crypto:1.0.0
 org.apache.commons:commons-csv:1.0
 org.apache.commons:commons-lang3:3.5
 org.apache.commons:commons-math:2.1
 org.apache.commons:commons-math3:3.2
-org.apache.commons:commons-math3:3.2
 org.apache.curator:curator-client:2.7.1
 org.apache.curator:curator-framework:2.7.1
 org.apache.curator:curator-recipes:2.7.1
@@ -279,21 +321,41 @@ org.apache.directory.server:apacheds-kerberos-codec:2.0.0-M15
 org.apache.hadoop:hadoop-annotations:2.2.0
 org.apache.hadoop:hadoop-auth:2.7.0
 org.apache.hadoop:hadoop-aws:2.7.0
+org.apache.hadoop:hadoop-client:2.7.0
 org.apache.hadoop:hadoop-common:2.7.0
 org.apache.hadoop:hadoop-hdfs:2.2.0
-org.apache.hadoop:hadoop-mapreduce-client-core:2.2.0
-org.apache.helix:helix-core:0.8.4
+org.apache.hadoop:hadoop-mapreduce-client-app:2.7.0
+org.apache.hadoop:hadoop-mapreduce-client-common:2.7.0
+org.apache.hadoop:hadoop-mapreduce-client-core:2.7.0
+org.apache.hadoop:hadoop-mapreduce-client-hs:2.7.0
+org.apache.hadoop:hadoop-mapreduce-client-jobclient:2.7.0
+org.apache.hadoop:hadoop-mapreduce-client-shuffle:2.7.0
+org.apache.hadoop:hadoop-minicluster:2.7.0
+org.apache.hadoop:hadoop-yarn-api:2.7.0
+org.apache.hadoop:hadoop-yarn-client:2.7.0
+org.apache.hadoop:hadoop-yarn-common:2.7.0
+org.apache.hadoop:hadoop-yarn-server-applicationhistoryservice:2.7.0
+org.apache.hadoop:hadoop-yarn-server-common:2.7.0
+org.apache.hadoop:hadoop-yarn-server-nodemanager:2.7.0
+org.apache.hadoop:hadoop-yarn-server-resourcemanager:2.7.0
+org.apache.hadoop:hadoop-yarn-server-web-proxy:2.7.0
+org.apache.helix:helix-core:0.9.4
 org.apache.hive:hive-storage-api:2.6.0
 org.apache.htrace:htrace-core:3.1.0-incubating
 org.apache.httpcomponents:httpclient:4.5.3
 org.apache.httpcomponents:httpcore:4.4.6
 org.apache.httpcomponents:httpmime:4.5.3
-org.apache.kafka:kafka_2.10:0.9.0.1
-org.apache.kafka:kafka-clients:0.9.0.1
+org.apache.kafka:kafka-clients:2.0.0
+org.apache.kafka:kafka_2.11:2.0.0
 org.apache.logging.log4j:log4j-1.2-api:2.11.2
 org.apache.logging.log4j:log4j-api:2.11.2
 org.apache.logging.log4j:log4j-core:2.11.2
 org.apache.logging.log4j:log4j-slf4j-impl:2.11.2
+org.apache.lucene:lucene-analyzers-common:8.2.0
+org.apache.lucene:lucene-core:8.2.0
+org.apache.lucene:lucene-queries:8.2.0
+org.apache.lucene:lucene-queryparser:8.2.0
+org.apache.lucene:lucene-sandbox:8.2.0
 org.apache.orc:orc-core:1.5.2
 org.apache.orc:orc-mapreduce:1.5.2
 org.apache.orc:orc-shims:1.5.2
@@ -304,19 +366,34 @@ org.apache.parquet:parquet-encoding:1.8.0
 org.apache.parquet:parquet-format:2.3.0-incubating
 org.apache.parquet:parquet-hadoop:1.8.0
 org.apache.parquet:parquet-jackson:1.8.0
+org.apache.spark:spark-core_2.11:2.2.0
+org.apache.spark:spark-launcher_2.11:2.2.0
+org.apache.spark:spark-network-common_2.11:2.2.0
+org.apache.spark:spark-network-shuffle_2.11:2.2.0
+org.apache.spark:spark-tags_2.11:2.2.0
+org.apache.spark:spark-unsafe_2.11:2.2.0
 org.apache.thrift:libthrift:0.12.0
 org.apache.velocity:velocity:1.7
+org.apache.xbean:xbean-asm5-shaded:4.4
 org.apache.yetus:audience-annotations:0.5.0
 org.apache.zookeeper:zookeeper:3.4.11
 org.codehaus.jackson:jackson-core-asl:1.9.13
+org.codehaus.jackson:jackson-jaxrs:1.9.13
 org.codehaus.jackson:jackson-mapper-asl:1.9.13
+org.codehaus.jackson:jackson-xc:1.9.13
+org.codehaus.jettison:jettison:1.1
 org.javassist:javassist:3.19.0-GA
+org.json4s:json4s-ast_2.11:3.2.11
+org.json4s:json4s-core_2.11:3.2.11
+org.json4s:json4s-jackson_2.11:3.2.11
+org.lz4:lz4-java:1.4.1
 org.mortbay.jetty:jetty:6.1.26
 org.mortbay.jetty:jetty-util:6.1.26
 org.mortbay.jetty:servlet-api:2.5-20081211
 org.objenesis:objenesis:2.1
 org.roaringbitmap:RoaringBitmap:0.8.0
 org.roaringbitmap:shims:0.8.0
+org.testng:testng:6.11
 org.webjars:swagger-ui:2.2.2
 org.xerial.java:xerial-core:2.1
 org.xerial.larray:larray:0.2.1
@@ -325,6 +402,9 @@ org.xerial.larray:larray-mmap:0.2.1
 org.xerial.snappy:snappy-java:1.1.1.7
 org.xerial:xerial-core:3.2.2
 org.yaml:snakeyaml:1.16
+oro:oro:2.0.8
+xerces:xercesImpl:2.9.1
+xml-apis:xml-apis:1.0.b2
 
 
 ------------------------------------------------------------------------------------
@@ -336,9 +416,15 @@ of these licenses.
 MIT License
 -----------
 args4j:args4j:2.32
+com.github.jnr:jnr-x86asm:1.0.2
 com.microsoft.azure:azure-data-lake-store-sdk:2.1.5
+net.razorvine:pyrolite:4.13
 net.sf.jopt-simple:jopt-simple:4.6
+org.mockito:mockito-core:2.10.0
+org.slf4j:jcl-over-slf4j:1.7.16
+org.slf4j:jul-to-slf4j:1.7.16
 org.slf4j:slf4j-api:1.7.25
+org.slf4j:slf4j-log4j12:1.7.16
 
 pinot-controller/src/main/resources/*/js/lib/codemirror/*
 pinot-controller/src/main/resources/*/js/lib/foundation/*
@@ -356,6 +442,7 @@ pinot-controller/src/main/resources/*/css/lib/normalize.css
 BSD 2-Clause
 ------------
 jline:jline:0.9.94
+net.sf.py4j:py4j:0.10.4
 org.reflections:reflections:0.9.11
 
 
@@ -370,7 +457,14 @@ com.jcabi:jcabi-log:0.17.1
 com.jcraft:jsch:0.1.42
 com.thoughtworks.paranamer:paranamer:2.3
 org.antlr:antlr4-runtime:4.6
+org.fusesource.leveldbjni:leveldbjni-all:1.8
+org.ow2.asm:asm:5.0.3
+org.ow2.asm:asm-analysis:5.0.3
+org.ow2.asm:asm-commons:5.0.3
+org.ow2.asm:asm-tree:5.0.3
+org.ow2.asm:asm-util:5.0.3
 org.scala-lang:scala-library:2.10.5
+org.threeten:threetenbp:1.3.3
 xmlenc:xmlenc:0.52
 
 pinot-controller/src/main/resources/*/js/lib/jquery.dataTables.min.js
@@ -380,6 +474,7 @@ Common Development and Distribution License (CDDL) 1.0
 ------------------------------------------------------
 (see license/LICENSE-cddl-1.0.txt)
 
+javax.activation:activation:1.1.1
 javax.servlet:javax.servlet-api:3.0.1
 javax.ws.rs:jsr311-api:1.1.1
 
@@ -389,8 +484,17 @@ Common Development and Distribution License (CDDL) 1.1
 (see license/LICENSE-cddl-1.1.txt)
 
 com.sun.jersey:jersey-core:1.9
+com.sun.jersey:jersey-json:1.9
 com.sun.jersey:jersey-server:1.9
+com.sun.jersey.contribs:jersey-guice:1.9
+com.sun.xml.bind:jaxb-core:2.3.0
+com.sun.xml.bind:jaxb-impl:2.3.0
+javax.annotation:javax.annotation-api:1.3.2
+javax.mail:mail:1.4.7
+javax.servlet:javax.servlet-api:3.0.1
 javax.ws.rs:javax.ws.rs-api:2.0.1
+javax.ws.rs:jsr311-api:1.1.1
+javax.xml.bind:jaxb-api:2.3.0
 org.glassfish.hk2:osgi-resource-locator:1.0.1
 org.glassfish.jersey.containers:jersey-container-servlet-core:2.25.1
 org.glassfish.tyrus.bundles:tyrus-standalone-client:1.5
@@ -400,6 +504,7 @@ Common Public License (CPL) 1.0
 -------------------------------
 (see license/LICENSE-cpl-1.0.txt)
 
+com.github.jnr:jnr-posix:3.0.12
 junit:junit:3.8.1
 
 
@@ -408,6 +513,7 @@ Eclipse Public License (EPL) 1.0
 (see license/LICENSE-epl-1.0.txt)
 
 org.aspectj:aspectjrt:1.8.4
+com.h2database:h2:1.4.193
 
 
 Eclipse Public License (EPL) 2.0
@@ -442,4 +548,5 @@ Public Domain
 -------------
 
 aopalliance:aopalliance:1.0
+net.iharder:base64:2.3.8
 
diff --git a/NOTICE b/NOTICE
index 84efe86..ce85af5 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Pinot (incubating)
-Copyright 2018-2019 The Apache Software Foundation
+Copyright 2018-2020 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
diff --git a/NOTICE-binary b/NOTICE-binary
index 377c7c1..f5b4a4b 100644
--- a/NOTICE-binary
+++ b/NOTICE-binary
@@ -9,25 +9,36 @@ The Apache Software Foundation (http://www.apache.org/).
 // Version 2.0, in this case for 
 // ------------------------------------------------------------------
 
-htrace-core
-Copyright 2015 The Apache Software Foundation
+The HermiteInterpolator class and its corresponding test have been imported from
+the orekit library distributed under the terms of the Apache 2 licence. Original
+source copyright:
+Copyright 2010-2012 CS Systèmes d'Information
+===============================================================================
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
 
+Apache Commons Configuration
+Copyright 2001-2008 The Apache Software Foundation
 
-Apache HttpClient Mime
-Copyright 1999-2017 The Apache Software Foundation
+This product includes software developed by
+The Apache Software Foundation (http://www.apache.org/).
 
-Apache HttpClient
-Copyright 1999-2017 The Apache Software Foundation
+Apache Commons Collections
+Copyright 2001-2008 The Apache Software Foundation
+
+Apache Jakarta Commons Digester
+Copyright 2001-2006 The Apache Software Foundation
+
+Apache Commons BeanUtils
+Copyright 2000-2010 The Apache Software Foundation
+
+Apache Commons BeanUtils
+Copyright 2000-2008 The Apache Software Foundation
 
 Apache Commons Codec
 Copyright 2002-2011 The Apache Software Foundation
 
-This product includes software developed by
-The Apache Software Foundation (http://www.apache.org/).
-
 --------------------------------------------------------------------------------
 src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains 
 test data from http://aspell.sourceforge.net/test/batch0.tab.
@@ -37,17 +48,14 @@ and distribution of this entire article is permitted in any medium,
 provided this notice is preserved.
 --------------------------------------------------------------------------------
 
-Apache HttpCore
-Copyright 2005-2017 The Apache Software Foundation
-
-Calcite Core
-Copyright 2012-2019 The Apache Software Foundation
+Apache Commons IO
+Copyright 2002-2012 The Apache Software Foundation
 
-Apache Calcite Avatica
-Copyright 2012-2018 The Apache Software Foundation
+Apache Commons Lang
+Copyright 2001-2011 The Apache Software Foundation
 
-Calcite Linq4j
-Copyright 2012-2019 The Apache Software Foundation
+Apache Commons Logging
+Copyright 2003-2014 The Apache Software Foundation
 
 Apache Commons Lang
 Copyright 2001-2016 The Apache Software Foundation
@@ -55,20 +63,6 @@ Copyright 2001-2016 The Apache Software Foundation
 This product includes software from the Spring Framework,
 under the Apache License 2.0 (see: StringUtils.containsWhitespace())
 
-Apache Thrift
-Copyright 2006-2017 The Apache Software Foundation.
-
-Apache Avro
-Copyright 2009-2014 The Apache Software Foundation
-
-This product currently only contains code developed by authors
-of specific components, as identified by the source code files;
-if such notes are missing files have been created by
-Tatu Saloranta.
-
-For additional credits (generally to people who reported problems)
-see CREDITS file.
-
 Apache Log4j SLF4J Binding
 Copyright 1999-2019 The Apache Software Foundation
 
@@ -84,27 +78,54 @@ Copyright 1999-2019 The Apache Software Foundation
 This product includes software developed by
 Joda.org (http://www.joda.org/).
 
-Apache Commons Configuration
-Copyright 2001-2008 The Apache Software Foundation
+# Jackson JSON processor
 
-Apache Commons Collections
-Copyright 2001-2008 The Apache Software Foundation
+Jackson is a high-performance, Free/Open Source JSON processing library.
+It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+been in development since 2007.
+It is currently developed by a community of developers, as well as supported
+commercially by FasterXML.com.
 
-Apache Jakarta Commons Digester
-Copyright 2001-2006 The Apache Software Foundation
+## Licensing
 
-Apache Commons BeanUtils
-Copyright 2000-2010 The Apache Software Foundation
+Jackson core and extension components may be licensed under different licenses.
+To find the details that apply to this artifact see the accompanying LICENSE file.
+For more information, including possible other licensing options, contact
+FasterXML.com (http://fasterxml.com).
 
-Apache Commons BeanUtils
-Copyright 2000-2008 The Apache Software Foundation
+## Credits
+
+A list of contributors may be found from CREDITS file, which is included
+in some artifacts (usually source distributions); but is always available
+from the source code management (SCM) system project uses.
+
+Apache HttpClient Mime
+Copyright 1999-2017 The Apache Software Foundation
+
+Apache HttpClient
+Copyright 1999-2017 The Apache Software Foundation
+
+Apache HttpCore
+Copyright 2005-2017 The Apache Software Foundation
+
+Calcite Core
+Copyright 2012-2019 The Apache Software Foundation
+
+Apache Calcite Avatica
+Copyright 2012-2018 The Apache Software Foundation
+
+Calcite Linq4j
+Copyright 2012-2019 The Apache Software Foundation
+
+Calcite Babel
+Copyright 2012-2019 The Apache Software Foundation
+
+Apache Thrift
+Copyright 2006-2017 The Apache Software Foundation.
 
 Apache Jakarta HttpClient
 Copyright 1999-2007 The Apache Software Foundation
 
-Apache Commons IO
-Copyright 2002-2012 The Apache Software Foundation
-
 Apache Commons Compress
 Copyright 2002-2014 The Apache Software Foundation
 
@@ -115,7 +136,15 @@ which has been placed in the public domain:
 "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
 
 Apache Helix :: Core
-Copyright 2019 The Apache Software Foundation
+Copyright 2020 The Apache Software Foundation
+
+This product currently only contains code developed by authors
+of specific components, as identified by the source code files;
+if such notes are missing files have been created by
+Tatu Saloranta.
+
+For additional credits (generally to people who reported problems)
+see CREDITS file.
 
                             The Netty Project
                             =================
@@ -233,56 +262,9 @@ framework implementation, which can be obtained at:
   * HOMEPAGE:
     * http://felix.apache.org/
 
-# Jackson JSON processor
-
-Jackson is a high-performance, Free/Open Source JSON processing library.
-It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
-been in development since 2007.
-It is currently developed by a community of developers, as well as supported
-commercially by FasterXML.com.
-
-## Licensing
-
-Jackson core and extension components may be licensed under different licenses.
-To find the details that apply to this artifact see the accompanying LICENSE file.
-For more information, including possible other licensing options, contact
-FasterXML.com (http://fasterxml.com).
-
-## Credits
-
-A list of contributors may be found from CREDITS file, which is included
-in some artifacts (usually source distributions); but is always available
-from the source code management (SCM) system project uses.
-
-Apache Commons Logging
-Copyright 2003-2014 The Apache Software Foundation
-
-Apache Commons Lang
-Copyright 2001-2011 The Apache Software Foundation
-
 Apache Yetus - Audience Annotations
 Copyright 2015-2017 The Apache Software Foundation
 
-Apache Commons CSV
-Copyright 2005-2014 The Apache Software Foundation
-
-src/main/resources/contract.txt
-This file was downloaded from http://www.ferc.gov/docs-filing/eqr/soft-tools/sample-csv/contract.txt and contains neither copyright notice nor license.
-
-src/main/resources/transaction.txt
-This file was downloaded from http://www.ferc.gov/docs-filing/eqr/soft-tools/sample-csv/transaction.txt and contains neither copyright notice nor license.
-
-src/test/resources/CSVFileParser/bom.csv
-src/test/resources/CSVFileParser/test.csv
-src/test/resources/CSVFileParser/test_default.txt
-src/test/resources/CSVFileParser/test_default_comment.txt
-src/test/resources/CSVFileParser/test_rfc4180.txt
-src/test/resources/CSVFileParser/test_rfc4180_trim.txt
-src/test/resources/CSVFileParser/testCSV85.csv
-src/test/resources/CSVFileParser/testCSV85_default.txt
-src/test/resources/CSVFileParser/testCSV85_ignoreEmpty.txt
-These files are used as test data and test result specifications.
-
 Apache Commons Math
 Copyright 2001-2010 The Apache Software Foundation
 
@@ -329,6 +311,204 @@ The complete text of licenses and disclaimers associated with the the original
 sources enumerated above at the time of code translation are in the LICENSE.txt
 file.
 
+Apache Lucene
+Copyright 2001-2019 The Apache Software Foundation
+
+Includes software from other Apache Software Foundation projects,
+including, but not limited to:
+ - Apache Ant
+ - Apache Jakarta Regexp
+ - Apache Commons
+ - Apache Xerces
+
+ICU4J, (under analysis/icu) is licensed under an MIT styles license
+and Copyright (c) 1995-2008 International Business Machines Corporation and others
+
+Some data files (under analysis/icu/src/data) are derived from Unicode data such
+as the Unicode Character Database. See http://unicode.org/copyright.html for more
+details.
+
+Brics Automaton (under core/src/java/org/apache/lucene/util/automaton) is
+BSD-licensed, created by Anders Møller. See http://www.brics.dk/automaton/
+
+The levenshtein automata tables (under core/src/java/org/apache/lucene/util/automaton) were
+automatically generated with the moman/finenight FSA library, created by
+Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
+see http://sites.google.com/site/rrettesite/moman and
+http://bitbucket.org/jpbarrette/moman/overview/
+
+The class org.apache.lucene.util.WeakIdentityMap was derived from
+the Apache CXF project and is Apache License 2.0.
+
+The Google Code Prettify is Apache License 2.0.
+See http://code.google.com/p/google-code-prettify/
+
+JUnit (junit-4.10) is licensed under the Common Public License v. 1.0
+See http://junit.sourceforge.net/cpl-v10.html
+
+This product includes code (JaspellTernarySearchTrie) from Java Spelling Checkin
+g Package (jaspell): http://jaspell.sourceforge.net/
+License: The BSD License (http://www.opensource.org/licenses/bsd-license.php)
+
+The snowball stemmers in
+  analysis/common/src/java/net/sf/snowball
+were developed by Martin Porter and Richard Boulton.
+The snowball stopword lists in
+  analysis/common/src/resources/org/apache/lucene/analysis/snowball
+were developed by Martin Porter and Richard Boulton.
+The full snowball package is available from
+  http://snowball.tartarus.org/
+
+The KStem stemmer in
+  analysis/common/src/org/apache/lucene/analysis/en
+was developed by Bob Krovetz and Sergio Guzman-Lara (CIIR-UMass Amherst)
+under the BSD-license.
+
+The Arabic,Persian,Romanian,Bulgarian, Hindi and Bengali analyzers (common) come with a default
+stopword list that is BSD-licensed created by Jacques Savoy.  These files reside in:
+analysis/common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/bn/stopwords.txt
+See http://members.unine.ch/jacques.savoy/clef/index.html.
+
+The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers
+(common) are based on BSD-licensed reference implementations created by Jacques Savoy and
+Ljiljana Dolamic. These files reside in:
+analysis/common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java
+
+The Stempel analyzer (stempel) includes BSD-licensed software developed
+by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil,
+and Edmond Nolan.
+
+The Polish analyzer (stempel) comes with a default
+stopword list that is BSD-licensed created by the Carrot2 project. The file resides
+in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt.
+See http://project.carrot2.org/license.html.
+
+The SmartChineseAnalyzer source code (smartcn) was
+provided by Xiaoping Gao and copyright 2009 by www.imdict.net.
+
+WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/)
+is derived from Unicode data such as the Unicode Character Database.
+See http://unicode.org/copyright.html for more details.
+
+The Morfologik analyzer (morfologik) includes BSD-licensed software
+developed by Dawid Weiss and Marcin Miłkowski (http://morfologik.blogspot.com/).
+
+Morfologik uses data from Polish ispell/myspell dictionary
+(http://www.sjp.pl/slownik/en/) licenced on the terms of (inter alia)
+LGPL and Creative Commons ShareAlike.
+
+Morfologic includes data from BSD-licensed dictionary of Polish (SGJP)
+(http://sgjp.pl/morfeusz/)
+
+Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original
+source code for this can be found at http://www.eclipse.org/jetty/downloads.php
+
+===========================================================================
+Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration
+===========================================================================
+
+This software includes a binary and/or source version of data from
+
+  mecab-ipadic-2.7.0-20070801
+
+which can be obtained from
+
+  http://atilika.com/releases/mecab-ipadic/mecab-ipadic-2.7.0-20070801.tar.gz
+
+or
+
+  http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz
+
+===========================================================================
+mecab-ipadic-2.7.0-20070801 Notice
+===========================================================================
+
+Nara Institute of Science and Technology (NAIST),
+the copyright holders, disclaims all warranties with regard to this
+software, including all implied warranties of merchantability and
+fitness, in no event shall NAIST be liable for
+any special, indirect or consequential damages or any damages
+whatsoever resulting from loss of use, data or profits, whether in an
+action of contract, negligence or other tortuous action, arising out
+of or in connection with the use or performance of this software.
+
+A large portion of the dictionary entries
+originate from ICOT Free Software.  The following conditions for ICOT
+Free Software applies to the current dictionary as well.
+
+Each User may also freely distribute the Program, whether in its
+original form or modified, to any third party or parties, PROVIDED
+that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear
+on, or be attached to, the Program, which is distributed substantially
+in the same form as set out herein and that such intended
+distribution, if actually made, will neither violate or otherwise
+contravene any of the laws and regulations of the countries having
+jurisdiction over the User or the intended distribution itself.
+
+NO WARRANTY
+
+The program was produced on an experimental basis in the course of the
+research and development conducted during the project and is provided
+to users as so produced on an experimental basis.  Accordingly, the
+program is provided without any warranty whatsoever, whether express,
+implied, statutory or otherwise.  The term "warranty" used herein
+includes, but is not limited to, any warranty of the quality,
+performance, merchantability and fitness for a particular purpose of
+the program and the nonexistence of any infringement or violation of
+any right of any third party.
+
+Each user of the program will agree and understand, and be deemed to
+have agreed and understood, that there is no warranty whatsoever for
+the program and, accordingly, the entire risk arising from or
+otherwise connected with the program is assumed by the user.
+
+Therefore, neither ICOT, the copyright holder, or any other
+organization that participated in or was otherwise related to the
+development of the program and their respective officials, directors,
+officers and other employees shall be held liable for any and all
+damages, including, without limitation, general, special, incidental
+and consequential damages, arising out of or otherwise in connection
+with the use or inability to use the program or any product, material
+or result produced or otherwise obtained by using the program,
+regardless of whether they have been advised of, or otherwise had
+knowledge of, the possibility of such damages at any time during the
+project or thereafter.  Each user will be deemed to have agreed to the
+foregoing by his or her commencement of use of the program.  The term
+"use" as used herein includes, but is not limited to, the use,
+modification, copying and distribution of the program and the
+production of secondary products from the program.
+
+In the case where the program, whether in its original form or
+modified, was distributed or delivered to or received by a user from
+any person, organization or entity other than ICOT, unless it makes or
+grants independently of ICOT any specific warranty to the user in
+writing, such person, organization or entity, will also be exempted
+from and not be held liable to the user for any such damages as noted
+above as far as the program is concerned.
+
+===========================================================================
+Nori Korean Morphological Analyzer - Apache Lucene Integration
+===========================================================================
+
+  mecab-ko-dic-2.0.3-20170922
+
+  https://bitbucket.org/eunjeon/mecab-ko-dic/downloads/mecab-ko-dic-2.0.3-20170922.tar.gz
+
 Apache Commons CLI
 Copyright 2001-2009 The Apache Software Foundation
 
@@ -343,58 +523,6 @@ Copyright 1999-2012 Apache Software Foundation
 ResolverUtil.java
 Copyright 2005-2006 Tim Fennell
 
-Apache Avro Mapred API
-Copyright 2009-2013 The Apache Software Foundation
-
-Apache Avro IPC
-Copyright 2009-2013 The Apache Software Foundation
-
-Apache Velocity
-
-Copyright (C) 2000-2007 The Apache Software Foundation
-
-ORC Core
-Copyright 2013-2018 The Apache Software Foundation
-
-ORC Shims
-Copyright 2013-2018 The Apache Software Foundation
-
-Hive Storage API
-Copyright 2018 The Apache Software Foundation
-
-ORC MapReduce
-Copyright 2013-2018 The Apache Software Foundation
-
-Objenesis
-Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita
-
-Google Guice - Extensions - Servlet
-Copyright 2006-2011 Google, Inc.
-
-Google Guice - Core Library
-Copyright 2006-2011 Google, Inc.
-
-Apache Parquet Avro
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Column
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Common
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Encodings
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Hadoop
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Jackson
-Copyright 2015 The Apache Software Foundation
-
-Apache Parquet Format (Incubating)
-Copyright 2015 The Apache Software Foundation
-
 Apache Commons Math
 Copyright 2001-2013 The Apache Software Foundation
 
@@ -456,37 +584,7 @@ terms of the Apache 2 licence. Original source copyright:
 Copyright 2010 CS Systèmes d'Information
 ===============================================================================
 
-The HermiteInterpolator class and its corresponding test have been imported from
-the orekit library distributed under the terms of the Apache 2 licence. Original
-source copyright:
-Copyright 2010-2012 CS Systèmes d'Information
-===============================================================================
-
 The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired
 by an original code donated by Sébastien Brisard.
 ===============================================================================
 
-Apache Commons Net
-Copyright 2001-2012 The Apache Software Foundation
-
-ApacheDS Protocol Kerberos Codec
-Copyright 2003-2013 The Apache Software Foundation
-
-ApacheDS I18n
-Copyright 2003-2013 The Apache Software Foundation
-
-Apache Directory API ASN.1 API
-Copyright 2003-2013 The Apache Software Foundation
-
-Apache Directory LDAP API Utilities
-Copyright 2003-2013 The Apache Software Foundation
-
-Curator Framework
-Copyright 2011-2015 The Apache Software Foundation
-
-Curator Client
-Copyright 2011-2015 The Apache Software Foundation
-
-Curator Recipes
-Copyright 2011-2015 The Apache Software Foundation
-
diff --git a/licenses-binary/LICENSE-fusesource.txt b/licenses-binary/LICENSE-fusesource.txt
new file mode 100644
index 0000000..8edd375
--- /dev/null
+++ b/licenses-binary/LICENSE-fusesource.txt
@@ -0,0 +1,27 @@
+Copyright (c) 2011 FuseSource Corp. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of FuseSource Corp. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses-binary/LICENSE-py4j.txt b/licenses-binary/LICENSE-py4j.txt
new file mode 100644
index 0000000..0f45e3e
--- /dev/null
+++ b/licenses-binary/LICENSE-py4j.txt
@@ -0,0 +1,26 @@
+Copyright (c) 2009-2018, Barthelemy Dagenais and individual contributors. All
+rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+- Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+- Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+- The name of the author may not be used to endorse or promote products
+  derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses-binary/LICENSE-threeten.txt b/licenses-binary/LICENSE-threeten.txt
new file mode 100644
index 0000000..bbed356
--- /dev/null
+++ b/licenses-binary/LICENSE-threeten.txt
@@ -0,0 +1,29 @@
+Copyright (c) 2007-present, Stephen Colebourne & Michael Nascimento Santos.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice,
+  this list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of JSR-310 nor the names of its contributors
+  may be used to endorse or promote products derived from this software
+  without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 02/02: update pinot assembly scripts

Posted by xi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch release-0.3.0
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit c0b0e6cee2ad01dcf52d18a99dcb239301e4e5a9
Author: Xiang Fu <fx...@gmail.com>
AuthorDate: Mon Mar 2 16:53:01 2020 -0800

    update pinot assembly scripts
---
 pinot-distribution/pinot-assembly.xml | 88 +++++++++++++++++++++++++----------
 pinot-plugins/pom.xml                 |  1 -
 2 files changed, 64 insertions(+), 25 deletions(-)

diff --git a/pinot-distribution/pinot-assembly.xml b/pinot-distribution/pinot-assembly.xml
index 1bc4051..c78889b 100644
--- a/pinot-distribution/pinot-assembly.xml
+++ b/pinot-distribution/pinot-assembly.xml
@@ -43,10 +43,74 @@
     <file>
       <source>${pinot.root}/DISCLAIMER</source>
     </file>
+    <!-- Include Pinot All-in-one jar -->
     <file>
       <source>${pinot.root}/pinot-distribution/target/pinot-distribution-${project.version}-shaded.jar</source>
       <destName>lib/pinot-all-${project.version}-jar-with-dependencies.jar</destName>
     </file>
+    <!-- Start Include Pinot Plugins-->
+    <!-- Start Include Pinot Stream Ingestion Plugins-->
+    <!-- Only Include Specified Kafka Version Plugin-->
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-stream-ingestion/pinot-kafka-${kafka.version}/target/pinot-kafka-${kafka.version}-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-stream-ingestion/pinot-kafka-${kafka.version}/pinot-kafka-${kafka.version}-${project.version}-shaded.jar</destName>
+    </file>
+    <!-- End Include Pinot Stream Ingestion Plugins-->
+    <!-- Start Include Pinot Batch Ingestion Plugins-->
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/target/pinot-batch-ingestion-standalone-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/pinot-batch-ingestion-standalone-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/target/pinot-batch-ingestion-hadoop-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/pinot-batch-ingestion-hadoop-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/target/pinot-batch-ingestion-spark-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/pinot-batch-ingestion-spark-${project.version}-shaded.jar</destName>
+    </file>
+    <!-- End Include Pinot Batch Ingestion Plugins-->
+    <!-- Start Include Pinot File System Plugins-->
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-file-system/pinot-adls/target/pinot-adls-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-file-system/pinot-adls/pinot-adls-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-file-system/pinot-gcs/target/pinot-gcs-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-file-system/pinot-gcs/pinot-gcs-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-file-system/pinot-hdfs/target/pinot-hdfs-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-file-system/pinot-hdfs/pinot-hdfs-${project.version}-shaded.jar</destName>
+    </file>
+    <!-- End Include Pinot File System Plugins-->
+    <!-- Start Include Pinot Input Format Plugins-->
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-avro/target/pinot-avro-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-avro/pinot-avro-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-csv/target/pinot-csv-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-csv/pinot-csv-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-json/target/pinot-json-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-json/pinot-json-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-orc/target/pinot-orc-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-orc/pinot-orc-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-parquet/target/pinot-parquet-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-parquet/pinot-parquet-${project.version}-shaded.jar</destName>
+    </file>
+    <file>
+      <source>${pinot.root}/pinot-plugins/pinot-input-format/pinot-thrift/target/pinot-thrift-${project.version}-shaded.jar</source>
+      <destName>plugins/pinot-input-format/pinot-thrift/pinot-thrift-${project.version}-shaded.jar</destName>
+    </file>
+    <!-- End Include Pinot Input Format Plugins-->
+    <!-- End Include Pinot Plugins-->
   </files>
   <fileSets>
     <!-- Rename licenses-binary directory to licenses and include it to a distribution tarbell -->
@@ -86,29 +150,5 @@
       <outputDirectory>bin</outputDirectory>
       <fileMode>0755</fileMode>
     </fileSet>
-    <fileSet>
-      <useDefaultExcludes>false</useDefaultExcludes>
-      <directory>${pinot.root}/pinot-plugins/target/plugins/</directory>
-      <outputDirectory>plugins</outputDirectory>
-      <excludes>
-        <exclude>**/*.pom</exclude>
-        <exclude>**/pinot-plugins/**</exclude>
-        <exclude>**/pinot-file-system/pinot-file-system/**</exclude>
-        <exclude>**/pinot-input-format/pinot-input-format/**</exclude>
-        <exclude>**/pinot-stream-ingestion/pinot-stream-ingestion/**</exclude>
-        <exclude>**/pinot-stream-ingestion/pinot-kafka-*/**</exclude>
-        <exclude>**/pinot-batch-ingestion/pinot-batch-ingestion/**</exclude>
-        <exclude>**/pinot-batch-ingestion/pinot-batch-ingestion-common/**</exclude>
-        <exclude>**/pinot-batch-ingestion/pinot-ingestion-common/**</exclude>
-        <exclude>**/pinot-batch-ingestion/pinot-hadoop/**</exclude>
-        <exclude>**/pinot-batch-ingestion/pinot-spark/**</exclude>
-        <exclude>**/pinot-batch-ingestion/v0_deprecated/**</exclude>
-      </excludes>
-    </fileSet>
-    <fileSet>
-      <useDefaultExcludes>false</useDefaultExcludes>
-      <directory>${pinot.root}/pinot-plugins/target/plugins/pinot-stream-ingestion/pinot-kafka-${kafka.version}</directory>
-      <outputDirectory>plugins/pinot-stream-ingestion/pinot-kafka-${kafka.version}</outputDirectory>
-    </fileSet>
   </fileSets>
 </assembly>
diff --git a/pinot-plugins/pom.xml b/pinot-plugins/pom.xml
index c92ac1e..24f20c3 100644
--- a/pinot-plugins/pom.xml
+++ b/pinot-plugins/pom.xml
@@ -102,7 +102,6 @@
                       <shadedPattern>shaded.org.apache.http</shadedPattern>
                     </relocation>
                   </relocations>
-                  <outputDirectory>${pinot.root}/pinot-plugins/target/plugins/${plugin.type}/${project.artifactId}</outputDirectory>
                 </configuration>
               </execution>
             </executions>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org