You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2020/03/30 00:04:06 UTC
[incubator-pinot] branch master updated: Move table config into
pinot-spi (#5194)
This is an automated email from the ASF dual-hosted git repository.
jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git
The following commit(s) were added to refs/heads/master by this push:
new 00fcb1d Move table config into pinot-spi (#5194)
00fcb1d is described below
commit 00fcb1d27f8daec48c1d87d46abfc8b60cfb32cf
Author: Xiaotian (Jackie) Jiang <17...@users.noreply.github.com>
AuthorDate: Sun Mar 29 17:03:55 2020 -0700
Move table config into pinot-spi (#5194)
Motivation:
- Table config should be moved to pinot-spi so that user interface can access it (e.g. segment generation spec)
Changes:
- Make all configs POJO like, and Json serializable for backward-compatibility
- De-couple the Helix properties and utils from the configs
- Add the TableConfigSerDeTest to check all the serialization/de-serialization
Side changes:
- Refactor DataSizeUtils (from DataSize), integrate StorageQuotaChecker and HelixExternalViewBasedQueryQuotaManager with the POJO like QuotaConfig
- TextIndexConfigValidator is integrated into `TableConfigUtils.validate(TableConfig tableConfig)`
BACKWARD-INCOMPATIBILITY:
- TableConfig no longer support de-serialization from json string of nested json string (i.e. no `\"` inside the json)
---
.../broker/api/resources/PinotBrokerDebug.java | 4 +-
...okerResourceOnlineOfflineStateModelFactory.java | 2 +-
.../broker/broker/helix/HelixBrokerStarter.java | 2 +-
.../HelixExternalViewBasedQueryQuotaManager.java | 54 +-
.../requesthandler/BaseBrokerRequestHandler.java | 9 +-
.../SingleConnectionBrokerRequestHandler.java | 2 +-
.../pinot/broker/routing/RoutingManager.java | 6 +-
.../instanceselector/InstanceSelectorFactory.java | 6 +-
.../segmentpruner/SegmentPrunerFactory.java | 10 +-
.../segmentselector/SegmentSelectorFactory.java | 5 +-
.../routing/timeboundary/TimeBoundaryManager.java | 4 +-
.../broker/broker/HelixBrokerStarterTest.java | 15 +-
...elixExternalViewBasedQueryQuotaManagerTest.java | 107 ++-
.../instanceselector/InstanceSelectorTest.java | 10 +-
.../routing/segmentpruner/SegmentPrunerTest.java | 12 +-
.../segmentselector/SegmentSelectorTest.java | 4 +-
.../timeboundary/TimeBoundaryManagerTest.java | 9 +-
.../InstanceAssignmentConfigUtils.java | 17 +-
.../common/assignment/InstancePartitionsUtils.java | 9 +-
.../apache/pinot/common/config/TableConfig.java | 733 ---------------------
.../common/config/TextIndexConfigValidator.java | 46 --
.../pinot/common/metadata/ZKMetadataProvider.java | 19 +-
.../metadata/instance/InstanceZKMetadata.java | 2 +-
.../apache/pinot/common/utils/CommonConstants.java | 15 -
.../pinot/common/utils/config/InstanceUtils.java | 80 +++
.../common/utils/config/TableConfigUtils.java | 192 ++++++
.../common/{ => utils}/config/TagNameUtils.java | 12 +-
.../pinot/common/utils/helix/HelixHelper.java | 2 +-
.../pinot/common/utils/helix/TableCache.java | 9 +-
.../pinot/common/config/QuotaConfigTest.java | 120 ----
.../pinot/common/config/TableConfigTest.java | 409 ------------
.../apache/pinot/common/utils/DataSizeTest.java | 45 --
.../pinot/common/utils/DataSizeUtilsTest.java | 92 +++
.../common/utils/config/TableConfigSerDeTest.java | 370 +++++++++++
.../common/utils/config/TableConfigUtilsTest.java | 66 ++
.../config/TagNameUtilsTest.java} | 6 +-
.../pinot/controller/LeadControllerManager.java | 2 +-
.../api/events/DefaultMetadataEventNotifier.java | 3 +-
.../api/events/MetadataEventNotifier.java | 3 +-
.../pinot/controller/api/resources/Constants.java | 6 +-
.../PinotInstanceAssignmentRestletResource.java | 12 +-
.../resources/PinotInstanceRestletResource.java | 9 +-
.../PinotLeadControllerRestletResource.java | 2 +-
.../api/resources/PinotQueryResource.java | 2 +-
.../api/resources/PinotSchemaRestletResource.java | 2 +-
.../api/resources/PinotSegmentRestletResource.java | 4 +-
.../PinotSegmentUploadDownloadRestletResource.java | 2 +-
.../api/resources/PinotTableIndexingConfigs.java | 18 +-
.../api/resources/PinotTableInstances.java | 4 +-
.../api/resources/PinotTableMetadataConfigs.java | 14 +-
.../api/resources/PinotTableRestletResource.java | 50 +-
.../api/resources/PinotTableSegmentConfigs.java | 22 +-
.../api/resources/PinotTaskRestletResource.java | 2 +-
.../api/resources/PinotTenantRestletResource.java | 6 +-
.../pinot/controller/api/resources/TableViews.java | 49 +-
.../controller/api/upload/SegmentValidator.java | 5 +-
.../pinot/controller/api/upload/ZKOperator.java | 2 +-
.../helix/ControllerRequestURLBuilder.java | 2 +-
.../controller/helix/SegmentStatusChecker.java | 4 +-
.../helix/core/PinotHelixResourceManager.java | 42 +-
.../helix/core/PinotTableIdealStateBuilder.java | 6 +-
.../helix/core/SegmentDeletionManager.java | 2 +-
.../instance/InstanceAssignmentDriver.java | 12 +-
.../InstanceReplicaGroupPartitionSelector.java | 2 +-
.../instance/InstanceTagPoolSelector.java | 6 +-
.../segment/OfflineSegmentAssignment.java | 6 +-
.../segment/RealtimeSegmentAssignment.java | 4 +-
.../core/assignment/segment/SegmentAssignment.java | 4 +-
.../segment/SegmentAssignmentFactory.java | 4 +-
.../helix/core/minion/ClusterInfoProvider.java | 6 +-
.../core/minion/PinotHelixTaskResourceManager.java | 2 +-
.../helix/core/minion/PinotTaskManager.java | 6 +-
.../generator/ConvertToRawIndexTaskGenerator.java | 10 +-
.../core/minion/generator/PinotTaskGenerator.java | 4 +-
.../core/minion/generator/TaskGeneratorUtils.java | 2 +-
.../realtime/PinotLLCRealtimeSegmentManager.java | 10 +-
.../core/realtime/PinotRealtimeSegmentManager.java | 11 +-
.../core/realtime/SegmentCompletionManager.java | 2 +-
.../helix/core/realtime/TableConfigCache.java | 2 +-
.../helix/core/rebalance/RebalanceResult.java | 2 +-
.../helix/core/rebalance/TableRebalancer.java | 8 +-
.../core/relocation/RealtimeSegmentRelocator.java | 6 +-
.../helix/core/retention/RetentionManager.java | 6 +-
.../controller/util/AutoAddInvertedIndex.java | 12 +-
.../controller/util/SegmentIntervalUtils.java | 2 +-
.../controller/util/TableRetentionValidator.java | 9 +-
.../pinot/controller/util/TableSizeReader.java | 10 +-
.../BrokerResourceValidationManager.java | 2 +-
.../validation/OfflineSegmentIntervalChecker.java | 14 +-
.../RealtimeSegmentValidationManager.java | 10 +-
.../controller/validation/StorageQuotaChecker.java | 70 +-
.../pinot/controller/api/PinotFileUploadTest.java | 7 +-
...PinotInstanceAssignmentRestletResourceTest.java | 25 +-
.../api/PinotInstanceRestletResourceTest.java | 11 +-
.../api/PinotSegmentRestletResourceTest.java | 10 +-
.../api/PinotTableRestletResourceTest.java | 110 ++--
.../api/PinotTenantRestletResourceTest.java | 10 +-
.../pinot/controller/api/TableSizeReaderTest.java | 4 +-
.../pinot/controller/api/TableViewsTest.java | 18 +-
.../controller/api/upload/ZKOperatorTest.java | 8 +-
.../helix/ControllerInstanceToggleTest.java | 15 +-
.../controller/helix/ControllerSentinelTestV2.java | 13 +-
.../controller/helix/ControllerTenantTest.java | 2 +-
.../pinot/controller/helix/ControllerTest.java | 14 +-
.../controller/helix/PinotResourceManagerTest.java | 10 +-
.../controller/helix/SegmentStatusCheckerTest.java | 2 +-
.../helix/core/PinotHelixResourceManagerTest.java | 28 +-
.../instance/InstanceAssignmentTest.java | 39 +-
...fflineNonReplicaGroupSegmentAssignmentTest.java | 9 +-
.../OfflineReplicaGroupSegmentAssignmentTest.java | 15 +-
...altimeNonReplicaGroupSegmentAssignmentTest.java | 11 +-
.../RealtimeReplicaGroupSegmentAssignmentTest.java | 9 +-
.../PinotLLCRealtimeSegmentManagerTest.java | 11 +-
.../segment/FlushThresholdUpdaterTest.java | 2 +-
.../core/rebalance/TableRebalancerClusterTest.java | 19 +-
.../helix/core/retention/RetentionManagerTest.java | 17 +-
.../validation/StorageQuotaCheckerTest.java | 183 ++---
.../validation/ValidationManagerTest.java | 15 +-
.../core/data/manager/BaseTableDataManager.java | 2 +-
.../pinot/core/data/manager/TableDataManager.java | 2 +-
.../manager/config/TableDataManagerConfig.java | 6 +-
.../manager/offline/TableDataManagerProvider.java | 4 +-
.../realtime/HLRealtimeSegmentDataManager.java | 10 +-
.../realtime/LLRealtimeSegmentDataManager.java | 10 +-
.../manager/realtime/RealtimeTableDataManager.java | 7 +-
.../generator/SegmentGeneratorConfig.java | 11 +-
.../indexsegment/mutable/MutableSegmentImpl.java | 4 +-
.../apache/pinot/core/minion}/PinotTaskConfig.java | 4 +-
.../apache/pinot/core/minion/SegmentConverter.java | 5 +-
.../minion/rollup/MergeRollupSegmentConverter.java | 2 +-
.../core/query/reduce/BrokerReduceService.java | 2 +-
.../query/reduce/SelectionDataTableReducer.java | 2 +-
.../converter/RealtimeSegmentConverter.java | 4 +-
.../core/segment/creator/StatsCollectorConfig.java | 6 +-
.../segment/index/loader/IndexLoadingConfig.java | 6 +-
.../v2/builder/StarTreeV2BuilderConfig.java | 2 +-
.../apache/pinot/core/transport/QueryRouter.java | 2 +-
.../pinot/core/transport/ServerInstance.java | 3 +-
.../core/transport/ServerRoutingInstance.java | 2 +-
.../apache/pinot/core/util/ReplicationUtils.java | 10 +-
.../ServerSegmentCompletionProtocolHandler.java | 2 +-
.../realtime/LLRealtimeSegmentDataManagerTest.java | 16 +-
.../core/realtime/stream/StreamConfigTest.java | 8 +-
.../index/creator/SegmentPartitionTest.java | 10 +-
.../pinot/core/transport/QueryRoutingTest.java | 2 +-
.../org/apache/pinot/queries/BaseQueriesTest.java | 2 +-
.../tests/BaseClusterIntegrationTest.java | 10 +-
.../pinot/integration/tests/ClusterTest.java | 39 +-
.../ControllerPeriodicTasksIntegrationTests.java | 20 +-
...vertToRawIndexMinionClusterIntegrationTest.java | 4 +-
.../DefaultCommitterRealtimeIntegrationTest.java | 41 +-
.../tests/HybridClusterIntegrationTest.java | 7 +-
.../tests/LLCRealtimeClusterIntegrationTest.java | 11 +-
.../LuceneRealtimeClusterIntegrationTest.java | 5 +-
.../tests/OfflineClusterIntegrationTest.java | 17 +-
.../tests/SegmentCompletionIntegrationTest.java | 4 +-
.../tests/SimpleMinionClusterIntegrationTest.java | 10 +-
.../minion/events/DefaultMinionEventObserver.java | 2 +-
.../pinot/minion/events/MinionEventObserver.java | 2 +-
.../BaseMultipleSegmentsConversionExecutor.java | 4 +-
.../BaseSingleSegmentConversionExecutor.java | 4 +-
.../executor/ConvertToRawIndexTaskExecutor.java | 4 +-
.../pinot/minion/executor/PinotTaskExecutor.java | 2 +-
.../pinot/minion/executor/PurgeTaskExecutor.java | 4 +-
.../minion/executor/SegmentConversionResult.java | 2 +-
.../minion/taskfactory/TaskFactoryRegistry.java | 2 +-
.../minion/executor/PurgeTaskExecutorTest.java | 12 +-
.../batch/common/SegmentGenerationTaskRunner.java | 8 +-
.../batch/common/SegmentGenerationUtils.java | 6 +-
.../batch/hadoop/HadoopSegmentCreationMapper.java | 6 +-
.../spark/SparkSegmentGenerationJobRunner.java | 4 +-
.../standalone/SegmentGenerationJobRunner.java | 6 +-
.../pinot/hadoop/job/HadoopSegmentCreationJob.java | 9 +-
.../hadoop/job/HadoopSegmentPreprocessingJob.java | 20 +-
.../hadoop/job/mappers/SegmentCreationMapper.java | 24 +-
...mentBuildPushOfflineClusterIntegrationTest.java | 6 +-
.../pinot/ingestion/common/ControllerRestApi.java | 2 +-
.../ingestion/common/DefaultControllerRestApi.java | 10 +-
.../pinot/ingestion/jobs/BaseSegmentJob.java | 6 +-
.../spark/jobs/SparkSegmentCreationFunction.java | 17 +-
.../pinot/spark/jobs/SparkSegmentCreationJob.java | 6 +-
.../starter/helix/HelixInstanceDataManager.java | 2 +-
.../server/starter/helix/HelixServerStarter.java | 4 +-
.../SegmentOnlineOfflineStateModelFactory.java | 4 +-
.../apache/pinot/spi}/config/BaseJsonConfig.java | 11 +-
.../pinot/spi}/config/ColumnPartitionConfig.java | 8 +-
.../apache/pinot/spi}/config/CompletionConfig.java | 3 +-
.../org/apache/pinot/spi}/config/FieldConfig.java | 15 +-
.../apache/pinot/spi}/config/IndexingConfig.java | 64 +-
.../org/apache/pinot/spi}/config/QueryConfig.java | 6 +-
.../org/apache/pinot/spi}/config/QuotaConfig.java | 82 ++-
.../spi}/config/ReplicaGroupStrategyConfig.java | 2 +-
.../apache/pinot/spi}/config/RoutingConfig.java | 2 +-
.../pinot/spi}/config/SegmentPartitionConfig.java | 9 +-
.../SegmentsValidationAndRetentionConfig.java | 3 +-
.../pinot/spi}/config/StarTreeIndexConfig.java | 2 +-
.../org/apache/pinot/spi/config/TableConfig.java | 220 +++++++
.../pinot/spi}/config/TableCustomConfig.java | 2 +-
.../apache/pinot/spi}/config/TableTaskConfig.java | 2 +-
.../org/apache/pinot/spi/config/TableType.java | 4 +-
.../pinot/spi}/config/TagOverrideConfig.java | 2 +-
.../org/apache/pinot/spi}/config/TenantConfig.java | 2 +-
.../org/apache/pinot/spi/config/api}/Instance.java | 54 +-
.../apache/pinot/spi/config/api/InstanceType.java | 6 +-
.../org/apache/pinot/spi/config/api}/Tenant.java | 4 +-
.../apache/pinot/spi/config/api}/TenantRole.java | 2 +-
.../assignment}/InstanceAssignmentConfig.java | 4 +-
.../assignment}/InstanceConstraintConfig.java | 4 +-
.../config}/assignment/InstancePartitionsType.java | 2 +-
.../InstanceReplicaGroupPartitionConfig.java | 4 +-
.../config/assignment}/InstanceTagPoolConfig.java | 4 +-
.../org/apache/pinot/spi/stream/StreamConfig.java | 21 +-
.../java/org/apache/pinot/spi/utils/DataSize.java | 91 ---
.../org/apache/pinot/spi/utils/DataSizeUtils.java | 94 +++
.../spi/utils/builder/TableConfigBuilder.java | 312 +++++++++
.../pinot/spi/utils/builder}/TableNameBuilder.java | 9 +-
.../pinot/spi}/config/IndexingConfigTest.java | 4 +-
.../apache/pinot/spi/config/QuotaConfigTest.java | 129 ++++
.../apache/pinot/spi/config/api}/TenantTest.java | 3 +-
.../apache/pinot/tools/ClusterStateVerifier.java | 2 +-
.../org/apache/pinot/tools/HybridQuickstart.java | 3 +-
.../apache/pinot/tools/PinotNumReplicaChanger.java | 4 +-
.../apache/pinot/tools/PinotTableRebalancer.java | 2 +-
.../apache/pinot/tools/QuickstartTableRequest.java | 2 +-
.../org/apache/pinot/tools/UpdateSegmentState.java | 5 +-
.../tools/admin/command/AddTenantCommand.java | 4 +-
.../command/BackfillDateTimeColumnCommand.java | 6 +-
.../tools/admin/command/MoveReplicaGroup.java | 14 +-
.../OfflineSegmentIntervalCheckerCommand.java | 6 +-
.../tools/admin/command/QuickstartRunner.java | 4 +-
.../command/RealtimeProvisioningHelperCommand.java | 12 +-
.../admin/command/ShowClusterInfoCommand.java | 2 +-
.../tools/admin/command/ValidateConfigCommand.java | 7 +-
.../tools/admin/command/VerifySegmentState.java | 2 +-
.../config/validator/TableConfigValidator.java | 3 +-
.../pinot/tools/perf/PerfBenchmarkDriver.java | 12 +-
.../tools/query/comparison/ClusterStarter.java | 15 +-
.../realtime/provisioning/MemoryEstimator.java | 40 +-
.../segment/converter/SegmentMergeCommand.java | 9 +-
239 files changed, 2727 insertions(+), 2746 deletions(-)
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java b/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java
index b48d0a4..b5d127a 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java
@@ -37,10 +37,10 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.pinot.broker.routing.RoutingManager;
import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryInfo;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.core.transport.ServerInstance;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
@Api(tags = "Debug")
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerResourceOnlineOfflineStateModelFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerResourceOnlineOfflineStateModelFactory.java
index 2990e2f..a619c13 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerResourceOnlineOfflineStateModelFactory.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BrokerResourceOnlineOfflineStateModelFactory.java
@@ -29,8 +29,8 @@ import org.apache.helix.participant.statemachine.Transition;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
import org.apache.pinot.broker.queryquota.HelixExternalViewBasedQueryQuotaManager;
import org.apache.pinot.broker.routing.RoutingManager;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.spi.config.TableConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
index 3f8f3b1..0fc14fb 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
@@ -49,7 +49,6 @@ import org.apache.pinot.broker.requesthandler.BrokerRequestHandler;
import org.apache.pinot.broker.requesthandler.SingleConnectionBrokerRequestHandler;
import org.apache.pinot.broker.routing.RoutingManager;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
@@ -59,6 +58,7 @@ import org.apache.pinot.common.utils.CommonConstants.Broker;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.NetUtil;
import org.apache.pinot.common.utils.ServiceStatus;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManager.java
index eba8833..d40eb0b 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManager.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManager.java
@@ -20,7 +20,6 @@ package org.apache.pinot.broker.queryquota;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
import com.google.common.util.concurrent.RateLimiter;
import java.util.Iterator;
import java.util.Map;
@@ -33,14 +32,15 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.model.ExternalView;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
import org.apache.pinot.broker.broker.helix.ClusterChangeHandler;
-import org.apache.pinot.common.config.QuotaConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.BrokerGauge;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -99,33 +99,11 @@ public class HelixExternalViewBasedQueryQuotaManager implements ClusterChangeHan
*/
public void initTableQueryQuota(TableConfig tableConfig, ExternalView brokerResource) {
String tableNameWithType = tableConfig.getTableName();
- String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
LOGGER.info("Initializing rate limiter for table {}", tableNameWithType);
- // Check whether qps quotas from both tables are the same.
- QuotaConfig offlineQuotaConfig;
- QuotaConfig realtimeQuotaConfig;
- CommonConstants.Helix.TableType tableType = tableConfig.getTableType();
- if (tableType == CommonConstants.Helix.TableType.OFFLINE) {
- offlineQuotaConfig = tableConfig.getQuotaConfig();
- realtimeQuotaConfig = getQuotaConfigFromPropertyStore(TableNameBuilder.REALTIME.tableNameWithType(rawTableName));
- } else {
- realtimeQuotaConfig = tableConfig.getQuotaConfig();
- offlineQuotaConfig = getQuotaConfigFromPropertyStore(TableNameBuilder.OFFLINE.tableNameWithType(rawTableName));
- }
- // Log a warning if MaxQueriesPerSecond are set different.
- if ((offlineQuotaConfig != null && !Strings.isNullOrEmpty(offlineQuotaConfig.getMaxQueriesPerSecond())) && (
- realtimeQuotaConfig != null && !Strings.isNullOrEmpty(realtimeQuotaConfig.getMaxQueriesPerSecond()))) {
- if (!offlineQuotaConfig.getMaxQueriesPerSecond().equals(realtimeQuotaConfig.getMaxQueriesPerSecond())) {
- LOGGER.warn(
- "Attention! The values of MaxQueriesPerSecond for table {} are set different! Offline table qps quota: {}, Real-time table qps quota: {}",
- rawTableName, offlineQuotaConfig.getMaxQueriesPerSecond(), realtimeQuotaConfig.getMaxQueriesPerSecond());
- }
- }
-
// Create rate limiter if query quota config is specified.
QuotaConfig quotaConfig = tableConfig.getQuotaConfig();
- if (quotaConfig == null || Strings.isNullOrEmpty(quotaConfig.getMaxQueriesPerSecond())) {
+ if (quotaConfig == null || quotaConfig.getMaxQueriesPerSecond() == null) {
LOGGER.info("No qps config specified for table: {}", tableNameWithType);
removeRateLimiter(tableNameWithType);
} else {
@@ -173,7 +151,7 @@ public class HelixExternalViewBasedQueryQuotaManager implements ClusterChangeHan
* @param quotaConfig quota config of the table.
*/
private void createRateLimiter(String tableNameWithType, ExternalView brokerResource, QuotaConfig quotaConfig) {
- if (quotaConfig == null || Strings.isNullOrEmpty(quotaConfig.getMaxQueriesPerSecond())) {
+ if (quotaConfig == null || quotaConfig.getMaxQueriesPerSecond() == null) {
LOGGER.info("No qps config specified for table: {}", tableNameWithType);
return;
}
@@ -200,14 +178,7 @@ public class HelixExternalViewBasedQueryQuotaManager implements ClusterChangeHan
LOGGER.info("The number of online brokers for table {} is {}", tableNameWithType, onlineCount);
// Get the dynamic rate
- double overallRate;
- if (quotaConfig.isMaxQueriesPerSecondValid()) {
- overallRate = Double.parseDouble(quotaConfig.getMaxQueriesPerSecond());
- } else {
- LOGGER.error("Failed to init qps quota: error when parsing qps quota: {} for table: {}",
- quotaConfig.getMaxQueriesPerSecond(), tableNameWithType);
- return;
- }
+ double overallRate = quotaConfig.getMaxQPS();
// Get stat from property store
String tableConfigPath = constructTableConfigPath(tableNameWithType);
@@ -237,11 +208,11 @@ public class HelixExternalViewBasedQueryQuotaManager implements ClusterChangeHan
QueryQuotaEntity offlineTableQueryQuotaEntity = null;
QueryQuotaEntity realtimeTableQueryQuotaEntity = null;
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
- if (tableType == CommonConstants.Helix.TableType.OFFLINE) {
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
+ if (tableType == TableType.OFFLINE) {
offlineTableName = tableName;
offlineTableQueryQuotaEntity = _rateLimiterMap.get(tableName);
- } else if (tableType == CommonConstants.Helix.TableType.REALTIME) {
+ } else if (tableType == TableType.REALTIME) {
realtimeTableName = tableName;
realtimeTableQueryQuotaEntity = _rateLimiterMap.get(tableName);
} else {
@@ -358,14 +329,13 @@ public class HelixExternalViewBasedQueryQuotaManager implements ClusterChangeHan
// Get latest quota config only if stat don't match.
if (stat.getVersion() != queryQuotaEntity.getTableConfigStatVersion()) {
QuotaConfig quotaConfig = getQuotaConfigFromPropertyStore(tableNameWithType);
- if (quotaConfig == null || quotaConfig.getMaxQueriesPerSecond() == null || !quotaConfig
- .isMaxQueriesPerSecondValid()) {
+ if (quotaConfig == null || quotaConfig.getMaxQueriesPerSecond() == null) {
LOGGER.info("No query quota config or the config is invalid for Table {}. Removing its rate limit.",
tableNameWithType);
it.remove();
continue;
}
- overallRate = Double.parseDouble(quotaConfig.getMaxQueriesPerSecond());
+ overallRate = quotaConfig.getMaxQPS();
} else {
overallRate = queryQuotaEntity.getOverallRate();
}
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index 16718eb..b1851f9 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -49,7 +49,6 @@ import org.apache.pinot.broker.broker.AccessControlFactory;
import org.apache.pinot.broker.queryquota.QueryQuotaManager;
import org.apache.pinot.broker.routing.RoutingManager;
import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryInfo;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.QueryException;
import org.apache.pinot.common.function.AggregationFunctionType;
import org.apache.pinot.common.metrics.BrokerMeter;
@@ -73,6 +72,8 @@ import org.apache.pinot.core.query.reduce.BrokerReduceService;
import org.apache.pinot.core.transport.ServerInstance;
import org.apache.pinot.core.util.QueryOptions;
import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -207,13 +208,13 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
// Get the tables hit by the request
String offlineTableName = null;
String realtimeTableName = null;
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
- if (tableType == CommonConstants.Helix.TableType.OFFLINE) {
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
+ if (tableType == TableType.OFFLINE) {
// Offline table
if (_routingManager.routingExists(tableName)) {
offlineTableName = tableName;
}
- } else if (tableType == CommonConstants.Helix.TableType.REALTIME) {
+ } else if (tableType == TableType.REALTIME) {
// Realtime table
if (_routingManager.routingExists(tableName)) {
realtimeTableName = tableName;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
index 8f0842c..5e1dd18 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
@@ -30,7 +30,6 @@ import org.apache.pinot.broker.api.RequestStatistics;
import org.apache.pinot.broker.broker.AccessControlFactory;
import org.apache.pinot.broker.queryquota.QueryQuotaManager;
import org.apache.pinot.broker.routing.RoutingManager;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.metrics.BrokerQueryPhase;
@@ -44,6 +43,7 @@ import org.apache.pinot.core.transport.QueryRouter;
import org.apache.pinot.core.transport.ServerInstance;
import org.apache.pinot.core.transport.ServerResponse;
import org.apache.pinot.core.transport.ServerRoutingInstance;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
/**
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingManager.java
index 8c267c6..75fa46d 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingManager.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/RoutingManager.java
@@ -46,9 +46,6 @@ import org.apache.pinot.broker.routing.segmentselector.SegmentSelector;
import org.apache.pinot.broker.routing.segmentselector.SegmentSelectorFactory;
import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryInfo;
import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryManager;
-import org.apache.pinot.common.config.QueryConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
@@ -57,6 +54,9 @@ import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
import org.apache.pinot.common.utils.HashUtil;
import org.apache.pinot.core.transport.ServerInstance;
+import org.apache.pinot.spi.config.QueryConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorFactory.java
index c20ff80..fe446f9 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorFactory.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.pinot.broker.routing.instanceselector;
-import org.apache.pinot.common.config.RoutingConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metrics.BrokerMetrics;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java
index 640be0b..c61c08f 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java
@@ -25,11 +25,11 @@ import java.util.Map;
import javax.annotation.Nullable;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.RoutingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorFactory.java
index ea611f6..27cb34f 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorFactory.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorFactory.java
@@ -17,9 +17,8 @@
* under the License.
*/
package org.apache.pinot.broker.routing.segmentselector;
-
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
public class SegmentSelectorFactory {
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManager.java
index 3651f9f..5845b92 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManager.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManager.java
@@ -30,10 +30,10 @@ import org.apache.helix.AccessOption;
import org.apache.helix.ZNRecord;
import org.apache.helix.model.ExternalView;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.data.Schema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
index e9e0ed9..eb88c3b 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
@@ -29,20 +29,21 @@ import org.apache.helix.model.IdealState;
import org.apache.pinot.broker.broker.helix.HelixBrokerStarter;
import org.apache.pinot.broker.routing.RoutingManager;
import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryInfo;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.request.BrokerRequest;
import org.apache.pinot.common.utils.CommonConstants.Helix;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
import org.apache.pinot.core.transport.ServerInstance;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -83,11 +84,11 @@ public class HelixBrokerStarterTest extends ControllerTest {
.addTime(TIME_COLUMN_NAME, TimeUnit.DAYS, FieldSpec.DataType.INT).build();
_helixResourceManager.addSchema(schema, true);
TableConfig offlineTableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN_NAME)
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN_NAME)
.setTimeType(TimeUnit.DAYS.name()).build();
_helixResourceManager.addTable(offlineTableConfig);
TableConfig realtimeTimeConfig =
- new TableConfig.Builder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN_NAME)
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN_NAME)
.setTimeType(TimeUnit.DAYS.name()).
setStreamConfigs(getStreamConfigs()).build();
_helixResourceManager.addTable(realtimeTimeConfig);
@@ -154,7 +155,7 @@ public class HelixBrokerStarterTest extends ControllerTest {
String newRawTableName = "newTable";
String newOfflineTableName = TableNameBuilder.OFFLINE.tableNameWithType(newRawTableName);
TableConfig newTableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(newRawTableName).setBrokerTenant("testBroker").build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(newRawTableName).setBrokerTenant("testBroker").build();
_helixResourceManager.addTable(newTableConfig);
// Broker tenant should be overridden to DefaultTenant
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManagerTest.java
index 53b40ec..d8b0d56 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManagerTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/queryquota/HelixExternalViewBasedQueryQuotaManagerTest.java
@@ -28,14 +28,17 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
import org.apache.helix.manager.zk.ZkClient;
import org.apache.helix.model.ExternalView;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.QuotaConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.StringUtil;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
@@ -117,7 +120,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
ExternalView brokerResource = generateBrokerResource(OFFLINE_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(tableConfig));
setQps(tableConfig);
_queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 1);
@@ -143,11 +147,11 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
QuotaConfig quotaConfig = new QuotaConfig("6G", null);
TableConfig realtimeTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
- ZKMetadataProvider
- .setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, realtimeTableConfig.toZNRecord());
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ ZKMetadataProvider.setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME,
+ TableConfigUtils.toZNRecord(realtimeTableConfig));
ExternalView brokerResource = generateBrokerResource(OFFLINE_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
@@ -164,11 +168,11 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
QuotaConfig quotaConfig = new QuotaConfig("6G", "100.00");
TableConfig realtimeTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
- ZKMetadataProvider
- .setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, realtimeTableConfig.toZNRecord());
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ ZKMetadataProvider.setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME,
+ TableConfigUtils.toZNRecord(realtimeTableConfig));
ExternalView brokerResource = generateBrokerResource(REALTIME_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
@@ -189,17 +193,18 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
QuotaConfig quotaConfig = new QuotaConfig("6G", "100.00");
TableConfig realtimeTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
TableConfig offlineTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ ZKMetadataProvider.setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME,
+ TableConfigUtils.toZNRecord(realtimeTableConfig));
ZKMetadataProvider
- .setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, realtimeTableConfig.toZNRecord());
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, offlineTableConfig.toZNRecord());
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(offlineTableConfig));
// Since each table has 2 online brokers, per broker rate becomes 100.0 / 2 = 50.0
_queryQuotaManager.initTableQueryQuota(offlineTableConfig, brokerResource);
@@ -225,7 +230,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
ExternalView brokerResource = generateBrokerResource(REALTIME_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(REALTIME_TABLE_NAME);
- ZKMetadataProvider.setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, TableConfigUtils.toZNRecord(tableConfig));
setQps(tableConfig);
_queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 1);
@@ -241,7 +247,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
ExternalView brokerResource = generateBrokerResource(REALTIME_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(REALTIME_TABLE_NAME);
- ZKMetadataProvider.setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setRealtimeTableConfig(_testPropertyStore, REALTIME_TABLE_NAME, TableConfigUtils.toZNRecord(tableConfig));
setQps(tableConfig);
_queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 1);
@@ -267,10 +274,11 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
QuotaConfig quotaConfig = new QuotaConfig("6G", null);
TableConfig offlineTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, offlineTableConfig.toZNRecord());
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ ZKMetadataProvider
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(offlineTableConfig));
ExternalView brokerResource = generateBrokerResource(REALTIME_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(REALTIME_TABLE_NAME);
@@ -283,10 +291,11 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
QuotaConfig quotaConfig = new QuotaConfig("6G", "100.00");
TableConfig offlineTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
- .setQuotaConfig(quotaConfig).setRetentionTimeUnit("DAYS").setRetentionTimeValue("1")
- .setSegmentPushType("APPEND").setBrokerTenant("testBroker").setServerTenant("testServer").build();
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, offlineTableConfig.toZNRecord());
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setQuotaConfig(quotaConfig)
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("1").setSegmentPushType("APPEND")
+ .setBrokerTenant("testBroker").setServerTenant("testServer").build();
+ ZKMetadataProvider
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(offlineTableConfig));
ExternalView brokerResource = generateBrokerResource(OFFLINE_TABLE_NAME);
TableConfig tableConfig = generateDefaultTableConfig(REALTIME_TABLE_NAME);
@@ -295,30 +304,6 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
}
@Test
- public void testInvalidQpsQuota()
- throws Exception {
- ExternalView brokerResource = generateBrokerResource(OFFLINE_TABLE_NAME);
- TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
- // Set invalid qps quota
- QuotaConfig quotaConfig = new QuotaConfig(null, "InvalidQpsQuota");
- tableConfig.setQuotaConfig(quotaConfig);
- _queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
- Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 0);
- }
-
- @Test
- public void testInvalidNegativeQpsQuota()
- throws Exception {
- ExternalView brokerResource = generateBrokerResource(OFFLINE_TABLE_NAME);
- TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
- // Set invalid negative qps quota
- QuotaConfig quotaConfig = new QuotaConfig(null, "-1.0");
- tableConfig.setQuotaConfig(quotaConfig);
- _queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
- Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 0);
- }
-
- @Test
public void testNoBrokerResource()
throws Exception {
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
@@ -332,7 +317,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
throws Exception {
ExternalView brokerResource = new ExternalView(CommonConstants.Helix.BROKER_RESOURCE_INSTANCE);
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(tableConfig));
setQps(tableConfig);
_queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
Assert.assertEquals(_queryQuotaManager.getRateLimiterMapSize(), 1);
@@ -344,7 +330,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
ExternalView brokerResource = new ExternalView(CommonConstants.Helix.BROKER_RESOURCE_INSTANCE);
brokerResource.setState(OFFLINE_TABLE_NAME, "broker_instance_2", "OFFLINE");
TableConfig tableConfig = generateDefaultTableConfig(OFFLINE_TABLE_NAME);
- ZKMetadataProvider.setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setOfflineTableConfig(_testPropertyStore, OFFLINE_TABLE_NAME, TableConfigUtils.toZNRecord(tableConfig));
setQps(tableConfig);
_queryQuotaManager.initTableQueryQuota(tableConfig, brokerResource);
@@ -354,8 +341,8 @@ public class HelixExternalViewBasedQueryQuotaManagerTest {
}
private TableConfig generateDefaultTableConfig(String tableName) {
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
- TableConfig.Builder builder = new TableConfig.Builder(tableType);
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
+ TableConfigBuilder builder = new TableConfigBuilder(tableType);
builder.setTableName(tableName);
return builder.build();
}
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorTest.java
index 8ed7259..f159fde 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/instanceselector/InstanceSelectorTest.java
@@ -27,11 +27,11 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.helix.model.ExternalView;
-import org.apache.pinot.common.config.RoutingConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.request.BrokerRequest;
-import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.testng.annotations.Test;
import static org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel.ERROR;
@@ -66,12 +66,12 @@ public class InstanceSelectorTest {
// Should be backward-compatible with legacy config
when(routingConfig.getInstanceSelectorType()).thenReturn(null);
- when(tableConfig.getTableType()).thenReturn(CommonConstants.Helix.TableType.OFFLINE);
+ when(tableConfig.getTableType()).thenReturn(TableType.OFFLINE);
when(routingConfig.getRoutingTableBuilderName())
.thenReturn(InstanceSelectorFactory.LEGACY_REPLICA_GROUP_OFFLINE_ROUTING);
assertTrue(InstanceSelectorFactory
.getInstanceSelector(tableConfig, brokerMetrics) instanceof ReplicaGroupInstanceSelector);
- when(tableConfig.getTableType()).thenReturn(CommonConstants.Helix.TableType.REALTIME);
+ when(tableConfig.getTableType()).thenReturn(TableType.REALTIME);
when(routingConfig.getRoutingTableBuilderName())
.thenReturn(InstanceSelectorFactory.LEGACY_REPLICA_GROUP_REALTIME_ROUTING);
assertTrue(InstanceSelectorFactory
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java
index 380319c..939a0e5 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java
@@ -31,19 +31,19 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
import org.apache.helix.manager.zk.ZkClient;
import org.apache.helix.model.ExternalView;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.RoutingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
import org.apache.pinot.common.request.BrokerRequest;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.ZkStarter;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.mockito.Mockito;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorTest.java
index 5b89b88..26ec30d 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentselector/SegmentSelectorTest.java
@@ -23,11 +23,11 @@ import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.helix.model.ExternalView;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.request.BrokerRequest;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.HLCSegmentName;
import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.testng.annotations.Test;
import static org.apache.pinot.broker.routing.segmentselector.RealtimeSegmentSelector.FORCE_HLC;
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
index afdc4e1..33b63c4 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
@@ -27,14 +27,15 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
import org.apache.helix.manager.zk.ZkClient;
import org.apache.helix.model.ExternalView;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.mockito.Mockito;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -141,7 +142,7 @@ public class TimeBoundaryManagerTest {
}
private TableConfig getTableConfig(String rawTableName, TimeUnit timeUnit, String pushFrequency) {
- return new TableConfig.Builder(TableType.OFFLINE).setTableName(rawTableName).setTimeColumnName(TIME_COLUMN)
+ return new TableConfigBuilder(TableType.OFFLINE).setTableName(rawTableName).setTimeColumnName(TIME_COLUMN)
.setTimeType(timeUnit.name()).setSegmentPushFrequency(pushFrequency).build();
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfigUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstanceAssignmentConfigUtils.java
similarity index 90%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfigUtils.java
rename to pinot-common/src/main/java/org/apache/pinot/common/assignment/InstanceAssignmentConfigUtils.java
index cf3f6fb..7eb6ccf 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfigUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstanceAssignmentConfigUtils.java
@@ -16,17 +16,20 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config.instance;
+package org.apache.pinot.common.assignment;
import com.google.common.base.Preconditions;
import java.util.Map;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.AssignmentStrategy;
+import org.apache.pinot.common.utils.config.TagNameUtils;
+import org.apache.pinot.spi.config.ReplicaGroupStrategyConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
public class InstanceAssignmentConfigUtils {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
index 955c5c7..c652ffe 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
@@ -28,12 +28,13 @@ import org.apache.helix.HelixManager;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.HelixPropertyStore;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.TenantConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
/**
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java b/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java
deleted file mode 100644
index 967e72b..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java
+++ /dev/null
@@ -1,733 +0,0 @@
-/**
- * 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.pinot.common.config;
-
-import com.fasterxml.jackson.annotation.JsonPropertyDescription;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.base.Preconditions;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.helix.ZNRecord;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.spi.utils.JsonUtils;
-
-
-@SuppressWarnings({"Duplicates", "unused"})
-public class TableConfig extends BaseJsonConfig {
- public static final String TABLE_NAME_KEY = "tableName";
- public static final String TABLE_TYPE_KEY = "tableType";
- public static final String VALIDATION_CONFIG_KEY = "segmentsConfig";
- public static final String TENANT_CONFIG_KEY = "tenants";
- public static final String INDEXING_CONFIG_KEY = "tableIndexConfig";
- public static final String CUSTOM_CONFIG_KEY = "metadata";
- public static final String QUOTA_CONFIG_KEY = "quota";
- public static final String TASK_CONFIG_KEY = "task";
- public static final String ROUTING_CONFIG_KEY = "routing";
- public static final String QUERY_CONFIG_KEY = "query";
- public static final String INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY = "instanceAssignmentConfigMap";
- public static final String FIELD_CONFIG_LIST_KEY = "fieldConfigList";
-
- private static final String FIELD_MISSING_MESSAGE_TEMPLATE = "Mandatory field '%s' is missing";
-
- @JsonPropertyDescription("The name for the table (with type suffix), e.g. \"myTable_OFFLINE\" (mandatory)")
- private String _tableName;
-
- @JsonPropertyDescription(value = "The type of the table (OFFLINE|REALTIME) (mandatory)")
- private TableType _tableType;
-
- private SegmentsValidationAndRetentionConfig _validationConfig;
- private TenantConfig _tenantConfig;
- private IndexingConfig _indexingConfig;
- private TableCustomConfig _customConfig;
-
- @JsonPropertyDescription("Resource quota associated with this table")
- private QuotaConfig _quotaConfig;
-
- private TableTaskConfig _taskConfig;
- private RoutingConfig _routingConfig;
- private QueryConfig _queryConfig;
-
- private Map<InstancePartitionsType, InstanceAssignmentConfig> _instanceAssignmentConfigMap;
- private List<FieldConfig> _fieldConfigList;
-
- /**
- * NOTE: DO NOT use this constructor, use builder instead. This constructor is for deserializer only.
- */
- public TableConfig() {
- // TODO: currently these 2 fields are annotated as non-null. Revisit to see whether that's necessary
- _tenantConfig = new TenantConfig(null, null, null);
- _customConfig = new TableCustomConfig(null);
- }
-
- private TableConfig(String tableName, TableType tableType, SegmentsValidationAndRetentionConfig validationConfig,
- TenantConfig tenantConfig, IndexingConfig indexingConfig, TableCustomConfig customConfig,
- @Nullable QuotaConfig quotaConfig, @Nullable TableTaskConfig taskConfig, @Nullable RoutingConfig routingConfig,
- @Nullable QueryConfig queryConfig,
- @Nullable Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap,
- @Nullable List<FieldConfig> fieldConfigList) {
- _tableName = TableNameBuilder.forType(tableType).tableNameWithType(tableName);
- _tableType = tableType;
- _validationConfig = validationConfig;
- _tenantConfig = tenantConfig;
- _indexingConfig = indexingConfig;
- _customConfig = customConfig;
- _quotaConfig = quotaConfig;
- _taskConfig = taskConfig;
- _routingConfig = routingConfig;
- _queryConfig = queryConfig;
- _instanceAssignmentConfigMap = instanceAssignmentConfigMap;
- _fieldConfigList = fieldConfigList;
- }
-
- public static TableConfig fromJsonString(String jsonString)
- throws IOException {
- return fromJsonConfig(JsonUtils.stringToJsonNode(jsonString));
- }
-
- public static TableConfig fromJsonConfig(JsonNode jsonConfig)
- throws IOException {
- // Mandatory fields
- JsonNode jsonTableType = jsonConfig.get(TABLE_TYPE_KEY);
- Preconditions
- .checkState(jsonTableType != null && !jsonTableType.isNull(), FIELD_MISSING_MESSAGE_TEMPLATE, TABLE_TYPE_KEY);
- TableType tableType = TableType.valueOf(jsonTableType.asText().toUpperCase());
-
- JsonNode jsonTableName = jsonConfig.get(TABLE_NAME_KEY);
- Preconditions
- .checkState(jsonTableName != null && !jsonTableName.isNull(), FIELD_MISSING_MESSAGE_TEMPLATE, TABLE_NAME_KEY);
- String tableName = TableNameBuilder.forType(tableType).tableNameWithType(jsonTableName.asText());
-
- SegmentsValidationAndRetentionConfig validationConfig =
- extractChildConfig(jsonConfig, VALIDATION_CONFIG_KEY, SegmentsValidationAndRetentionConfig.class);
- Preconditions.checkState(validationConfig != null, FIELD_MISSING_MESSAGE_TEMPLATE, VALIDATION_CONFIG_KEY);
-
- TenantConfig tenantConfig = extractChildConfig(jsonConfig, TENANT_CONFIG_KEY, TenantConfig.class);
- Preconditions.checkState(tenantConfig != null, FIELD_MISSING_MESSAGE_TEMPLATE, TENANT_CONFIG_KEY);
-
- IndexingConfig indexingConfig = extractChildConfig(jsonConfig, INDEXING_CONFIG_KEY, IndexingConfig.class);
- Preconditions.checkState(indexingConfig != null, FIELD_MISSING_MESSAGE_TEMPLATE, INDEXING_CONFIG_KEY);
-
- TableCustomConfig customConfig = extractChildConfig(jsonConfig, CUSTOM_CONFIG_KEY, TableCustomConfig.class);
- Preconditions.checkState(customConfig != null, FIELD_MISSING_MESSAGE_TEMPLATE, CUSTOM_CONFIG_KEY);
-
- // Optional fields
- QuotaConfig quotaConfig = extractChildConfig(jsonConfig, QUOTA_CONFIG_KEY, QuotaConfig.class);
- if (quotaConfig != null) {
- quotaConfig.validate();
- }
-
- TableTaskConfig taskConfig = extractChildConfig(jsonConfig, TASK_CONFIG_KEY, TableTaskConfig.class);
-
- RoutingConfig routingConfig = extractChildConfig(jsonConfig, ROUTING_CONFIG_KEY, RoutingConfig.class);
-
- QueryConfig queryConfig = extractChildConfig(jsonConfig, QUERY_CONFIG_KEY, QueryConfig.class);
-
- Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap =
- extractChildConfig(jsonConfig, INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY,
- new TypeReference<Map<InstancePartitionsType, InstanceAssignmentConfig>>() {
- });
-
- List<FieldConfig> fieldConfigList =
- extractChildConfig(jsonConfig, FIELD_CONFIG_LIST_KEY, new TypeReference<List<FieldConfig>>() {
- });
-
- return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
- quotaConfig, taskConfig, routingConfig, queryConfig, instanceAssignmentConfigMap, fieldConfigList);
- }
-
- /**
- * Extracts the child config from the table config. Returns {@code null} if child config does not exist.
- * <p>
- * NOTE: For historical reason, we support two kinds of nested config values: normal json and serialized json string
- */
- @Nullable
- private static <T> T extractChildConfig(JsonNode jsonConfig, String childConfigKey, Class<T> childConfigClass)
- throws IOException {
- JsonNode childConfigNode = jsonConfig.get(childConfigKey);
- if (childConfigNode == null || childConfigNode.isNull()) {
- return null;
- }
- if (childConfigNode.isObject() || childConfigNode.isContainerNode()) {
- return JsonUtils.jsonNodeToObject(childConfigNode, childConfigClass);
- } else {
- return JsonUtils.stringToObject(childConfigNode.asText(), childConfigClass);
- }
- }
-
- /**
- * Extracts the child config from the table config. Returns {@code null} if child config does not exist.
- * <p>
- * NOTE: For historical reason, we support two kinds of nested config values: normal json and serialized json string
- */
- @Nullable
- private static <T> T extractChildConfig(JsonNode jsonConfig, String childConfigKey,
- TypeReference<T> childConfigTypeReference)
- throws IOException {
- JsonNode childConfigNode = jsonConfig.get(childConfigKey);
- if (childConfigNode == null || childConfigNode.isNull()) {
- return null;
- }
- if (childConfigNode.isObject() || childConfigNode.isContainerNode()) {
- return JsonUtils.jsonNodeToObject(childConfigNode, childConfigTypeReference);
- } else {
- return JsonUtils.stringToObject(childConfigNode.asText(), childConfigTypeReference);
- }
- }
-
- public ObjectNode toJsonConfig() {
- validate();
-
- ObjectNode jsonConfig = JsonUtils.newObjectNode();
-
- // Mandatory fields
- jsonConfig.put(TABLE_NAME_KEY, _tableName);
- jsonConfig.put(TABLE_TYPE_KEY, _tableType.toString());
- jsonConfig.set(VALIDATION_CONFIG_KEY, JsonUtils.objectToJsonNode(_validationConfig));
- jsonConfig.set(TENANT_CONFIG_KEY, JsonUtils.objectToJsonNode(_tenantConfig));
- jsonConfig.set(INDEXING_CONFIG_KEY, JsonUtils.objectToJsonNode(_indexingConfig));
- jsonConfig.set(CUSTOM_CONFIG_KEY, JsonUtils.objectToJsonNode(_customConfig));
-
- // Optional fields
- if (_quotaConfig != null) {
- jsonConfig.set(QUOTA_CONFIG_KEY, JsonUtils.objectToJsonNode(_quotaConfig));
- }
- if (_taskConfig != null) {
- jsonConfig.set(TASK_CONFIG_KEY, JsonUtils.objectToJsonNode(_taskConfig));
- }
- if (_routingConfig != null) {
- jsonConfig.set(ROUTING_CONFIG_KEY, JsonUtils.objectToJsonNode(_routingConfig));
- }
- if (_queryConfig != null) {
- jsonConfig.set(QUERY_CONFIG_KEY, JsonUtils.objectToJsonNode(_queryConfig));
- }
- if (_instanceAssignmentConfigMap != null) {
- jsonConfig.set(INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY, JsonUtils.objectToJsonNode(_instanceAssignmentConfigMap));
- }
- if (_fieldConfigList != null) {
- jsonConfig.put(FIELD_CONFIG_LIST_KEY, JsonUtils.objectToJsonNode(_fieldConfigList));
- }
-
- return jsonConfig;
- }
-
- @Override
- public JsonNode toJsonNode() {
- return toJsonConfig();
- }
-
- public String toJsonConfigString() {
- return toJsonConfig().toString();
- }
-
- public static TableConfig fromZnRecord(ZNRecord znRecord)
- throws IOException {
- Map<String, String> simpleFields = znRecord.getSimpleFields();
-
- // Mandatory fields
- String tableTypeString = simpleFields.get(TABLE_TYPE_KEY);
- Preconditions.checkState(tableTypeString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TABLE_TYPE_KEY);
- TableType tableType = TableType.valueOf(tableTypeString.toUpperCase());
-
- String tableNameString = simpleFields.get(TABLE_NAME_KEY);
- Preconditions.checkState(tableNameString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TABLE_NAME_KEY);
- String tableName = TableNameBuilder.forType(tableType).tableNameWithType(tableNameString);
-
- String validationConfigString = simpleFields.get(VALIDATION_CONFIG_KEY);
- Preconditions.checkState(validationConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, VALIDATION_CONFIG_KEY);
- SegmentsValidationAndRetentionConfig validationConfig =
- JsonUtils.stringToObject(validationConfigString, SegmentsValidationAndRetentionConfig.class);
-
- String tenantConfigString = simpleFields.get(TENANT_CONFIG_KEY);
- Preconditions.checkState(tenantConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TENANT_CONFIG_KEY);
- TenantConfig tenantConfig = JsonUtils.stringToObject(tenantConfigString, TenantConfig.class);
-
- String indexingConfigString = simpleFields.get(INDEXING_CONFIG_KEY);
- Preconditions.checkState(indexingConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, INDEXING_CONFIG_KEY);
- IndexingConfig indexingConfig = JsonUtils.stringToObject(indexingConfigString, IndexingConfig.class);
-
- String customConfigString = simpleFields.get(CUSTOM_CONFIG_KEY);
- Preconditions.checkState(customConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, CUSTOM_CONFIG_KEY);
- TableCustomConfig customConfig = JsonUtils.stringToObject(customConfigString, TableCustomConfig.class);
-
- // Optional fields
- QuotaConfig quotaConfig = null;
- String quotaConfigString = simpleFields.get(QUOTA_CONFIG_KEY);
- if (quotaConfigString != null) {
- quotaConfig = JsonUtils.stringToObject(quotaConfigString, QuotaConfig.class);
- quotaConfig.validate();
- }
-
- TableTaskConfig taskConfig = null;
- String taskConfigString = simpleFields.get(TASK_CONFIG_KEY);
- if (taskConfigString != null) {
- taskConfig = JsonUtils.stringToObject(taskConfigString, TableTaskConfig.class);
- }
-
- RoutingConfig routingConfig = null;
- String routingConfigString = simpleFields.get(ROUTING_CONFIG_KEY);
- if (routingConfigString != null) {
- routingConfig = JsonUtils.stringToObject(routingConfigString, RoutingConfig.class);
- }
-
- QueryConfig queryConfig = null;
- String queryConfigString = simpleFields.get(QUERY_CONFIG_KEY);
- if (queryConfigString != null) {
- queryConfig = JsonUtils.stringToObject(queryConfigString, QueryConfig.class);
- }
-
- Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap = null;
- String instanceAssignmentConfigMapString = simpleFields.get(INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY);
- if (instanceAssignmentConfigMapString != null) {
- instanceAssignmentConfigMap = JsonUtils.stringToObject(instanceAssignmentConfigMapString,
- new TypeReference<Map<InstancePartitionsType, InstanceAssignmentConfig>>() {
- });
- }
-
- List<FieldConfig> fieldConfigList = null;
- String fieldConfigListString = simpleFields.get(FIELD_CONFIG_LIST_KEY);
- if (fieldConfigListString != null) {
- fieldConfigList = JsonUtils.stringToObject(fieldConfigListString, new TypeReference<List<FieldConfig>>() {
- });
- }
-
- return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
- quotaConfig, taskConfig, routingConfig, queryConfig, instanceAssignmentConfigMap, fieldConfigList);
- }
-
- public ZNRecord toZNRecord()
- throws JsonProcessingException {
- validate();
-
- Map<String, String> simpleFields = new HashMap<>();
-
- // Mandatory fields
- simpleFields.put(TABLE_NAME_KEY, _tableName);
- simpleFields.put(TABLE_TYPE_KEY, _tableType.toString());
- simpleFields.put(VALIDATION_CONFIG_KEY, JsonUtils.objectToString(_validationConfig));
- simpleFields.put(TENANT_CONFIG_KEY, JsonUtils.objectToString(_tenantConfig));
- simpleFields.put(INDEXING_CONFIG_KEY, JsonUtils.objectToString(_indexingConfig));
- simpleFields.put(CUSTOM_CONFIG_KEY, JsonUtils.objectToString(_customConfig));
-
- // Optional fields
- if (_quotaConfig != null) {
- simpleFields.put(QUOTA_CONFIG_KEY, JsonUtils.objectToString(_quotaConfig));
- }
- if (_taskConfig != null) {
- simpleFields.put(TASK_CONFIG_KEY, JsonUtils.objectToString(_taskConfig));
- }
- if (_routingConfig != null) {
- simpleFields.put(ROUTING_CONFIG_KEY, JsonUtils.objectToString(_routingConfig));
- }
- if (_queryConfig != null) {
- simpleFields.put(QUERY_CONFIG_KEY, JsonUtils.objectToString(_queryConfig));
- }
- if (_instanceAssignmentConfigMap != null) {
- simpleFields.put(INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY, JsonUtils.objectToString(_instanceAssignmentConfigMap));
- }
- if (_fieldConfigList != null) {
- simpleFields.put(FIELD_CONFIG_LIST_KEY, JsonUtils.objectToString(_fieldConfigList));
- }
-
- ZNRecord znRecord = new ZNRecord(_tableName);
- znRecord.setSimpleFields(simpleFields);
- return znRecord;
- }
-
- /**
- * Validates the table config.
- * TODO: revisit to see whether all the following fields are mandatory
- */
- public void validate() {
- Preconditions.checkState(_tableName != null, "Table name is missing");
- Preconditions.checkState(_tableType != null, "Table type is missing");
- Preconditions.checkState(_validationConfig != null, "Validation config is missing");
- Preconditions.checkState(_tenantConfig != null, "Tenant config is missing");
- Preconditions.checkState(_indexingConfig != null, "Indexing config is missing");
- Preconditions.checkState(_customConfig != null, "Custom config is missing");
- }
-
- public String getTableName() {
- return _tableName;
- }
-
- public void setTableName(String tableName) {
- _tableName = tableName;
- }
-
- public TableType getTableType() {
- return _tableType;
- }
-
- public void setTableType(TableType tableType) {
- _tableType = tableType;
- }
-
- public SegmentsValidationAndRetentionConfig getValidationConfig() {
- return _validationConfig;
- }
-
- public void setValidationConfig(SegmentsValidationAndRetentionConfig validationConfig) {
- _validationConfig = validationConfig;
- }
-
- public TenantConfig getTenantConfig() {
- return _tenantConfig;
- }
-
- public void setTenantConfig(TenantConfig tenantConfig) {
- _tenantConfig = tenantConfig;
- }
-
- public IndexingConfig getIndexingConfig() {
- return _indexingConfig;
- }
-
- public void setIndexingConfig(IndexingConfig indexingConfig) {
- _indexingConfig = indexingConfig;
- }
-
- public TableCustomConfig getCustomConfig() {
- return _customConfig;
- }
-
- public void setCustomConfig(TableCustomConfig customConfig) {
- _customConfig = customConfig;
- }
-
- @Nullable
- public QuotaConfig getQuotaConfig() {
- return _quotaConfig;
- }
-
- public void setQuotaConfig(QuotaConfig quotaConfig) {
- _quotaConfig = quotaConfig;
- }
-
- @Nullable
- public TableTaskConfig getTaskConfig() {
- return _taskConfig;
- }
-
- public void setTaskConfig(TableTaskConfig taskConfig) {
- _taskConfig = taskConfig;
- }
-
- @Nullable
- public RoutingConfig getRoutingConfig() {
- return _routingConfig;
- }
-
- public void setRoutingConfig(RoutingConfig routingConfig) {
- _routingConfig = routingConfig;
- }
-
- @Nullable
- public QueryConfig getQueryConfig() {
- return _queryConfig;
- }
-
- public void setQueryConfig(QueryConfig queryConfig) {
- _queryConfig = queryConfig;
- }
-
- @Nullable
- public Map<InstancePartitionsType, InstanceAssignmentConfig> getInstanceAssignmentConfigMap() {
- return _instanceAssignmentConfigMap;
- }
-
- public void setInstanceAssignmentConfigMap(
- Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap) {
- _instanceAssignmentConfigMap = instanceAssignmentConfigMap;
- }
-
- @Nullable
- public List<FieldConfig> getFieldConfigList() {
- return _fieldConfigList;
- }
-
- public static class Builder {
- private static final String DEFAULT_SEGMENT_PUSH_TYPE = "APPEND";
- private static final String REFRESH_SEGMENT_PUSH_TYPE = "REFRESH";
- private static final String DEFAULT_SEGMENT_ASSIGNMENT_STRATEGY = "BalanceNumSegmentAssignmentStrategy";
- private static final String DEFAULT_NUM_REPLICAS = "1";
- private static final String DEFAULT_LOAD_MODE = "HEAP";
- private static final String MMAP_LOAD_MODE = "MMAP";
-
- private final TableType _tableType;
- private String _tableName;
- private boolean _isLLC;
-
- // Validation config related
- private String _timeColumnName;
- private String _timeType;
- private String _retentionTimeUnit;
- private String _retentionTimeValue;
- private String _segmentPushFrequency;
- private String _segmentPushType = DEFAULT_SEGMENT_PUSH_TYPE;
- private String _segmentAssignmentStrategy = DEFAULT_SEGMENT_ASSIGNMENT_STRATEGY;
- private String _schemaName;
- private String _numReplicas = DEFAULT_NUM_REPLICAS;
-
- // Tenant config related
- private String _brokerTenant;
- private String _serverTenant;
- private TagOverrideConfig _tagOverrideConfig;
-
- // Indexing config related
- private String _loadMode = DEFAULT_LOAD_MODE;
- private String _segmentVersion;
- private String _sortedColumn;
- private List<String> _invertedIndexColumns;
- private List<String> _noDictionaryColumns;
- private List<String> _onHeapDictionaryColumns;
- private List<String> _bloomFilterColumns;
- private Map<String, String> _streamConfigs;
- private SegmentPartitionConfig _segmentPartitionConfig;
-
- private TableCustomConfig _customConfig;
- private QuotaConfig _quotaConfig;
- private TableTaskConfig _taskConfig;
- private RoutingConfig _routingConfig;
- private QueryConfig _queryConfig;
- private Map<InstancePartitionsType, InstanceAssignmentConfig> _instanceAssignmentConfigMap;
- private List<FieldConfig> _fieldConfigList;
-
- public Builder(TableType tableType) {
- _tableType = tableType;
- }
-
- public Builder setTableName(String tableName) {
- _tableName = tableName;
- return this;
- }
-
- public Builder setLLC(boolean isLLC) {
- Preconditions.checkState(_tableType == TableType.REALTIME);
- _isLLC = isLLC;
- return this;
- }
-
- public Builder setTimeColumnName(String timeColumnName) {
- _timeColumnName = timeColumnName;
- return this;
- }
-
- public Builder setTimeType(String timeType) {
- _timeType = timeType;
- return this;
- }
-
- public Builder setRetentionTimeUnit(String retentionTimeUnit) {
- _retentionTimeUnit = retentionTimeUnit;
- return this;
- }
-
- public Builder setRetentionTimeValue(String retentionTimeValue) {
- _retentionTimeValue = retentionTimeValue;
- return this;
- }
-
- public Builder setSegmentPushType(String segmentPushType) {
- if (REFRESH_SEGMENT_PUSH_TYPE.equalsIgnoreCase(segmentPushType)) {
- _segmentPushType = REFRESH_SEGMENT_PUSH_TYPE;
- } else {
- _segmentPushType = DEFAULT_SEGMENT_PUSH_TYPE;
- }
- return this;
- }
-
- public Builder setSegmentPushFrequency(String segmentPushFrequency) {
- _segmentPushFrequency = segmentPushFrequency;
- return this;
- }
-
- public Builder setSegmentAssignmentStrategy(String segmentAssignmentStrategy) {
- _segmentAssignmentStrategy = segmentAssignmentStrategy;
- return this;
- }
-
- public Builder setSchemaName(String schemaName) {
- _schemaName = schemaName;
- return this;
- }
-
- public Builder setNumReplicas(int numReplicas) {
- Preconditions.checkArgument(numReplicas > 0);
- _numReplicas = String.valueOf(numReplicas);
- return this;
- }
-
- public Builder setBrokerTenant(String brokerTenant) {
- _brokerTenant = brokerTenant;
- return this;
- }
-
- public Builder setServerTenant(String serverTenant) {
- _serverTenant = serverTenant;
- return this;
- }
-
- public Builder setTagOverrideConfig(TagOverrideConfig tagOverrideConfig) {
- _tagOverrideConfig = tagOverrideConfig;
- return this;
- }
-
- public Builder setLoadMode(String loadMode) {
- if (MMAP_LOAD_MODE.equalsIgnoreCase(loadMode)) {
- _loadMode = MMAP_LOAD_MODE;
- } else {
- _loadMode = DEFAULT_LOAD_MODE;
- }
- return this;
- }
-
- public Builder setSegmentVersion(String segmentVersion) {
- _segmentVersion = segmentVersion;
- return this;
- }
-
- public Builder setSortedColumn(String sortedColumn) {
- _sortedColumn = sortedColumn;
- return this;
- }
-
- public Builder setInvertedIndexColumns(List<String> invertedIndexColumns) {
- _invertedIndexColumns = invertedIndexColumns;
- return this;
- }
-
- public Builder setNoDictionaryColumns(List<String> noDictionaryColumns) {
- _noDictionaryColumns = noDictionaryColumns;
- return this;
- }
-
- public Builder setOnHeapDictionaryColumns(List<String> onHeapDictionaryColumns) {
- _onHeapDictionaryColumns = onHeapDictionaryColumns;
- return this;
- }
-
- public Builder setBloomFilterColumns(List<String> bloomFilterColumns) {
- _bloomFilterColumns = bloomFilterColumns;
- return this;
- }
-
- public Builder setStreamConfigs(Map<String, String> streamConfigs) {
- Preconditions.checkState(_tableType == TableType.REALTIME);
- _streamConfigs = streamConfigs;
- return this;
- }
-
- public Builder setSegmentPartitionConfig(SegmentPartitionConfig segmentPartitionConfig) {
- _segmentPartitionConfig = segmentPartitionConfig;
- return this;
- }
-
- public Builder setCustomConfig(TableCustomConfig customConfig) {
- _customConfig = customConfig;
- return this;
- }
-
- public Builder setQuotaConfig(QuotaConfig quotaConfig) {
- _quotaConfig = quotaConfig;
- return this;
- }
-
- public Builder setTaskConfig(TableTaskConfig taskConfig) {
- _taskConfig = taskConfig;
- return this;
- }
-
- public Builder setRoutingConfig(RoutingConfig routingConfig) {
- _routingConfig = routingConfig;
- return this;
- }
-
- public Builder setQueryConfig(QueryConfig queryConfig) {
- _queryConfig = queryConfig;
- return this;
- }
-
- public Builder setInstanceAssignmentConfigMap(
- Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap) {
- _instanceAssignmentConfigMap = instanceAssignmentConfigMap;
- return this;
- }
-
- public Builder setFieldConfigList(List<FieldConfig> fieldConfigList) {
- _fieldConfigList = fieldConfigList;
- return this;
- }
-
- public TableConfig build() {
- // Validation config
- SegmentsValidationAndRetentionConfig validationConfig = new SegmentsValidationAndRetentionConfig();
- validationConfig.setTimeColumnName(_timeColumnName);
- validationConfig.setTimeType(_timeType);
- validationConfig.setRetentionTimeUnit(_retentionTimeUnit);
- validationConfig.setRetentionTimeValue(_retentionTimeValue);
- validationConfig.setSegmentPushFrequency(_segmentPushFrequency);
- validationConfig.setSegmentPushType(_segmentPushType);
- validationConfig.setSegmentAssignmentStrategy(_segmentAssignmentStrategy);
- validationConfig.setSchemaName(_schemaName);
- validationConfig.setReplication(_numReplicas);
- if (_isLLC) {
- validationConfig.setReplicasPerPartition(_numReplicas);
- }
-
- // Tenant config
- TenantConfig tenantConfig = new TenantConfig(_brokerTenant, _serverTenant, _tagOverrideConfig);
-
- // Indexing config
- IndexingConfig indexingConfig = new IndexingConfig();
- indexingConfig.setLoadMode(_loadMode);
- indexingConfig.setSegmentFormatVersion(_segmentVersion);
- if (_sortedColumn != null) {
- indexingConfig.setSortedColumn(Collections.singletonList(_sortedColumn));
- }
- indexingConfig.setInvertedIndexColumns(_invertedIndexColumns);
- indexingConfig.setNoDictionaryColumns(_noDictionaryColumns);
- indexingConfig.setOnHeapDictionaryColumns(_onHeapDictionaryColumns);
- indexingConfig.setBloomFilterColumns(_bloomFilterColumns);
- indexingConfig.setStreamConfigs(_streamConfigs);
- indexingConfig.setSegmentPartitionConfig(_segmentPartitionConfig);
-
- if (_customConfig == null) {
- _customConfig = new TableCustomConfig(null);
- }
-
- // eventually this validation will be generic but since we are initially
- // using FieldConfig only for text columns (and migrate to expand its usage
- // soon after), just validate the field config list from text index creation
- // perspective.
- TextIndexConfigValidator.validate(_fieldConfigList, _noDictionaryColumns);
-
- return new TableConfig(_tableName, _tableType, validationConfig, tenantConfig, indexingConfig, _customConfig,
- _quotaConfig, _taskConfig, _routingConfig, _queryConfig, _instanceAssignmentConfigMap, _fieldConfigList);
- }
- }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TextIndexConfigValidator.java b/pinot-common/src/main/java/org/apache/pinot/common/config/TextIndexConfigValidator.java
deleted file mode 100644
index 1a48163..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TextIndexConfigValidator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.pinot.common.config;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-
-public class TextIndexConfigValidator {
- public static void validate(List<FieldConfig> fieldConfigList, List<String> noDictionaryColumns) {
- if (fieldConfigList != null) {
- Set<String> noDict = new HashSet<>();
- if (noDictionaryColumns != null) {
- noDict.addAll(noDictionaryColumns);
- }
- for (FieldConfig fieldConfig : fieldConfigList) {
- String column = fieldConfig.getName();
- if (fieldConfig.getIndexType() == FieldConfig.IndexType.TEXT) {
- // validate both places until we get rid of IndexingConfig way of specifying info and
- // move completely to FieldConfig
- if (fieldConfig.getEncodingType() != FieldConfig.EncodingType.RAW || !noDict.contains(fieldConfig.getName())) {
- throw new UnsupportedOperationException(
- "Dictionary encoded index is not supported for text column: " + column + " currently. Only raw index is supported. Please use EncodingType as RAW in FieldConfig");
- }
- }
- }
- }
- }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java
index 9e33b41..e09ebba 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java
@@ -27,17 +27,18 @@ import org.I0Itec.zkclient.exception.ZkBadVersionException;
import org.apache.helix.AccessOption;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.SchemaUtils;
import org.apache.pinot.common.utils.SegmentName;
import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -212,7 +213,7 @@ public class ZKMetadataProvider {
return null;
}
try {
- return TableConfig.fromZnRecord(znRecord);
+ return TableConfigUtils.fromZNRecord(znRecord);
} catch (Exception e) {
LOGGER.error("Caught exception while getting table configuration for table: {}", tableNameWithType, e);
return null;
@@ -268,16 +269,16 @@ public class ZKMetadataProvider {
}
// For backward compatible where schema name is not the same as raw table name
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
// Try to fetch realtime schema first
- if (tableType == null || tableType == CommonConstants.Helix.TableType.REALTIME) {
+ if (tableType == null || tableType == TableType.REALTIME) {
TableConfig realtimeTableConfig = getRealtimeTableConfig(propertyStore, tableName);
if (realtimeTableConfig != null) {
schema = getSchema(propertyStore, realtimeTableConfig.getValidationConfig().getSchemaName());
}
}
// Try to fetch offline schema if realtime schema does not exist
- if (schema == null && (tableType == null || tableType == CommonConstants.Helix.TableType.OFFLINE)) {
+ if (schema == null && (tableType == null || tableType == TableType.OFFLINE)) {
schema = getSchema(propertyStore, TableNameBuilder.OFFLINE.tableNameWithType(tableName));
}
if (schema != null) {
@@ -434,7 +435,7 @@ public class ZKMetadataProvider {
String controllerConfigPath = constructPropertyStorePathForControllerConfig(CLUSTER_TENANT_ISOLATION_ENABLED_KEY);
if (propertyStore.exists(controllerConfigPath, AccessOption.PERSISTENT)) {
ZNRecord znRecord = propertyStore.get(controllerConfigPath, null, AccessOption.PERSISTENT);
- if (znRecord.getSimpleFields().keySet().contains(CLUSTER_TENANT_ISOLATION_ENABLED_KEY)) {
+ if (znRecord.getSimpleFields().containsKey(CLUSTER_TENANT_ISOLATION_ENABLED_KEY)) {
return znRecord.getBooleanField(CLUSTER_TENANT_ISOLATION_ENABLED_KEY, true);
} else {
return true;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/instance/InstanceZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/instance/InstanceZKMetadata.java
index 06d13f0..8ab499a 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/instance/InstanceZKMetadata.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/instance/InstanceZKMetadata.java
@@ -21,9 +21,9 @@ package org.apache.pinot.common.metadata.instance;
import java.util.HashMap;
import java.util.Map;
import org.apache.helix.ZNRecord;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadata;
import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import static org.apache.pinot.spi.utils.EqualityUtils.hashCodeOf;
import static org.apache.pinot.spi.utils.EqualityUtils.isEqual;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
index 1fa19b5..880a48a 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
@@ -95,21 +95,6 @@ public class CommonConstants {
public static final String ADMIN_PORT_KEY = "adminPort";
}
- public enum InstanceType {
- CONTROLLER, BROKER, SERVER, MINION
- }
-
- public enum TableType {
- OFFLINE, REALTIME;
-
- public ServerType getServerType() {
- if (this == OFFLINE) {
- return ServerType.OFFLINE;
- }
- return ServerType.REALTIME;
- }
- }
-
public static final String SET_INSTANCE_ID_TO_HOSTNAME_KEY = "pinot.set.instance.id.to.hostname";
public static final String KEY_OF_SERVER_NETTY_PORT = "pinot.server.netty.port";
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/InstanceUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/InstanceUtils.java
new file mode 100644
index 0000000..7e13222
--- /dev/null
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/InstanceUtils.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.pinot.common.utils.config;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.CommonConstants.Helix;
+import org.apache.pinot.spi.config.api.Instance;
+
+
+public class InstanceUtils {
+ private InstanceUtils() {
+ }
+
+ public static final String POOL_KEY = "pool";
+
+ /**
+ * Returns the Helix instance id (e.g. {@code Server_localhost_1234}) for the given instance.
+ */
+ public static String getHelixInstanceId(Instance instance) {
+ String prefix;
+ switch (instance.getType()) {
+ case CONTROLLER:
+ prefix = Helix.PREFIX_OF_CONTROLLER_INSTANCE;
+ break;
+ case BROKER:
+ prefix = Helix.PREFIX_OF_BROKER_INSTANCE;
+ break;
+ case SERVER:
+ prefix = Helix.PREFIX_OF_SERVER_INSTANCE;
+ break;
+ case MINION:
+ prefix = Helix.PREFIX_OF_MINION_INSTANCE;
+ break;
+ default:
+ throw new IllegalStateException();
+ }
+ return prefix + instance.getHost() + "_" + instance.getPort();
+ }
+
+ /**
+ * Returns the Helix InstanceConfig for the given instance.
+ */
+ public static InstanceConfig toHelixInstanceConfig(Instance instance) {
+ InstanceConfig instanceConfig = InstanceConfig.toInstanceConfig(getHelixInstanceId(instance));
+ List<String> tags = instance.getTags();
+ if (tags != null) {
+ for (String tag : tags) {
+ instanceConfig.addTag(tag);
+ }
+ }
+ Map<String, Integer> pools = instance.getPools();
+ if (pools != null && !pools.isEmpty()) {
+ Map<String, String> mapValue = new TreeMap<>();
+ for (Map.Entry<String, Integer> entry : pools.entrySet()) {
+ mapValue.put(entry.getKey(), entry.getValue().toString());
+ }
+ instanceConfig.getRecord().setMapField(POOL_KEY, mapValue);
+ }
+ return instanceConfig;
+ }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigUtils.java
new file mode 100644
index 0000000..dc78de1
--- /dev/null
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigUtils.java
@@ -0,0 +1,192 @@
+/**
+ * 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.pinot.common.utils.config;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.helix.ZNRecord;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.QueryConfig;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableCustomConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.JsonUtils;
+
+
+public class TableConfigUtils {
+ private TableConfigUtils() {
+ }
+
+ private static final String FIELD_MISSING_MESSAGE_TEMPLATE = "Mandatory field '%s' is missing";
+
+ public static TableConfig fromZNRecord(ZNRecord znRecord)
+ throws IOException {
+ Map<String, String> simpleFields = znRecord.getSimpleFields();
+
+ // Mandatory fields
+ String tableName = znRecord.getId();
+
+ String tableType = simpleFields.get(TableConfig.TABLE_TYPE_KEY);
+ Preconditions.checkState(tableType != null, FIELD_MISSING_MESSAGE_TEMPLATE, TableConfig.TABLE_TYPE_KEY);
+
+ String validationConfigString = simpleFields.get(TableConfig.VALIDATION_CONFIG_KEY);
+ Preconditions
+ .checkState(validationConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TableConfig.VALIDATION_CONFIG_KEY);
+ SegmentsValidationAndRetentionConfig validationConfig =
+ JsonUtils.stringToObject(validationConfigString, SegmentsValidationAndRetentionConfig.class);
+
+ String tenantConfigString = simpleFields.get(TableConfig.TENANT_CONFIG_KEY);
+ Preconditions.checkState(tenantConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TableConfig.TENANT_CONFIG_KEY);
+ TenantConfig tenantConfig = JsonUtils.stringToObject(tenantConfigString, TenantConfig.class);
+
+ String indexingConfigString = simpleFields.get(TableConfig.INDEXING_CONFIG_KEY);
+ Preconditions
+ .checkState(indexingConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TableConfig.INDEXING_CONFIG_KEY);
+ IndexingConfig indexingConfig = JsonUtils.stringToObject(indexingConfigString, IndexingConfig.class);
+
+ String customConfigString = simpleFields.get(TableConfig.CUSTOM_CONFIG_KEY);
+ Preconditions.checkState(customConfigString != null, FIELD_MISSING_MESSAGE_TEMPLATE, TableConfig.CUSTOM_CONFIG_KEY);
+ TableCustomConfig customConfig = JsonUtils.stringToObject(customConfigString, TableCustomConfig.class);
+
+ // Optional fields
+ QuotaConfig quotaConfig = null;
+ String quotaConfigString = simpleFields.get(TableConfig.QUOTA_CONFIG_KEY);
+ if (quotaConfigString != null) {
+ quotaConfig = JsonUtils.stringToObject(quotaConfigString, QuotaConfig.class);
+ }
+
+ TableTaskConfig taskConfig = null;
+ String taskConfigString = simpleFields.get(TableConfig.TASK_CONFIG_KEY);
+ if (taskConfigString != null) {
+ taskConfig = JsonUtils.stringToObject(taskConfigString, TableTaskConfig.class);
+ }
+
+ RoutingConfig routingConfig = null;
+ String routingConfigString = simpleFields.get(TableConfig.ROUTING_CONFIG_KEY);
+ if (routingConfigString != null) {
+ routingConfig = JsonUtils.stringToObject(routingConfigString, RoutingConfig.class);
+ }
+
+ QueryConfig queryConfig = null;
+ String queryConfigString = simpleFields.get(TableConfig.QUERY_CONFIG_KEY);
+ if (queryConfigString != null) {
+ queryConfig = JsonUtils.stringToObject(queryConfigString, QueryConfig.class);
+ }
+
+ Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap = null;
+ String instanceAssignmentConfigMapString = simpleFields.get(TableConfig.INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY);
+ if (instanceAssignmentConfigMapString != null) {
+ instanceAssignmentConfigMap = JsonUtils.stringToObject(instanceAssignmentConfigMapString,
+ new TypeReference<Map<InstancePartitionsType, InstanceAssignmentConfig>>() {
+ });
+ }
+
+ List<FieldConfig> fieldConfigList = null;
+ String fieldConfigListString = simpleFields.get(TableConfig.FIELD_CONFIG_LIST_KEY);
+ if (fieldConfigListString != null) {
+ fieldConfigList = JsonUtils.stringToObject(fieldConfigListString, new TypeReference<List<FieldConfig>>() {
+ });
+ }
+
+ return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
+ quotaConfig, taskConfig, routingConfig, queryConfig, instanceAssignmentConfigMap, fieldConfigList);
+ }
+
+ public static ZNRecord toZNRecord(TableConfig tableConfig)
+ throws JsonProcessingException {
+ Map<String, String> simpleFields = new HashMap<>();
+
+ // Mandatory fields
+ simpleFields.put(TableConfig.TABLE_NAME_KEY, tableConfig.getTableName());
+ simpleFields.put(TableConfig.TABLE_TYPE_KEY, tableConfig.getTableType().toString());
+ simpleFields.put(TableConfig.VALIDATION_CONFIG_KEY, tableConfig.getValidationConfig().toJsonString());
+ simpleFields.put(TableConfig.TENANT_CONFIG_KEY, tableConfig.getTenantConfig().toJsonString());
+ simpleFields.put(TableConfig.INDEXING_CONFIG_KEY, tableConfig.getIndexingConfig().toJsonString());
+ simpleFields.put(TableConfig.CUSTOM_CONFIG_KEY, tableConfig.getCustomConfig().toJsonString());
+
+ // Optional fields
+ QuotaConfig quotaConfig = tableConfig.getQuotaConfig();
+ if (quotaConfig != null) {
+ simpleFields.put(TableConfig.QUOTA_CONFIG_KEY, quotaConfig.toJsonString());
+ }
+ TableTaskConfig taskConfig = tableConfig.getTaskConfig();
+ if (taskConfig != null) {
+ simpleFields.put(TableConfig.TASK_CONFIG_KEY, taskConfig.toJsonString());
+ }
+ RoutingConfig routingConfig = tableConfig.getRoutingConfig();
+ if (routingConfig != null) {
+ simpleFields.put(TableConfig.ROUTING_CONFIG_KEY, routingConfig.toJsonString());
+ }
+ QueryConfig queryConfig = tableConfig.getQueryConfig();
+ if (queryConfig != null) {
+ simpleFields.put(TableConfig.QUERY_CONFIG_KEY, queryConfig.toJsonString());
+ }
+ Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap =
+ tableConfig.getInstanceAssignmentConfigMap();
+ if (instanceAssignmentConfigMap != null) {
+ simpleFields
+ .put(TableConfig.INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY, JsonUtils.objectToString(instanceAssignmentConfigMap));
+ }
+ List<FieldConfig> fieldConfigList = tableConfig.getFieldConfigList();
+ if (fieldConfigList != null) {
+ simpleFields.put(TableConfig.FIELD_CONFIG_LIST_KEY, JsonUtils.objectToString(fieldConfigList));
+ }
+
+ ZNRecord znRecord = new ZNRecord(tableConfig.getTableName());
+ znRecord.setSimpleFields(simpleFields);
+ return znRecord;
+ }
+
+ /**
+ * Validates the table config with the following rules:
+ * <ul>
+ * <li>Text index column must be raw</li>
+ * </ul>
+ */
+ public static void validate(TableConfig tableConfig) {
+ List<FieldConfig> fieldConfigList = tableConfig.getFieldConfigList();
+ if (fieldConfigList != null) {
+ List<String> noDictionaryColumns = tableConfig.getIndexingConfig().getNoDictionaryColumns();
+ for (FieldConfig fieldConfig : fieldConfigList) {
+ if (fieldConfig.getIndexType() == FieldConfig.IndexType.TEXT) {
+ // For Text index column, it must be raw (no-dictionary)
+ // NOTE: Check both encodingType and noDictionaryColumns before migrating indexing configs into field configs
+ String column = fieldConfig.getName();
+ if (fieldConfig.getEncodingType() != FieldConfig.EncodingType.RAW || noDictionaryColumns == null
+ || !noDictionaryColumns.contains(column)) {
+ throw new IllegalStateException(
+ "Text index column: " + column + " must be raw (no-dictionary) in both FieldConfig and IndexingConfig");
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TagNameUtils.java
similarity index 94%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java
rename to pinot-common/src/main/java/org/apache/pinot/common/utils/config/TagNameUtils.java
index de5c6b3..35a6edf 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TagNameUtils.java
@@ -16,11 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.common.utils.config;
import javax.annotation.Nullable;
-import org.apache.pinot.common.utils.ServerType;
-import org.apache.pinot.common.utils.TenantRole;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TagOverrideConfig;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.api.TenantRole;
/**
@@ -41,8 +43,8 @@ public class TagNameUtils {
public final static String DEFAULT_TENANT_NAME = "DefaultTenant";
private final static String BROKER_TAG_SUFFIX = "_" + TenantRole.BROKER;
- private final static String OFFLINE_SERVER_TAG_SUFFIX = "_" + ServerType.OFFLINE;
- private final static String REALTIME_SERVER_TAG_SUFFIX = "_" + ServerType.REALTIME;
+ private final static String OFFLINE_SERVER_TAG_SUFFIX = "_" + TableType.OFFLINE;
+ private final static String REALTIME_SERVER_TAG_SUFFIX = "_" + TableType.REALTIME;
/**
* Returns whether the given tag is a broker tag.
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/HelixHelper.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/HelixHelper.java
index f354671..064a699 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/HelixHelper.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/HelixHelper.java
@@ -41,8 +41,8 @@ import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
import org.apache.helix.model.IdealState;
import org.apache.helix.model.InstanceConfig;
import org.apache.helix.model.builder.HelixConfigScopeBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.pinot.spi.utils.retry.RetryPolicy;
import org.slf4j.Logger;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/TableCache.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/TableCache.java
index 32fb21b..7138e5f 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/TableCache.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/TableCache.java
@@ -18,7 +18,6 @@
*/
package org.apache.pinot.common.utils.helix;
-import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@@ -27,13 +26,13 @@ import org.I0Itec.zkclient.IZkChildListener;
import org.I0Itec.zkclient.IZkDataListener;
import org.apache.helix.AccessOption;
import org.apache.helix.ZNRecord;
-import org.apache.helix.store.HelixPropertyListener;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.SchemaUtils;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -97,7 +96,7 @@ public class TableCache {
if (children != null) {
for (ZNRecord znRecord : children) {
try {
- TableConfig tableConfig = TableConfig.fromZnRecord(znRecord);
+ TableConfig tableConfig = TableConfigUtils.fromZNRecord(znRecord);
String tableNameWithType = tableConfig.getTableName();
_tableConfigMap.put(tableNameWithType, tableConfig);
String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/config/QuotaConfigTest.java b/pinot-common/src/test/java/org/apache/pinot/common/config/QuotaConfigTest.java
deleted file mode 100644
index 6dd3af7..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/config/QuotaConfigTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * 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.pinot.common.config;
-
-import java.io.IOException;
-import org.apache.commons.configuration.ConfigurationRuntimeException;
-import org.apache.pinot.spi.utils.JsonUtils;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-public class QuotaConfigTest {
-
- @Test
- public void testQuotaConfig()
- throws IOException {
- {
- String quotaConfigStr = "{\"storage\" : \"100g\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
-
- Assert.assertEquals(quotaConfig.getStorage(), "100g");
- Assert.assertEquals(quotaConfig.storageSizeBytes(), 100 * 1024 * 1024 * 1024L);
- }
- {
- String quotaConfigStr = "{}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNull(quotaConfig.getStorage());
- Assert.assertEquals(quotaConfig.storageSizeBytes(), -1);
- }
- }
-
- @Test
- public void testBadQuotaConfig()
- throws IOException {
- {
- String quotaConfigStr = "{\"storage\" : \"124GB3GB\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNotNull(quotaConfig.getStorage());
- Assert.assertEquals(quotaConfig.storageSizeBytes(), -1);
- }
- }
-
- @Test(expectedExceptions = ConfigurationRuntimeException.class)
- public void testBadConfig()
- throws IOException {
- String quotaConfigStr = "{\"storage\":\"-1M\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- quotaConfig.validate();
- }
-
- @Test
- public void testQpsQuota()
- throws IOException {
- {
- String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"100.00\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
-
- Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
- Assert.assertEquals(quotaConfig.getMaxQueriesPerSecond(), "100.00");
- Assert.assertTrue(quotaConfig.isMaxQueriesPerSecondValid());
- }
- {
- String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"0.5\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
-
- Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
- Assert.assertEquals(quotaConfig.getMaxQueriesPerSecond(), "0.5");
- Assert.assertTrue(quotaConfig.isMaxQueriesPerSecondValid());
- }
- {
- String quotaConfigStr = "{}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNull(quotaConfig.getMaxQueriesPerSecond());
- Assert.assertTrue(quotaConfig.isMaxQueriesPerSecondValid());
- }
- }
-
- @Test(expectedExceptions = ConfigurationRuntimeException.class)
- public void testInvalidQpsQuota()
- throws IOException {
- String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"InvalidQpsQuota\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
- quotaConfig.validate();
- }
-
- @Test(expectedExceptions = ConfigurationRuntimeException.class)
- public void testNegativeQpsQuota()
- throws IOException {
- String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"-1.0\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
- quotaConfig.validate();
- }
-
- @Test(expectedExceptions = ConfigurationRuntimeException.class)
- public void testBadQpsQuota()
- throws IOException {
- String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"1.0Test\"}";
- QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
- Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
- quotaConfig.validate();
- }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java b/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java
deleted file mode 100644
index 3911b37..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java
+++ /dev/null
@@ -1,409 +0,0 @@
-/**
- * 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.pinot.common.config;
-
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.config.instance.InstanceConstraintConfig;
-import org.apache.pinot.common.config.instance.InstanceReplicaGroupPartitionConfig;
-import org.apache.pinot.common.config.instance.InstanceTagPoolConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.testng.annotations.Test;
-
-import static org.testng.Assert.*;
-
-
-public class TableConfigTest {
-
- @Test
- public void testSerializeMandatoryFields()
- throws Exception {
- TableConfig tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setTableName(null);
- testSerializeMandatoryFields(tableConfig, "Table name");
-
- tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setTableType(null);
- testSerializeMandatoryFields(tableConfig, "Table type");
-
- tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setValidationConfig(null);
- testSerializeMandatoryFields(tableConfig, "Validation config");
-
- tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setTenantConfig(null);
- testSerializeMandatoryFields(tableConfig, "Tenant config");
-
- tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setIndexingConfig(null);
- testSerializeMandatoryFields(tableConfig, "Indexing config");
-
- tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- tableConfig.setCustomConfig(null);
- testSerializeMandatoryFields(tableConfig, "Custom config");
- }
-
- private void testSerializeMandatoryFields(TableConfig tableConfig, String expectedMessage)
- throws Exception {
- try {
- tableConfig.toJsonConfig();
- fail();
- } catch (IllegalStateException e) {
- assertTrue(e.getMessage().contains(expectedMessage));
- }
- try {
- tableConfig.toZNRecord();
- fail();
- } catch (IllegalStateException e) {
- assertTrue(e.getMessage().contains(expectedMessage));
- }
- }
-
- @Test
- public void testDeserializeMandatoryFields()
- throws Exception {
- TableConfig tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable").build();
- ObjectNode jsonTableConfig = tableConfig.toJsonConfig();
- TableConfig.fromJsonConfig(jsonTableConfig);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.TABLE_TYPE_KEY);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.TABLE_NAME_KEY);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.VALIDATION_CONFIG_KEY);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.TENANT_CONFIG_KEY);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.INDEXING_CONFIG_KEY);
-
- testDeserializeMandatoryFields(jsonTableConfig.deepCopy(), TableConfig.CUSTOM_CONFIG_KEY);
- }
-
- private void testDeserializeMandatoryFields(ObjectNode jsonTableConfig, String mandatoryFieldKey)
- throws Exception {
- jsonTableConfig.remove(mandatoryFieldKey);
- try {
- TableConfig.fromJsonConfig(jsonTableConfig);
- fail();
- } catch (IllegalStateException e) {
- assertTrue(e.getMessage().contains(mandatoryFieldKey));
- }
- }
-
- @Test
- public void testSerializeDeserialize()
- throws Exception {
- TableConfig.Builder tableConfigBuilder = new TableConfig.Builder(TableType.OFFLINE).setTableName("myTable");
- {
- // Only mandatory configs
- TableConfig tableConfig = tableConfigBuilder.build();
-
- assertEquals(tableConfig.getTableName(), "myTable_OFFLINE");
- assertEquals(tableConfig.getTableType(), TableType.OFFLINE);
- assertEquals(tableConfig.getIndexingConfig().getLoadMode(), "HEAP");
- assertNull(tableConfig.getQuotaConfig());
-
- // Serialize
- ObjectNode jsonTableConfig = tableConfig.toJsonConfig();
- // All nested configs should be json objects instead of serialized strings
- assertTrue(jsonTableConfig.get(TableConfig.VALIDATION_CONFIG_KEY) instanceof ObjectNode);
- assertTrue(jsonTableConfig.get(TableConfig.TENANT_CONFIG_KEY) instanceof ObjectNode);
- assertTrue(jsonTableConfig.get(TableConfig.INDEXING_CONFIG_KEY) instanceof ObjectNode);
- assertTrue(jsonTableConfig.get(TableConfig.CUSTOM_CONFIG_KEY) instanceof ObjectNode);
-
- // De-serialize
- TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(jsonTableConfig);
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNull(tableConfigToCompare.getQuotaConfig());
- assertNull(tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig());
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNull(tableConfigToCompare.getQuotaConfig());
- assertNull(tableConfig.getValidationConfig().getReplicaGroupStrategyConfig());
- }
- {
- // With quota config
- QuotaConfig quotaConfig = new QuotaConfig("30G", "100.00");
- TableConfig tableConfig = tableConfigBuilder.setQuotaConfig(quotaConfig).build();
-
- assertEquals(tableConfig.getTableName(), "myTable_OFFLINE");
- assertEquals(tableConfig.getTableType(), TableType.OFFLINE);
- assertEquals(tableConfig.getIndexingConfig().getLoadMode(), "HEAP");
- assertNotNull(tableConfig.getQuotaConfig());
- assertEquals(tableConfig.getQuotaConfig().getStorage(), "30G");
- assertEquals(tableConfig.getQuotaConfig().getMaxQueriesPerSecond(), "100.00");
-
- // Serialize then de-serialize
- TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getQuotaConfig());
- assertEquals(tableConfigToCompare.getQuotaConfig().getStorage(), tableConfig.getQuotaConfig().getStorage());
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getQuotaConfig());
- assertEquals(tableConfigToCompare.getQuotaConfig().getStorage(), tableConfig.getQuotaConfig().getStorage());
- }
- {
- // With tenant config
- TableConfig tableConfig =
- tableConfigBuilder.setServerTenant("aServerTenant").setBrokerTenant("aBrokerTenant").build();
-
- assertEquals(tableConfig.getTableName(), "myTable_OFFLINE");
- assertEquals(tableConfig.getTableType(), TableType.OFFLINE);
- assertEquals(tableConfig.getIndexingConfig().getLoadMode(), "HEAP");
- assertNotNull(tableConfig.getTenantConfig());
- assertEquals(tableConfig.getTenantConfig().getServer(), "aServerTenant");
- assertEquals(tableConfig.getTenantConfig().getBroker(), "aBrokerTenant");
- assertNull(tableConfig.getTenantConfig().getTagOverrideConfig());
-
- // Serialize then de-serialize
- TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getTenantConfig());
- assertEquals(tableConfigToCompare.getTenantConfig().getServer(), tableConfig.getTenantConfig().getServer());
- assertEquals(tableConfigToCompare.getTenantConfig().getBroker(), tableConfig.getTenantConfig().getBroker());
- assertNull(tableConfig.getTenantConfig().getTagOverrideConfig());
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getTenantConfig());
- assertEquals(tableConfigToCompare.getTenantConfig().getServer(), tableConfig.getTenantConfig().getServer());
- assertEquals(tableConfigToCompare.getTenantConfig().getBroker(), tableConfig.getTenantConfig().getBroker());
- assertNull(tableConfig.getTenantConfig().getTagOverrideConfig());
-
- TagOverrideConfig tagOverrideConfig = new TagOverrideConfig("aRTConsumingTag_REALTIME", null);
- tableConfig = tableConfigBuilder.setTagOverrideConfig(tagOverrideConfig).build();
-
- assertEquals(tableConfig.getTableName(), "myTable_OFFLINE");
- assertEquals(tableConfig.getTableType(), TableType.OFFLINE);
- assertNotNull(tableConfig.getTenantConfig());
- assertEquals(tableConfig.getTenantConfig().getServer(), "aServerTenant");
- assertEquals(tableConfig.getTenantConfig().getBroker(), "aBrokerTenant");
- assertNotNull(tableConfig.getTenantConfig().getTagOverrideConfig());
- assertEquals(tableConfig.getTenantConfig().getTagOverrideConfig().getRealtimeConsuming(),
- "aRTConsumingTag_REALTIME");
- assertNull(tableConfig.getTenantConfig().getTagOverrideConfig().getRealtimeCompleted());
-
- // Serialize then de-serialize
- tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getTenantConfig());
- assertEquals(tableConfigToCompare.getTenantConfig().getServer(), tableConfig.getTenantConfig().getServer());
- assertEquals(tableConfigToCompare.getTenantConfig().getBroker(), tableConfig.getTenantConfig().getBroker());
- assertNotNull(tableConfigToCompare.getTenantConfig().getTagOverrideConfig());
- assertEquals(tableConfig.getTenantConfig().getTagOverrideConfig(),
- tableConfigToCompare.getTenantConfig().getTagOverrideConfig());
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getTenantConfig());
- assertEquals(tableConfigToCompare.getTenantConfig().getServer(), tableConfig.getTenantConfig().getServer());
- assertEquals(tableConfigToCompare.getTenantConfig().getBroker(), tableConfig.getTenantConfig().getBroker());
- assertNotNull(tableConfigToCompare.getTenantConfig().getTagOverrideConfig());
- assertEquals(tableConfig.getTenantConfig().getTagOverrideConfig(),
- tableConfigToCompare.getTenantConfig().getTagOverrideConfig());
- }
- {
- // With SegmentAssignmentStrategyConfig
- ReplicaGroupStrategyConfig replicaGroupConfig = new ReplicaGroupStrategyConfig("memberId", 5);
-
- TableConfig tableConfig =
- tableConfigBuilder.setSegmentAssignmentStrategy("ReplicaGroupSegmentAssignmentStrategy").build();
- tableConfig.getValidationConfig().setReplicaGroupStrategyConfig(replicaGroupConfig);
-
- // Serialize then de-serialize
- TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- checkTableConfigWithAssignmentConfig(tableConfig, tableConfigToCompare);
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- checkTableConfigWithAssignmentConfig(tableConfig, tableConfigToCompare);
- }
- {
- // With completion config
- CompletionConfig completionConfig = new CompletionConfig("DEFAULT");
-
- TableConfig tableConfig = tableConfigBuilder.build();
- tableConfig.getValidationConfig().setCompletionConfig(completionConfig);
-
- // Serialize then de-serialize
- TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- checkTableConfigWithCompletionConfig(tableConfig, tableConfigToCompare);
-
- tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- checkTableConfigWithCompletionConfig(tableConfig, tableConfigToCompare);
- }
- {
- // With routing config
- RoutingConfig routingConfig =
- new RoutingConfig("builder", Arrays.asList("pruner0", "pruner1", "pruner2"), "selector");
- TableConfig tableConfig = tableConfigBuilder.setRoutingConfig(routingConfig).build();
-
- // Serialize then de-serialize
- assertEquals(TableConfig.fromJsonConfig(tableConfig.toJsonConfig()).getRoutingConfig(),
- tableConfig.getRoutingConfig());
- assertEquals(TableConfig.fromZnRecord(tableConfig.toZNRecord()).getRoutingConfig(),
- tableConfig.getRoutingConfig());
- }
- {
- // With query config
- QueryConfig queryConfig = new QueryConfig(1000L);
- TableConfig tableConfig = tableConfigBuilder.setQueryConfig(queryConfig).build();
-
- // Serialize then de-serialize
- assertEquals(TableConfig.fromJsonConfig(tableConfig.toJsonConfig()).getQueryConfig(),
- tableConfig.getQueryConfig());
- assertEquals(TableConfig.fromZnRecord(tableConfig.toZNRecord()).getQueryConfig(), tableConfig.getQueryConfig());
- }
- {
- // With instance assignment config
- InstanceAssignmentConfig instanceAssignmentConfig =
- new InstanceAssignmentConfig(new InstanceTagPoolConfig("tenant_OFFLINE", true, 3, null),
- new InstanceConstraintConfig(Arrays.asList("constraint1", "constraint2")),
- new InstanceReplicaGroupPartitionConfig(true, 0, 3, 5, 0, 0));
- TableConfig tableConfig = tableConfigBuilder.setInstanceAssignmentConfigMap(
- Collections.singletonMap(InstancePartitionsType.OFFLINE, instanceAssignmentConfig)).build();
-
- // Serialize then de-serialize
- checkTableConfigWithInstanceAssignmentConfigMap(TableConfig.fromJsonConfig(tableConfig.toJsonConfig()));
- checkTableConfigWithInstanceAssignmentConfigMap(TableConfig.fromZnRecord(tableConfig.toZNRecord()));
- }
- {
- // With field config
- Map<String, String> properties = new HashMap<>();
- properties.put(FieldConfig.TEXT_INDEX_REALTIME_READER_REFRESH_KEY, "100");
- FieldConfig fieldConfigTextCol =
- new FieldConfig("text_col", FieldConfig.EncodingType.RAW, FieldConfig.IndexType.TEXT, properties);
- FieldConfig fieldConfigInvCol =
- new FieldConfig("inv_index_col", FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.INVERTED, null);
- FieldConfig fieldConfigRawCol = new FieldConfig("raw_index_col", FieldConfig.EncodingType.RAW, null, null);
- Map<String, String> properties1 = new HashMap<>();
- properties1.put(FieldConfig.VAR_LENGTH_DICTIONARY_COLUMN_KEY, "true");
- FieldConfig fieldConfigSortedCol =
- new FieldConfig("sorted_index_col", FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.SORTED,
- properties1);
- List<String> noDict = new ArrayList<>();
- noDict.add("text_col");
- TableConfig tableConfig = tableConfigBuilder.setFieldConfigList(
- Lists.newArrayList(fieldConfigTextCol, fieldConfigInvCol, fieldConfigRawCol, fieldConfigSortedCol))
- .setNoDictionaryColumns(noDict).build();
-
- TableConfig toCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
- compareConfigHavingFieldConfig(toCompare, properties, properties1);
- toCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
- compareConfigHavingFieldConfig(toCompare, properties, properties1);
-
- TableConfig tableConfigWithoutFieldConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName("foo").build();
- toCompare = TableConfig.fromJsonConfig(tableConfigWithoutFieldConfig.toJsonConfig());
- assertNull(toCompare.getFieldConfigList());
- toCompare = TableConfig.fromZnRecord(tableConfigWithoutFieldConfig.toZNRecord());
- assertNull(toCompare.getFieldConfigList());
- }
- }
-
- private void compareConfigHavingFieldConfig(TableConfig toCompare, Map<String, String> properties,
- Map<String, String> properties1) {
- List<FieldConfig> fieldConfigs = toCompare.getFieldConfigList();
- assertNotNull(fieldConfigs);
- assertEquals(4, fieldConfigs.size());
-
- FieldConfig config = fieldConfigs.get(0);
- checkFieldConfigList(config, properties, "text_col", FieldConfig.EncodingType.RAW, FieldConfig.IndexType.TEXT);
- config = fieldConfigs.get(1);
- checkFieldConfigList(config, null, "inv_index_col", FieldConfig.EncodingType.DICTIONARY,
- FieldConfig.IndexType.INVERTED);
- config = fieldConfigs.get(2);
- checkFieldConfigList(config, null, "raw_index_col", FieldConfig.EncodingType.RAW, null);
- config = fieldConfigs.get(3);
- checkFieldConfigList(config, properties1, "sorted_index_col", FieldConfig.EncodingType.DICTIONARY,
- FieldConfig.IndexType.SORTED);
- }
-
- private void checkFieldConfigList(FieldConfig config, Map<String, String> expectedProperties, String expectedColumn,
- FieldConfig.EncodingType expectedEncodingType, FieldConfig.IndexType expectedIndexType) {
- assertEquals(expectedColumn, config.getName());
- assertEquals(expectedEncodingType, config.getEncodingType());
- assertEquals(expectedIndexType, config.getIndexType());
- Map<String, String> properties = config.getProperties();
- assertEquals(properties, expectedProperties);
- }
-
- private void checkTableConfigWithAssignmentConfig(TableConfig tableConfig, TableConfig tableConfigToCompare) {
- // Check that the segment assignment configuration does exist.
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig());
- assertEquals(tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig(),
- tableConfig.getValidationConfig().getReplicaGroupStrategyConfig());
-
- // Check that the configurations are correct.
- ReplicaGroupStrategyConfig strategyConfig =
- tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig();
- assertEquals(strategyConfig.getNumInstancesPerPartition(), 5);
- assertEquals(strategyConfig.getPartitionColumn(), "memberId");
- }
-
- private void checkTableConfigWithCompletionConfig(TableConfig tableConfig, TableConfig tableConfigToCompare) {
- // Check that the segment completion configuration does exist.
- assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
- assertNotNull(tableConfigToCompare.getValidationConfig().getCompletionConfig());
- assertEquals(tableConfigToCompare.getValidationConfig().getCompletionConfig(),
- tableConfig.getValidationConfig().getCompletionConfig());
-
- // Check that the configurations are correct.
- CompletionConfig completionConfig = tableConfigToCompare.getValidationConfig().getCompletionConfig();
- assertEquals(completionConfig.getCompletionMode(), "DEFAULT");
- }
-
- private void checkTableConfigWithInstanceAssignmentConfigMap(TableConfig tableConfig) {
- Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap =
- tableConfig.getInstanceAssignmentConfigMap();
- assertNotNull(instanceAssignmentConfigMap);
- assertEquals(instanceAssignmentConfigMap.size(), 1);
- assertTrue(instanceAssignmentConfigMap.containsKey(InstancePartitionsType.OFFLINE));
- InstanceAssignmentConfig instanceAssignmentConfig = instanceAssignmentConfigMap.get(InstancePartitionsType.OFFLINE);
-
- InstanceTagPoolConfig tagPoolConfig = instanceAssignmentConfig.getTagPoolConfig();
- assertEquals(tagPoolConfig.getTag(), "tenant_OFFLINE");
- assertTrue(tagPoolConfig.isPoolBased());
- assertEquals(tagPoolConfig.getNumPools(), 3);
- assertNull(tagPoolConfig.getPools());
-
- InstanceConstraintConfig constraintConfig = instanceAssignmentConfig.getConstraintConfig();
- assertEquals(constraintConfig.getConstraints(), Arrays.asList("constraint1", "constraint2"));
-
- InstanceReplicaGroupPartitionConfig replicaGroupPartitionConfig =
- instanceAssignmentConfig.getReplicaGroupPartitionConfig();
- assertTrue(replicaGroupPartitionConfig.isReplicaGroupBased());
- assertEquals(replicaGroupPartitionConfig.getNumInstances(), 0);
- assertEquals(replicaGroupPartitionConfig.getNumReplicaGroups(), 3);
- assertEquals(replicaGroupPartitionConfig.getNumInstancesPerReplicaGroup(), 5);
- assertEquals(replicaGroupPartitionConfig.getNumPartitions(), 0);
- assertEquals(replicaGroupPartitionConfig.getNumInstancesPerPartition(), 0);
- }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeTest.java
deleted file mode 100644
index f0fbce1..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeTest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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.pinot.common.utils;
-
-import org.apache.pinot.spi.utils.DataSize;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-public class DataSizeTest {
-
- @Test
- public void testToBytes() {
-
- Assert.assertEquals(DataSize.toBytes("128M"), 128 * 1024 * 1024L);
- Assert.assertEquals(DataSize.toBytes("1024"), 1024L);
-
- Assert.assertEquals(DataSize.toBytes("1.5G"), (long) (1.5 * 1024 * 1024 * 1024L));
-
- Assert.assertEquals(DataSize.toBytes("123"), 123);
- Assert.assertEquals(DataSize.toBytes("123P"), -1);
- Assert.assertEquals(DataSize.toBytes("-123M"), -1);
- Assert.assertEquals(DataSize.toBytes("12G3G"), -1);
-
- Assert.assertEquals(DataSize.toBytes("123k"), 123 * 1024L);
-
- Assert.assertEquals(DataSize.toBytes("123t"), 123 * 1024L * 1024 * 1024 * 1024);
- }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeUtilsTest.java
new file mode 100644
index 0000000..3c69484
--- /dev/null
+++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/DataSizeUtilsTest.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.pinot.common.utils;
+
+import org.apache.pinot.spi.utils.DataSizeUtils;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+
+
+public class DataSizeUtilsTest {
+
+ @Test
+ public void testToBytes() {
+ assertEquals(DataSizeUtils.toBytes("128"), 128L);
+ assertEquals(DataSizeUtils.toBytes("128B"), 128L);
+ assertEquals(DataSizeUtils.toBytes("128b"), 128L);
+
+ assertEquals(DataSizeUtils.toBytes("128.3"), 128L);
+ assertEquals(DataSizeUtils.toBytes("128.3B"), 128L);
+ assertEquals(DataSizeUtils.toBytes("128.3b"), 128L);
+
+ assertEquals(DataSizeUtils.toBytes("0"), 0L);
+ assertEquals(DataSizeUtils.toBytes("0B"), 0L);
+ assertEquals(DataSizeUtils.toBytes("0b"), 0L);
+
+ assertEquals(DataSizeUtils.toBytes("0.3"), 0L);
+ assertEquals(DataSizeUtils.toBytes("0.3B"), 0L);
+ assertEquals(DataSizeUtils.toBytes("0.3b"), 0L);
+
+ assertEquals(DataSizeUtils.toBytes("128K"), 128L * 1024);
+ assertEquals(DataSizeUtils.toBytes("128k"), 128L * 1024);
+ assertEquals(DataSizeUtils.toBytes("128KB"), 128L * 1024);
+ assertEquals(DataSizeUtils.toBytes("128kb"), 128L * 1024);
+ assertEquals(DataSizeUtils.toBytes("128.125K"), (long) (128.125 * 1024));
+
+ assertEquals(DataSizeUtils.toBytes("128M"), 128L * 1024 * 1024);
+ assertEquals(DataSizeUtils.toBytes("128G"), 128L * 1024 * 1024 * 1024);
+ assertEquals(DataSizeUtils.toBytes("128T"), 128L * 1024 * 1024 * 1024 * 1024);
+ assertEquals(DataSizeUtils.toBytes("128P"), 128L * 1024 * 1024 * 1024 * 1024 * 1024);
+
+ testIllegalDataSize(" 128");
+ testIllegalDataSize(" 128B");
+ testIllegalDataSize("128 ");
+ testIllegalDataSize("128 B");
+ testIllegalDataSize("-128");
+ testIllegalDataSize("-128B");
+ testIllegalDataSize(".3");
+ testIllegalDataSize(".3B");
+ testIllegalDataSize("128.");
+ testIllegalDataSize("128.B");
+ testIllegalDataSize("128.3.");
+ testIllegalDataSize("128.3.B");
+ }
+
+ private void testIllegalDataSize(String dataSizeString) {
+ try {
+ DataSizeUtils.toBytes(dataSizeString);
+ fail();
+ } catch (IllegalArgumentException e) {
+ // Expected
+ }
+ }
+
+ @Test
+ public void testFromBytes() {
+ assertEquals(DataSizeUtils.fromBytes(128L), "128B");
+ assertEquals(DataSizeUtils.fromBytes(128L * 1024), "128K");
+ assertEquals(DataSizeUtils.fromBytes(128L * 1024 * 1024), "128M");
+ assertEquals(DataSizeUtils.fromBytes(128L * 1024 * 1024 * 1024), "128G");
+ assertEquals(DataSizeUtils.fromBytes(128L * 1024 * 1024 * 1024 * 1024), "128T");
+ assertEquals(DataSizeUtils.fromBytes(128L * 1024 * 1024 * 1024 * 1024 * 1024), "128P");
+ assertEquals(DataSizeUtils.fromBytes((long) (128.125 * 1024)), "128.12K");
+ }
+}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
new file mode 100644
index 0000000..64c2f5b
--- /dev/null
+++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
@@ -0,0 +1,370 @@
+/**
+ * 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.pinot.common.utils.config;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.spi.config.CompletionConfig;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.QueryConfig;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.ReplicaGroupStrategyConfig;
+import org.apache.pinot.spi.config.RoutingConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TagOverrideConfig;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstanceConstraintConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.*;
+
+
+public class TableConfigSerDeTest {
+
+ @Test
+ public void testSerDe()
+ throws IOException {
+ TableConfigBuilder tableConfigBuilder = new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable");
+ {
+ // Default table config
+ TableConfig tableConfig = tableConfigBuilder.build();
+
+ checkDefaultTableConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkDefaultTableConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkDefaultTableConfig(tableConfigToCompare);
+
+ // Backward-compatible for raw table name and lower case table type
+ ObjectNode tableConfigJson = (ObjectNode) tableConfigBuilder.build().toJsonNode();
+ tableConfigJson.put(TableConfig.TABLE_NAME_KEY, "testTable");
+ tableConfigJson.put(TableConfig.TABLE_TYPE_KEY, "offline");
+ tableConfigToCompare = JsonUtils.jsonNodeToObject(tableConfigJson, TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkDefaultTableConfig(tableConfigToCompare);
+ }
+ {
+ // With quota config
+ QuotaConfig quotaConfig = new QuotaConfig("30g", "100.00");
+ TableConfig tableConfig = tableConfigBuilder.setQuotaConfig(quotaConfig).build();
+
+ checkQuotaConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkQuotaConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkQuotaConfig(tableConfigToCompare);
+ }
+ {
+ // With tenant config
+ TableConfig tableConfig =
+ tableConfigBuilder.setServerTenant("aServerTenant").setBrokerTenant("aBrokerTenant").build();
+
+ checkTenantConfigWithoutTagOverride(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkTenantConfigWithoutTagOverride(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkTenantConfigWithoutTagOverride(tableConfigToCompare);
+
+ // With tag override config
+ TagOverrideConfig tagOverrideConfig = new TagOverrideConfig("aRTConsumingTag_REALTIME", null);
+ tableConfig = tableConfigBuilder.setTagOverrideConfig(tagOverrideConfig).build();
+
+ checkTenantConfigWithTagOverride(tableConfig);
+
+ // Serialize then de-serialize
+ tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkTenantConfigWithTagOverride(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkTenantConfigWithTagOverride(tableConfigToCompare);
+ }
+ {
+ // With SegmentAssignmentStrategyConfig
+ ReplicaGroupStrategyConfig replicaGroupStrategyConfig = new ReplicaGroupStrategyConfig("memberId", 5);
+ TableConfig tableConfig = tableConfigBuilder.setSegmentAssignmentStrategy("ReplicaGroupSegmentAssignmentStrategy")
+ .setReplicaGroupStrategyConfig(replicaGroupStrategyConfig).build();
+
+ checkSegmentAssignmentStrategyConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkSegmentAssignmentStrategyConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkSegmentAssignmentStrategyConfig(tableConfigToCompare);
+ }
+ {
+ // With completion config
+ CompletionConfig completionConfig = new CompletionConfig("DEFAULT");
+ TableConfig tableConfig = tableConfigBuilder.setCompletionConfig(completionConfig).build();
+
+ checkCompletionConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkCompletionConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkCompletionConfig(tableConfigToCompare);
+ }
+ {
+ // With routing config
+ RoutingConfig routingConfig =
+ new RoutingConfig("builder", Arrays.asList("pruner0", "pruner1", "pruner2"), "selector");
+ TableConfig tableConfig = tableConfigBuilder.setRoutingConfig(routingConfig).build();
+
+ checkRoutingConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkRoutingConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkRoutingConfig(tableConfigToCompare);
+ }
+ {
+ // With query config
+ QueryConfig queryConfig = new QueryConfig(1000L);
+ TableConfig tableConfig = tableConfigBuilder.setQueryConfig(queryConfig).build();
+
+ checkQueryConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkQueryConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkQueryConfig(tableConfigToCompare);
+ }
+ {
+ // With instance assignment config
+ InstanceAssignmentConfig instanceAssignmentConfig =
+ new InstanceAssignmentConfig(new InstanceTagPoolConfig("tenant_OFFLINE", true, 3, null),
+ new InstanceConstraintConfig(Arrays.asList("constraint1", "constraint2")),
+ new InstanceReplicaGroupPartitionConfig(true, 0, 3, 5, 0, 0));
+ TableConfig tableConfig = tableConfigBuilder.setInstanceAssignmentConfigMap(
+ Collections.singletonMap(InstancePartitionsType.OFFLINE, instanceAssignmentConfig)).build();
+
+ checkInstanceAssignmentConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkInstanceAssignmentConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkInstanceAssignmentConfig(tableConfigToCompare);
+ }
+ {
+ // With field config
+ Map<String, String> properties = new HashMap<>();
+ properties.put("foo", "bar");
+ properties.put("foobar", "potato");
+ List<FieldConfig> fieldConfigList = Arrays.asList(
+ new FieldConfig("column1", FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.INVERTED, properties),
+ new FieldConfig("column2", null, null, null));
+ TableConfig tableConfig = tableConfigBuilder.setFieldConfigList(fieldConfigList).build();
+
+ checkFieldConfig(tableConfig);
+
+ // Serialize then de-serialize
+ TableConfig tableConfigToCompare = JsonUtils.stringToObject(tableConfig.toJsonString(), TableConfig.class);
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkFieldConfig(tableConfigToCompare);
+
+ tableConfigToCompare = TableConfigUtils.fromZNRecord(TableConfigUtils.toZNRecord(tableConfig));
+ assertEquals(tableConfigToCompare, tableConfig);
+ checkFieldConfig(tableConfigToCompare);
+ }
+ }
+
+ private void checkDefaultTableConfig(TableConfig tableConfig) {
+ // Check mandatory fields
+ assertEquals(tableConfig.getTableName(), "testTable_OFFLINE");
+ assertEquals(tableConfig.getTableType(), TableType.OFFLINE);
+ assertNotNull(tableConfig.getValidationConfig());
+ assertNotNull(tableConfig.getTenantConfig());
+ assertNotNull(tableConfig.getIndexingConfig());
+ assertNotNull(tableConfig.getCustomConfig());
+
+ // Check optional fields
+ assertNull(tableConfig.getQuotaConfig());
+ assertNull(tableConfig.getRoutingConfig());
+ assertNull(tableConfig.getQueryConfig());
+ assertNull(tableConfig.getInstanceAssignmentConfigMap());
+ assertNull(tableConfig.getFieldConfigList());
+
+ // Serialize
+ ObjectNode tableConfigJson = (ObjectNode) tableConfig.toJsonNode();
+ assertEquals(tableConfigJson.get(TableConfig.TABLE_NAME_KEY).asText(), "testTable_OFFLINE");
+ assertEquals(tableConfigJson.get(TableConfig.TABLE_TYPE_KEY).asText(), "OFFLINE");
+ assertTrue(tableConfigJson.get(TableConfig.VALIDATION_CONFIG_KEY) instanceof ObjectNode);
+ assertTrue(tableConfigJson.get(TableConfig.TENANT_CONFIG_KEY) instanceof ObjectNode);
+ assertTrue(tableConfigJson.get(TableConfig.INDEXING_CONFIG_KEY) instanceof ObjectNode);
+ assertTrue(tableConfigJson.get(TableConfig.CUSTOM_CONFIG_KEY) instanceof ObjectNode);
+ assertFalse(tableConfigJson.has(TableConfig.QUOTA_CONFIG_KEY));
+ assertFalse(tableConfigJson.has(TableConfig.TASK_CONFIG_KEY));
+ assertFalse(tableConfigJson.has(TableConfig.ROUTING_CONFIG_KEY));
+ assertFalse(tableConfigJson.has(TableConfig.QUERY_CONFIG_KEY));
+ assertFalse(tableConfigJson.has(TableConfig.INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY));
+ assertFalse(tableConfigJson.has(TableConfig.FIELD_CONFIG_LIST_KEY));
+ }
+
+ private void checkQuotaConfig(TableConfig tableConfig) {
+ QuotaConfig quotaConfig = tableConfig.getQuotaConfig();
+ assertNotNull(quotaConfig);
+ assertEquals(quotaConfig.getStorage(), "30G");
+ assertEquals(quotaConfig.getMaxQueriesPerSecond(), "100.0");
+ }
+
+ private void checkTenantConfigWithoutTagOverride(TableConfig tableConfig) {
+ TenantConfig tenantConfig = tableConfig.getTenantConfig();
+ assertNotNull(tenantConfig);
+ assertEquals(tenantConfig.getServer(), "aServerTenant");
+ assertEquals(tenantConfig.getBroker(), "aBrokerTenant");
+ assertNull(tenantConfig.getTagOverrideConfig());
+ }
+
+ private void checkTenantConfigWithTagOverride(TableConfig tableConfig) {
+ TenantConfig tenantConfig = tableConfig.getTenantConfig();
+ assertNotNull(tenantConfig);
+ assertEquals(tenantConfig.getServer(), "aServerTenant");
+ assertEquals(tenantConfig.getBroker(), "aBrokerTenant");
+ assertNotNull(tenantConfig.getTagOverrideConfig());
+ assertEquals(tenantConfig.getTagOverrideConfig().getRealtimeConsuming(), "aRTConsumingTag_REALTIME");
+ assertNull(tenantConfig.getTagOverrideConfig().getRealtimeCompleted());
+ }
+
+ private void checkSegmentAssignmentStrategyConfig(TableConfig tableConfig) {
+ SegmentsValidationAndRetentionConfig validationConfig = tableConfig.getValidationConfig();
+ assertEquals(validationConfig.getSegmentAssignmentStrategy(), "ReplicaGroupSegmentAssignmentStrategy");
+ ReplicaGroupStrategyConfig replicaGroupStrategyConfig = validationConfig.getReplicaGroupStrategyConfig();
+ assertNotNull(replicaGroupStrategyConfig);
+ assertEquals(replicaGroupStrategyConfig.getPartitionColumn(), "memberId");
+ assertEquals(replicaGroupStrategyConfig.getNumInstancesPerPartition(), 5);
+ }
+
+ private void checkCompletionConfig(TableConfig tableConfig) {
+ CompletionConfig completionConfig = tableConfig.getValidationConfig().getCompletionConfig();
+ assertNotNull(completionConfig);
+ assertEquals(completionConfig.getCompletionMode(), "DEFAULT");
+ }
+
+ private void checkRoutingConfig(TableConfig tableConfig) {
+ RoutingConfig routingConfig = tableConfig.getRoutingConfig();
+ assertNotNull(routingConfig);
+ assertEquals(routingConfig.getRoutingTableBuilderName(), "builder");
+ assertEquals(routingConfig.getSegmentPrunerTypes(), Arrays.asList("pruner0", "pruner1", "pruner2"));
+ assertEquals(routingConfig.getInstanceSelectorType(), "selector");
+ }
+
+ private void checkQueryConfig(TableConfig tableConfig) {
+ QueryConfig queryConfig = tableConfig.getQueryConfig();
+ assertNotNull(queryConfig);
+ assertEquals(queryConfig.getTimeoutMs(), Long.valueOf(1000L));
+ }
+
+ private void checkInstanceAssignmentConfig(TableConfig tableConfig) {
+ Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap =
+ tableConfig.getInstanceAssignmentConfigMap();
+ assertNotNull(instanceAssignmentConfigMap);
+ assertEquals(instanceAssignmentConfigMap.size(), 1);
+ assertTrue(instanceAssignmentConfigMap.containsKey(InstancePartitionsType.OFFLINE));
+ InstanceAssignmentConfig instanceAssignmentConfig = instanceAssignmentConfigMap.get(InstancePartitionsType.OFFLINE);
+
+ InstanceTagPoolConfig tagPoolConfig = instanceAssignmentConfig.getTagPoolConfig();
+ assertEquals(tagPoolConfig.getTag(), "tenant_OFFLINE");
+ assertTrue(tagPoolConfig.isPoolBased());
+ assertEquals(tagPoolConfig.getNumPools(), 3);
+ assertNull(tagPoolConfig.getPools());
+
+ InstanceConstraintConfig constraintConfig = instanceAssignmentConfig.getConstraintConfig();
+ assertNotNull(constraintConfig);
+ assertEquals(constraintConfig.getConstraints(), Arrays.asList("constraint1", "constraint2"));
+
+ InstanceReplicaGroupPartitionConfig replicaGroupPartitionConfig =
+ instanceAssignmentConfig.getReplicaGroupPartitionConfig();
+ assertTrue(replicaGroupPartitionConfig.isReplicaGroupBased());
+ assertEquals(replicaGroupPartitionConfig.getNumInstances(), 0);
+ assertEquals(replicaGroupPartitionConfig.getNumReplicaGroups(), 3);
+ assertEquals(replicaGroupPartitionConfig.getNumInstancesPerReplicaGroup(), 5);
+ assertEquals(replicaGroupPartitionConfig.getNumPartitions(), 0);
+ assertEquals(replicaGroupPartitionConfig.getNumInstancesPerPartition(), 0);
+ }
+
+ private void checkFieldConfig(TableConfig tableConfig) {
+ List<FieldConfig> fieldConfigList = tableConfig.getFieldConfigList();
+ assertNotNull(fieldConfigList);
+ assertEquals(fieldConfigList.size(), 2);
+
+ FieldConfig firstFieldConfig = fieldConfigList.get(0);
+ assertEquals(firstFieldConfig.getName(), "column1");
+ assertEquals(firstFieldConfig.getEncodingType(), FieldConfig.EncodingType.DICTIONARY);
+ assertEquals(firstFieldConfig.getIndexType(), FieldConfig.IndexType.INVERTED);
+ Map<String, String> expectedProperties = new HashMap<>();
+ expectedProperties.put("foo", "bar");
+ expectedProperties.put("foobar", "potato");
+ assertEquals(firstFieldConfig.getProperties(), expectedProperties);
+
+ FieldConfig secondFieldConfig = fieldConfigList.get(1);
+ assertEquals(secondFieldConfig.getName(), "column2");
+ assertNull(secondFieldConfig.getEncodingType());
+ assertNull(secondFieldConfig.getIndexType());
+ assertNull(secondFieldConfig.getProperties());
+ }
+}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java
new file mode 100644
index 0000000..5ae446c
--- /dev/null
+++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.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.pinot.common.utils.config;
+
+import java.util.Collections;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.fail;
+
+
+public class TableConfigUtilsTest {
+
+ @Test
+ public void testValidate() {
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").build();
+
+ // Default table config
+ TableConfigUtils.validate(tableConfig);
+
+ // Invalid field config (dictionary encoding & text index)
+ try {
+ FieldConfig textFieldConfig =
+ new FieldConfig("text", FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.TEXT, null);
+ tableConfig.setFieldConfigList(Collections.singletonList(textFieldConfig));
+ TableConfigUtils.validate(tableConfig);
+ fail();
+ } catch (IllegalStateException e) {
+ // Expected
+ }
+
+ // Valid field config (no-dictionary & text index) but raw index not configured in indexing config
+ try {
+ FieldConfig textFieldConfig =
+ new FieldConfig("text", FieldConfig.EncodingType.RAW, FieldConfig.IndexType.TEXT, null);
+ tableConfig.setFieldConfigList(Collections.singletonList(textFieldConfig));
+ TableConfigUtils.validate(tableConfig);
+ fail();
+ } catch (IllegalStateException e) {
+ // Expected
+ }
+
+ // Valid field config (no-dictionary & text index) and indexing config
+ tableConfig.getIndexingConfig().setNoDictionaryColumns(Collections.singletonList("text"));
+ TableConfigUtils.validate(tableConfig);
+ }
+}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/config/TagOverrideConfigTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TagNameUtilsTest.java
similarity index 94%
rename from pinot-common/src/test/java/org/apache/pinot/common/config/TagOverrideConfigTest.java
rename to pinot-common/src/test/java/org/apache/pinot/common/utils/config/TagNameUtilsTest.java
index 77de610..30ec9c5 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/config/TagOverrideConfigTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TagNameUtilsTest.java
@@ -16,17 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.common.utils.config;
import java.util.ArrayList;
import java.util.List;
+import org.apache.pinot.spi.config.TagOverrideConfig;
+import org.apache.pinot.spi.config.TenantConfig;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
import static org.testng.Assert.assertEquals;
-public class TagOverrideConfigTest {
+public class TagNameUtilsTest {
@DataProvider(name = "tagOverrideConfigTestDataProvider")
public Object[][] tagOverrideConfigTestDataProvider() {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/LeadControllerManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/LeadControllerManager.java
index d73bece..c103f82 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/LeadControllerManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/LeadControllerManager.java
@@ -22,12 +22,12 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.helix.HelixManager;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/DefaultMetadataEventNotifier.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/DefaultMetadataEventNotifier.java
index 1030582..d65a396 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/DefaultMetadataEventNotifier.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/DefaultMetadataEventNotifier.java
@@ -17,8 +17,7 @@
* under the License.
*/
package org.apache.pinot.controller.api.events;
-
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/MetadataEventNotifier.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/MetadataEventNotifier.java
index 12e1a7f..1095043 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/MetadataEventNotifier.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/events/MetadataEventNotifier.java
@@ -17,8 +17,7 @@
* under the License.
*/
package org.apache.pinot.controller.api.events;
-
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/Constants.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/Constants.java
index 2aea519..6c508cf 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/Constants.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/Constants.java
@@ -20,7 +20,7 @@ package org.apache.pinot.controller.api.resources;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response.Status;
-import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.spi.config.TableType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,12 +40,12 @@ public class Constants {
public static final String LEAD_CONTROLLER_TAG = "Leader";
public static final String TABLE_NAME = "tableName";
- public static CommonConstants.Helix.TableType validateTableType(String tableTypeStr) {
+ public static TableType validateTableType(String tableTypeStr) {
if (tableTypeStr == null || tableTypeStr.isEmpty()) {
return null;
}
try {
- return CommonConstants.Helix.TableType.valueOf(tableTypeStr.toUpperCase());
+ return TableType.valueOf(tableTypeStr.toUpperCase());
} catch (IllegalArgumentException e) {
LOGGER.info("Illegal table type '{}'", tableTypeStr);
throw new WebApplicationException("Illegal table type '" + tableTypeStr + "'", Status.BAD_REQUEST);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java
index e085053..4f3a854 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java
@@ -41,16 +41,16 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssignmentDriver;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java
index 99384ff..1ee5df6 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java
@@ -37,10 +37,11 @@ import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.Instance;
-import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.common.utils.config.InstanceUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
+import org.apache.pinot.spi.config.api.Instance;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -97,7 +98,7 @@ public class PinotInstanceRestletResource {
response.put("enabled", instanceConfig.getInstanceEnabled());
response.put("port", instanceConfig.getPort());
response.set("tags", JsonUtils.objectToJsonNode(instanceConfig.getTags()));
- response.set("pools", JsonUtils.objectToJsonNode(instanceConfig.getRecord().getMapField(Instance.POOL_KEY)));
+ response.set("pools", JsonUtils.objectToJsonNode(instanceConfig.getRecord().getMapField(InstanceUtils.POOL_KEY)));
return response.toString();
}
@@ -108,7 +109,7 @@ public class PinotInstanceRestletResource {
@ApiOperation(value = "Create a new instance", consumes = MediaType.APPLICATION_JSON, notes = "Creates a new instance with given instance config")
@ApiResponses(value = {@ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 409, message = "Instance already exists"), @ApiResponse(code = 500, message = "Internal error")})
public SuccessResponse addInstance(Instance instance) {
- LOGGER.info("Instance creation request received for instance: {}", instance.getInstanceId());
+ LOGGER.info("Instance creation request received for instance: {}", InstanceUtils.getHelixInstanceId(instance));
if (!pinotHelixResourceManager.addInstance(instance).isSuccessful()) {
throw new ControllerApplicationException(LOGGER, "Instance already exists", Response.Status.CONFLICT);
}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java
index b5f0216..5fb9b15 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java
@@ -40,10 +40,10 @@ import javax.ws.rs.core.Response;
import org.apache.helix.HelixManager;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.MasterSlaveSMD;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.helix.LeadControllerUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
index eadec54..004f863 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
@@ -43,7 +43,6 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.HttpHeaders;
import org.apache.helix.model.InstanceConfig;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.QueryException;
import org.apache.pinot.common.request.BrokerRequest;
import org.apache.pinot.common.utils.CommonConstants;
@@ -52,6 +51,7 @@ import org.apache.pinot.controller.api.access.AccessControlFactory;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.pql.parsers.Pql2Compiler;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.sql.parsers.CalciteSqlCompiler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
index 4ebb314..74c0bdb 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
@@ -41,7 +41,6 @@ import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.exception.SchemaNotFoundException;
import org.apache.pinot.common.exception.TableNotFoundException;
import org.apache.pinot.common.metrics.ControllerMeter;
@@ -49,6 +48,7 @@ import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.api.events.MetadataEventNotifierFactory;
import org.apache.pinot.controller.api.events.SchemaEventType;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.utils.JsonUtils;
import org.glassfish.jersey.media.multipart.FormDataBodyPart;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
index 69b6637..dd34307 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
@@ -44,17 +44,17 @@ import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.TableNotFoundException;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.SegmentName;
import org.apache.pinot.common.utils.URIUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
index 7e63ea5..576390d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
@@ -54,7 +54,6 @@ import javax.ws.rs.core.StreamingOutput;
import org.apache.commons.httpclient.HttpConnectionManager;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.utils.CommonConstants;
@@ -77,6 +76,7 @@ import org.apache.pinot.spi.crypt.PinotCrypter;
import org.apache.pinot.spi.crypt.PinotCrypterFactory;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.glassfish.grizzly.http.server.Request;
import org.glassfish.jersey.media.multipart.FormDataBodyPart;
import org.glassfish.jersey.media.multipart.FormDataMultiPart;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableIndexingConfigs.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableIndexingConfigs.java
index 46affda..4a1319d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableIndexingConfigs.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableIndexingConfigs.java
@@ -23,7 +23,6 @@ import io.swagger.annotations.ApiOperation;
import io.swagger.annotations.ApiParam;
import io.swagger.annotations.ApiResponse;
import io.swagger.annotations.ApiResponses;
-import java.io.IOException;
import javax.inject.Inject;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
@@ -31,8 +30,10 @@ import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -53,15 +54,18 @@ public class PinotTableIndexingConfigs {
@ApiResponses(value = {@ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 404, message = "Table not found"), @ApiResponse(code = 500, message = "Server error updating configuration")})
public SuccessResponse updateIndexingConfig(
@ApiParam(value = "Table name (without type)", required = true) @PathParam("tableName") String tableName,
- String body) {
+ String tableConfigString) {
+ TableConfig tableConfig;
+ try {
+ tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, "Invalid table config", Response.Status.BAD_REQUEST, e);
+ }
try {
- TableConfig tableConfig = TableConfig.fromJsonString(body);
pinotHelixResourceManager.updateIndexingConfigFor(tableConfig.getTableName(), tableConfig.getTableType(),
tableConfig.getIndexingConfig());
return new SuccessResponse("Updated indexing config for table " + tableName);
- } catch (IOException e) {
- String errStr = "Error converting request to table config for table: " + tableName;
- throw new ControllerApplicationException(LOGGER, errStr, Response.Status.BAD_REQUEST, e);
} catch (Exception e) {
String errStr = "Failed to update indexing config for table: " + tableName;
throw new ControllerApplicationException(LOGGER, errStr, Response.Status.INTERNAL_SERVER_ERROR, e);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableInstances.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableInstances.java
index c9da5b5..a1424e9 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableInstances.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableInstances.java
@@ -33,9 +33,9 @@ import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.JsonUtils;
@Api(tags = Constants.TABLE_TAG)
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableMetadataConfigs.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableMetadataConfigs.java
index bb67def..f882d37 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableMetadataConfigs.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableMetadataConfigs.java
@@ -29,8 +29,10 @@ import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -49,9 +51,15 @@ public class PinotTableMetadataConfigs {
@Produces(MediaType.APPLICATION_JSON)
@ApiOperation(value = "Update table metadata", notes = "Updates table configuration")
@ApiResponses(value = {@ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 500, message = "Internal server error"), @ApiResponse(code = 404, message = "Table not found")})
- public SuccessResponse updateTableMetadata(@PathParam("tableName") String tableName, String requestBody) {
+ public SuccessResponse updateTableMetadata(@PathParam("tableName") String tableName, String tableConfigString) {
+ TableConfig tableConfig;
+ try {
+ tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, "Invalid table config", Response.Status.BAD_REQUEST, e);
+ }
try {
- TableConfig tableConfig = TableConfig.fromJsonString(requestBody);
pinotHelixResourceManager.updateMetadataConfigFor(tableConfig.getTableName(), tableConfig.getTableType(),
tableConfig.getCustomConfig());
return new SuccessResponse("Successfully updated " + tableName + " configuration");
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
index f9a806a..a95e7a4 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
@@ -45,20 +45,20 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.configuration.BaseConfiguration;
import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.TableNotFoundException;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceResult;
import org.apache.pinot.core.util.ReplicationUtils;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.LoggerFactory;
@@ -110,13 +110,13 @@ public class PinotTableRestletResource {
public SuccessResponse addTable(String tableConfigStr) {
// TODO introduce a table config ctor with json string.
TableConfig tableConfig;
- String tableName;
try {
- tableConfig = TableConfig.fromJsonString(tableConfigStr);
- tableName = tableConfig.getTableName();
+ tableConfig = JsonUtils.stringToObject(tableConfigStr, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
} catch (Exception e) {
throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST, e);
}
+ String tableName = tableConfig.getTableName();
try {
ensureMinReplicas(tableConfig);
verifyTableConfigs(tableConfig);
@@ -144,15 +144,15 @@ public class PinotTableRestletResource {
public String listTableConfigs(@ApiParam(value = "realtime|offline") @QueryParam("type") String tableTypeStr) {
try {
List<String> tableNames;
- CommonConstants.Helix.TableType tableType = null;
+ TableType tableType = null;
if (tableTypeStr != null) {
- tableType = CommonConstants.Helix.TableType.valueOf(tableTypeStr.toUpperCase());
+ tableType = TableType.valueOf(tableTypeStr.toUpperCase());
}
if (tableType == null) {
tableNames = _pinotHelixResourceManager.getAllRawTables();
} else {
- if (tableType == CommonConstants.Helix.TableType.REALTIME) {
+ if (tableType == TableType.REALTIME) {
tableNames = _pinotHelixResourceManager.getAllRealtimeTables();
} else {
tableNames = _pinotHelixResourceManager.getAllOfflineTables();
@@ -170,18 +170,18 @@ public class PinotTableRestletResource {
try {
ObjectNode ret = JsonUtils.newObjectNode();
- if ((tableTypeStr == null || CommonConstants.Helix.TableType.OFFLINE.name().equalsIgnoreCase(tableTypeStr))
+ if ((tableTypeStr == null || TableType.OFFLINE.name().equalsIgnoreCase(tableTypeStr))
&& _pinotHelixResourceManager.hasOfflineTable(tableName)) {
TableConfig tableConfig = _pinotHelixResourceManager.getOfflineTableConfig(tableName);
Preconditions.checkNotNull(tableConfig);
- ret.set(CommonConstants.Helix.TableType.OFFLINE.name(), tableConfig.toJsonConfig());
+ ret.set(TableType.OFFLINE.name(), tableConfig.toJsonNode());
}
- if ((tableTypeStr == null || CommonConstants.Helix.TableType.REALTIME.name().equalsIgnoreCase(tableTypeStr))
+ if ((tableTypeStr == null || TableType.REALTIME.name().equalsIgnoreCase(tableTypeStr))
&& _pinotHelixResourceManager.hasRealtimeTable(tableName)) {
TableConfig tableConfig = _pinotHelixResourceManager.getRealtimeTableConfig(tableName);
Preconditions.checkNotNull(tableConfig);
- ret.set(CommonConstants.Helix.TableType.REALTIME.name(), tableConfig.toJsonConfig());
+ ret.set(TableType.REALTIME.name(), tableConfig.toJsonNode());
}
return ret.toString();
} catch (Exception e) {
@@ -303,13 +303,14 @@ public class PinotTableRestletResource {
@ApiOperation(value = "Updates table config for a table", notes = "Updates table config for a table")
public SuccessResponse updateTableConfig(
@ApiParam(value = "Name of the table to update", required = true) @PathParam("tableName") String tableName,
- String tableConfigStr)
+ String tableConfigString)
throws Exception {
TableConfig tableConfig;
try {
- tableConfig = TableConfig.fromJsonString(tableConfigStr);
+ tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
} catch (Exception e) {
- throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST);
+ throw new ControllerApplicationException(LOGGER, "Invalid table config", Response.Status.BAD_REQUEST, e);
}
try {
@@ -349,16 +350,17 @@ public class PinotTableRestletResource {
+ " This allows us to validate table config before apply.")
public String checkTableConfig(String tableConfigStr) {
try {
+ TableConfig tableConfig = JsonUtils.stringToObject(tableConfigStr, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
ObjectNode tableConfigValidateStr = JsonUtils.newObjectNode();
- TableConfig tableConfig = TableConfig.fromJsonString(tableConfigStr);
- if (tableConfig.getTableType() == CommonConstants.Helix.TableType.OFFLINE) {
- tableConfigValidateStr.set(CommonConstants.Helix.TableType.OFFLINE.name(), tableConfig.toJsonConfig());
+ if (tableConfig.getTableType() == TableType.OFFLINE) {
+ tableConfigValidateStr.set(TableType.OFFLINE.name(), tableConfig.toJsonNode());
} else {
- tableConfigValidateStr.set(CommonConstants.Helix.TableType.REALTIME.name(), tableConfig.toJsonConfig());
+ tableConfigValidateStr.set(TableType.REALTIME.name(), tableConfig.toJsonNode());
}
return tableConfigValidateStr.toString();
} catch (Exception e) {
- throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST);
+ throw new ControllerApplicationException(LOGGER, "Invalid table config", Response.Status.BAD_REQUEST, e);
}
}
@@ -421,7 +423,7 @@ public class PinotTableRestletResource {
LOGGER.info("Validating table configs for Table: {}", rawTableName);
TableConfig tableConfigToCompare = null;
- if (newTableConfig.getTableType() == CommonConstants.Helix.TableType.REALTIME) {
+ if (newTableConfig.getTableType() == TableType.REALTIME) {
if (_pinotHelixResourceManager.hasOfflineTable(rawTableName)) {
tableConfigToCompare = _pinotHelixResourceManager.getOfflineTableConfig(rawTableName);
}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSegmentConfigs.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSegmentConfigs.java
index b764252..4ec9565 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSegmentConfigs.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSegmentConfigs.java
@@ -23,7 +23,6 @@ import io.swagger.annotations.ApiOperation;
import io.swagger.annotations.ApiParam;
import io.swagger.annotations.ApiResponse;
import io.swagger.annotations.ApiResponses;
-import java.io.IOException;
import javax.inject.Inject;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
@@ -31,10 +30,11 @@ import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,20 +56,20 @@ public class PinotTableSegmentConfigs {
@ApiOperation(value = "Update segments configuration", notes = "Updates segmentsConfig section (validation and retention) of a table")
@ApiResponses(value = {@ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 404, message = "Table not found"), @ApiResponse(code = 500, message = "Internal server error")})
public SuccessResponse put(@ApiParam(value = "Table name", required = true) @PathParam("tableName") String tableName,
- String requestBody) {
+ String tableConfigString) {
+ TableConfig tableConfig;
+ try {
+ tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
+ TableConfigUtils.validate(tableConfig);
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, "Invalid table config", Response.Status.BAD_REQUEST, e);
+ }
try {
- TableConfig tableConfig = TableConfig.fromJsonString(requestBody);
pinotHelixResourceManager
.updateSegmentsValidationAndRetentionConfigFor(tableConfig.getTableName(), tableConfig.getTableType(),
tableConfig.getValidationConfig());
return new SuccessResponse("Update segmentsConfig for table: " + tableName);
- } catch (IOException e) {
- metrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_SCHEMA_UPDATE_ERROR, 1L);
- throw new ControllerApplicationException(LOGGER,
- String.format("Invalid json while updating segments config for table: %s", tableName),
- Response.Status.BAD_REQUEST, e);
} catch (Exception e) {
- metrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_SCHEMA_UPDATE_ERROR, 1L);
throw new ControllerApplicationException(LOGGER,
String.format("Failed to update segments config for table: %s", tableName),
Response.Status.INTERNAL_SERVER_ERROR, e);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java
index a7cbac8..745ff30 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java
@@ -34,9 +34,9 @@ import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.QueryParam;
import org.apache.helix.task.TaskState;
-import org.apache.pinot.common.config.PinotTaskConfig;
import org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
import org.apache.pinot.controller.helix.core.minion.PinotTaskManager;
+import org.apache.pinot.core.minion.PinotTaskConfig;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java
index f412aea..7da9c6a 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java
@@ -44,13 +44,13 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.Tenant;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.utils.TenantRole;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.api.Tenant;
+import org.apache.pinot.spi.config.api.TenantRole;
import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableViews.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableViews.java
index 89a7fcd..af8d4a3 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableViews.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableViews.java
@@ -35,9 +35,9 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -67,7 +67,7 @@ public class TableViews {
public TableView getIdealState(
@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
@ApiParam(value = "realtime|offline", required = false) @QueryParam("tableType") String tableTypeStr) {
- CommonConstants.Helix.TableType tableType = validateTableType(tableTypeStr);
+ TableType tableType = validateTableType(tableTypeStr);
return getTableState(tableName, IDEALSTATE, tableType);
}
@@ -77,16 +77,14 @@ public class TableViews {
@ApiOperation(value = "Get table external view", notes = "Get table external view")
public TableView getExternalView(
@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
- @ApiParam(value = "realtime|offline", required = false) @QueryParam("tableType") String tableTypeStr
-
- ) {
- CommonConstants.Helix.TableType tableType = validateTableType(tableTypeStr);
+ @ApiParam(value = "realtime|offline", required = false) @QueryParam("tableType") String tableTypeStr) {
+ TableType tableType = validateTableType(tableTypeStr);
return getTableState(tableName, EXTERNALVIEW, tableType);
}
// we use name "view" to closely match underlying names and to not
// confuse with table state of enable/disable
- private TableView getTableState(String tableName, String view, CommonConstants.Helix.TableType tableType) {
+ private TableView getTableState(String tableName, String view, TableType tableType) {
TableView tableView;
if (view.equalsIgnoreCase(IDEALSTATE)) {
tableView = getTableIdealState(tableName, tableType);
@@ -103,35 +101,34 @@ public class TableViews {
return tableView;
}
- private TableView getTableIdealState(String tableNameOptType, CommonConstants.Helix.TableType tableType) {
+ private TableView getTableIdealState(String tableNameOptType, TableType tableType) {
TableView tableView = new TableView();
- if (tableType == null || tableType == CommonConstants.Helix.TableType.OFFLINE) {
- tableView.offline = getIdealState(tableNameOptType, CommonConstants.Helix.TableType.OFFLINE);
+ if (tableType == null || tableType == TableType.OFFLINE) {
+ tableView.offline = getIdealState(tableNameOptType, TableType.OFFLINE);
}
- if (tableType == null || tableType == CommonConstants.Helix.TableType.REALTIME) {
- tableView.realtime = getIdealState(tableNameOptType, CommonConstants.Helix.TableType.REALTIME);
+ if (tableType == null || tableType == TableType.REALTIME) {
+ tableView.realtime = getIdealState(tableNameOptType, TableType.REALTIME);
}
return tableView;
}
- private TableView getTableExternalView(@Nonnull String tableNameOptType,
- @Nullable CommonConstants.Helix.TableType tableType) {
+ private TableView getTableExternalView(@Nonnull String tableNameOptType, @Nullable TableType tableType) {
TableView tableView = new TableView();
- if (tableType == null || tableType == CommonConstants.Helix.TableType.OFFLINE) {
- tableView.offline = getExternalView(tableNameOptType, CommonConstants.Helix.TableType.OFFLINE);
+ if (tableType == null || tableType == TableType.OFFLINE) {
+ tableView.offline = getExternalView(tableNameOptType, TableType.OFFLINE);
}
- if (tableType == null || tableType == CommonConstants.Helix.TableType.REALTIME) {
- tableView.realtime = getExternalView(tableNameOptType, CommonConstants.Helix.TableType.REALTIME);
+ if (tableType == null || tableType == TableType.REALTIME) {
+ tableView.realtime = getExternalView(tableNameOptType, TableType.REALTIME);
}
return tableView;
}
- private CommonConstants.Helix.TableType validateTableType(String tableTypeStr) {
+ private TableType validateTableType(String tableTypeStr) {
if (tableTypeStr == null) {
return null;
}
try {
- return CommonConstants.Helix.TableType.valueOf(tableTypeStr.toUpperCase());
+ return TableType.valueOf(tableTypeStr.toUpperCase());
} catch (IllegalArgumentException e) {
String errStr = "Illegal table type '" + tableTypeStr + "'";
throw new ControllerApplicationException(LOGGER, errStr, Response.Status.BAD_REQUEST, e);
@@ -140,7 +137,7 @@ public class TableViews {
@Nullable
public Map<String, Map<String, String>> getIdealState(@Nonnull String tableNameOptType,
- @Nullable CommonConstants.Helix.TableType tableType) {
+ @Nullable TableType tableType) {
String tableNameWithType = getTableNameWithType(tableNameOptType, tableType);
IdealState resourceIdealState = _pinotHelixResourceManager.getHelixAdmin()
.getResourceIdealState(_pinotHelixResourceManager.getHelixClusterName(), tableNameWithType);
@@ -148,18 +145,16 @@ public class TableViews {
}
@Nullable
- public Map<String, Map<String, String>> getExternalView(@Nonnull String tableNameOptType,
- CommonConstants.Helix.TableType tableType) {
+ public Map<String, Map<String, String>> getExternalView(@Nonnull String tableNameOptType, TableType tableType) {
String tableNameWithType = getTableNameWithType(tableNameOptType, tableType);
ExternalView resourceEV = _pinotHelixResourceManager.getHelixAdmin()
.getResourceExternalView(_pinotHelixResourceManager.getHelixClusterName(), tableNameWithType);
return resourceEV == null ? null : resourceEV.getRecord().getMapFields();
}
- private String getTableNameWithType(@Nonnull String tableNameOptType,
- @Nullable CommonConstants.Helix.TableType tableType) {
+ private String getTableNameWithType(@Nonnull String tableNameOptType, @Nullable TableType tableType) {
if (tableType != null) {
- if (tableType == CommonConstants.Helix.TableType.OFFLINE) {
+ if (tableType == TableType.OFFLINE) {
return TableNameBuilder.OFFLINE.tableNameWithType(tableNameOptType);
} else {
return TableNameBuilder.REALTIME.tableNameWithType(tableNameOptType);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
index 07ec187..9228f55 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
@@ -22,7 +22,7 @@ import java.io.File;
import java.util.concurrent.Executor;
import javax.ws.rs.core.Response;
import org.apache.commons.httpclient.HttpConnectionManager;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.commons.io.FileUtils;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.ControllerMetrics;
@@ -32,6 +32,7 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.util.TableSizeReader;
import org.apache.pinot.controller.validation.StorageQuotaChecker;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.utils.TimeUtils;
import org.joda.time.Interval;
import org.slf4j.Logger;
@@ -114,7 +115,7 @@ public class SegmentValidator {
new TableSizeReader(_executor, _connectionManager, _controllerMetrics, _pinotHelixResourceManager);
StorageQuotaChecker quotaChecker =
new StorageQuotaChecker(offlineTableConfig, tableSizeReader, _controllerMetrics, _isLeaderForTable);
- return quotaChecker.isSegmentStorageWithinQuota(segmentFile, metadata.getName(),
+ return quotaChecker.isSegmentStorageWithinQuota(metadata.getName(), FileUtils.sizeOfDirectory(segmentFile),
_controllerConf.getServerAdminRequestTimeoutSeconds() * 1000);
}
}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java
index b2599e2..b33161b 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java
@@ -23,7 +23,6 @@ import java.net.URI;
import javax.ws.rs.core.HttpHeaders;
import javax.ws.rs.core.Response;
import org.apache.helix.ZNRecord;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier;
import org.apache.pinot.common.metrics.ControllerMeter;
@@ -35,6 +34,7 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestURLBuilder.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestURLBuilder.java
index 2541c61..47bb181 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestURLBuilder.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestURLBuilder.java
@@ -20,10 +20,10 @@ package org.apache.pinot.controller.helix;
import javax.annotation.Nullable;
import org.apache.commons.lang.StringUtils;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.utils.StringUtil;
import org.apache.pinot.common.utils.URIUtils;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
public class ControllerRequestURLBuilder {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
index 9261420..6bbe41c 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
@@ -23,15 +23,15 @@ import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
index 31d78c1..ffe59a8 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
@@ -58,19 +58,9 @@ import org.apache.helix.model.IdealState;
import org.apache.helix.model.InstanceConfig;
import org.apache.helix.model.LiveInstance;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.Instance;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableCustomConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.Tenant;
-import org.apache.pinot.common.config.TenantConfig;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.exception.SchemaNotFoundException;
import org.apache.pinot.common.exception.TableNotFoundException;
@@ -84,9 +74,11 @@ import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.BrokerOnlineOfflineStateModel;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.SegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Server;
import org.apache.pinot.common.utils.SchemaUtils;
+import org.apache.pinot.common.utils.config.InstanceUtils;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
import org.apache.pinot.common.utils.helix.PinotHelixPropertyStoreZnRecordProvider;
import org.apache.pinot.common.utils.helix.TableCache;
@@ -103,8 +95,18 @@ import org.apache.pinot.controller.helix.core.rebalance.TableRebalancer;
import org.apache.pinot.controller.helix.core.util.ZKMetadataUtils;
import org.apache.pinot.controller.helix.starter.HelixConfig;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableCustomConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.api.Instance;
+import org.apache.pinot.spi.config.api.Tenant;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.pinot.spi.utils.retry.RetryPolicy;
import org.apache.zookeeper.data.Stat;
@@ -350,11 +352,11 @@ public class PinotHelixResourceManager {
*/
public synchronized PinotResourceManagerResponse addInstance(Instance instance) {
List<String> instances = getAllInstances();
- String instanceIdToAdd = instance.getInstanceId();
+ String instanceIdToAdd = InstanceUtils.getHelixInstanceId(instance);
if (instances.contains(instanceIdToAdd)) {
return PinotResourceManagerResponse.failure("Instance " + instanceIdToAdd + " already exists");
} else {
- _helixAdmin.addInstance(_helixClusterName, instance.toInstanceConfig());
+ _helixAdmin.addInstance(_helixClusterName, InstanceUtils.toHelixInstanceConfig(instance));
return PinotResourceManagerResponse.SUCCESS;
}
}
@@ -1080,7 +1082,8 @@ public class PinotHelixResourceManager {
_helixAdmin.addResource(_helixClusterName, tableNameWithType, offlineIdealState);
// lets add table configs
- ZKMetadataProvider.setOfflineTableConfig(_propertyStore, tableNameWithType, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setOfflineTableConfig(_propertyStore, tableNameWithType, TableConfigUtils.toZNRecord(tableConfig));
// Assign instances
assignInstances(tableConfig, true);
@@ -1105,7 +1108,8 @@ public class PinotHelixResourceManager {
}
// lets add table configs
- ZKMetadataProvider.setRealtimeTableConfig(_propertyStore, tableNameWithType, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setRealtimeTableConfig(_propertyStore, tableNameWithType, TableConfigUtils.toZNRecord(tableConfig));
// Assign instances before setting up the real-time cluster so that new LLC CONSUMING segment can be assigned
// based on the instance partitions
@@ -1310,7 +1314,8 @@ public class PinotHelixResourceManager {
TableType tableType = tableConfig.getTableType();
switch (tableType) {
case OFFLINE:
- ZKMetadataProvider.setOfflineTableConfig(_propertyStore, tableNameWithType, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setOfflineTableConfig(_propertyStore, tableNameWithType, TableConfigUtils.toZNRecord(tableConfig));
// Update IdealState replication
IdealState idealState = _helixAdmin.getResourceIdealState(_helixClusterName, tableNameWithType);
@@ -1331,7 +1336,8 @@ public class PinotHelixResourceManager {
case REALTIME:
IndexingConfig indexingConfig = tableConfig.getIndexingConfig();
verifyIndexingConfig(tableNameWithType, indexingConfig);
- ZKMetadataProvider.setRealtimeTableConfig(_propertyStore, tableNameWithType, tableConfig.toZNRecord());
+ ZKMetadataProvider
+ .setRealtimeTableConfig(_propertyStore, tableNameWithType, TableConfigUtils.toZNRecord(tableConfig));
// Assign instances before setting up the real-time cluster so that new LLC CONSUMING segment can be assigned
// based on the instance partitions
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
index 5269a90..a17c79c 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
@@ -25,16 +25,16 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.model.IdealState;
import org.apache.helix.model.builder.CustomModeISBuilder;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
-import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.stream.PartitionCountFetcher;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java
index 3ae4f32..9c8c286 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java
@@ -37,12 +37,12 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.utils.SegmentName;
import org.apache.pinot.common.utils.URIUtils;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.joda.time.DateTime;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentDriver.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentDriver.java
index 62dd7b9..d18f214 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentDriver.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentDriver.java
@@ -22,13 +22,13 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
-import org.apache.pinot.common.config.instance.InstanceConstraintConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstanceConstraintConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceReplicaGroupPartitionSelector.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceReplicaGroupPartitionSelector.java
index a4e26aa..9567724 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceReplicaGroupPartitionSelector.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceReplicaGroupPartitionSelector.java
@@ -25,7 +25,7 @@ import java.util.Map;
import java.util.TreeMap;
import org.apache.helix.model.InstanceConfig;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.config.instance.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceTagPoolSelector.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceTagPoolSelector.java
index 50aa095..6d819c7 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceTagPoolSelector.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceTagPoolSelector.java
@@ -26,8 +26,8 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.Instance;
-import org.apache.pinot.common.config.instance.InstanceTagPoolConfig;
+import org.apache.pinot.common.utils.config.InstanceUtils;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -72,7 +72,7 @@ public class InstanceTagPoolSelector {
// Extract the pool information from the instance configs
for (InstanceConfig instanceConfig : candidateInstanceConfigs) {
- Map<String, String> poolMap = instanceConfig.getRecord().getMapField(Instance.POOL_KEY);
+ Map<String, String> poolMap = instanceConfig.getRecord().getMapField(InstanceUtils.POOL_KEY);
if (poolMap != null && poolMap.containsKey(tag)) {
int pool = Integer.parseInt(poolMap.get(tag));
poolToInstanceConfigsMap.computeIfAbsent(pool, k -> new ArrayList<>()).add(instanceConfig);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineSegmentAssignment.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineSegmentAssignment.java
index 3f5ea8b..fb0187b 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineSegmentAssignment.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineSegmentAssignment.java
@@ -30,13 +30,13 @@ import java.util.TreeMap;
import org.apache.commons.configuration.Configuration;
import org.apache.helix.HelixManager;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.utils.Pairs;
+import org.apache.pinot.spi.config.ReplicaGroupStrategyConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeSegmentAssignment.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeSegmentAssignment.java
index 1006f44..9447b9b 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeSegmentAssignment.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeSegmentAssignment.java
@@ -29,11 +29,11 @@ import java.util.TreeMap;
import org.apache.commons.configuration.Configuration;
import org.apache.helix.HelixManager;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignment.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignment.java
index c934de0..a9b1787 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignment.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignment.java
@@ -23,8 +23,8 @@ import java.util.Map;
import org.apache.commons.configuration.Configuration;
import org.apache.helix.HelixManager;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentFactory.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentFactory.java
index db531bb..1145268 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentFactory.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentFactory.java
@@ -19,8 +19,8 @@
package org.apache.pinot.controller.helix.core.assignment.segment;
import org.apache.helix.HelixManager;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/ClusterInfoProvider.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/ClusterInfoProvider.java
index 508dad6..a30cac5 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/ClusterInfoProvider.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/ClusterInfoProvider.java
@@ -22,15 +22,15 @@ import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.helix.task.TaskState;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.data.Schema;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java
index 18f7b17..5dadef9 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java
@@ -34,8 +34,8 @@ import org.apache.helix.task.TaskConfig;
import org.apache.helix.task.TaskDriver;
import org.apache.helix.task.TaskState;
import org.apache.helix.task.WorkflowConfig;
-import org.apache.pinot.common.config.PinotTaskConfig;
import org.apache.pinot.common.utils.CommonConstants.Helix;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
index 8ceffff..1e0e5db 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
@@ -23,9 +23,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableTaskConfig;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.ControllerConf;
@@ -34,6 +31,9 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator;
import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorRegistry;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/ConvertToRawIndexTaskGenerator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/ConvertToRawIndexTaskGenerator.java
index 22bdcc5..ac1d9af 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/ConvertToRawIndexTaskGenerator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/ConvertToRawIndexTaskGenerator.java
@@ -25,14 +25,14 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.annotation.Nonnull;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableTaskConfig;
import org.apache.pinot.common.data.Segment;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.helix.core.minion.ClusterInfoProvider;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.config.TableType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -64,7 +64,7 @@ public class ConvertToRawIndexTaskGenerator implements PinotTaskGenerator {
for (TableConfig tableConfig : tableConfigs) {
// Only generate tasks for OFFLINE tables
String offlineTableName = tableConfig.getTableName();
- if (tableConfig.getTableType() != CommonConstants.Helix.TableType.OFFLINE) {
+ if (tableConfig.getTableType() != TableType.OFFLINE) {
LOGGER.warn("Skip generating ConvertToRawIndexTask for non-OFFLINE table: {}", offlineTableName);
continue;
}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/PinotTaskGenerator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/PinotTaskGenerator.java
index fbbe335..c4b282c 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/PinotTaskGenerator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/PinotTaskGenerator.java
@@ -21,8 +21,8 @@ package org.apache.pinot.controller.helix.core.minion.generator;
import java.util.List;
import javax.annotation.Nonnull;
import org.apache.helix.task.JobConfig;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.TableConfig;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/TaskGeneratorUtils.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/TaskGeneratorUtils.java
index 43dfe73..31f0c70 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/TaskGeneratorUtils.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/TaskGeneratorUtils.java
@@ -23,11 +23,11 @@ import java.util.Map;
import java.util.Set;
import javax.annotation.Nonnull;
import org.apache.helix.task.TaskState;
-import org.apache.pinot.common.config.PinotTaskConfig;
import org.apache.pinot.common.data.Segment;
import org.apache.pinot.controller.helix.core.minion.ClusterInfoProvider;
import org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
public class TaskGeneratorUtils {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
index e465d21..f349d69 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
@@ -41,12 +41,7 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.model.IdealState;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
@@ -72,6 +67,10 @@ import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpd
import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpdater;
import org.apache.pinot.controller.util.SegmentCompletionUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
import org.apache.pinot.spi.stream.OffsetCriteria;
@@ -79,6 +78,7 @@ import org.apache.pinot.spi.stream.PartitionLevelStreamConfig;
import org.apache.pinot.spi.stream.PartitionOffsetFetcher;
import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.stream.StreamConfigProperties;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
index b3b5d85..89fcdbb 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
@@ -37,26 +37,27 @@ import org.apache.helix.manager.zk.ZkClient;
import org.apache.helix.model.IdealState;
import org.apache.helix.store.HelixPropertyListener;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
import org.apache.pinot.common.utils.CommonConstants.Segment.SegmentType;
import org.apache.pinot.common.utils.HLCSegmentName;
import org.apache.pinot.common.utils.SegmentName;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
-import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotTableIdealStateBuilder;
import org.apache.pinot.core.query.utils.Pair;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -332,7 +333,7 @@ public class PinotRealtimeSegmentManager implements HelixPropertyListener, IZkCh
try {
String znRecordId = tableConfigZnRecord.getId();
if (TableNameBuilder.getTableTypeFromTableName(znRecordId) == TableType.REALTIME) {
- TableConfig tableConfig = TableConfig.fromZnRecord(tableConfigZnRecord);
+ TableConfig tableConfig = TableConfigUtils.fromZNRecord(tableConfigZnRecord);
StreamConfig metadata = new StreamConfig(tableConfig.getTableName(),
tableConfig.getIndexingConfig().getStreamConfigs());
if (metadata.hasHighLevelConsumerType()) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
index b467495..e233544 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
@@ -28,7 +28,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.helix.HelixManager;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
@@ -37,6 +36,7 @@ import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.realtime.segment.CommittingSegmentDescriptor;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
index ddff959..b8a1958 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
@@ -27,8 +27,8 @@ import java.util.concurrent.TimeUnit;
import javax.annotation.Nonnull;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.spi.config.TableConfig;
/**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/RebalanceResult.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/RebalanceResult.java
index 0544814..6c4190a 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/RebalanceResult.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/RebalanceResult.java
@@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
@JsonIgnoreProperties(ignoreUnknown = true)
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancer.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancer.java
index fc053c9..f768757 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancer.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancer.java
@@ -32,17 +32,17 @@ import org.apache.helix.PropertyKey;
import org.apache.helix.ZNRecord;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssignmentDriver;
import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentFactory;
import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.apache.pinot.spi.stream.StreamConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
index ff8f9a1..03981ba 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
@@ -22,9 +22,7 @@ import com.google.common.base.Preconditions;
import java.util.concurrent.ExecutorService;
import org.apache.commons.configuration.BaseConfiguration;
import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.LeadControllerManager;
@@ -33,8 +31,10 @@ import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTas
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceResult;
import org.apache.pinot.controller.helix.core.rebalance.TableRebalancer;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.utils.TimeUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
index a154b83..f566638 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
@@ -25,9 +25,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMetrics;
@@ -40,6 +37,9 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
import org.apache.pinot.controller.helix.core.retention.strategy.RetentionStrategy;
import org.apache.pinot.controller.helix.core.retention.strategy.TimeRetentionStrategy;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/AutoAddInvertedIndex.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/AutoAddInvertedIndex.java
index 4e13e02..21fbb98 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/AutoAddInvertedIndex.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/AutoAddInvertedIndex.java
@@ -38,15 +38,15 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.manager.zk.ZKHelixAdmin;
import org.apache.helix.manager.zk.ZNRecordSerializer;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.controller.helix.ControllerRequestURLBuilder;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.TimeFieldSpec;
-import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.spi.utils.JsonUtils;
-import org.apache.pinot.controller.helix.ControllerRequestURLBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -339,7 +339,7 @@ public class AutoAddInvertedIndex {
httpURLConnection.setRequestMethod("PUT");
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(httpURLConnection.getOutputStream(), "UTF-8"));
- writer.write(tableConfig.toJsonConfigString());
+ writer.write(tableConfig.toJsonString());
writer.flush();
BufferedReader reader = new BufferedReader(new InputStreamReader(httpURLConnection.getInputStream(), "UTF-8"));
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/SegmentIntervalUtils.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/SegmentIntervalUtils.java
index b9cfc11..44b2a2f 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/SegmentIntervalUtils.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/SegmentIntervalUtils.java
@@ -19,7 +19,7 @@
package org.apache.pinot.controller.util;
import org.apache.commons.lang3.StringUtils;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
import org.joda.time.Duration;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableRetentionValidator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableRetentionValidator.java
index 1b68e20..c451de4 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableRetentionValidator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableRetentionValidator.java
@@ -29,12 +29,13 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.manager.zk.ZKHelixAdmin;
import org.apache.helix.manager.zk.ZNRecordSerializer;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
+import org.apache.pinot.common.utils.config.TableConfigUtils;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.utils.TimeUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -184,7 +185,7 @@ public class TableRetentionValidator {
private TableConfig getTableConfig(String tableName)
throws Exception {
- return TableConfig.fromZnRecord(
+ return TableConfigUtils.fromZNRecord(
_propertyStore.get(ZKMetadataProvider.constructPropertyStorePathForResourceConfig(tableName), null, 0));
}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableSizeReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableSizeReader.java
index 8b9b624..8ef5b55 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableSizeReader.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableSizeReader.java
@@ -30,14 +30,14 @@ import javax.annotation.Nonnegative;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.commons.httpclient.HttpConnectionManager;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.restlet.resources.SegmentSizeInfo;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.api.resources.ServerTableSizeReader;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -79,11 +79,11 @@ public class TableSizeReader {
boolean hasRealtimeTable = false;
boolean hasOfflineTable = false;
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
if (tableType != null) {
- hasRealtimeTable = tableType == CommonConstants.Helix.TableType.REALTIME;
- hasOfflineTable = tableType == CommonConstants.Helix.TableType.OFFLINE;
+ hasRealtimeTable = tableType == TableType.REALTIME;
+ hasOfflineTable = tableType == TableType.OFFLINE;
} else {
hasRealtimeTable = _helixResourceManager.hasRealtimeTable(tableName);
hasOfflineTable = _helixResourceManager.hasOfflineTable(tableName);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
index a82a161..649542d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
@@ -21,12 +21,12 @@ package org.apache.pinot.controller.validation;
import java.util.List;
import java.util.Set;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
+import org.apache.pinot.spi.config.TableConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
index d73db4b..56e01ab 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
@@ -22,19 +22,19 @@ import com.google.common.annotations.VisibleForTesting;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.metrics.ValidationMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.spi.utils.TimeUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
import org.apache.pinot.controller.util.SegmentIntervalUtils;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.base.BaseInterval;
@@ -62,8 +62,8 @@ public class OfflineSegmentIntervalChecker extends ControllerPeriodicTask<Void>
@Override
protected void processTable(String tableNameWithType) {
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType);
- if (tableType == CommonConstants.Helix.TableType.OFFLINE) {
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType);
+ if (tableType == TableType.OFFLINE) {
TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType);
if (tableConfig == null) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
index 7e52296..fec1c74 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
@@ -22,12 +22,9 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.util.List;
import java.util.concurrent.TimeUnit;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.metrics.ValidationMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.HLCSegmentName;
import org.apache.pinot.common.utils.SegmentName;
import org.apache.pinot.controller.ControllerConf;
@@ -35,8 +32,11 @@ import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.stream.PartitionLevelStreamConfig;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -82,8 +82,8 @@ public class RealtimeSegmentValidationManager extends ControllerPeriodicTask<Rea
@Override
protected void processTable(String tableNameWithType, Context context) {
- CommonConstants.Helix.TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType);
- if (tableType == CommonConstants.Helix.TableType.REALTIME) {
+ TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType);
+ if (tableType == TableType.REALTIME) {
TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType);
if (tableConfig == null) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java
index 65b77ab..d3e7bae 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java
@@ -19,16 +19,13 @@
package org.apache.pinot.controller.validation;
import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import java.io.File;
-import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.QuotaConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.spi.utils.DataSize;
import org.apache.pinot.controller.util.TableSizeReader;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -72,18 +69,11 @@ public class StorageQuotaChecker {
}
/**
- * check if the segment represented by segmentFile is within the storage quota
- * @param segmentFile untarred segment. This should not be null.
- * segmentFile must exist on disk and must be a directory
- * @param segmentName name of the segment being added
- * @param timeoutMs timeout in milliseconds for reading table sizes from server
- *
+ * Returns whether the new added segment is within the storage quota.
*/
- public QuotaCheckerResponse isSegmentStorageWithinQuota(File segmentFile, String segmentName, int timeoutMs)
+ public QuotaCheckerResponse isSegmentStorageWithinQuota(String segmentName, long segmentSizeInBytes, int timeoutMs)
throws InvalidConfigException {
Preconditions.checkArgument(timeoutMs > 0, "Timeout value must be > 0, input: %s", timeoutMs);
- Preconditions.checkArgument(segmentFile.exists(), "Segment file: %s does not exist", segmentFile);
- Preconditions.checkArgument(segmentFile.isDirectory(), "Segment file: %s is not a directory", segmentFile);
// 1. Read table config
// 2. read table size from all the servers
@@ -93,7 +83,7 @@ public class StorageQuotaChecker {
int numReplicas = _tableConfig.getValidationConfig().getReplicationNumber();
final String tableNameWithType = _tableConfig.getTableName();
- if (quotaConfig == null || Strings.isNullOrEmpty(quotaConfig.getStorage())) {
+ if (quotaConfig == null || quotaConfig.getStorage() == null) {
// no quota configuration...so ignore for backwards compatibility
String message =
String.format("Storage quota is not configured for table: %s, skipping the check", tableNameWithType);
@@ -101,18 +91,9 @@ public class StorageQuotaChecker {
return success(message);
}
- long allowedStorageBytes = numReplicas * quotaConfig.storageSizeBytes();
- if (allowedStorageBytes <= 0) {
- String message = String
- .format("Invalid storage quota: %s for table: %s, skipping the check", quotaConfig.getStorage(),
- tableNameWithType);
- LOGGER.warn(message);
- return success(message);
- }
+ long allowedStorageBytes = numReplicas * quotaConfig.getStorageInBytes();
_controllerMetrics.setValueOfTableGauge(tableNameWithType, ControllerGauge.TABLE_QUOTA, allowedStorageBytes);
- long incomingSegmentSizeBytes = FileUtils.sizeOfDirectory(segmentFile);
-
// read table size
TableSizeReader.TableSubTypeSizeDetails tableSubtypeSize;
try {
@@ -130,9 +111,9 @@ public class StorageQuotaChecker {
if (tableSubtypeSize.missingSegments > 0) {
if (tableSubtypeSize.estimatedSizeInBytes > allowedStorageBytes) {
return failure(
- "Table " + tableNameWithType + " already over quota. Estimated size for all replicas is " + DataSize
+ "Table " + tableNameWithType + " already over quota. Estimated size for all replicas is " + DataSizeUtils
.fromBytes(tableSubtypeSize.estimatedSizeInBytes) + ". Configured size for " + numReplicas + " is "
- + DataSize.fromBytes(allowedStorageBytes));
+ + DataSizeUtils.fromBytes(allowedStorageBytes));
} else {
return success("Missing size report for " + tableSubtypeSize.missingSegments
+ " segments. Bypassing storage quota check for " + tableNameWithType);
@@ -162,7 +143,7 @@ public class StorageQuotaChecker {
// Note: incomingSegmentSizeBytes is uncompressed data size for just 1 replica,
// while estimatedFinalSizeBytes is for all replicas of all segments put together.
- long totalIncomingSegmentSizeBytes = incomingSegmentSizeBytes * numReplicas;
+ long totalIncomingSegmentSizeBytes = segmentSizeInBytes * numReplicas;
long estimatedFinalSizeBytes =
tableSubtypeSize.estimatedSizeInBytes - existingSegmentSizeBytes + totalIncomingSegmentSizeBytes;
if (estimatedFinalSizeBytes <= allowedStorageBytes) {
@@ -171,20 +152,20 @@ public class StorageQuotaChecker {
// append use case
message = String.format(
"Appending Segment %s of Table %s is within quota. Total allowed storage size: %s ( = configured quota: %s * number replicas: %d). New estimated table size of all replicas: %s. Current table size of all replicas: %s. Incoming uncompressed segment size of all replicas: %s ( = single incoming uncompressed segment size: %s * number replicas: %d). Formula: New estimated size = current table size + incoming segment size",
- segmentName, tableNameWithType, DataSize.fromBytes(allowedStorageBytes),
- DataSize.fromBytes(quotaConfig.storageSizeBytes()), numReplicas,
- DataSize.fromBytes(estimatedFinalSizeBytes), DataSize.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
- DataSize.fromBytes(totalIncomingSegmentSizeBytes), DataSize.fromBytes(incomingSegmentSizeBytes),
+ segmentName, tableNameWithType, DataSizeUtils.fromBytes(allowedStorageBytes), quotaConfig.getStorage(),
+ numReplicas, DataSizeUtils.fromBytes(estimatedFinalSizeBytes),
+ DataSizeUtils.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
+ DataSizeUtils.fromBytes(totalIncomingSegmentSizeBytes), DataSizeUtils.fromBytes(segmentSizeInBytes),
numReplicas);
} else {
// refresh use case
message = String.format(
"Refreshing Segment %s of Table %s is within quota. Total allowed storage size: %s ( = configured quota: %s * number replicas: %d). New estimated table size of all replicas: %s. Current table size of all replicas: %s. Incoming uncompressed segment size of all replicas: %s ( = single incoming uncompressed segment size: %s * number replicas: %d). Existing same segment size of all replicas: %s. Formula: New estimated size = current table size - existing same segment size + incom [...]
- segmentName, tableNameWithType, DataSize.fromBytes(allowedStorageBytes),
- DataSize.fromBytes(quotaConfig.storageSizeBytes()), numReplicas,
- DataSize.fromBytes(estimatedFinalSizeBytes), DataSize.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
- DataSize.fromBytes(totalIncomingSegmentSizeBytes), DataSize.fromBytes(incomingSegmentSizeBytes),
- numReplicas, DataSize.fromBytes(existingSegmentSizeBytes));
+ segmentName, tableNameWithType, DataSizeUtils.fromBytes(allowedStorageBytes), quotaConfig.getStorage(),
+ numReplicas, DataSizeUtils.fromBytes(estimatedFinalSizeBytes),
+ DataSizeUtils.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
+ DataSizeUtils.fromBytes(totalIncomingSegmentSizeBytes), DataSizeUtils.fromBytes(segmentSizeInBytes),
+ numReplicas, DataSizeUtils.fromBytes(existingSegmentSizeBytes));
}
LOGGER.info(message);
return success(message);
@@ -193,15 +174,16 @@ public class StorageQuotaChecker {
if (tableSubtypeSize.estimatedSizeInBytes > allowedStorageBytes) {
message = String.format(
"Table %s already over quota. Existing estimated uncompressed table size of all replicas: %s > total allowed storage size: %s ( = configured quota: %s * num replicas: %d). Check if indexes were enabled recently and adjust table quota accordingly.",
- tableNameWithType, DataSize.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
- DataSize.fromBytes(allowedStorageBytes), DataSize.fromBytes(quotaConfig.storageSizeBytes()), numReplicas);
+ tableNameWithType, DataSizeUtils.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
+ DataSizeUtils.fromBytes(allowedStorageBytes), quotaConfig.getStorage(), numReplicas);
} else {
message = String.format(
"Storage quota exceeded for Table %s. New estimated size: %s > total allowed storage size: %s, where new estimated size = existing estimated uncompressed size of all replicas: %s - existing segment sizes of all replicas: %s + (incoming uncompressed segment size: %s * number replicas: %d), total allowed storage size = configured quota: %s * number replicas: %d",
- tableNameWithType, DataSize.fromBytes(estimatedFinalSizeBytes), DataSize.fromBytes(allowedStorageBytes),
- DataSize.fromBytes(tableSubtypeSize.estimatedSizeInBytes), DataSize.fromBytes(existingSegmentSizeBytes),
- DataSize.fromBytes(incomingSegmentSizeBytes), numReplicas,
- DataSize.fromBytes(quotaConfig.storageSizeBytes()), numReplicas);
+ tableNameWithType, DataSizeUtils.fromBytes(estimatedFinalSizeBytes),
+ DataSizeUtils.fromBytes(allowedStorageBytes),
+ DataSizeUtils.fromBytes(tableSubtypeSize.estimatedSizeInBytes),
+ DataSizeUtils.fromBytes(existingSegmentSizeBytes), DataSizeUtils.fromBytes(segmentSizeInBytes), numReplicas,
+ quotaConfig.getStorage(), numReplicas);
}
LOGGER.warn(message);
return failure(message);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotFileUploadTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotFileUploadTest.java
index d0c7511..b3c07c7 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotFileUploadTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotFileUploadTest.java
@@ -23,9 +23,10 @@ import org.apache.http.HttpResponse;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.entity.StringEntity;
import org.apache.http.impl.client.DefaultHttpClient;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.helix.ControllerTest;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -48,7 +49,7 @@ public class PinotFileUploadTest extends ControllerTest {
addFakeServerInstancesToAutoJoinHelixCluster(5, true);
// Adding table
- TableConfig tableConfig = new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TABLE_NAME)
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
.setSegmentAssignmentStrategy("RandomAssignmentStrategy").setNumReplicas(2).build();
_helixResourceManager.addTable(tableConfig);
}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
index a2e3533..9fe105d 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
@@ -25,22 +25,23 @@ import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.Tenant;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.config.instance.InstanceReplicaGroupPartitionConfig;
-import org.apache.pinot.common.config.instance.InstanceTagPoolConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.common.utils.TenantRole;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.api.Tenant;
+import org.apache.pinot.spi.config.api.TenantRole;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
import org.apache.pinot.spi.data.FieldSpec.DataType;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -82,11 +83,11 @@ public class PinotInstanceAssignmentRestletResourceTest extends ControllerTest {
.build();
_helixResourceManager.addSchema(schema, true);
TableConfig offlineTableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setBrokerTenant(TENANT_NAME)
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setBrokerTenant(TENANT_NAME)
.setServerTenant(TENANT_NAME).build();
_helixResourceManager.addTable(offlineTableConfig);
TableConfig realtimeTableConfig =
- new TableConfig.Builder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setBrokerTenant(TENANT_NAME)
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setBrokerTenant(TENANT_NAME)
.setServerTenant(TENANT_NAME).setLLC(true)
.setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()).build();
_helixResourceManager.addTable(realtimeTableConfig);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceRestletResourceTest.java
index 6965e2a..fed6291 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceRestletResourceTest.java
@@ -26,18 +26,16 @@ import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.Instance;
import org.apache.pinot.common.utils.CommonConstants.Helix;
-import org.apache.pinot.common.utils.CommonConstants.Helix.InstanceType;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.controller.helix.ControllerTest;
+import org.apache.pinot.spi.config.api.Instance;
+import org.apache.pinot.spi.config.api.InstanceType;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.util.TestUtils;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
@@ -130,8 +128,7 @@ public class PinotInstanceRestletResourceTest extends ControllerTest {
@Override
public Boolean apply(@Nullable Void aVoid) {
try {
- String getResponse =
- sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation(instanceName));
+ String getResponse = sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation(instanceName));
JsonNode instance = JsonUtils.stringToJsonNode(getResponse);
boolean result =
(instance.get("instanceName") != null) && (instance.get("instanceName").asText().equals(instanceName))
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotSegmentRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotSegmentRestletResourceTest.java
index 330f0a0..d60b028 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotSegmentRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotSegmentRestletResourceTest.java
@@ -20,13 +20,14 @@ package org.apache.pinot.controller.api;
import java.util.HashMap;
import java.util.Map;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -55,8 +56,7 @@ public class PinotSegmentRestletResourceTest extends ControllerTest {
throws Exception {
// Adding table
TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TABLE_NAME).setNumReplicas(1)
- .build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setNumReplicas(1).build();
_helixResourceManager.addTable(tableConfig);
// Wait for the table addition
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
index 8e77387..4bac330 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
@@ -19,18 +19,20 @@
package org.apache.pinot.controller.api;
import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
import java.io.FileNotFoundException;
import java.io.IOException;
-import org.apache.pinot.common.config.QuotaConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.common.utils.StringUtil;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceResult;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -48,8 +50,8 @@ public class PinotTableRestletResourceTest extends ControllerTest {
private static final String OFFLINE_TABLE_NAME = "testOfflineTable";
private static final String REALTIME_TABLE_NAME = "testRealtimeTable";
- private final TableConfig.Builder _offlineBuilder = new TableConfig.Builder(TableType.OFFLINE);
- private final TableConfig.Builder _realtimeBuilder = new TableConfig.Builder(TableType.REALTIME);
+ private final TableConfigBuilder _offlineBuilder = new TableConfigBuilder(TableType.OFFLINE);
+ private final TableConfigBuilder _realtimeBuilder = new TableConfigBuilder(TableType.REALTIME);
private String _createTableUrl;
@BeforeClass
@@ -79,11 +81,12 @@ public class PinotTableRestletResourceTest extends ControllerTest {
public void testCreateTable()
throws Exception {
// Create an OFFLINE table with an invalid name which should fail
- // NOTE: Set bad table name inside table config builder is not allowed, so have to explicitly set in table config
+ // NOTE: Set bad table name inside table config builder is not allowed, so have to set in json node
TableConfig offlineTableConfig = _offlineBuilder.build();
- offlineTableConfig.setTableName("bad__table__name");
+ ObjectNode offlineTableConfigJson = (ObjectNode) offlineTableConfig.toJsonNode();
+ offlineTableConfigJson.put(TableConfig.TABLE_NAME_KEY, "bad__table__name");
try {
- sendPostRequest(_createTableUrl, offlineTableConfig.toJsonConfigString());
+ sendPostRequest(_createTableUrl, offlineTableConfigJson.toString());
Assert.fail("Creation of an OFFLINE table with two underscores in the table name does not fail");
} catch (IOException e) {
// Expected 400 Bad Request
@@ -91,13 +94,13 @@ public class PinotTableRestletResourceTest extends ControllerTest {
}
// Create an OFFLINE table with a valid name which should succeed
- offlineTableConfig.setTableName("valid_table_name");
- String offlineTableJSONConfigString = offlineTableConfig.toJsonConfigString();
- sendPostRequest(_createTableUrl, offlineTableJSONConfigString);
+ offlineTableConfig = _offlineBuilder.setTableName("valid_table_name").build();
+ String offlineTableConfigString = offlineTableConfig.toJsonString();
+ sendPostRequest(_createTableUrl, offlineTableConfigString);
// Create an OFFLINE table that already exists which should fail
try {
- sendPostRequest(_createTableUrl, offlineTableJSONConfigString);
+ sendPostRequest(_createTableUrl, offlineTableConfigString);
Assert.fail("Creation of an existing OFFLINE table does not fail");
} catch (IOException e) {
// Expected 409 Conflict
@@ -106,9 +109,8 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Create an OFFLINE table with invalid replication config
offlineTableConfig.getValidationConfig().setReplication("abc");
- offlineTableConfig.setTableName("invalid_replication_table");
try {
- sendPostRequest(_createTableUrl, offlineTableConfig.toJsonConfigString());
+ sendPostRequest(_createTableUrl, offlineTableConfig.toJsonString());
Assert.fail("Creation of an invalid OFFLINE table does not fail");
} catch (IOException e) {
// Expected 400 Bad Request
@@ -116,11 +118,12 @@ public class PinotTableRestletResourceTest extends ControllerTest {
}
// Create a REALTIME table with an invalid name which should fail
- // NOTE: Set bad table name inside table config builder is not allowed, so have to explicitly set in table config
+ // NOTE: Set bad table name inside table config builder is not allowed, so have to set in json node
TableConfig realtimeTableConfig = _realtimeBuilder.build();
- realtimeTableConfig.setTableName("bad__table__name");
+ ObjectNode realtimeTableConfigJson = (ObjectNode) realtimeTableConfig.toJsonNode();
+ realtimeTableConfigJson.put(TableConfig.TABLE_NAME_KEY, "bad__table__name");
try {
- sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonConfigString());
+ sendPostRequest(_createTableUrl, realtimeTableConfigJson.toString());
Assert.fail("Creation of a REALTIME table with two underscores in the table name does not fail");
} catch (IOException e) {
// Expected 400 Bad Request
@@ -128,10 +131,9 @@ public class PinotTableRestletResourceTest extends ControllerTest {
}
// Creating a REALTIME table without a valid schema should fail
- _realtimeBuilder.setSchemaName("invalidSchemaName");
- TableConfig invalidConfig = _realtimeBuilder.build();
+ realtimeTableConfig = _realtimeBuilder.setTableName("noSchema").setSchemaName("noSchema").build();
try {
- sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonConfigString());
+ sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonString());
Assert.fail("Creation of a REALTIME table without a valid schema does not fail");
} catch (IOException e) {
// Expected 400 Bad Request
@@ -139,24 +141,17 @@ public class PinotTableRestletResourceTest extends ControllerTest {
}
// Creating a REALTIME table with a different schema name in the config should succeed (backwards compatibility mode)
- String schemaName = "differentRTSchema";
- _realtimeBuilder.setSchemaName(schemaName);
- // use a different table name so it doesn't associate a previously uploaded schema with the table
- _realtimeBuilder.setTableName("RT_TABLE");
- addDummySchema(schemaName);
- TableConfig diffConfig = _realtimeBuilder.build();
- sendPostRequest(_createTableUrl, diffConfig.toJsonConfigString());
+ realtimeTableConfig = _realtimeBuilder.setSchemaName(REALTIME_TABLE_NAME).build();
+ sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonString());
// Create a REALTIME table with a valid name and schema which should succeed
- _realtimeBuilder.setTableName(REALTIME_TABLE_NAME);
- _realtimeBuilder.setSchemaName(REALTIME_TABLE_NAME);
- TableConfig config = _realtimeBuilder.build();
- String realtimeTableJSONConfigString = config.toJsonConfigString();
- sendPostRequest(_createTableUrl, realtimeTableJSONConfigString);
+ realtimeTableConfig = _realtimeBuilder.setTableName(REALTIME_TABLE_NAME).build();
+ String realtimeTableConfigString = realtimeTableConfig.toJsonString();
+ sendPostRequest(_createTableUrl, realtimeTableConfigString);
// TODO: check whether we should allow POST request to create REALTIME table that already exists
// Create a REALTIME table that already exists which should succeed
- sendPostRequest(_createTableUrl, realtimeTableJSONConfigString);
+ sendPostRequest(_createTableUrl, realtimeTableConfigString);
}
@Test
@@ -169,7 +164,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
private void testTableMinReplicationInternal(String tableName, int tableReplication)
throws Exception {
String tableJSONConfigString =
- _offlineBuilder.setTableName(tableName).setNumReplicas(tableReplication).build().toJsonConfigString();
+ _offlineBuilder.setTableName(tableName).setNumReplicas(tableReplication).build().toJsonString();
sendPostRequest(_createTableUrl, tableJSONConfigString);
// table creation should succeed
TableConfig tableConfig = getTableConfig(tableName, "OFFLINE");
@@ -178,7 +173,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
addDummySchema(tableName);
tableJSONConfigString =
- _realtimeBuilder.setTableName(tableName).setNumReplicas(tableReplication).build().toJsonConfigString();
+ _realtimeBuilder.setTableName(tableName).setNumReplicas(tableReplication).build().toJsonString();
sendPostRequest(_createTableUrl, tableJSONConfigString);
tableConfig = getTableConfig(tableName, "REALTIME");
Assert.assertEquals(tableConfig.getValidationConfig().getReplicationNumber(),
@@ -191,16 +186,15 @@ public class PinotTableRestletResourceTest extends ControllerTest {
private TableConfig getTableConfig(String tableName, String tableType)
throws Exception {
String tableConfigString = sendGetRequest(_controllerRequestURLBuilder.forTableGet(tableName));
- return TableConfig.fromJsonConfig(JsonUtils.stringToJsonNode(tableConfigString).get(tableType));
+ return JsonUtils.jsonNodeToObject(JsonUtils.stringToJsonNode(tableConfigString).get(tableType), TableConfig.class);
}
@Test
public void testUpdateTableConfig()
throws Exception {
String tableName = "updateTC";
- String tableJSONConfigString =
- _offlineBuilder.setTableName(tableName).setNumReplicas(2).build().toJsonConfigString();
- sendPostRequest(_createTableUrl, tableJSONConfigString);
+ String tableConfigString = _offlineBuilder.setTableName(tableName).setNumReplicas(2).build().toJsonString();
+ sendPostRequest(_createTableUrl, tableConfigString);
// table creation should succeed
TableConfig tableConfig = getTableConfig(tableName, "OFFLINE");
Assert.assertEquals(tableConfig.getValidationConfig().getRetentionTimeValue(), "5");
@@ -210,7 +204,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
tableConfig.getValidationConfig().setRetentionTimeValue("10");
JsonNode jsonResponse = JsonUtils.stringToJsonNode(
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonConfigString()));
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonString()));
Assert.assertTrue(jsonResponse.has("status"));
TableConfig modifiedConfig = getTableConfig(tableName, "OFFLINE");
@@ -219,27 +213,27 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Realtime
addDummySchema(tableName);
- tableJSONConfigString = _realtimeBuilder.setTableName(tableName).setNumReplicas(2).build().toJsonConfigString();
- sendPostRequest(_createTableUrl, tableJSONConfigString);
+ tableConfigString = _realtimeBuilder.setTableName(tableName).setNumReplicas(2).build().toJsonString();
+ sendPostRequest(_createTableUrl, tableConfigString);
tableConfig = getTableConfig(tableName, "REALTIME");
Assert.assertEquals(tableConfig.getValidationConfig().getRetentionTimeValue(), "5");
Assert.assertEquals(tableConfig.getValidationConfig().getRetentionTimeUnit(), "DAYS");
Assert.assertNull(tableConfig.getQuotaConfig());
- QuotaConfig quota = new QuotaConfig("10G", "100.00");
+ QuotaConfig quota = new QuotaConfig("10G", "100.0");
tableConfig.setQuotaConfig(quota);
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonConfigString());
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonString());
modifiedConfig = getTableConfig(tableName, "REALTIME");
Assert.assertNotNull(modifiedConfig.getQuotaConfig());
Assert.assertEquals(modifiedConfig.getQuotaConfig().getStorage(), "10G");
- Assert.assertEquals(modifiedConfig.getQuotaConfig().getMaxQueriesPerSecond(), "100.00");
+ Assert.assertEquals(modifiedConfig.getQuotaConfig().getMaxQueriesPerSecond(), "100.0");
boolean notFoundException = false;
try {
// table does not exist
- tableConfig.setTableName("noSuchTable_REALTIME");
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig("noSuchTable"),
- tableConfig.toJsonConfigString());
+ ObjectNode tableConfigJson = (ObjectNode) tableConfig.toJsonNode();
+ tableConfigJson.put(TableConfig.TABLE_NAME_KEY, "noSuchTable_REALTIME");
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig("noSuchTable"), tableConfigJson.toString());
} catch (Exception e) {
Assert.assertTrue(e instanceof FileNotFoundException);
notFoundException = true;
@@ -257,7 +251,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
public void rebalanceTableWithoutSegments()
throws Exception {
// Create the table
- sendPostRequest(_createTableUrl, _offlineBuilder.build().toJsonConfigString());
+ sendPostRequest(_createTableUrl, _offlineBuilder.build().toJsonString());
// Rebalance should return status NO_OP
RebalanceResult rebalanceResult = JsonUtils.stringToObject(
@@ -271,7 +265,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
throws IOException {
// Case 1: Create a REALTIME table and delete it directly w/o using query param.
TableConfig realtimeTableConfig = _realtimeBuilder.setTableName("table0").build();
- String creationResponse = sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonConfigString());
+ String creationResponse = sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table0_REALTIME succesfully added\"}");
// Delete realtime table using REALTIME suffix.
@@ -281,7 +275,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Case 2: Create an offline table and delete it directly w/o using query param.
TableConfig offlineTableConfig = _offlineBuilder.setTableName("table0").build();
- creationResponse = sendPostRequest(_createTableUrl, offlineTableConfig.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, offlineTableConfig.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table0_OFFLINE succesfully added\"}");
// Delete offline table using OFFLINE suffix.
@@ -290,11 +284,11 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Case 3: Create REALTIME and OFFLINE tables and delete both of them.
TableConfig rtConfig1 = _realtimeBuilder.setTableName("table1").build();
- creationResponse = sendPostRequest(_createTableUrl, rtConfig1.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, rtConfig1.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table1_REALTIME succesfully added\"}");
TableConfig offlineConfig1 = _offlineBuilder.setTableName("table1").build();
- creationResponse = sendPostRequest(_createTableUrl, offlineConfig1.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, offlineConfig1.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table1_OFFLINE succesfully added\"}");
deleteResponse = sendDeleteRequest(StringUtil.join("/", this._controllerBaseApiUrl, "tables", "table1"));
@@ -302,11 +296,11 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Case 4: Create REALTIME and OFFLINE tables and delete the realtime/offline table using query params.
TableConfig rtConfig2 = _realtimeBuilder.setTableName("table2").build();
- creationResponse = sendPostRequest(_createTableUrl, rtConfig2.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, rtConfig2.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table2_REALTIME succesfully added\"}");
TableConfig offlineConfig2 = _offlineBuilder.setTableName("table2").build();
- creationResponse = sendPostRequest(_createTableUrl, offlineConfig2.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, offlineConfig2.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table2_OFFLINE succesfully added\"}");
// The conflict between param type and table name suffix causes no table being deleted.
@@ -336,11 +330,11 @@ public class PinotTableRestletResourceTest extends ControllerTest {
// Case 6: Create REALTIME and OFFLINE tables and delete the realtime/offline table using query params and suffixes.
TableConfig rtConfig3 = _realtimeBuilder.setTableName("table3").build();
- creationResponse = sendPostRequest(_createTableUrl, rtConfig3.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, rtConfig3.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table3_REALTIME succesfully added\"}");
TableConfig offlineConfig3 = _offlineBuilder.setTableName("table3").build();
- creationResponse = sendPostRequest(_createTableUrl, offlineConfig3.toJsonConfigString());
+ creationResponse = sendPostRequest(_createTableUrl, offlineConfig3.toJsonString());
Assert.assertEquals(creationResponse, "{\"status\":\"Table table3_OFFLINE succesfully added\"}");
deleteResponse =
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTenantRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTenantRestletResourceTest.java
index fcda2c2..fae611b 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTenantRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTenantRestletResourceTest.java
@@ -19,11 +19,11 @@
package org.apache.pinot.controller.api;
import com.fasterxml.jackson.databind.JsonNode;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.helix.ControllerTest;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -54,7 +54,7 @@ public class PinotTenantRestletResourceTest extends ControllerTest {
// Add a table
sendPostRequest(_controllerRequestURLBuilder.forTableCreate(),
- new TableConfig.Builder(TableType.OFFLINE).setTableName("testTable").build().toJsonConfigString());
+ new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").build().toJsonString());
// There should be 1 table on the tenant
tableList = JsonUtils.stringToJsonNode(sendGetRequest(listTablesUrl));
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableSizeReaderTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableSizeReaderTest.java
index 53ec5d4..71f054a 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableSizeReaderTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableSizeReaderTest.java
@@ -36,15 +36,15 @@ import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import org.apache.commons.httpclient.HttpConnectionManager;
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.restlet.resources.SegmentSizeInfo;
import org.apache.pinot.common.restlet.resources.TableSizeInfo;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.util.TableSizeReader;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.mockito.ArgumentMatchers;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableViewsTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableViewsTest.java
index 0aa2576..a0f7298 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableViewsTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableViewsTest.java
@@ -22,15 +22,17 @@ import java.net.HttpURLConnection;
import java.net.URL;
import java.util.Map;
import org.apache.helix.InstanceType;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.controller.api.resources.TableViews;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -55,23 +57,21 @@ public class TableViewsTest extends ControllerTest {
// Create the offline table and add one segment
TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(OFFLINE_TABLE_NAME)
- .setNumReplicas(2).build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(OFFLINE_TABLE_NAME).setNumReplicas(2).build();
Assert.assertEquals(_helixManager.getInstanceType(), InstanceType.CONTROLLER);
_helixResourceManager.addTable(tableConfig);
_helixResourceManager.addNewSegment(OFFLINE_TABLE_NAME,
SegmentMetadataMockUtils.mockSegmentMetadata(OFFLINE_TABLE_NAME, OFFLINE_SEGMENT_NAME), "downloadUrl");
// Create the hybrid table
- tableConfig = new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(HYBRID_TABLE_NAME)
- .setNumReplicas(2).build();
+ tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(HYBRID_TABLE_NAME).setNumReplicas(2).build();
_helixResourceManager.addTable(tableConfig);
// add schema for realtime table
addDummySchema(HYBRID_TABLE_NAME);
StreamConfig streamConfig = FakeStreamConfigUtils.getDefaultHighLevelStreamConfigs();
- tableConfig = new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(HYBRID_TABLE_NAME)
- .setNumReplicas(2).setStreamConfigs(streamConfig.getStreamConfigsMap()).build();
+ tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(HYBRID_TABLE_NAME).setNumReplicas(2)
+ .setStreamConfigs(streamConfig.getStreamConfigsMap()).build();
_helixResourceManager.addTable(tableConfig);
// Wait for external view get updated
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/upload/ZKOperatorTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/upload/ZKOperatorTest.java
index 3d28fd3..31985fe 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/upload/ZKOperatorTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/upload/ZKOperatorTest.java
@@ -19,14 +19,15 @@
package org.apache.pinot.controller.api.upload;
import javax.ws.rs.core.HttpHeaders;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -51,8 +52,7 @@ public class ZKOperatorTest extends ControllerTest {
addFakeBrokerInstancesToAutoJoinHelixCluster(1, true);
addFakeServerInstancesToAutoJoinHelixCluster(1, true);
- TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
_helixResourceManager.addTable(tableConfig);
}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerInstanceToggleTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerInstanceToggleTest.java
index 2ccf9fd..af9039e 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerInstanceToggleTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerInstanceToggleTest.java
@@ -21,12 +21,14 @@ package org.apache.pinot.controller.helix;
import java.io.IOException;
import java.util.Set;
import org.apache.helix.model.ExternalView;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
@@ -55,10 +57,9 @@ public class ControllerInstanceToggleTest extends ControllerTest {
public void testInstanceToggle()
throws Exception {
// Create an offline table
- String tableJSONConfigString =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
- .setNumReplicas(NUM_INSTANCES).build().toJsonConfigString();
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableJSONConfigString);
+ TableConfig tableConfig =
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_INSTANCES).build();
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonString());
Assert.assertEquals(
_helixAdmin.getResourceIdealState(getHelixClusterName(), CommonConstants.Helix.BROKER_RESOURCE_INSTANCE)
.getPartitionSet().size(), 1);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerSentinelTestV2.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerSentinelTestV2.java
index e00eea8..3057d48 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerSentinelTestV2.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerSentinelTestV2.java
@@ -19,10 +19,12 @@
package org.apache.pinot.controller.helix;
import java.io.IOException;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -45,10 +47,9 @@ public class ControllerSentinelTestV2 extends ControllerTest {
throws IOException {
// Create offline table creation request
String tableName = "testTable";
- String tableJSONConfigString =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(tableName).setNumReplicas(3)
- .build().toJsonConfigString();
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableJSONConfigString);
+ TableConfig tableConfig =
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setNumReplicas(3).build();
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonString());
Assert.assertEquals(
_helixAdmin.getResourceIdealState(getHelixClusterName(), CommonConstants.Helix.BROKER_RESOURCE_INSTANCE)
.getPartitionSet().size(), 1);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTenantTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTenantTest.java
index c2406d3..fa82432 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTenantTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTenantTest.java
@@ -20,9 +20,9 @@ package org.apache.pinot.controller.helix;
import com.fasterxml.jackson.databind.JsonNode;
import java.io.IOException;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.spi.utils.JsonUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
index 99263e0..faa36dd 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
@@ -59,18 +59,18 @@ import org.apache.helix.participant.statemachine.StateModelFactory;
import org.apache.helix.participant.statemachine.StateModelInfo;
import org.apache.helix.participant.statemachine.Transition;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.Tenant;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.spi.data.DimensionFieldSpec;
-import org.apache.pinot.spi.data.FieldSpec;
-import org.apache.pinot.spi.data.MetricFieldSpec;
-import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.common.utils.TenantRole;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.ControllerStarter;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.api.Tenant;
+import org.apache.pinot.spi.config.api.TenantRole;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.MetricFieldSpec;
+import org.apache.pinot.spi.data.Schema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/PinotResourceManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/PinotResourceManagerTest.java
index 06b5de8..53e32de 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/PinotResourceManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/PinotResourceManagerTest.java
@@ -23,11 +23,12 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
@@ -54,8 +55,7 @@ public class PinotResourceManagerTest extends ControllerTest {
1);
// Adding table
- TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TABLE_NAME).build();
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).build();
_helixResourceManager.addTable(tableConfig);
}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java
index 066040e..f61b064 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java
@@ -28,7 +28,6 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
@@ -37,6 +36,7 @@ import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.LeadControllerManager;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.Assert;
import org.testng.annotations.Test;
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerTest.java
index ed0d8ca..9e2d190 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerTest.java
@@ -40,23 +40,25 @@ import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
import org.apache.helix.model.InstanceConfig;
import org.apache.helix.model.MasterSlaveSMD;
-import org.apache.pinot.common.config.Instance;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.TagOverrideConfig;
-import org.apache.pinot.common.config.Tenant;
-import org.apache.pinot.common.config.TenantConfig;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
-import org.apache.pinot.common.utils.TenantRole;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.LeadControllerUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.helix.ControllerTest;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TagOverrideConfig;
+import org.apache.pinot.spi.config.TenantConfig;
+import org.apache.pinot.spi.config.api.Instance;
+import org.apache.pinot.spi.config.api.InstanceType;
+import org.apache.pinot.spi.config.api.Tenant;
+import org.apache.pinot.spi.config.api.TenantRole;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
@@ -171,7 +173,7 @@ public class PinotHelixResourceManagerTest extends ControllerTest {
Assert.assertFalse(instances.contains(instanceName));
// Add new instance.
- Instance instance = new Instance("localhost", biggerRandomNumber, CommonConstants.Helix.InstanceType.SERVER,
+ Instance instance = new Instance("localhost", biggerRandomNumber, InstanceType.SERVER,
Collections.singletonList(UNTAGGED_SERVER_INSTANCE), null);
_helixResourceManager.addInstance(instance);
@@ -194,7 +196,7 @@ public class PinotHelixResourceManagerTest extends ControllerTest {
Assert.assertTrue(response.isSuccessful());
// Create the table
- TableConfig tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName(TABLE_NAME).setNumReplicas(3)
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setNumReplicas(3)
.setBrokerTenant(BROKER_TENANT_NAME).setServerTenant(SERVER_TENANT_NAME).build();
_helixResourceManager.addTable(tableConfig);
@@ -312,7 +314,7 @@ public class PinotHelixResourceManagerTest extends ControllerTest {
_helixResourceManager.createBrokerTenant(brokerTenant);
String rawTableName = "testTable";
- TableConfig offlineTableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName(rawTableName).build();
+ TableConfig offlineTableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(rawTableName).build();
// Empty broker tag (DefaultTenant_BROKER)
try {
@@ -336,7 +338,7 @@ public class PinotHelixResourceManagerTest extends ControllerTest {
_helixResourceManager.validateTableTenantConfig(offlineTableConfig);
TableConfig realtimeTableConfig =
- new TableConfig.Builder(TableType.REALTIME).setTableName(rawTableName).setBrokerTenant(BROKER_TENANT_NAME)
+ new TableConfigBuilder(TableType.REALTIME).setTableName(rawTableName).setBrokerTenant(BROKER_TENANT_NAME)
.setServerTenant(SERVER_TENANT_NAME).build();
// Empty server tag (serverTenant_REALTIME)
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentTest.java
index 2fdf175..e3a4581 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/instance/InstanceAssignmentTest.java
@@ -23,20 +23,21 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.assignment.InstanceAssignmentConfigUtils;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.Instance;
-import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfigUtils;
-import org.apache.pinot.common.config.instance.InstanceReplicaGroupPartitionConfig;
-import org.apache.pinot.common.config.instance.InstanceTagPoolConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.AssignmentStrategy;
+import org.apache.pinot.common.utils.config.InstanceUtils;
+import org.apache.pinot.common.utils.config.TagNameUtils;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.ReplicaGroupStrategyConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.Test;
import static org.testng.Assert.assertEquals;
@@ -54,7 +55,7 @@ public class InstanceAssignmentTest {
public void testDefaultOfflineReplicaGroup() {
int numReplicas = 3;
TableConfig tableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setServerTenant(TENANT_NAME)
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setServerTenant(TENANT_NAME)
.setNumReplicas(numReplicas)
.setSegmentAssignmentStrategy(AssignmentStrategy.REPLICA_GROUP_SEGMENT_ASSIGNMENT_STRATEGY).build();
int numInstancesPerPartition = 2;
@@ -136,7 +137,7 @@ public class InstanceAssignmentTest {
instanceConfig.addTag(OFFLINE_TAG);
int pool = i / numInstancesPerPool;
instanceConfig.getRecord()
- .setMapField(Instance.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, Integer.toString(pool)));
+ .setMapField(InstanceUtils.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, Integer.toString(pool)));
instanceConfigs.add(instanceConfig);
}
@@ -146,7 +147,7 @@ public class InstanceAssignmentTest {
int numReplicaGroups = numPools;
InstanceReplicaGroupPartitionConfig replicaPartitionConfig =
new InstanceReplicaGroupPartitionConfig(true, 0, numReplicaGroups, 0, 0, 0);
- TableConfig tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME)
.setInstanceAssignmentConfigMap(Collections.singletonMap(InstancePartitionsType.OFFLINE,
new InstanceAssignmentConfig(tagPoolConfig, null, replicaPartitionConfig))).build();
InstanceAssignmentDriver driver = new InstanceAssignmentDriver(tableConfig);
@@ -172,7 +173,7 @@ public class InstanceAssignmentTest {
instanceConfig.addTag(OFFLINE_TAG);
int pool = numPools - 1;
instanceConfig.getRecord()
- .setMapField(Instance.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, Integer.toString(pool)));
+ .setMapField(InstanceUtils.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, Integer.toString(pool)));
instanceConfigs.add(instanceConfig);
}
@@ -256,7 +257,7 @@ public class InstanceAssignmentTest {
@Test
public void testIllegalConfig() {
- TableConfig tableConfig = new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
InstanceAssignmentDriver driver = new InstanceAssignmentDriver(tableConfig);
int numInstances = 10;
@@ -319,9 +320,9 @@ public class InstanceAssignmentTest {
for (int i = 0; i < numInstances; i++) {
InstanceConfig instanceConfig = instanceConfigs.get(i);
if (i < numInstances / 2) {
- instanceConfig.getRecord().setMapField(Instance.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, "0"));
+ instanceConfig.getRecord().setMapField(InstanceUtils.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, "0"));
} else {
- instanceConfig.getRecord().setMapField(Instance.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, "1"));
+ instanceConfig.getRecord().setMapField(InstanceUtils.POOL_KEY, Collections.singletonMap(OFFLINE_TAG, "1"));
}
}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupSegmentAssignmentTest.java
index 1ead5ff..6ecfbd0 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupSegmentAssignmentTest.java
@@ -24,10 +24,11 @@ import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.SegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -55,7 +56,7 @@ public class OfflineNonReplicaGroupSegmentAssignmentTest {
@BeforeClass
public void setUp() {
TableConfig tableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS).build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS).build();
_segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(null, tableConfig);
// {
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineReplicaGroupSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineReplicaGroupSegmentAssignmentTest.java
index 8fbd52e..e2f2c5a 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineReplicaGroupSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineReplicaGroupSegmentAssignmentTest.java
@@ -28,17 +28,18 @@ import org.apache.helix.HelixManager;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.SegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.AssignmentStrategy;
+import org.apache.pinot.spi.config.ReplicaGroupStrategyConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -80,7 +81,7 @@ public class OfflineReplicaGroupSegmentAssignmentTest {
@BeforeClass
public void setUp() {
TableConfig tableConfigWithoutPartitions =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME_WITHOUT_PARTITION)
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME_WITHOUT_PARTITION)
.setNumReplicas(NUM_REPLICAS)
.setSegmentAssignmentStrategy(AssignmentStrategy.REPLICA_GROUP_SEGMENT_ASSIGNMENT_STRATEGY).build();
_segmentAssignmentWithoutPartition =
@@ -132,7 +133,7 @@ public class OfflineReplicaGroupSegmentAssignmentTest {
ReplicaGroupStrategyConfig replicaGroupStrategyConfig =
new ReplicaGroupStrategyConfig(PARTITION_COLUMN, numInstancesPerPartition);
TableConfig tableConfigWithPartitions =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME_WITH_PARTITION)
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME_WITH_PARTITION)
.setNumReplicas(NUM_REPLICAS)
.setSegmentAssignmentStrategy(AssignmentStrategy.REPLICA_GROUP_SEGMENT_ASSIGNMENT_STRATEGY).build();
tableConfigWithPartitions.getValidationConfig().setReplicaGroupStrategyConfig(replicaGroupStrategyConfig);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupSegmentAssignmentTest.java
index ef8b044..f6e8b42 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupSegmentAssignmentTest.java
@@ -24,12 +24,13 @@ import java.util.Map;
import java.util.TreeMap;
import org.apache.commons.configuration.BaseConfiguration;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -68,8 +69,8 @@ public class RealtimeNonReplicaGroupSegmentAssignmentTest {
}
TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(RAW_TABLE_NAME)
- .setNumReplicas(NUM_REPLICAS).setLLC(true).build();
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS)
+ .setLLC(true).build();
_segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(null, tableConfig);
_instancePartitionsMap = new TreeMap<>();
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeReplicaGroupSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeReplicaGroupSegmentAssignmentTest.java
index 92f728a..80558eb 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeReplicaGroupSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeReplicaGroupSegmentAssignmentTest.java
@@ -25,13 +25,14 @@ import java.util.Map;
import java.util.TreeMap;
import org.apache.commons.configuration.BaseConfiguration;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.AssignmentStrategy;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -70,7 +71,7 @@ public class RealtimeReplicaGroupSegmentAssignmentTest {
}
TableConfig tableConfig =
- new TableConfig.Builder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS)
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS)
.setLLC(true).setSegmentAssignmentStrategy(AssignmentStrategy.REPLICA_GROUP_SEGMENT_ASSIGNMENT_STRATEGY)
.build();
_segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(null, tableConfig);
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
index a44912c..2470fe8 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
@@ -35,14 +35,10 @@ import org.apache.commons.configuration.BaseConfiguration;
import org.apache.commons.io.FileUtils;
import org.apache.helix.model.IdealState;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.controller.ControllerConf;
@@ -53,10 +49,15 @@ import org.apache.pinot.controller.util.SegmentCompletionUtils;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
import org.apache.pinot.spi.stream.OffsetCriteria;
import org.apache.pinot.spi.stream.PartitionLevelStreamConfig;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.zookeeper.data.Stat;
import org.joda.time.Interval;
import org.testng.annotations.AfterClass;
@@ -794,7 +795,7 @@ public class PinotLLCRealtimeSegmentManagerTest {
void makeTableConfig() {
Map<String, String> streamConfigs = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap();
_tableConfig =
- new TableConfig.Builder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(_numReplicas)
+ new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(_numReplicas)
.setLLC(true).setStreamConfigs(streamConfigs).build();
_streamConfig = new PartitionLevelStreamConfig(_tableConfig.getTableName(),
_tableConfig.getIndexingConfig().getStreamConfigs());
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
index a81dce6..42d2358 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
@@ -19,11 +19,11 @@
package org.apache.pinot.controller.helix.core.realtime.segment;
import java.util.Arrays;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.spi.stream.PartitionLevelStreamConfig;
import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.annotations.Test;
import static org.mockito.Mockito.mock;
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterTest.java
index c60cc84..37ab0a5 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterTest.java
@@ -24,18 +24,19 @@ import java.util.Map;
import org.apache.commons.configuration.BaseConfiguration;
import org.apache.commons.configuration.Configuration;
import org.apache.pinot.common.assignment.InstancePartitions;
-import org.apache.pinot.common.assignment.InstancePartitionsType;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.instance.InstanceAssignmentConfig;
-import org.apache.pinot.common.config.instance.InstanceReplicaGroupPartitionConfig;
-import org.apache.pinot.common.config.instance.InstanceTagPoolConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.config.assignment.InstanceReplicaGroupPartitionConfig;
+import org.apache.pinot.spi.config.assignment.InstanceTagPoolConfig;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -80,7 +81,7 @@ public class TableRebalancerClusterTest extends ControllerTest {
TableRebalancer tableRebalancer = new TableRebalancer(_helixManager);
TableConfig tableConfig =
- new TableConfig.Builder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS).build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS).build();
// Rebalance should fail without creating the table
RebalanceResult rebalanceResult = tableRebalancer.rebalance(tableConfig, new BaseConfiguration());
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java
index de07131..ba91bce 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java
@@ -25,8 +25,6 @@ import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.helix.HelixAdmin;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
@@ -40,6 +38,10 @@ import org.apache.pinot.controller.helix.core.PinotTableIdealStateBuilder;
import org.apache.pinot.controller.helix.core.SegmentDeletionManager;
import org.apache.pinot.controller.helix.core.util.ZKMetadataUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.mockito.ArgumentMatchers;
@@ -153,14 +155,13 @@ public class RetentionManagerTest {
testDifferentTimeUnits(pastDaysSinceEpoch, TimeUnit.DAYS, daysSinceEpochTimeStamp);
}
- private TableConfig createOfflineTableConfig()
- throws Exception {
- return new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TEST_TABLE_NAME)
- .setRetentionTimeUnit("DAYS").setRetentionTimeValue("365").setNumReplicas(2).build();
+ private TableConfig createOfflineTableConfig() {
+ return new TableConfigBuilder(TableType.OFFLINE).setTableName(TEST_TABLE_NAME).setRetentionTimeUnit("DAYS")
+ .setRetentionTimeValue("365").setNumReplicas(2).build();
}
private TableConfig createRealtimeTableConfig1(int replicaCount) {
- return new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(TEST_TABLE_NAME).setLLC(true)
+ return new TableConfigBuilder(TableType.REALTIME).setTableName(TEST_TABLE_NAME).setLLC(true)
.setRetentionTimeUnit("DAYS").setRetentionTimeValue("5").setNumReplicas(replicaCount).build();
}
@@ -197,7 +198,7 @@ public class RetentionManagerTest {
}
return null;
}
- }).when(resourceManager).deleteSegments(anyString(), ArgumentMatchers.<String>anyList());
+ }).when(resourceManager).deleteSegments(anyString(), ArgumentMatchers.anyList());
}
// This test makes sure that we clean up the segments marked OFFLINE in realtime for more than 7 days
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/validation/StorageQuotaCheckerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/validation/StorageQuotaCheckerTest.java
index a4a2a3d..f76bfb4 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/validation/StorageQuotaCheckerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/validation/StorageQuotaCheckerTest.java
@@ -19,162 +19,109 @@
package org.apache.pinot.controller.validation;
import com.yammer.metrics.core.MetricsRegistry;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.QuotaConfig;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
+import java.util.Collections;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.util.TableSizeReader;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
+import org.apache.pinot.spi.config.QuotaConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
public class StorageQuotaCheckerTest {
+ private static final String OFFLINE_TABLE_NAME = "testTable_OFFLINE";
+ private static final String SEGMENT_NAME = "testSegment";
+ private static final long SEGMENT_SIZE_IN_BYTES = 1024;
+ private static final int NUM_REPLICAS = 2;
+
private TableSizeReader _tableSizeReader;
private TableConfig _tableConfig;
private ControllerMetrics _controllerMetrics;
- private boolean _isLeaderForTable;
- private QuotaConfig _quotaConfig;
- private SegmentsValidationAndRetentionConfig _validationConfig;
- private static final File TEST_DIR = new File(StorageQuotaCheckerTest.class.getName());
+ private StorageQuotaChecker _storageQuotaChecker;
@BeforeClass
public void setUp() {
+ _tableConfig =
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(OFFLINE_TABLE_NAME).setNumReplicas(NUM_REPLICAS).build();
_tableSizeReader = mock(TableSizeReader.class);
- _tableConfig = mock(TableConfig.class);
- _quotaConfig = mock(QuotaConfig.class);
_controllerMetrics = new ControllerMetrics(new MetricsRegistry());
- _validationConfig = mock(SegmentsValidationAndRetentionConfig.class);
- _isLeaderForTable = true;
- when(_tableConfig.getValidationConfig()).thenReturn(_validationConfig);
- when(_validationConfig.getReplicationNumber()).thenReturn(2);
- TEST_DIR.mkdirs();
+ _storageQuotaChecker = new StorageQuotaChecker(_tableConfig, _tableSizeReader, _controllerMetrics, true);
}
- @AfterClass
- public void tearDown() {
- FileUtils.deleteQuietly(TEST_DIR);
+ private boolean isSegmentWithinQuota()
+ throws InvalidConfigException {
+ return _storageQuotaChecker
+ .isSegmentStorageWithinQuota(SEGMENT_NAME, SEGMENT_SIZE_IN_BYTES, 1000).isSegmentWithinQuota;
}
@Test
public void testNoQuota()
throws InvalidConfigException {
- StorageQuotaChecker checker =
- new MockStorageQuotaChecker(_tableConfig, _tableSizeReader, _controllerMetrics, _isLeaderForTable);
- when(_tableConfig.getQuotaConfig()).thenReturn(null);
- StorageQuotaChecker.QuotaCheckerResponse res = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment", 1000);
- Assert.assertTrue(res.isSegmentWithinQuota);
+ _tableConfig.setQuotaConfig(null);
+ assertTrue(isSegmentWithinQuota());
}
@Test
public void testNoStorageQuotaConfig()
throws InvalidConfigException {
- StorageQuotaChecker checker =
- new MockStorageQuotaChecker(_tableConfig, _tableSizeReader, _controllerMetrics, _isLeaderForTable);
- when(_tableConfig.getQuotaConfig()).thenReturn(_quotaConfig);
- when(_quotaConfig.storageSizeBytes()).thenReturn(-1L);
- StorageQuotaChecker.QuotaCheckerResponse res = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment", 1000);
- Assert.assertTrue(res.isSegmentWithinQuota);
+ _tableConfig.setQuotaConfig(new QuotaConfig(null, null));
+ assertTrue(isSegmentWithinQuota());
}
- public void setupTableSegmentSize(final long tableSize, final long segmentSize, final int missing)
+ public void mockTableSizeResult(long tableSizeInBytes, int numMissingSegments)
throws InvalidConfigException {
- when(_tableSizeReader.getTableSubtypeSize("testTable", 1000))
- .thenAnswer(new Answer<TableSizeReader.TableSubTypeSizeDetails>() {
- @Override
- public TableSizeReader.TableSubTypeSizeDetails answer(InvocationOnMock invocationOnMock)
- throws Throwable {
- TableSizeReader.TableSubTypeSizeDetails sizeDetails = new TableSizeReader.TableSubTypeSizeDetails();
- sizeDetails.estimatedSizeInBytes = tableSize;
- TableSizeReader.SegmentSizeDetails segSizeDetails = new TableSizeReader.SegmentSizeDetails();
- segSizeDetails.estimatedSizeInBytes = segmentSize;
- sizeDetails.segments.put("segment1", segSizeDetails);
- sizeDetails.missingSegments = missing;
- return sizeDetails;
- }
- });
+ TableSizeReader.TableSubTypeSizeDetails tableSizeResult = new TableSizeReader.TableSubTypeSizeDetails();
+ tableSizeResult.estimatedSizeInBytes = tableSizeInBytes;
+ tableSizeResult.segments = Collections.emptyMap();
+ tableSizeResult.missingSegments = numMissingSegments;
+ when(_tableSizeReader.getTableSubtypeSize(OFFLINE_TABLE_NAME, 1000)).thenReturn(tableSizeResult);
}
@Test
public void testWithinQuota()
- throws IOException, InvalidConfigException {
- File tempFile = new File(TEST_DIR, "small_file");
- tempFile.createNewFile();
- byte[] data = new byte[1024];
- Arrays.fill(data, (byte) 1);
- try (FileOutputStream ostr = new FileOutputStream(tempFile)) {
- ostr.write(data);
- }
- String tableName = "testTable";
- setupTableSegmentSize(4800L, 900L, 0);
- when(_tableConfig.getTableName()).thenReturn(tableName);
- when(_tableConfig.getQuotaConfig()).thenReturn(_quotaConfig);
- when(_quotaConfig.storageSizeBytes()).thenReturn(3000L);
- when(_quotaConfig.getStorage()).thenReturn("3K");
- StorageQuotaChecker checker =
- new MockStorageQuotaChecker(_tableConfig, _tableSizeReader, _controllerMetrics, _isLeaderForTable);
- StorageQuotaChecker.QuotaCheckerResponse response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertTrue(response.isSegmentWithinQuota);
- Assert.assertEquals(
- _controllerMetrics.getValueOfTableGauge(tableName, ControllerGauge.TABLE_STORAGE_QUOTA_UTILIZATION), 80L);
-
- // Quota exceeded.
- when(_quotaConfig.storageSizeBytes()).thenReturn(2800L);
- when(_quotaConfig.getStorage()).thenReturn("2.8K");
- response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertFalse(response.isSegmentWithinQuota);
- Assert.assertEquals(
- _controllerMetrics.getValueOfTableGauge(tableName, ControllerGauge.TABLE_STORAGE_QUOTA_UTILIZATION), 85L);
-
- // Table already over quota.
- setupTableSegmentSize(6000L, 900L, 0);
- when(_quotaConfig.storageSizeBytes()).thenReturn(2800L);
- when(_quotaConfig.getStorage()).thenReturn("2.8K");
- response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertFalse(response.isSegmentWithinQuota);
- Assert.assertEquals(
- _controllerMetrics.getValueOfTableGauge(tableName, ControllerGauge.TABLE_STORAGE_QUOTA_UTILIZATION), 107L);
-
- // no response from any server
- setupTableSegmentSize(-1, -1, 0);
- when(_quotaConfig.storageSizeBytes()).thenReturn(2800L);
- when(_quotaConfig.getStorage()).thenReturn("2.8K");
- response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertTrue(response.isSegmentWithinQuota);
-
- // partial response from servers, but table already over quota
- setupTableSegmentSize(6000L, 900L, -2);
- when(_quotaConfig.storageSizeBytes()).thenReturn(2800L);
- when(_quotaConfig.getStorage()).thenReturn("2.8K");
- response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertFalse(response.isSegmentWithinQuota);
-
- // partial response from servers, but current estimate within quota
- setupTableSegmentSize(2000L, 900L, -2);
- when(_quotaConfig.storageSizeBytes()).thenReturn(2800L);
- when(_quotaConfig.getStorage()).thenReturn("2.8K");
- response = checker.isSegmentStorageWithinQuota(TEST_DIR, "segment1", 1000);
- Assert.assertTrue(response.isSegmentWithinQuota);
- }
-
- private class MockStorageQuotaChecker extends StorageQuotaChecker {
-
- public MockStorageQuotaChecker(TableConfig tableConfig, TableSizeReader tableSizeReader,
- ControllerMetrics controllerMetrics, boolean isLeaderForTable) {
- super(tableConfig, tableSizeReader, controllerMetrics, isLeaderForTable);
- }
+ throws InvalidConfigException {
+ _tableConfig.setQuotaConfig(new QuotaConfig("2.8K", null));
+
+ // No response from server, should pass without updating metrics
+ mockTableSizeResult(-1, 0);
+ assertTrue(isSegmentWithinQuota());
+ assertEquals(
+ _controllerMetrics.getValueOfTableGauge(OFFLINE_TABLE_NAME, ControllerGauge.OFFLINE_TABLE_ESTIMATED_SIZE), 0);
+
+ // Within quota but with missing segments, should pass without updating metrics
+ mockTableSizeResult(4 * 1024, 1);
+ assertTrue(isSegmentWithinQuota());
+ assertEquals(
+ _controllerMetrics.getValueOfTableGauge(OFFLINE_TABLE_NAME, ControllerGauge.OFFLINE_TABLE_ESTIMATED_SIZE), 0);
+
+ // Exceed quota and with missing segments, should fail without updating metrics
+ mockTableSizeResult(8 * 1024, 1);
+ assertFalse(isSegmentWithinQuota());
+ assertEquals(
+ _controllerMetrics.getValueOfTableGauge(OFFLINE_TABLE_NAME, ControllerGauge.OFFLINE_TABLE_ESTIMATED_SIZE), 0);
+
+ // Within quota without missing segments, should pass and update metrics
+ mockTableSizeResult(3 * 1024, 0);
+ assertTrue(isSegmentWithinQuota());
+ assertEquals(
+ _controllerMetrics.getValueOfTableGauge(OFFLINE_TABLE_NAME, ControllerGauge.OFFLINE_TABLE_ESTIMATED_SIZE),
+ 3 * 1024);
+
+ // Exceed quota without missing segments, should fail and update metrics
+ mockTableSizeResult(4 * 1024, 0);
+ assertFalse(isSegmentWithinQuota());
+ assertEquals(
+ _controllerMetrics.getValueOfTableGauge(OFFLINE_TABLE_NAME, ControllerGauge.OFFLINE_TABLE_ESTIMATED_SIZE),
+ 4 * 1024);
}
}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/validation/ValidationManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/validation/ValidationManagerTest.java
index 637d4d2..fa74eab 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/validation/ValidationManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/validation/ValidationManagerTest.java
@@ -24,18 +24,19 @@ import org.apache.helix.HelixAdmin;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.HLCSegmentName;
import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.joda.time.DateTime;
import org.joda.time.Duration;
@@ -68,8 +69,7 @@ public class ValidationManagerTest extends ControllerTest {
addFakeServerInstancesToAutoJoinHelixCluster(2, true);
_offlineTableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TEST_TABLE_NAME).setNumReplicas(2)
- .build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(TEST_TABLE_NAME).setNumReplicas(2).build();
_helixResourceManager.addTable(_offlineTableConfig);
}
@@ -87,8 +87,7 @@ public class ValidationManagerTest extends ControllerTest {
_helixResourceManager.rebuildBrokerResourceFromHelixTags(partitionName);
// Add another table that needs to be rebuilt
- TableConfig offlineTableConfigTwo =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(TEST_TABLE_TWO).build();
+ TableConfig offlineTableConfigTwo = new TableConfigBuilder(TableType.OFFLINE).setTableName(TEST_TABLE_TWO).build();
_helixResourceManager.addTable(offlineTableConfigTwo);
String partitionNameTwo = offlineTableConfigTwo.getTableName();
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
index a804add..6ea0b9a 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
@@ -27,7 +27,6 @@ import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metrics.ServerGauge;
import org.apache.pinot.common.metrics.ServerMeter;
import org.apache.pinot.common.metrics.ServerMetrics;
@@ -35,6 +34,7 @@ import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
index 7b1c79c..79a3445 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
@@ -24,11 +24,11 @@ import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.config.TableConfig;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
index 1c01ef3..dac01ea 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
@@ -22,9 +22,9 @@ import com.google.common.base.Preconditions;
import javax.annotation.Nonnull;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
index aa36055..8ddd960 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
@@ -23,11 +23,11 @@ import javax.annotation.Nonnull;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.core.data.manager.TableDataManager;
import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager;
+import org.apache.pinot.spi.config.TableType;
/**
@@ -50,7 +50,7 @@ public class TableDataManagerProvider {
@Nonnull String instanceId, @Nonnull ZkHelixPropertyStore<ZNRecord> propertyStore,
@Nonnull ServerMetrics serverMetrics) {
TableDataManager tableDataManager;
- switch (CommonConstants.Helix.TableType.valueOf(tableDataManagerConfig.getTableDataManagerType())) {
+ switch (TableType.valueOf(tableDataManagerConfig.getTableDataManagerType())) {
case OFFLINE:
tableDataManager = new OfflineTableDataManager();
break;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
index acd8ca5..4cb7af2 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
@@ -28,9 +28,6 @@ import java.util.TimerTask;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ServerGauge;
@@ -38,7 +35,6 @@ import org.apache.pinot.common.metrics.ServerMeter;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
import org.apache.pinot.common.utils.CommonConstants.Segment.SegmentType;
-import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
import org.apache.pinot.core.data.recordtransformer.RecordTransformer;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
@@ -46,11 +42,15 @@ import org.apache.pinot.core.indexsegment.mutable.MutableSegment;
import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
import org.apache.pinot.core.realtime.converter.RealtimeSegmentConverter;
import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.stream.StreamConsumerFactory;
import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider;
import org.apache.pinot.spi.stream.StreamLevelConsumer;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.slf4j.Logger;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
index 13aea50..e5c078b 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
@@ -36,11 +36,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.FileUtils;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.CompletionConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ServerGauge;
@@ -62,6 +57,11 @@ import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
import org.apache.pinot.core.segment.creator.impl.V1Constants;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.CompletionConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.spi.stream.MessageBatch;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
index 94c9614..2457bda 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
@@ -31,10 +31,9 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import javax.annotation.concurrent.ThreadSafe;
+import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.io.FileUtils;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
@@ -52,6 +51,8 @@ import org.apache.pinot.core.realtime.impl.RealtimeSegmentStatsHistory;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
import org.apache.pinot.core.segment.index.loader.LoaderUtils;
import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnProviderFactory;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
@@ -340,7 +341,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
// 1. Make sure that the sorted column is not a multi-value field.
List<String> sortedColumns = indexingConfig.getSortedColumn();
boolean isValid = true;
- if (!sortedColumns.isEmpty()) {
+ if (CollectionUtils.isNotEmpty(sortedColumns)) {
final String sortedColumn = sortedColumns.get(0);
if (sortedColumns.size() > 1) {
_logger.warn("More than one sorted column configured. Using {}", sortedColumn);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
index ddf166e..4f9fa01 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
@@ -35,17 +35,16 @@ import java.util.stream.Collectors;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.commons.lang.StringUtils;
-import org.apache.pinot.common.config.FieldConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.StarTreeIndexConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.core.io.compression.ChunkCompressorFactory;
import org.apache.pinot.core.segment.name.FixedSegmentNameGenerator;
import org.apache.pinot.core.segment.name.SegmentNameGenerator;
import org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator;
import org.apache.pinot.core.startree.v2.builder.StarTreeV2BuilderConfig;
-import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.StarTreeIndexConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.FieldSpec.FieldType;
import org.apache.pinot.spi.data.Schema;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
index e53119a..ba42e8b 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
@@ -33,8 +33,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
import org.apache.pinot.core.common.DataSource;
import org.apache.pinot.core.data.partition.PartitionFunction;
import org.apache.pinot.core.indexsegment.IndexSegmentUtils;
@@ -69,6 +67,8 @@ import org.apache.pinot.core.startree.v2.StarTreeV2;
import org.apache.pinot.core.util.FixedIntArray;
import org.apache.pinot.core.util.FixedIntArrayOffHeapIdMap;
import org.apache.pinot.core.util.IdMap;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
import org.apache.pinot.spi.data.DimensionFieldSpec;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.MetricFieldSpec;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/PinotTaskConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/minion/PinotTaskConfig.java
similarity index 95%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/PinotTaskConfig.java
rename to pinot-core/src/main/java/org/apache/pinot/core/minion/PinotTaskConfig.java
index 9c82d7e..6e91bd8 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/PinotTaskConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/minion/PinotTaskConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.core.minion;
import java.util.HashMap;
import java.util.Map;
@@ -74,7 +74,7 @@ public class PinotTaskConfig {
}
if (obj instanceof PinotTaskConfig) {
PinotTaskConfig that = (PinotTaskConfig) obj;
- return this._taskType.equals(that._taskType) && this._configs.equals(that._configs);
+ return _taskType.equals(that._taskType) && _configs.equals(that._configs);
}
return false;
}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
index 0a02062..87253d2 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
@@ -23,7 +23,7 @@ import java.io.File;
import java.util.ArrayList;
import java.util.List;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.IndexingConfig;
+import org.apache.commons.collections.CollectionUtils;
import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
import org.apache.pinot.core.minion.segment.DefaultRecordPartitioner;
@@ -33,6 +33,7 @@ import org.apache.pinot.core.minion.segment.RecordPartitioner;
import org.apache.pinot.core.minion.segment.RecordTransformer;
import org.apache.pinot.core.minion.segment.ReducerRecordReader;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.spi.config.IndexingConfig;
import org.apache.pinot.spi.data.readers.RecordReader;
@@ -119,7 +120,7 @@ public class SegmentConverter {
List<String> invertedIndexColumns = _indexingConfig.getInvertedIndexColumns();
// Check if the table config has any index configured
- if ((sortedColumn != null && !sortedColumn.isEmpty()) || invertedIndexColumns != null) {
+ if (CollectionUtils.isNotEmpty(sortedColumn) || CollectionUtils.isNotEmpty(invertedIndexColumns)) {
String indexGenerationOutputPath = _workingDir.getPath() + File.separator + INDEX_PREFIX + currentPartition;
try (
PinotSegmentRecordReader recordReader = new PinotSegmentRecordReader(outputSegment, null, sortedColumn)) {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/minion/rollup/MergeRollupSegmentConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/minion/rollup/MergeRollupSegmentConverter.java
index 5979af5..65262f1 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/minion/rollup/MergeRollupSegmentConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/minion/rollup/MergeRollupSegmentConverter.java
@@ -24,12 +24,12 @@ import java.util.List;
import java.util.Map;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.IndexingConfig;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.core.minion.SegmentConverter;
import org.apache.pinot.core.minion.segment.RecordAggregator;
import org.apache.pinot.core.minion.segment.RecordTransformer;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.spi.config.IndexingConfig;
import org.apache.pinot.spi.data.DimensionFieldSpec;
import org.apache.pinot.spi.data.Schema;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
index bcaf738..69c9af5 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
@@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.calcite.sql.SqlKind;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.metrics.BrokerTimer;
@@ -38,6 +37,7 @@ import org.apache.pinot.common.utils.DataSchema;
import org.apache.pinot.common.utils.DataTable;
import org.apache.pinot.core.transport.ServerRoutingInstance;
import org.apache.pinot.core.util.QueryOptions;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java
index 1cea825..266d6db 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java
@@ -24,7 +24,6 @@ import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.QueryException;
import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
@@ -40,6 +39,7 @@ import org.apache.pinot.core.query.selection.SelectionOperatorService;
import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
import org.apache.pinot.core.transport.ServerRoutingInstance;
import org.apache.pinot.core.util.QueryOptions;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
index 32af6d5..a315183 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
@@ -24,8 +24,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
import org.apache.pinot.common.metrics.ServerGauge;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
@@ -35,6 +33,8 @@ import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
import org.apache.pinot.core.io.compression.ChunkCompressorFactory;
import org.apache.pinot.core.realtime.converter.stats.RealtimeSegmentSegmentCreationDataSource;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.TimeFieldSpec;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/StatsCollectorConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/StatsCollectorConfig.java
index 9bf2c41..0ebc459 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/StatsCollectorConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/StatsCollectorConfig.java
@@ -21,11 +21,11 @@ package org.apache.pinot.core.segment.creator;
import com.google.common.base.Preconditions;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.spi.data.FieldSpec;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.core.data.partition.PartitionFunction;
import org.apache.pinot.core.data.partition.PartitionFunctionFactory;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/IndexLoadingConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/IndexLoadingConfig.java
index 1cc04df..c3a4076 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/IndexLoadingConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/IndexLoadingConfig.java
@@ -27,13 +27,13 @@ import java.util.Map;
import java.util.Set;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.FieldConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.segment.ReadMode;
import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
import org.apache.pinot.core.segment.index.loader.columnminmaxvalue.ColumnMinMaxValueGeneratorMode;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.TableConfig;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/startree/v2/builder/StarTreeV2BuilderConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/startree/v2/builder/StarTreeV2BuilderConfig.java
index 5ad839a..eef298a 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/startree/v2/builder/StarTreeV2BuilderConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/startree/v2/builder/StarTreeV2BuilderConfig.java
@@ -24,8 +24,8 @@ import java.util.List;
import java.util.Set;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
-import org.apache.pinot.common.config.StarTreeIndexConfig;
import org.apache.pinot.core.startree.v2.AggregationFunctionColumnPair;
+import org.apache.pinot.spi.config.StarTreeIndexConfig;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java
index 6b35e06..9c6ddb9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java
@@ -28,8 +28,8 @@ import org.apache.pinot.common.metrics.BrokerMeter;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.request.BrokerRequest;
import org.apache.pinot.common.request.InstanceRequest;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.spi.config.TableType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java
index 872428e..6c3174e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java
@@ -21,6 +21,7 @@ package org.apache.pinot.core.transport;
import com.google.common.annotations.VisibleForTesting;
import org.apache.helix.model.InstanceConfig;
import org.apache.pinot.common.utils.CommonConstants.Helix;
+import org.apache.pinot.spi.config.TableType;
public class ServerInstance {
@@ -67,7 +68,7 @@ public class ServerInstance {
return _port;
}
- public ServerRoutingInstance toServerRoutingInstance(Helix.TableType tableType) {
+ public ServerRoutingInstance toServerRoutingInstance(TableType tableType) {
return new ServerRoutingInstance(_hostname, _port, tableType);
}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java
index d617c03..b731357 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java
@@ -22,7 +22,7 @@ import com.google.common.net.InternetDomainName;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import javax.annotation.concurrent.ThreadSafe;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.TableType;
/**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java
index 96a0d28..8f18f78 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java
@@ -18,9 +18,9 @@
*/
package org.apache.pinot.core.util;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.stream.StreamConfig;
@@ -30,7 +30,7 @@ import org.apache.pinot.spi.stream.StreamConfig;
public class ReplicationUtils {
/**
- * Decides if {@link SegmentsValidationAndRetentionConfig::getReplicationNumber} should be used
+ * Decides if {@link SegmentsValidationAndRetentionConfig ::getReplicationNumber} should be used
*/
public static boolean useReplication(TableConfig tableConfig) {
@@ -44,7 +44,7 @@ public class ReplicationUtils {
}
/**
- * Decides if {@link SegmentsValidationAndRetentionConfig::getReplicasPerPartitionNumber} should be used
+ * Decides if {@link SegmentsValidationAndRetentionConfig ::getReplicasPerPartitionNumber} should be used
*/
public static boolean useReplicasPerPartition(TableConfig tableConfig) {
diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
index 7c2ff68..cf8df0e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
+++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
@@ -23,7 +23,6 @@ import java.net.URI;
import java.util.Map;
import javax.net.ssl.SSLContext;
import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metrics.ServerMeter;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
@@ -31,6 +30,7 @@ import org.apache.pinot.common.utils.ClientSSLContextGenerator;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.pql.parsers.utils.Pair;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
index 715c833..4acbc22 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
@@ -30,8 +30,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
@@ -43,16 +41,21 @@ import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
import org.apache.pinot.core.realtime.impl.RealtimeSegmentStatsHistory;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory;
import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamMessageDecoder;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.stream.PermanentConsumerException;
import org.apache.pinot.spi.stream.StreamConfigProperties;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
// TODO Write more tests for other parts of the class
@@ -85,8 +88,7 @@ public class LLRealtimeSegmentDataManagerTest {
+ " \"tableIndexConfig\": {\n" + " \"invertedIndexColumns\": [" + " ], \n"
+ " \"lazyLoad\": \"false\", \n" + " \"loadMode\": \"HEAP\", \n"
+ " \"segmentFormatVersion\": null, \n" + " \"sortedColumn\": [], \n" + " \"streamConfigs\": {\n"
- + " \"" + StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + "\": \"" + String
- .valueOf(maxRowsInSegment) + "\", \n" + " \"" + StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME
+ + " \"" + StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + "\": \"" + maxRowsInSegment + "\", \n" + " \"" + StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME
+ "\": \"" + maxTimeForSegmentCloseMs + "\", \n"
+ " \"stream.fakeStream.broker.list\": \"broker:7777\", \n"
+ " \"stream.fakeStream.consumer.prop.auto.offset.reset\": \"smallest\", \n"
@@ -113,7 +115,7 @@ public class LLRealtimeSegmentDataManagerTest {
private TableConfig createTableConfig()
throws Exception {
- return TableConfig.fromJsonString(_tableConfigJson);
+ return JsonUtils.stringToObject(_tableConfigJson, TableConfig.class);
}
private RealtimeTableDataManager createTableDataManager() {
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/stream/StreamConfigTest.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/stream/StreamConfigTest.java
index 4afad60..af09f30 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/stream/StreamConfigTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/stream/StreamConfigTest.java
@@ -20,14 +20,14 @@ package org.apache.pinot.core.realtime.stream;
import java.util.HashMap;
import java.util.Map;
+import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory;
+import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamMessageDecoder;
import org.apache.pinot.spi.stream.OffsetCriteria;
import org.apache.pinot.spi.stream.PartitionLevelStreamConfig;
import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.stream.StreamConfigProperties;
-import org.apache.pinot.spi.utils.DataSize;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.pinot.spi.utils.TimeUtils;
-import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory;
-import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamMessageDecoder;
import org.testng.Assert;
import org.testng.annotations.Test;
@@ -238,7 +238,7 @@ public class StreamConfigTest {
Assert.assertEquals(streamConfig.getFlushThresholdRows(), Integer.parseInt(flushThresholdRows));
Assert.assertEquals(streamConfig.getFlushThresholdTimeMillis(),
(long) TimeUtils.convertPeriodToMillis(flushThresholdTime));
- Assert.assertEquals(streamConfig.getFlushSegmentDesiredSizeBytes(), DataSize.toBytes(flushSegmentSize));
+ Assert.assertEquals(streamConfig.getFlushSegmentDesiredSizeBytes(), DataSizeUtils.toBytes(flushSegmentSize));
// Backward compatibility check for flushThresholdTime
flushThresholdTime = "18000000";
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentPartitionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentPartitionTest.java
index 5ef3be7..cb19e85 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentPartitionTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentPartitionTest.java
@@ -29,8 +29,6 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
import org.apache.pinot.common.request.FilterOperator;
import org.apache.pinot.common.segment.ReadMode;
import org.apache.pinot.core.data.partition.ModuloPartitionFunction;
@@ -41,10 +39,13 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
import org.apache.pinot.core.segment.index.metadata.ColumnMetadata;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
import org.apache.pinot.spi.data.DimensionFieldSpec;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -137,7 +138,8 @@ public class SegmentPartitionTest {
}
SegmentPartitionConfig expectedConfig = new SegmentPartitionConfig(expectedMap);
- SegmentPartitionConfig actualConfig = SegmentPartitionConfig.fromJsonString(expectedConfig.toJsonString());
+ SegmentPartitionConfig actualConfig =
+ JsonUtils.stringToObject(expectedConfig.toJsonString(), SegmentPartitionConfig.class);
for (Map.Entry<String, ColumnPartitionConfig> entry : actualConfig.getColumnPartitionMap().entrySet()) {
String partitionColumn = entry.getKey();
@@ -156,7 +158,7 @@ public class SegmentPartitionTest {
"{\"columnPartitionMap\":{\"column_0\":{\"functionName\":\"function\",\"numPartitions\":10}}}";
assertEquals(jsonStringWithoutNewField,
- SegmentPartitionConfig.fromJsonString(jsonStringWithNewField).toJsonString());
+ JsonUtils.stringToObject(jsonStringWithNewField, SegmentPartitionConfig.class).toJsonString());
}
private String buildQuery(String tableName, String columnName, int predicateValue) {
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java
index cfd4daa..051763f 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java
@@ -25,11 +25,11 @@ import java.util.Map;
import org.apache.pinot.common.metrics.BrokerMetrics;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.request.BrokerRequest;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.DataTable;
import org.apache.pinot.core.common.datatable.DataTableImplV2;
import org.apache.pinot.core.query.scheduler.QueryScheduler;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.TableType;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
index 3cbf6dd..a325a97 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.pinot.common.request.BrokerRequest;
import org.apache.pinot.common.response.broker.BrokerResponseNative;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.CommonConstants.Server;
import org.apache.pinot.common.utils.DataTable;
import org.apache.pinot.core.common.Operator;
@@ -37,6 +36,7 @@ import org.apache.pinot.core.plan.maker.PlanMaker;
import org.apache.pinot.core.query.reduce.BrokerReduceService;
import org.apache.pinot.core.transport.ServerRoutingInstance;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.sql.parsers.CalciteSqlCompiler;
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java
index a6dcc19..747aefa 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java
@@ -32,13 +32,13 @@ import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import org.apache.commons.io.FileUtils;
import org.apache.pinot.client.ConnectionFactory;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableTaskConfig;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.common.utils.config.TagNameUtils;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.stream.StreamDataServerStartable;
import org.apache.pinot.tools.utils.KafkaStarterUtils;
import org.apache.pinot.util.TestUtils;
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
index 65cefa7..4613369 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
@@ -45,13 +45,6 @@ import org.apache.commons.io.FileUtils;
import org.apache.http.HttpStatus;
import org.apache.pinot.broker.broker.helix.HelixBrokerStarter;
import org.apache.pinot.broker.requesthandler.PinotQueryRequest;
-import org.apache.pinot.common.config.FieldConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TableTaskConfig;
-import org.apache.pinot.common.config.TenantConfig;
import org.apache.pinot.common.utils.CommonConstants.Broker;
import org.apache.pinot.common.utils.CommonConstants.Helix;
import org.apache.pinot.common.utils.CommonConstants.Minion;
@@ -68,6 +61,13 @@ import org.apache.pinot.plugin.inputformat.avro.AvroUtils;
import org.apache.pinot.plugin.stream.kafka.KafkaStreamConfigProperties;
import org.apache.pinot.server.starter.helix.DefaultHelixStarterServerConfig;
import org.apache.pinot.server.starter.helix.HelixServerStarter;
+import org.apache.pinot.spi.config.FieldConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TenantConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.spi.data.readers.RecordExtractor;
@@ -75,6 +75,8 @@ import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.stream.StreamConfigProperties;
import org.apache.pinot.spi.stream.StreamMessageDecoder;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
@@ -308,7 +310,7 @@ public abstract class ClusterTest extends ControllerTest {
TableConfig tableConfig =
getOfflineTableConfig(tableName, timeColumnName, timeType, brokerTenant, serverTenant, loadMode, segmentVersion,
invertedIndexColumns, bloomFilterColumns, taskConfig, segmentPartitionConfig, sortedColumn, "daily");
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonConfigString());
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonString());
}
protected void updateOfflineTable(String tableName, String timeColumnName, String timeType, String brokerTenant,
@@ -319,14 +321,14 @@ public abstract class ClusterTest extends ControllerTest {
TableConfig tableConfig =
getOfflineTableConfig(tableName, timeColumnName, timeType, brokerTenant, serverTenant, loadMode, segmentVersion,
invertedIndexColumns, bloomFilterColumns, taskConfig, segmentPartitionConfig, sortedColumn, "daily");
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonConfigString());
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJsonString());
}
private static TableConfig getOfflineTableConfig(String tableName, String timeColumnName, String timeType,
String brokerTenant, String serverTenant, String loadMode, SegmentVersion segmentVersion,
List<String> invertedIndexColumns, List<String> bloomFilterColumns, TableTaskConfig taskConfig,
SegmentPartitionConfig segmentPartitionConfig, String sortedColumn, String segmentPushFrequency) {
- return new TableConfig.Builder(Helix.TableType.OFFLINE).setTableName(tableName).setTimeColumnName(timeColumnName)
+ return new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setTimeColumnName(timeColumnName)
.setTimeType(timeType).setSegmentPushFrequency(segmentPushFrequency).setNumReplicas(1)
.setBrokerTenant(brokerTenant).setServerTenant(serverTenant).setLoadMode(loadMode)
.setSegmentVersion(segmentVersion.toString()).setInvertedIndexColumns(invertedIndexColumns)
@@ -405,7 +407,8 @@ public abstract class ClusterTest extends ControllerTest {
String kafkaTopic, int realtimeSegmentFlushRows, File avroFile, String timeColumnName, String timeType,
String schemaName, String brokerTenant, String serverTenant, String loadMode, String sortedColumn,
List<String> invertedIndexColumns, List<String> bloomFilterColumns, List<String> noDictionaryColumns,
- TableTaskConfig taskConfig, String streamConsumerFactoryName, int numReplicas, List<FieldConfig> fieldConfigListForTextColumns)
+ TableTaskConfig taskConfig, String streamConsumerFactoryName, int numReplicas,
+ List<FieldConfig> fieldConfigListForTextColumns)
throws Exception {
Map<String, String> streamConfigs = new HashMap<>();
String streamType = "kafka";
@@ -443,7 +446,7 @@ public abstract class ClusterTest extends ControllerTest {
streamConfigs.put(StreamConfigProperties
.constructStreamProperty(streamType, StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA), "smallest");
- TableConfig tableConfig = new TableConfig.Builder(Helix.TableType.REALTIME).setTableName(tableName).setLLC(useLlc)
+ TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setLLC(useLlc)
.setTimeColumnName(timeColumnName).setTimeType(timeType).setSchemaName(schemaName).setBrokerTenant(brokerTenant)
.setServerTenant(serverTenant).setLoadMode(loadMode).setSortedColumn(sortedColumn)
.setInvertedIndexColumns(invertedIndexColumns).setBloomFilterColumns(bloomFilterColumns)
@@ -453,7 +456,7 @@ public abstract class ClusterTest extends ControllerTest {
// save the realtime table config
_realtimeTableConfig = tableConfig;
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonConfigString());
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonString());
}
protected void updateRealtimeTableConfig(String tablename, List<String> invertedIndexCols,
@@ -464,8 +467,7 @@ public abstract class ClusterTest extends ControllerTest {
config.setInvertedIndexColumns(invertedIndexCols);
config.setBloomFilterColumns(bloomFilterCols);
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tablename),
- _realtimeTableConfig.toJsonConfigString());
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tablename), _realtimeTableConfig.toJsonString());
}
protected void updateRealtimeTableTenant(String tableName, TenantConfig tenantConfig)
@@ -473,8 +475,7 @@ public abstract class ClusterTest extends ControllerTest {
_realtimeTableConfig.setTenantConfig(tenantConfig);
- sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName),
- _realtimeTableConfig.toJsonConfigString());
+ sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), _realtimeTableConfig.toJsonString());
}
protected void dropRealtimeTable(String tableName)
@@ -532,12 +533,12 @@ public abstract class ClusterTest extends ControllerTest {
/**
* Queries the broker's sql query endpoint (/sql)
*/
- static JsonNode postSqlQuery(String query, String brokerBaseApiUrl) throws Exception {
+ static JsonNode postSqlQuery(String query, String brokerBaseApiUrl)
+ throws Exception {
ObjectNode payload = JsonUtils.newObjectNode();
payload.put("sql", query);
payload.put("queryOptions", "groupByMode=sql;responseFormat=sql");
return JsonUtils.stringToJsonNode(sendPostRequest(brokerBaseApiUrl + "/query/sql", payload.toString()));
}
-
}
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ControllerPeriodicTasksIntegrationTests.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ControllerPeriodicTasksIntegrationTests.java
index 4b45175..3c15337 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ControllerPeriodicTasksIntegrationTests.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ControllerPeriodicTasksIntegrationTests.java
@@ -34,22 +34,23 @@ import javax.annotation.Nullable;
import org.apache.commons.io.FileUtils;
import org.apache.helix.model.IdealState;
import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
-import org.apache.pinot.common.config.TagOverrideConfig;
-import org.apache.pinot.common.config.TenantConfig;
import org.apache.pinot.common.metrics.ControllerGauge;
import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.metrics.ValidationMetrics;
-import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.common.utils.helix.HelixHelper;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.validation.OfflineSegmentIntervalChecker;
import org.apache.pinot.controller.validation.RealtimeSegmentValidationManager;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.config.TagOverrideConfig;
+import org.apache.pinot.spi.config.TenantConfig;
import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.spi.utils.retry.RetryPolicies;
import org.apache.pinot.tools.utils.KafkaStarterUtils;
import org.apache.pinot.util.TestUtils;
@@ -429,8 +430,7 @@ public class ControllerPeriodicTasksIntegrationTests extends BaseClusterIntegrat
// Check that the first table we added doesn't need to be rebuilt(case where ideal state brokers and brokers in broker resource are the same.
String table1 = (String) context.getAttribute("testTableOne");
String table2 = (String) context.getAttribute("testTableTwo");
- TableConfig tableConfigOne =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(table1).build();
+ TableConfig tableConfigOne = new TableConfigBuilder(TableType.OFFLINE).setTableName(table1).build();
String partitionNameOne = tableConfigOne.getTableName();
// Ensure that the broker resource is not rebuilt.
@@ -442,8 +442,8 @@ public class ControllerPeriodicTasksIntegrationTests extends BaseClusterIntegrat
// Add another table that needs to be rebuilt
TableConfig offlineTableConfigTwo =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName(table2)
- .setBrokerTenant(TENANT_NAME).setServerTenant(TENANT_NAME).build();
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(table2).setBrokerTenant(TENANT_NAME)
+ .setServerTenant(TENANT_NAME).build();
_helixResourceManager.addTable(offlineTableConfigTwo);
String partitionNameTwo = offlineTableConfigTwo.getTableName();
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ConvertToRawIndexMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ConvertToRawIndexMinionClusterIntegrationTest.java
index 4058f16..5090a5f 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ConvertToRawIndexMinionClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ConvertToRawIndexMinionClusterIntegrationTest.java
@@ -27,8 +27,6 @@ import java.util.Map;
import javax.annotation.Nullable;
import org.apache.commons.lang.StringUtils;
import org.apache.helix.task.TaskState;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TableTaskConfig;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
@@ -37,6 +35,8 @@ import org.apache.pinot.core.common.MinionConstants;
import org.apache.pinot.core.common.MinionConstants.ConvertToRawIndexTask;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/DefaultCommitterRealtimeIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/DefaultCommitterRealtimeIntegrationTest.java
index 6b32044..c3e0ab8 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/DefaultCommitterRealtimeIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/DefaultCommitterRealtimeIntegrationTest.java
@@ -24,29 +24,30 @@ import com.yammer.metrics.core.MetricsRegistry;
import java.io.File;
import java.util.List;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
-import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
-import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
import org.apache.pinot.core.data.manager.realtime.SegmentCommitter;
import org.apache.pinot.core.data.manager.realtime.SegmentCommitterFactory;
import org.apache.pinot.core.data.readers.GenericRowRecordReader;
import org.apache.pinot.core.data.readers.PinotSegmentUtil;
-import org.apache.pinot.spi.data.readers.RecordReader;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
import org.apache.pinot.server.realtime.ControllerLeaderLocator;
import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
/**
@@ -92,15 +93,16 @@ public class DefaultCommitterRealtimeIntegrationTest extends RealtimeClusterInte
}
@Test
- public void testDefaultCommitter() throws Exception {
+ public void testDefaultCommitter()
+ throws Exception {
IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
ServerMetrics serverMetrics = new ServerMetrics(new MetricsRegistry());
- ServerSegmentCompletionProtocolHandler
- protocolHandler = new ServerSegmentCompletionProtocolHandler(serverMetrics, getTableName());
+ ServerSegmentCompletionProtocolHandler protocolHandler =
+ new ServerSegmentCompletionProtocolHandler(serverMetrics, getTableName());
SegmentCompletionProtocol.Response prevResponse = new SegmentCompletionProtocol.Response();
- LLRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = mock(
- LLRealtimeSegmentDataManager.SegmentBuildDescriptor.class);
+ LLRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor =
+ mock(LLRealtimeSegmentDataManager.SegmentBuildDescriptor.class);
RealtimeSegmentZKMetadata metadata = _helixResourceManager.getRealtimeSegmentMetadata(getTableName()).get(0);
@@ -122,17 +124,17 @@ public class DefaultCommitterRealtimeIntegrationTest extends RealtimeClusterInte
when(segmentBuildDescriptor.getWaitTimeMillis()).thenReturn(0L);
// Get realtime segment name
- String segmentList = sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPIWithTableType(getTableName(), "REALTIME"));
- JsonNode realtimeSegmentsList =
- getSegmentsFromJsonSegmentAPI(segmentList, CommonConstants.Helix.TableType.REALTIME.toString());
+ String segmentList =
+ sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPIWithTableType(getTableName(), "REALTIME"));
+ JsonNode realtimeSegmentsList = getSegmentsFromJsonSegmentAPI(segmentList, TableType.REALTIME.toString());
String segmentName = realtimeSegmentsList.get(0).asText();
// Send segmentConsumed request
- sendGetRequest("http://localhost:" + DEFAULT_CONTROLLER_PORT +
- "/segmentConsumed?instance=" + instanceId + "&name=" + segmentName + "&offset=" + END_OFFSET);
+ sendGetRequest("http://localhost:" + DEFAULT_CONTROLLER_PORT + "/segmentConsumed?instance=" + instanceId + "&name="
+ + segmentName + "&offset=" + END_OFFSET);
- SegmentCommitterFactory
- segmentCommitterFactory = new SegmentCommitterFactory(LOGGER, indexLoadingConfig, protocolHandler);
+ SegmentCommitterFactory segmentCommitterFactory =
+ new SegmentCommitterFactory(LOGGER, indexLoadingConfig, protocolHandler);
SegmentCommitter segmentCommitter = segmentCommitterFactory.createDefaultSegmentCommitter(params);
segmentCommitter.commit(END_OFFSET, 3, segmentBuildDescriptor);
}
@@ -153,7 +155,8 @@ public class DefaultCommitterRealtimeIntegrationTest extends RealtimeClusterInte
}
@Override
- public void tearDown() throws Exception {
+ public void tearDown()
+ throws Exception {
super.tearDown();
_indexDir.deleteOnExit();
_realtimeSegmentUntarred.deleteOnExit();
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java
index 94606f3..c11e435 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java
@@ -28,11 +28,12 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.tools.utils.KafkaStarterUtils;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
@@ -181,7 +182,7 @@ public class HybridClusterIntegrationTest extends BaseClusterIntegrationTestSet
throws Exception {
{
String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder.
- forSegmentListAPIWithTableType(getTableName(), CommonConstants.Helix.TableType.OFFLINE.toString()));
+ forSegmentListAPIWithTableType(getTableName(), TableType.OFFLINE.toString()));
JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
// There should be one element in the array
JsonNode element = array.get(0);
@@ -190,7 +191,7 @@ public class HybridClusterIntegrationTest extends BaseClusterIntegrationTestSet
}
{
String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder.
- forSegmentListAPIWithTableType(getTableName(), CommonConstants.Helix.TableType.REALTIME.toString()));
+ forSegmentListAPIWithTableType(getTableName(), TableType.REALTIME.toString()));
JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
// There should be one element in the array
JsonNode element = array.get(0);
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
index c7ba4e6..7e86020 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
@@ -28,11 +28,12 @@ import org.apache.avro.reflect.Nullable;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.io.FileUtils;
import org.apache.helix.ZNRecord;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -156,9 +157,9 @@ public class LLCRealtimeClusterIntegrationTest extends RealtimeClusterIntegratio
@Test(expectedExceptions = IOException.class)
public void testAddHLCTableShouldFail()
throws IOException {
- TableConfig tableConfig = new TableConfig.Builder(TableType.REALTIME).setTableName("testTable")
+ TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName("testTable")
.setStreamConfigs(Collections.singletonMap("stream.kafka.consumer.type", "HIGHLEVEL")).build();
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonConfigString());
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfig.toJsonString());
}
@Test
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
index 5958056..af2b399 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
@@ -23,10 +23,8 @@ import com.fasterxml.jackson.databind.node.TextNode;
import com.google.common.collect.Lists;
import java.io.BufferedReader;
import java.io.File;
-import java.io.FileInputStream;
import java.io.InputStream;
import java.io.InputStreamReader;
-import java.net.URL;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
@@ -37,9 +35,8 @@ import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.FieldConfig;
import org.apache.pinot.plugin.inputformat.avro.AvroUtils;
-import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.FieldConfig;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.TimeGranularitySpec;
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
index 354e676..8f23be5 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java
@@ -34,16 +34,18 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.commons.io.FileUtils;
import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.QueryConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.exception.QueryException;
import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.ServiceStatus;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.pinot.spi.config.QueryConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableType;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
@@ -177,13 +179,12 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
@Test
public void testInvalidTableConfig() {
- TableConfig tableConfig =
- new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE).setTableName("badTable").build();
- ObjectNode jsonConfig = tableConfig.toJsonConfig();
+ TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("badTable").build();
+ ObjectNode tableConfigJson = (ObjectNode) tableConfig.toJsonNode();
// Remove a mandatory field
- jsonConfig.remove(TableConfig.VALIDATION_CONFIG_KEY);
+ tableConfigJson.remove(TableConfig.VALIDATION_CONFIG_KEY);
try {
- sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), jsonConfig.toString());
+ sendPostRequest(_controllerRequestURLBuilder.forTableCreate(), tableConfigJson.toString());
fail();
} catch (IOException e) {
// Should get response code 400 (BAD_REQUEST)
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentCompletionIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentCompletionIntegrationTest.java
index 3e04c0e..0dc4560 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentCompletionIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentCompletionIntegrationTest.java
@@ -32,19 +32,19 @@ import org.apache.helix.participant.statemachine.StateModel;
import org.apache.helix.participant.statemachine.StateModelFactory;
import org.apache.helix.participant.statemachine.StateModelInfo;
import org.apache.helix.participant.statemachine.Transition;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.common.utils.NetUtil;
import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.helix.core.PinotHelixSegmentOnlineOfflineStateModelGenerator;
import org.apache.pinot.controller.validation.RealtimeSegmentValidationManager;
import org.apache.pinot.server.realtime.ControllerLeaderLocator;
import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
import org.apache.pinot.server.starter.helix.SegmentOnlineOfflineStateModelFactory;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SimpleMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SimpleMinionClusterIntegrationTest.java
index 3db3a72..ff32a92 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SimpleMinionClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SimpleMinionClusterIntegrationTest.java
@@ -28,22 +28,22 @@ import java.util.concurrent.atomic.AtomicBoolean;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.helix.task.TaskState;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TableTaskConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.controller.helix.core.minion.ClusterInfoProvider;
import org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
import org.apache.pinot.controller.helix.core.minion.PinotTaskManager;
import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.minion.events.MinionEventObserver;
import org.apache.pinot.minion.events.MinionEventObserverFactory;
import org.apache.pinot.minion.exception.TaskCancelledException;
import org.apache.pinot.minion.executor.BaseTaskExecutor;
import org.apache.pinot.minion.executor.PinotTaskExecutor;
import org.apache.pinot.minion.executor.PinotTaskExecutorFactory;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableTaskConfig;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.util.TestUtils;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/events/DefaultMinionEventObserver.java b/pinot-minion/src/main/java/org/apache/pinot/minion/events/DefaultMinionEventObserver.java
index cf0d913..04b5c8e 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/events/DefaultMinionEventObserver.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/events/DefaultMinionEventObserver.java
@@ -20,7 +20,7 @@ package org.apache.pinot.minion.events;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.PinotTaskConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
/**
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/events/MinionEventObserver.java b/pinot-minion/src/main/java/org/apache/pinot/minion/events/MinionEventObserver.java
index c23375d..9a3b9ff 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/events/MinionEventObserver.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/events/MinionEventObserver.java
@@ -20,7 +20,7 @@ package org.apache.pinot.minion.events;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.PinotTaskConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
/**
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseMultipleSegmentsConversionExecutor.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseMultipleSegmentsConversionExecutor.java
index 9970627..df60476 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseMultipleSegmentsConversionExecutor.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseMultipleSegmentsConversionExecutor.java
@@ -29,13 +29,13 @@ import javax.annotation.Nonnull;
import org.apache.commons.io.FileUtils;
import org.apache.http.NameValuePair;
import org.apache.http.message.BasicNameValuePair;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.minion.exception.TaskCancelledException;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseSingleSegmentConversionExecutor.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseSingleSegmentConversionExecutor.java
index 1564a4f..32c2c4c 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseSingleSegmentConversionExecutor.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/BaseSingleSegmentConversionExecutor.java
@@ -30,14 +30,14 @@ import org.apache.http.HttpHeaders;
import org.apache.http.NameValuePair;
import org.apache.http.message.BasicHeader;
import org.apache.http.message.BasicNameValuePair;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.minion.exception.TaskCancelledException;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/ConvertToRawIndexTaskExecutor.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/ConvertToRawIndexTaskExecutor.java
index 6e1b4b8..44dd50a 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/ConvertToRawIndexTaskExecutor.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/ConvertToRawIndexTaskExecutor.java
@@ -22,11 +22,11 @@ import java.io.File;
import java.util.Collections;
import java.util.Map;
import javax.annotation.Nonnull;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.core.minion.RawIndexConverter;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
public class ConvertToRawIndexTaskExecutor extends BaseSingleSegmentConversionExecutor {
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PinotTaskExecutor.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PinotTaskExecutor.java
index f4c0288..fdb436c 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PinotTaskExecutor.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PinotTaskExecutor.java
@@ -19,7 +19,7 @@
package org.apache.pinot.minion.executor;
import javax.annotation.Nonnull;
-import org.apache.pinot.common.config.PinotTaskConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
/**
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PurgeTaskExecutor.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PurgeTaskExecutor.java
index 3f39a14..8d8feeb 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PurgeTaskExecutor.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/PurgeTaskExecutor.java
@@ -22,11 +22,11 @@ import java.io.File;
import java.util.Collections;
import java.util.Map;
import javax.annotation.Nonnull;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier;
import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.core.minion.SegmentPurger;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
public class PurgeTaskExecutor extends BaseSingleSegmentConversionExecutor {
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/SegmentConversionResult.java b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/SegmentConversionResult.java
index fd3eb6a..a87cc99 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/executor/SegmentConversionResult.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/executor/SegmentConversionResult.java
@@ -21,7 +21,7 @@ package org.apache.pinot.minion.executor;
import java.io.File;
import java.util.HashMap;
import java.util.Map;
-import org.apache.pinot.common.config.PinotTaskConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
/**
diff --git a/pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java b/pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java
index 86d2a15..a4a9552 100644
--- a/pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java
+++ b/pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java
@@ -26,7 +26,7 @@ import org.apache.helix.task.Task;
import org.apache.helix.task.TaskConfig;
import org.apache.helix.task.TaskFactory;
import org.apache.helix.task.TaskResult;
-import org.apache.pinot.common.config.PinotTaskConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.minion.MinionContext;
import org.apache.pinot.minion.events.EventObserverFactoryRegistry;
import org.apache.pinot.minion.events.MinionEventObserver;
diff --git a/pinot-minion/src/test/java/org/apache/pinot/minion/executor/PurgeTaskExecutorTest.java b/pinot-minion/src/test/java/org/apache/pinot/minion/executor/PurgeTaskExecutorTest.java
index b44d053..cb69aad 100644
--- a/pinot-minion/src/test/java/org/apache/pinot/minion/executor/PurgeTaskExecutorTest.java
+++ b/pinot-minion/src/test/java/org/apache/pinot/minion/executor/PurgeTaskExecutorTest.java
@@ -23,18 +23,18 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.PinotTaskConfig;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.spi.data.DimensionFieldSpec;
-import org.apache.pinot.spi.data.FieldSpec;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.core.common.MinionConstants;
-import org.apache.pinot.spi.data.readers.GenericRow;
import org.apache.pinot.core.data.readers.GenericRowRecordReader;
import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.minion.PinotTaskConfig;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
import org.apache.pinot.minion.MinionContext;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
index bed1e86..fe50b79 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationTaskRunner.java
@@ -26,13 +26,13 @@ import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
import org.apache.pinot.core.segment.name.NormalizedDateSegmentNameGenerator;
import org.apache.pinot.core.segment.name.SegmentNameGenerator;
import org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.TimeFieldSpec;
import org.apache.pinot.spi.data.readers.RecordReader;
@@ -63,7 +63,7 @@ public class SegmentGenerationTaskRunner implements Serializable {
public String run()
throws Exception {
- TableConfig tableConfig = TableConfig.fromJsonConfig(_taskSpec.getTableConfig());
+ TableConfig tableConfig = JsonUtils.jsonNodeToObject(_taskSpec.getTableConfig(), TableConfig.class);
String tableName = tableConfig.getTableName();
Schema schema = _taskSpec.getSchema();
@@ -106,7 +106,7 @@ public class SegmentGenerationTaskRunner implements Serializable {
private SegmentNameGenerator getSegmentNameGerator()
throws IOException {
- TableConfig tableConfig = TableConfig.fromJsonConfig(_taskSpec.getTableConfig());
+ TableConfig tableConfig = JsonUtils.jsonNodeToObject(_taskSpec.getTableConfig(), TableConfig.class);
String tableName = tableConfig.getTableName();
Schema schema = _taskSpec.getSchema();
diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationUtils.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationUtils.java
index 6657e28..104d987 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationUtils.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/SegmentGenerationUtils.java
@@ -27,10 +27,11 @@ import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import org.apache.commons.io.IOUtils;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.JsonUtils;
public class SegmentGenerationUtils {
@@ -121,7 +122,7 @@ public class SegmentGenerationUtils {
tableJsonNode = tableJsonNode.get(OFFLINE);
}
try {
- return TableConfig.fromJsonConfig(tableJsonNode);
+ return JsonUtils.jsonNodeToObject(tableJsonNode, TableConfig.class);
} catch (IOException e) {
throw new RuntimeException("Failed to decode table config from JSON - '" + tableJsonNode + "'", e);
}
@@ -146,5 +147,4 @@ public class SegmentGenerationUtils {
URI relativeOutputURI = outputDir.resolve(relativePath).resolve(".");
return relativeOutputURI;
}
-
}
diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentCreationMapper.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentCreationMapper.java
index 1836fd8..44c4237 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentCreationMapper.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentCreationMapper.java
@@ -35,7 +35,7 @@ import org.apache.pinot.spi.filesystem.PinotFSFactory;
import org.apache.pinot.spi.ingestion.batch.spec.Constants;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationTaskSpec;
-import org.apache.pinot.spi.utils.DataSize;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.yaml.snakeyaml.Yaml;
@@ -166,7 +166,7 @@ public class HadoopSegmentCreationMapper extends Mapper<LongWritable, Text, Long
long uncompressedSegmentSize = FileUtils.sizeOf(localSegmentDir);
long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile);
LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName,
- DataSize.fromBytes(uncompressedSegmentSize), DataSize.fromBytes(compressedSegmentSize));
+ DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize));
//move segment to output PinotFS
URI outputSegmentTarURI = SegmentGenerationUtils.getRelativeOutputPath(inputDirURI, inputFileURI, outputDirURI)
.resolve(segmentTarFileName);
@@ -224,4 +224,4 @@ public class HadoopSegmentCreationMapper extends Mapper<LongWritable, Text, Long
}
}
}
-}
\ No newline at end of file
+}
diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark/SparkSegmentGenerationJobRunner.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark/SparkSegmentGenerationJobRunner.java
index a18308f..ef8bcce 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark/SparkSegmentGenerationJobRunner.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark/SparkSegmentGenerationJobRunner.java
@@ -43,7 +43,7 @@ import org.apache.pinot.spi.ingestion.batch.spec.PinotFSSpec;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationTaskSpec;
import org.apache.pinot.spi.plugin.PluginManager;
-import org.apache.pinot.spi.utils.DataSize;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -274,7 +274,7 @@ public class SparkSegmentGenerationJobRunner implements IngestionJobRunner, Seri
long uncompressedSegmentSize = FileUtils.sizeOf(localSegmentDir);
long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile);
LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName,
- DataSize.fromBytes(uncompressedSegmentSize), DataSize.fromBytes(compressedSegmentSize));
+ DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize));
//move segment to output PinotFS
URI outputSegmentTarURI =
SegmentGenerationUtils.getRelativeOutputPath(finalInputDirURI, inputFileURI, finalOutputDirURI)
diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/src/main/java/org/apache/pinot/plugin/ingestion/batch/standalone/SegmentGenerationJobRunner.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/src/main/java/org/apache/pinot/plugin/ingestion/batch/standalone/SegmentGenerationJobRunner.java
index b98a661..9be8c40 100644
--- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/src/main/java/org/apache/pinot/plugin/ingestion/batch/standalone/SegmentGenerationJobRunner.java
+++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/src/main/java/org/apache/pinot/plugin/ingestion/batch/standalone/SegmentGenerationJobRunner.java
@@ -28,10 +28,10 @@ import java.util.List;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.configuration.MapConfiguration;
import org.apache.commons.io.FileUtils;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationTaskRunner;
import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationUtils;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.filesystem.PinotFS;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
@@ -41,7 +41,7 @@ import org.apache.pinot.spi.ingestion.batch.spec.PinotClusterSpec;
import org.apache.pinot.spi.ingestion.batch.spec.PinotFSSpec;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationTaskSpec;
-import org.apache.pinot.spi.utils.DataSize;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -198,7 +198,7 @@ public class SegmentGenerationJobRunner implements IngestionJobRunner {
long uncompressedSegmentSize = FileUtils.sizeOf(localSegmentDir);
long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile);
LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName,
- DataSize.fromBytes(uncompressedSegmentSize), DataSize.fromBytes(compressedSegmentSize));
+ DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize));
//move segment to output PinotFS
URI outputSegmentTarURI = SegmentGenerationUtils.getRelativeOutputPath(inputDirURI, inputFileURI, outputDirURI)
.resolve(segmentTarFileName);
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentCreationJob.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentCreationJob.java
index 8403a5c..6ab4571 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentCreationJob.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentCreationJob.java
@@ -36,14 +36,14 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.StringUtil;
import org.apache.pinot.hadoop.job.mappers.SegmentCreationMapper;
import org.apache.pinot.hadoop.utils.PinotHadoopJobPreparationHelper;
import org.apache.pinot.ingestion.common.JobConfigConstants;
import org.apache.pinot.ingestion.jobs.SegmentCreationJob;
import org.apache.pinot.ingestion.utils.JobPreparationHelper;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
public class HadoopSegmentCreationJob extends SegmentCreationJob {
@@ -101,7 +101,7 @@ public class HadoopSegmentCreationJob extends SegmentCreationJob {
TableConfig tableConfig = getTableConfig();
if (tableConfig != null) {
validateTableConfig(tableConfig);
- jobConf.set(JobConfigConstants.TABLE_CONFIG, tableConfig.toJsonConfigString());
+ jobConf.set(JobConfigConstants.TABLE_CONFIG, tableConfig.toJsonString());
}
jobConf.set(JobConfigConstants.SCHEMA, getSchema().toSingleLineJsonString());
@@ -159,7 +159,8 @@ public class HadoopSegmentCreationJob extends SegmentCreationJob {
throws IOException {
if (_depsJarDir != null) {
PinotHadoopJobPreparationHelper
- .addDepsJarToDistributedCacheHelper(FileSystem.get(new Path(_depsJarDir).toUri(), getConf()), job, new Path(_depsJarDir));
+ .addDepsJarToDistributedCacheHelper(FileSystem.get(new Path(_depsJarDir).toUri(), getConf()), job,
+ new Path(_depsJarDir));
}
}
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentPreprocessingJob.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentPreprocessingJob.java
index 6c5c33c..12b3a11 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentPreprocessingJob.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/HadoopSegmentPreprocessingJob.java
@@ -48,12 +48,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TableCustomConfig;
import org.apache.pinot.core.data.partition.PartitionFunctionFactory;
import org.apache.pinot.hadoop.io.CombineAvroKeyInputFormat;
import org.apache.pinot.hadoop.job.mappers.SegmentPreprocessingMapper;
@@ -63,6 +57,12 @@ import org.apache.pinot.hadoop.utils.PinotHadoopJobPreparationHelper;
import org.apache.pinot.ingestion.common.ControllerRestApi;
import org.apache.pinot.ingestion.common.JobConfigConstants;
import org.apache.pinot.ingestion.jobs.SegmentPreprocessingJob;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.IndexingConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.config.TableCustomConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -220,9 +220,11 @@ public class HadoopSegmentPreprocessingJob extends SegmentPreprocessingJob {
// Fetch sorting info from table config.
IndexingConfig indexingConfig = _tableConfig.getIndexingConfig();
List<String> sortedColumns = indexingConfig.getSortedColumn();
- Preconditions.checkArgument(sortedColumns.size() <= 1, "There should be at most 1 sorted column in the table.");
- if (sortedColumns.size() == 1) {
- _sortedColumn = sortedColumns.get(0);
+ if (sortedColumns != null) {
+ Preconditions.checkArgument(sortedColumns.size() <= 1, "There should be at most 1 sorted column in the table.");
+ if (sortedColumns.size() == 1) {
+ _sortedColumn = sortedColumns.get(0);
+ }
}
}
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
index b43a83c..9e7f787 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/mappers/SegmentCreationMapper.java
@@ -34,17 +34,7 @@ import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig;
-import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.data.TimeFieldSpec;
-import org.apache.pinot.spi.utils.DataSize;
-import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
-import org.apache.pinot.spi.data.readers.FileFormat;
-import org.apache.pinot.spi.data.readers.RecordReaderConfig;
-import org.apache.pinot.plugin.inputformat.thrift.ThriftRecordReaderConfig;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
@@ -52,6 +42,16 @@ import org.apache.pinot.core.segment.name.NormalizedDateSegmentNameGenerator;
import org.apache.pinot.core.segment.name.SegmentNameGenerator;
import org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator;
import org.apache.pinot.ingestion.common.JobConfigConstants;
+import org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig;
+import org.apache.pinot.plugin.inputformat.thrift.ThriftRecordReaderConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeFieldSpec;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.RecordReaderConfig;
+import org.apache.pinot.spi.utils.DataSizeUtils;
+import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -113,7 +113,7 @@ public class SegmentCreationMapper extends Mapper<LongWritable, Text, LongWritab
// Optional
String tableConfigString = _jobConf.get(JobConfigConstants.TABLE_CONFIG);
if (tableConfigString != null) {
- _tableConfig = TableConfig.fromJsonString(tableConfigString);
+ _tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
}
String readerConfigFile = _jobConf.get(JobConfigConstants.PATH_TO_READER_CONFIG);
if (readerConfigFile != null) {
@@ -269,7 +269,7 @@ public class SegmentCreationMapper extends Mapper<LongWritable, Text, LongWritab
long uncompressedSegmentSize = FileUtils.sizeOf(localSegmentDir);
long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile);
_logger.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName,
- DataSize.fromBytes(uncompressedSegmentSize), DataSize.fromBytes(compressedSegmentSize));
+ DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize));
Path hdfsSegmentTarFile = new Path(_hdfsSegmentTarDir, segmentTarFileName);
if (_useRelativePath) {
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/test/java/org/apache/pinot/integration/tests/HadoopSegmentBuildPushOfflineClusterIntegrationTest.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/test/java/org/apache/pinot/integration/tests/HadoopSegmentBuildPushOfflineClusterIntegrationTest.java
index 390f056..f93aa7f 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/test/java/org/apache/pinot/integration/tests/HadoopSegmentBuildPushOfflineClusterIntegrationTest.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-hadoop/src/test/java/org/apache/pinot/integration/tests/HadoopSegmentBuildPushOfflineClusterIntegrationTest.java
@@ -38,9 +38,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
-import org.apache.pinot.common.config.ColumnPartitionConfig;
-import org.apache.pinot.common.config.SegmentPartitionConfig;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.core.data.partition.PartitionFunction;
import org.apache.pinot.core.data.partition.PartitionFunctionFactory;
import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
@@ -48,6 +45,9 @@ import org.apache.pinot.hadoop.job.HadoopSegmentCreationJob;
import org.apache.pinot.hadoop.job.HadoopSegmentPreprocessingJob;
import org.apache.pinot.ingestion.common.JobConfigConstants;
import org.apache.pinot.ingestion.jobs.SegmentTarPushJob;
+import org.apache.pinot.spi.config.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.SegmentPartitionConfig;
+import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.util.TestUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/ControllerRestApi.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/ControllerRestApi.java
index bb33974..236374e 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/ControllerRestApi.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/ControllerRestApi.java
@@ -22,7 +22,7 @@ import java.io.Closeable;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/DefaultControllerRestApi.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/DefaultControllerRestApi.java
index ee0e02d..4ef276f 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/DefaultControllerRestApi.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/common/DefaultControllerRestApi.java
@@ -27,10 +27,10 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.common.utils.SimpleHttpResponse;
import org.apache.pinot.ingestion.utils.PushLocation;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
@@ -66,7 +66,7 @@ public class DefaultControllerRestApi implements ControllerRestApi {
.getRetrieveTableConfigHttpURI(pushLocation.getHost(), pushLocation.getPort(), _rawTableName));
JsonNode offlineJsonTableConfig = JsonUtils.stringToJsonNode(response.getResponse()).get(OFFLINE);
if (offlineJsonTableConfig != null) {
- TableConfig offlineTableConfig = TableConfig.fromJsonConfig(offlineJsonTableConfig);
+ TableConfig offlineTableConfig = JsonUtils.jsonNodeToObject(offlineJsonTableConfig, TableConfig.class);
LOGGER.info("Got table config: {}", offlineTableConfig);
return offlineTableConfig;
}
@@ -121,7 +121,9 @@ public class DefaultControllerRestApi implements ControllerRestApi {
LOGGER.error("Caught exception while pushing segment: {} to location: {}, retry {}/{}", segmentName,
pushLocation, retry, _retry, e);
if (retry == _retry) {
- throw new RuntimeException(String.format("Failed to push segment %s to %s with %d retries",segmentName, pushLocation, retry), e);
+ throw new RuntimeException(
+ String.format("Failed to push segment %s to %s with %d retries", segmentName, pushLocation, retry),
+ e);
}
try {
// Exponential back-off, max sleep time is 64 seconds.
@@ -206,4 +208,4 @@ public class DefaultControllerRestApi implements ControllerRestApi {
throws Exception {
return JsonUtils.stringToJsonNode(json).get(0).get(tableType);
}
-}
\ No newline at end of file
+}
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/jobs/BaseSegmentJob.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/jobs/BaseSegmentJob.java
index f3b9c75..b9faf12 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/jobs/BaseSegmentJob.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-ingestion-common/src/main/java/org/apache/pinot/ingestion/jobs/BaseSegmentJob.java
@@ -31,13 +31,13 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.pinot.common.Utils;
import org.apache.pinot.ingestion.common.ControllerRestApi;
import org.apache.pinot.ingestion.common.DefaultControllerRestApi;
import org.apache.pinot.ingestion.common.JobConfigConstants;
-import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.ingestion.utils.PushLocation;
+import org.apache.pinot.spi.config.TableConfig;
+import org.apache.pinot.spi.data.Schema;
import org.joda.time.DateTime;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
index c564bba..667d0b0 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationFunction.java
@@ -31,29 +31,30 @@ import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.spi.utils.DataSize;
import org.apache.pinot.common.utils.TarGzCompressionUtils;
-import org.apache.pinot.spi.data.readers.FileFormat;
-import org.apache.pinot.plugin.inputformat.thrift.ThriftRecordReaderConfig;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
import org.apache.pinot.core.segment.name.NormalizedDateSegmentNameGenerator;
import org.apache.pinot.core.segment.name.SegmentNameGenerator;
import org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator;
-import org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig;
import org.apache.pinot.ingestion.common.JobConfigConstants;
+import org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig;
+import org.apache.pinot.plugin.inputformat.thrift.ThriftRecordReaderConfig;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.data.TimeFieldSpec;
+import org.apache.pinot.spi.data.readers.FileFormat;
import org.apache.pinot.spi.data.readers.RecordReaderConfig;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.pinot.spark.jobs.SparkSegmentCreationJob.getRelativeOutputPath;
+
public class SparkSegmentCreationFunction implements Serializable {
protected static final String LOCAL_TEMP_DIR = "pinot_spark_tmp";
@@ -95,7 +96,7 @@ public class SparkSegmentCreationFunction implements Serializable {
// Optional
String tableConfigString = _jobConf.get(JobConfigConstants.TABLE_CONFIG);
if (tableConfigString != null) {
- _tableConfig = TableConfig.fromJsonString(tableConfigString);
+ _tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
}
String readerConfigFile = _jobConf.get(JobConfigConstants.PATH_TO_READER_CONFIG);
if (readerConfigFile != null) {
@@ -237,7 +238,7 @@ public class SparkSegmentCreationFunction implements Serializable {
long uncompressedSegmentSize = FileUtils.sizeOf(localSegmentDir);
long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile);
_logger.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName,
- DataSize.fromBytes(uncompressedSegmentSize), DataSize.fromBytes(compressedSegmentSize));
+ DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize));
Path hdfsSegmentTarFile = new Path(_hdfsSegmentTarDir, segmentTarFileName);
if (_useRelativePath) {
diff --git a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationJob.java b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationJob.java
index 774cffa..733a3ec 100644
--- a/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationJob.java
+++ b/pinot-plugins/pinot-batch-ingestion/v0_deprecated/pinot-spark/src/main/java/org/apache/pinot/spark/jobs/SparkSegmentCreationJob.java
@@ -29,14 +29,14 @@ import java.util.Properties;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.utils.StringUtil;
import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
import org.apache.pinot.ingestion.common.JobConfigConstants;
import org.apache.pinot.ingestion.jobs.SegmentCreationJob;
import org.apache.pinot.ingestion.utils.JobPreparationHelper;
import org.apache.pinot.spark.utils.PinotSparkJobPreparationHelper;
+import org.apache.pinot.spi.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -109,7 +109,7 @@ public class SparkSegmentCreationJob extends SegmentCreationJob {
TableConfig tableConfig = getTableConfig();
if (tableConfig != null) {
validateTableConfig(tableConfig);
- _properties.put(JobConfigConstants.TABLE_CONFIG, tableConfig.toJsonConfigString());
+ _properties.put(JobConfigConstants.TABLE_CONFIG, tableConfig.toJsonString());
}
_properties.put(JobConfigConstants.SCHEMA, getSchema().toSingleLineJsonString());
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
index ddfbd14..beb3c51 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
@@ -34,7 +34,6 @@ import org.apache.commons.io.FileUtils;
import org.apache.helix.HelixManager;
import org.apache.helix.ZNRecord;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.TableConfig;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.ServerMetrics;
import org.apache.pinot.common.utils.CommonConstants;
@@ -49,6 +48,7 @@ import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
import org.apache.pinot.core.segment.index.loader.LoaderUtils;
import org.apache.pinot.core.segment.index.metadata.SegmentMetadata;
+import org.apache.pinot.spi.config.TableConfig;
import org.apache.pinot.spi.data.Schema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
index f46023d..a61bdf5 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
@@ -48,8 +48,6 @@ import org.apache.helix.model.Message;
import org.apache.helix.model.builder.HelixConfigScopeBuilder;
import org.apache.helix.participant.statemachine.StateModelFactory;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.ServerMeter;
import org.apache.pinot.common.metrics.ServerMetrics;
@@ -57,6 +55,7 @@ import org.apache.pinot.common.utils.CommonConstants;
import org.apache.pinot.common.utils.NetUtil;
import org.apache.pinot.common.utils.ServiceStatus;
import org.apache.pinot.common.utils.ServiceStatus.Status;
+import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.core.data.manager.InstanceDataManager;
import org.apache.pinot.core.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState;
import org.apache.pinot.core.segment.memory.PinotDataBuffer;
@@ -65,6 +64,7 @@ import org.apache.pinot.server.realtime.ControllerLeaderLocator;
import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
import org.apache.pinot.server.starter.ServerInstance;
import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
index 910a6b9..60ae0a4 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
@@ -29,16 +29,16 @@ import org.apache.helix.participant.statemachine.StateModelFactory;
import org.apache.helix.participant.statemachine.StateModelInfo;
import org.apache.helix.participant.statemachine.Transition;
import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.TableNameBuilder;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
import org.apache.pinot.common.utils.LLCSegmentName;
import org.apache.pinot.common.utils.SegmentName;
import org.apache.pinot.core.data.manager.InstanceDataManager;
import org.apache.pinot.core.data.manager.SegmentDataManager;
import org.apache.pinot.core.data.manager.TableDataManager;
import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.spi.config.TableType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/BaseJsonConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/BaseJsonConfig.java
similarity index 83%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/BaseJsonConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/BaseJsonConfig.java
index 401f109..33661f6 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/BaseJsonConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/BaseJsonConfig.java
@@ -16,9 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
import org.apache.pinot.spi.utils.JsonUtils;
@@ -27,6 +29,7 @@ import org.apache.pinot.spi.utils.JsonUtils;
* Base implementation for the JSON based configurations.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonInclude(JsonInclude.Include.NON_NULL)
public abstract class BaseJsonConfig {
public JsonNode toJsonNode() {
@@ -34,7 +37,11 @@ public abstract class BaseJsonConfig {
}
public String toJsonString() {
- return toJsonNode().toString();
+ try {
+ return JsonUtils.objectToString(this);
+ } catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
}
@Override
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/ColumnPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/ColumnPartitionConfig.java
similarity index 90%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/ColumnPartitionConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/ColumnPartitionConfig.java
index af95926..18d51a3 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/ColumnPartitionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/ColumnPartitionConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -27,12 +27,6 @@ public class ColumnPartitionConfig extends BaseJsonConfig {
private final String _functionName;
private final int _numPartitions;
- /**
- * Constructor for the class.
- *
- * @param functionName Name of the partition function.
- * @param numPartitions Number of partitions for this column.
- */
@JsonCreator
public ColumnPartitionConfig(@JsonProperty(value = "functionName", required = true) String functionName,
@JsonProperty(value = "numPartitions", required = true) int numPartitions) {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/CompletionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/CompletionConfig.java
similarity index 96%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/CompletionConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/CompletionConfig.java
index fcc1b1f..53f811d9 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/CompletionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/CompletionConfig.java
@@ -16,13 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import com.google.common.base.Preconditions;
-import javax.annotation.Nullable;
/**
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/FieldConfig.java
similarity index 92%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/FieldConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/FieldConfig.java
index 45e3c8e..460db74 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/FieldConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/FieldConfig.java
@@ -16,10 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
import java.util.Map;
import javax.annotation.Nullable;
@@ -40,6 +41,7 @@ public class FieldConfig extends BaseJsonConfig {
@JsonProperty(value = "encodingType") @Nullable EncodingType encodingType,
@JsonProperty(value = "indexType") @Nullable IndexType indexType,
@JsonProperty(value = "properties") @Nullable Map<String, String> properties) {
+ Preconditions.checkArgument(name != null, "'name' must be configured");
_name = name;
_encodingType = encodingType;
_indexType = indexType;
@@ -56,18 +58,21 @@ public class FieldConfig extends BaseJsonConfig {
INVERTED, SORTED, TEXT
}
+ public String getName() {
+ return _name;
+ }
+
+ @Nullable
public EncodingType getEncodingType() {
return _encodingType;
}
+ @Nullable
public IndexType getIndexType() {
return _indexType;
}
- public String getName() {
- return _name;
- }
-
+ @Nullable
public Map<String, String> getProperties() {
return _properties;
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/IndexingConfig.java
similarity index 90%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/IndexingConfig.java
index 6c416e7..e5dd595 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/IndexingConfig.java
@@ -16,22 +16,21 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
-import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import javax.annotation.Nullable;
public class IndexingConfig extends BaseJsonConfig {
private List<String> _invertedIndexColumns;
private boolean _autoGeneratedInvertedIndex;
private boolean _createInvertedIndexDuringSegmentGeneration;
- private List<String> _sortedColumn = new ArrayList<>();
- private List<String> _bloomFilterColumns = new ArrayList<>();
+ private List<String> _sortedColumn;
+ private List<String> _bloomFilterColumns;
private String _loadMode;
- private Map<String, String> _streamConfigs = new HashMap<>();
+ private Map<String, String> _streamConfigs;
private String _segmentFormatVersion;
private String _columnMinMaxValueGeneratorMode;
private List<String> _noDictionaryColumns; // TODO: replace this with noDictionaryConfig.
@@ -49,6 +48,7 @@ public class IndexingConfig extends BaseJsonConfig {
*/
private List<String> _varLengthDictionaryColumns;
+ @Nullable
public List<String> getInvertedIndexColumns() {
return _invertedIndexColumns;
}
@@ -73,6 +73,7 @@ public class IndexingConfig extends BaseJsonConfig {
_createInvertedIndexDuringSegmentGeneration = createInvertedIndexDuringSegmentGeneration;
}
+ @Nullable
public List<String> getSortedColumn() {
return _sortedColumn;
}
@@ -81,6 +82,7 @@ public class IndexingConfig extends BaseJsonConfig {
_sortedColumn = sortedColumn;
}
+ @Nullable
public List<String> getBloomFilterColumns() {
return _bloomFilterColumns;
}
@@ -89,6 +91,7 @@ public class IndexingConfig extends BaseJsonConfig {
_bloomFilterColumns = bloomFilterColumns;
}
+ @Nullable
public String getLoadMode() {
return _loadMode;
}
@@ -97,6 +100,7 @@ public class IndexingConfig extends BaseJsonConfig {
_loadMode = loadMode;
}
+ @Nullable
public Map<String, String> getStreamConfigs() {
return _streamConfigs;
}
@@ -105,6 +109,7 @@ public class IndexingConfig extends BaseJsonConfig {
_streamConfigs = streamConfigs;
}
+ @Nullable
public String getSegmentFormatVersion() {
return _segmentFormatVersion;
}
@@ -113,6 +118,7 @@ public class IndexingConfig extends BaseJsonConfig {
_segmentFormatVersion = segmentFormatVersion;
}
+ @Nullable
public String getColumnMinMaxValueGeneratorMode() {
return _columnMinMaxValueGeneratorMode;
}
@@ -121,30 +127,43 @@ public class IndexingConfig extends BaseJsonConfig {
_columnMinMaxValueGeneratorMode = columnMinMaxValueGeneratorMode;
}
+ @Nullable
public List<String> getNoDictionaryColumns() {
return _noDictionaryColumns;
}
+ public void setNoDictionaryColumns(List<String> noDictionaryColumns) {
+ _noDictionaryColumns = noDictionaryColumns;
+ }
+
+ @Nullable
public Map<String, String> getNoDictionaryConfig() {
return _noDictionaryConfig;
}
+ public void setNoDictionaryConfig(Map<String, String> noDictionaryConfig) {
+ _noDictionaryConfig = noDictionaryConfig;
+ }
+
+ @Nullable
public List<String> getOnHeapDictionaryColumns() {
return _onHeapDictionaryColumns;
}
- public void setNoDictionaryColumns(List<String> noDictionaryColumns) {
- _noDictionaryColumns = noDictionaryColumns;
+ public void setOnHeapDictionaryColumns(List<String> onHeapDictionaryColumns) {
+ _onHeapDictionaryColumns = onHeapDictionaryColumns;
}
- public void setnoDictionaryConfig(Map<String, String> noDictionaryConfig) {
- _noDictionaryConfig = noDictionaryConfig;
+ @Nullable
+ public List<String> getVarLengthDictionaryColumns() {
+ return _varLengthDictionaryColumns;
}
- public void setOnHeapDictionaryColumns(List<String> onHeapDictionaryColumns) {
- _onHeapDictionaryColumns = onHeapDictionaryColumns;
+ public void setVarLengthDictionaryColumns(List<String> varLengthDictionaryColumns) {
+ _varLengthDictionaryColumns = varLengthDictionaryColumns;
}
+ @Nullable
public List<StarTreeIndexConfig> getStarTreeIndexConfigs() {
return _starTreeIndexConfigs;
}
@@ -153,28 +172,21 @@ public class IndexingConfig extends BaseJsonConfig {
_starTreeIndexConfigs = starTreeIndexConfigs;
}
- public void setSegmentPartitionConfig(SegmentPartitionConfig config) {
- _segmentPartitionConfig = config;
- }
-
- public void setAggregateMetrics(boolean value) {
- _aggregateMetrics = value;
- }
-
+ @Nullable
public SegmentPartitionConfig getSegmentPartitionConfig() {
return _segmentPartitionConfig;
}
- public boolean isAggregateMetrics() {
- return _aggregateMetrics;
+ public void setSegmentPartitionConfig(SegmentPartitionConfig segmentPartitionConfig) {
+ _segmentPartitionConfig = segmentPartitionConfig;
}
- public List<String> getVarLengthDictionaryColumns() {
- return _varLengthDictionaryColumns;
+ public boolean isAggregateMetrics() {
+ return _aggregateMetrics;
}
- public void setVarLengthDictionaryColumns(List<String> varLengthDictionaryColumns) {
- _varLengthDictionaryColumns = varLengthDictionaryColumns;
+ public void setAggregateMetrics(boolean value) {
+ _aggregateMetrics = value;
}
public boolean isNullHandlingEnabled() {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/QueryConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/QueryConfig.java
similarity index 92%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/QueryConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/QueryConfig.java
index 84fb6e9..34fb5ac 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/QueryConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/QueryConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -38,9 +38,7 @@ public class QueryConfig extends BaseJsonConfig {
@JsonCreator
public QueryConfig(@JsonProperty("timeoutMs") @Nullable Long timeoutMs) {
- if (timeoutMs != null) {
- Preconditions.checkArgument(timeoutMs > 0, "Invalid timeoutMs: %s", timeoutMs);
- }
+ Preconditions.checkArgument(timeoutMs == null || timeoutMs > 0, "Invalid 'timeoutMs': %s", timeoutMs);
_timeoutMs = timeoutMs;
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/QuotaConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/QuotaConfig.java
similarity index 50%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/QuotaConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/QuotaConfig.java
index a7f81e4..6bd9294 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/QuotaConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/QuotaConfig.java
@@ -16,35 +16,60 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import com.google.common.base.Preconditions;
import javax.annotation.Nullable;
-import org.apache.commons.configuration.ConfigurationRuntimeException;
-import org.apache.pinot.spi.utils.DataSize;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.pinot.spi.utils.DataSizeUtils;
/**
* Class representing table quota configuration
*/
public class QuotaConfig extends BaseJsonConfig {
- private static final Logger LOGGER = LoggerFactory.getLogger(QuotaConfig.class);
+ private static final long INVALID_STORAGE_IN_BYTES = -1L;
+ private static final double INVALID_MAX_QPS = -1.0;
@JsonPropertyDescription("Storage allocated for this table, e.g. \"10G\"")
private final String _storage;
private final String _maxQueriesPerSecond;
+ // NOTE: These two fields are not to be serialized
+ private transient final long _storageInBytes;
+ private transient final double _maxQPS;
+
@JsonCreator
public QuotaConfig(@JsonProperty("storage") @Nullable String storage,
@JsonProperty("maxQueriesPerSecond") @Nullable String maxQueriesPerSecond) {
- _storage = storage;
- _maxQueriesPerSecond = maxQueriesPerSecond;
+ // Validate and standardize the value
+ if (storage != null) {
+ try {
+ _storageInBytes = DataSizeUtils.toBytes(storage);
+ } catch (Exception e) {
+ throw new IllegalArgumentException("Invalid 'storage': " + storage);
+ }
+ _storage = DataSizeUtils.fromBytes(_storageInBytes);
+ } else {
+ _storageInBytes = INVALID_STORAGE_IN_BYTES;
+ _storage = null;
+ }
+ if (maxQueriesPerSecond != null) {
+ try {
+ _maxQPS = Double.parseDouble(maxQueriesPerSecond);
+ Preconditions.checkArgument(_maxQPS > 0);
+ } catch (Exception e) {
+ throw new IllegalArgumentException("Invalid 'maxQueriesPerSecond': " + storage);
+ }
+ _maxQueriesPerSecond = Double.toString(_maxQPS);
+ } else {
+ _maxQPS = INVALID_MAX_QPS;
+ _maxQueriesPerSecond = null;
+ }
}
@Nullable
@@ -57,46 +82,13 @@ public class QuotaConfig extends BaseJsonConfig {
return _maxQueriesPerSecond;
}
- /**
- * Get the storage quota configured value in bytes
- * @return configured size in bytes or -1 if the value is missing or
- * unparseable
- */
- public long storageSizeBytes() {
- return DataSize.toBytes(_storage);
- }
-
- public void validate() {
- if (!isStorageValid()) {
- LOGGER.error("Failed to convert storage quota config: {} to bytes", _storage);
- throw new ConfigurationRuntimeException("Failed to convert storage quota config: " + _storage + " to bytes");
- }
- if (!isMaxQueriesPerSecondValid()) {
- LOGGER.error("Failed to convert qps quota config: {}", _maxQueriesPerSecond);
- throw new ConfigurationRuntimeException("Failed to convert qps quota config: " + _maxQueriesPerSecond);
- }
- }
-
@JsonIgnore
- public boolean isStorageValid() {
- return _storage == null || DataSize.toBytes(_storage) >= 0;
+ public long getStorageInBytes() {
+ return _storageInBytes;
}
@JsonIgnore
- public boolean isMaxQueriesPerSecondValid() {
- Double qps = null;
- if (_maxQueriesPerSecond != null) {
- try {
- qps = Double.parseDouble(_maxQueriesPerSecond);
- } catch (NumberFormatException e) {
- LOGGER.error("Failed to convert qps quota config: {}", _maxQueriesPerSecond);
- return false;
- }
- if (qps <= 0) {
- LOGGER.error("Failed to convert qps quota config: {}", _maxQueriesPerSecond);
- return false;
- }
- }
- return _maxQueriesPerSecond == null || qps > 0;
+ public double getMaxQPS() {
+ return _maxQPS;
}
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/ReplicaGroupStrategyConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/ReplicaGroupStrategyConfig.java
similarity index 98%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/ReplicaGroupStrategyConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/ReplicaGroupStrategyConfig.java
index 55bbbc3..573d34c 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/ReplicaGroupStrategyConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/ReplicaGroupStrategyConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/RoutingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/RoutingConfig.java
similarity index 98%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/RoutingConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/RoutingConfig.java
index 5b93b6c..11a2ab7 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/RoutingConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/RoutingConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/SegmentPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentPartitionConfig.java
similarity index 89%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/SegmentPartitionConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentPartitionConfig.java
index 172db19..bfed1fa 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/SegmentPartitionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentPartitionConfig.java
@@ -16,15 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
-import java.io.IOException;
import java.util.Map;
import javax.annotation.Nullable;
-import org.apache.pinot.spi.utils.JsonUtils;
public class SegmentPartitionConfig extends BaseJsonConfig {
@@ -39,11 +37,6 @@ public class SegmentPartitionConfig extends BaseJsonConfig {
_columnPartitionMap = columnPartitionMap;
}
- public static SegmentPartitionConfig fromJsonString(String jsonString)
- throws IOException {
- return JsonUtils.stringToObject(jsonString, SegmentPartitionConfig.class);
- }
-
public Map<String, ColumnPartitionConfig> getColumnPartitionMap() {
return _columnPartitionMap;
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/SegmentsValidationAndRetentionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentsValidationAndRetentionConfig.java
similarity index 97%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/SegmentsValidationAndRetentionConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentsValidationAndRetentionConfig.java
index d12b7a4..a848d68 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/SegmentsValidationAndRetentionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/SegmentsValidationAndRetentionConfig.java
@@ -16,13 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonIgnore;
import java.util.concurrent.TimeUnit;
import org.apache.pinot.spi.utils.TimeUtils;
+// TODO: Consider break this config into multiple configs
public class SegmentsValidationAndRetentionConfig extends BaseJsonConfig {
private String _retentionTimeUnit;
private String _retentionTimeValue;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/StarTreeIndexConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/StarTreeIndexConfig.java
similarity index 98%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/StarTreeIndexConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/StarTreeIndexConfig.java
index 84ff0fa..a6ae33b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/StarTreeIndexConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/StarTreeIndexConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfig.java
new file mode 100644
index 0000000..45cebc8
--- /dev/null
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfig.java
@@ -0,0 +1,220 @@
+/**
+ * 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.pinot.spi.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.assignment.InstanceAssignmentConfig;
+import org.apache.pinot.spi.config.assignment.InstancePartitionsType;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+
+
+@SuppressWarnings("unused")
+public class TableConfig extends BaseJsonConfig {
+ public static final String TABLE_NAME_KEY = "tableName";
+ public static final String TABLE_TYPE_KEY = "tableType";
+ public static final String VALIDATION_CONFIG_KEY = "segmentsConfig";
+ public static final String TENANT_CONFIG_KEY = "tenants";
+ public static final String INDEXING_CONFIG_KEY = "tableIndexConfig";
+ public static final String CUSTOM_CONFIG_KEY = "metadata";
+ public static final String QUOTA_CONFIG_KEY = "quota";
+ public static final String TASK_CONFIG_KEY = "task";
+ public static final String ROUTING_CONFIG_KEY = "routing";
+ public static final String QUERY_CONFIG_KEY = "query";
+ public static final String INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY = "instanceAssignmentConfigMap";
+ public static final String FIELD_CONFIG_LIST_KEY = "fieldConfigList";
+
+ // Double underscore is reserved for real-time segment name delimiter
+ private static final String TABLE_NAME_FORBIDDEN_SUBSTRING = "__";
+
+ /* MANDATORY FIELDS */
+
+ @JsonPropertyDescription("The name for the table (with type suffix), e.g. \"myTable_OFFLINE\" (mandatory)")
+ private final String _tableName;
+
+ @JsonPropertyDescription(value = "The type of the table (OFFLINE|REALTIME) (mandatory)")
+ private final TableType _tableType;
+
+ private SegmentsValidationAndRetentionConfig _validationConfig;
+ private TenantConfig _tenantConfig;
+ private IndexingConfig _indexingConfig;
+
+ // TODO: Make TableCustomConfig optional and use another key other than 'metadata'
+ private TableCustomConfig _customConfig;
+
+ /* OPTIONAL FIELDS */
+
+ @JsonPropertyDescription("Resource quota associated with this table")
+ private QuotaConfig _quotaConfig;
+
+ private TableTaskConfig _taskConfig;
+ private RoutingConfig _routingConfig;
+ private QueryConfig _queryConfig;
+ private Map<InstancePartitionsType, InstanceAssignmentConfig> _instanceAssignmentConfigMap;
+ private List<FieldConfig> _fieldConfigList;
+
+ @JsonCreator
+ public TableConfig(@JsonProperty(value = TABLE_NAME_KEY, required = true) String tableName,
+ @JsonProperty(value = TABLE_TYPE_KEY, required = true) String tableType,
+ @JsonProperty(value = VALIDATION_CONFIG_KEY, required = true) SegmentsValidationAndRetentionConfig validationConfig,
+ @JsonProperty(value = TENANT_CONFIG_KEY, required = true) TenantConfig tenantConfig,
+ @JsonProperty(value = INDEXING_CONFIG_KEY, required = true) IndexingConfig indexingConfig,
+ @JsonProperty(value = CUSTOM_CONFIG_KEY, required = true) TableCustomConfig customConfig,
+ @JsonProperty(QUOTA_CONFIG_KEY) @Nullable QuotaConfig quotaConfig,
+ @JsonProperty(TASK_CONFIG_KEY) @Nullable TableTaskConfig taskConfig,
+ @JsonProperty(ROUTING_CONFIG_KEY) @Nullable RoutingConfig routingConfig,
+ @JsonProperty(QUERY_CONFIG_KEY) @Nullable QueryConfig queryConfig,
+ @JsonProperty(INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY) @Nullable Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap,
+ @JsonProperty(FIELD_CONFIG_LIST_KEY) @Nullable List<FieldConfig> fieldConfigList) {
+ Preconditions.checkArgument(tableName != null, "'tableName' must be configured");
+ Preconditions.checkArgument(!tableName.contains(TABLE_NAME_FORBIDDEN_SUBSTRING),
+ "'tableName' cannot contain double underscore ('__')");
+ Preconditions.checkArgument(tableType != null, "'tableType' must be configured");
+ Preconditions.checkArgument(validationConfig != null, "'segmentsConfig' must be configured");
+ Preconditions.checkArgument(tenantConfig != null, "'tenants' must be configured");
+ Preconditions.checkArgument(indexingConfig != null, "'tableIndexConfig' must be configured");
+ Preconditions.checkArgument(customConfig != null, "'metadata' must be configured");
+
+ // NOTE: Handle lower case table type and raw table name for backward-compatibility
+ _tableType = TableType.valueOf(tableType.toUpperCase());
+ _tableName = TableNameBuilder.forType(_tableType).tableNameWithType(tableName);
+ _validationConfig = validationConfig;
+ _tenantConfig = tenantConfig;
+ _indexingConfig = indexingConfig;
+ _customConfig = customConfig;
+ _quotaConfig = quotaConfig;
+ _taskConfig = taskConfig;
+ _routingConfig = routingConfig;
+ _queryConfig = queryConfig;
+ _instanceAssignmentConfigMap = instanceAssignmentConfigMap;
+ _fieldConfigList = fieldConfigList;
+ }
+
+ @JsonProperty(TABLE_NAME_KEY)
+ public String getTableName() {
+ return _tableName;
+ }
+
+ @JsonProperty(TABLE_TYPE_KEY)
+ public TableType getTableType() {
+ return _tableType;
+ }
+
+ @JsonProperty(VALIDATION_CONFIG_KEY)
+ public SegmentsValidationAndRetentionConfig getValidationConfig() {
+ return _validationConfig;
+ }
+
+ public void setValidationConfig(SegmentsValidationAndRetentionConfig validationConfig) {
+ _validationConfig = validationConfig;
+ }
+
+ @JsonProperty(TENANT_CONFIG_KEY)
+ public TenantConfig getTenantConfig() {
+ return _tenantConfig;
+ }
+
+ public void setTenantConfig(TenantConfig tenantConfig) {
+ _tenantConfig = tenantConfig;
+ }
+
+ @JsonProperty(INDEXING_CONFIG_KEY)
+ public IndexingConfig getIndexingConfig() {
+ return _indexingConfig;
+ }
+
+ public void setIndexingConfig(IndexingConfig indexingConfig) {
+ _indexingConfig = indexingConfig;
+ }
+
+ @JsonProperty(CUSTOM_CONFIG_KEY)
+ public TableCustomConfig getCustomConfig() {
+ return _customConfig;
+ }
+
+ public void setCustomConfig(TableCustomConfig customConfig) {
+ _customConfig = customConfig;
+ }
+
+ @JsonProperty(QUOTA_CONFIG_KEY)
+ @Nullable
+ public QuotaConfig getQuotaConfig() {
+ return _quotaConfig;
+ }
+
+ public void setQuotaConfig(QuotaConfig quotaConfig) {
+ _quotaConfig = quotaConfig;
+ }
+
+ @JsonProperty(TASK_CONFIG_KEY)
+ @Nullable
+ public TableTaskConfig getTaskConfig() {
+ return _taskConfig;
+ }
+
+ public void setTaskConfig(TableTaskConfig taskConfig) {
+ _taskConfig = taskConfig;
+ }
+
+ @JsonProperty(ROUTING_CONFIG_KEY)
+ @Nullable
+ public RoutingConfig getRoutingConfig() {
+ return _routingConfig;
+ }
+
+ public void setRoutingConfig(RoutingConfig routingConfig) {
+ _routingConfig = routingConfig;
+ }
+
+ @JsonProperty(QUERY_CONFIG_KEY)
+ @Nullable
+ public QueryConfig getQueryConfig() {
+ return _queryConfig;
+ }
+
+ public void setQueryConfig(QueryConfig queryConfig) {
+ _queryConfig = queryConfig;
+ }
+
+ @JsonProperty(INSTANCE_ASSIGNMENT_CONFIG_MAP_KEY)
+ @Nullable
+ public Map<InstancePartitionsType, InstanceAssignmentConfig> getInstanceAssignmentConfigMap() {
+ return _instanceAssignmentConfigMap;
+ }
+
+ public void setInstanceAssignmentConfigMap(
+ Map<InstancePartitionsType, InstanceAssignmentConfig> instanceAssignmentConfigMap) {
+ _instanceAssignmentConfigMap = instanceAssignmentConfigMap;
+ }
+
+ @JsonProperty(FIELD_CONFIG_LIST_KEY)
+ @Nullable
+ public List<FieldConfig> getFieldConfigList() {
+ return _fieldConfigList;
+ }
+
+ public void setFieldConfigList(List<FieldConfig> fieldConfigList) {
+ _fieldConfigList = fieldConfigList;
+ }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TableCustomConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableCustomConfig.java
similarity index 97%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/TableCustomConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/TableCustomConfig.java
index 6126a07..8fa968b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TableCustomConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableCustomConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TableTaskConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableTaskConfig.java
similarity index 97%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/TableTaskConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/TableTaskConfig.java
index c580128..eda00b9 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TableTaskConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableTaskConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/ServerType.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableType.java
similarity index 92%
rename from pinot-common/src/main/java/org/apache/pinot/common/utils/ServerType.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/TableType.java
index b17ff06..06a74c4 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/ServerType.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableType.java
@@ -16,8 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.utils;
+package org.apache.pinot.spi.config;
-public enum ServerType {
+public enum TableType {
OFFLINE, REALTIME
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TagOverrideConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TagOverrideConfig.java
similarity index 98%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/TagOverrideConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/TagOverrideConfig.java
index dc9d117..d1dffde 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TagOverrideConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TagOverrideConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TenantConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TenantConfig.java
similarity index 97%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/TenantConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/TenantConfig.java
index fdd3ae6..287a459 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TenantConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TenantConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/Instance.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Instance.java
similarity index 60%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/Instance.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Instance.java
index 228d9c7..0113902 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/Instance.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Instance.java
@@ -16,19 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config.api;
import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import java.util.List;
import java.util.Map;
-import java.util.TreeMap;
import javax.annotation.Nullable;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix;
-import org.apache.pinot.common.utils.CommonConstants.Helix.InstanceType;
+import org.apache.pinot.spi.config.BaseJsonConfig;
/**
@@ -47,8 +43,6 @@ import org.apache.pinot.common.utils.CommonConstants.Helix.InstanceType;
* </pre>
*/
public class Instance extends BaseJsonConfig {
- public static final String POOL_KEY = "pool";
-
private final String _host;
private final int _port;
private final InstanceType _type;
@@ -69,69 +63,25 @@ public class Instance extends BaseJsonConfig {
_pools = pools;
}
- @JsonProperty
public String getHost() {
return _host;
}
- @JsonProperty
public int getPort() {
return _port;
}
- @JsonProperty
public InstanceType getType() {
return _type;
}
@Nullable
- @JsonProperty
public List<String> getTags() {
return _tags;
}
@Nullable
- @JsonProperty
public Map<String, Integer> getPools() {
return _pools;
}
-
- @JsonIgnore
- public String getInstanceId() {
- String prefix;
- switch (_type) {
- case CONTROLLER:
- prefix = Helix.PREFIX_OF_CONTROLLER_INSTANCE;
- break;
- case BROKER:
- prefix = Helix.PREFIX_OF_BROKER_INSTANCE;
- break;
- case SERVER:
- prefix = Helix.PREFIX_OF_SERVER_INSTANCE;
- break;
- case MINION:
- prefix = Helix.PREFIX_OF_MINION_INSTANCE;
- break;
- default:
- throw new IllegalStateException();
- }
- return prefix + _host + "_" + _port;
- }
-
- public InstanceConfig toInstanceConfig() {
- InstanceConfig instanceConfig = InstanceConfig.toInstanceConfig(getInstanceId());
- if (_tags != null) {
- for (String tag : _tags) {
- instanceConfig.addTag(tag);
- }
- }
- if (_pools != null && !_pools.isEmpty()) {
- Map<String, String> mapValue = new TreeMap<>();
- for (Map.Entry<String, Integer> entry : _pools.entrySet()) {
- mapValue.put(entry.getKey(), entry.getValue().toString());
- }
- instanceConfig.getRecord().setMapField(POOL_KEY, mapValue);
- }
- return instanceConfig;
- }
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/InstanceType.java
similarity index 88%
copy from pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java
copy to pinot-spi/src/main/java/org/apache/pinot/spi/config/api/InstanceType.java
index 116bbdb..4ff029d 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/InstanceType.java
@@ -16,8 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.utils;
+package org.apache.pinot.spi.config.api;
-public enum TenantRole {
- SERVER, BROKER
+public enum InstanceType {
+ CONTROLLER, BROKER, SERVER, MINION
}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/Tenant.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Tenant.java
similarity index 96%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/Tenant.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Tenant.java
index eadb08e..6f11d17 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/Tenant.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/Tenant.java
@@ -16,13 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config;
+package org.apache.pinot.spi.config.api;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
-import org.apache.pinot.common.utils.TenantRole;
+import org.apache.pinot.spi.config.BaseJsonConfig;
public class Tenant extends BaseJsonConfig {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/TenantRole.java
similarity index 95%
rename from pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/api/TenantRole.java
index 116bbdb..a65369a 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/TenantRole.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/api/TenantRole.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.utils;
+package org.apache.pinot.spi.config.api;
public enum TenantRole {
SERVER, BROKER
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceAssignmentConfig.java
similarity index 96%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceAssignmentConfig.java
index 02974a0..1adb40b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceAssignmentConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceAssignmentConfig.java
@@ -16,14 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config.instance;
+package org.apache.pinot.spi.config.assignment;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import com.google.common.base.Preconditions;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.BaseJsonConfig;
+import org.apache.pinot.spi.config.BaseJsonConfig;
public class InstanceAssignmentConfig extends BaseJsonConfig {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceConstraintConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceConstraintConfig.java
similarity index 93%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceConstraintConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceConstraintConfig.java
index ea1f41c..74bbe05 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceConstraintConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceConstraintConfig.java
@@ -16,14 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config.instance;
+package org.apache.pinot.spi.config.assignment;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import com.google.common.base.Preconditions;
import java.util.List;
-import org.apache.pinot.common.config.BaseJsonConfig;
+import org.apache.pinot.spi.config.BaseJsonConfig;
public class InstanceConstraintConfig extends BaseJsonConfig {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsType.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstancePartitionsType.java
similarity index 96%
rename from pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsType.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstancePartitionsType.java
index a65f616..1150536 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsType.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstancePartitionsType.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.assignment;
+package org.apache.pinot.spi.config.assignment;
/**
* The type of the instance partitions.
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceReplicaGroupPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceReplicaGroupPartitionConfig.java
similarity index 96%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceReplicaGroupPartitionConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceReplicaGroupPartitionConfig.java
index 327df10..2c2374b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceReplicaGroupPartitionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceReplicaGroupPartitionConfig.java
@@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config.instance;
+package org.apache.pinot.spi.config.assignment;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonPropertyDescription;
-import org.apache.pinot.common.config.BaseJsonConfig;
+import org.apache.pinot.spi.config.BaseJsonConfig;
public class InstanceReplicaGroupPartitionConfig extends BaseJsonConfig {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceTagPoolConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceTagPoolConfig.java
similarity index 95%
rename from pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceTagPoolConfig.java
rename to pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceTagPoolConfig.java
index 863814d..f1661e5 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/instance/InstanceTagPoolConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/assignment/InstanceTagPoolConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.common.config.instance;
+package org.apache.pinot.spi.config.assignment;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import com.google.common.base.Preconditions;
import java.util.List;
import javax.annotation.Nullable;
-import org.apache.pinot.common.config.BaseJsonConfig;
+import org.apache.pinot.spi.config.BaseJsonConfig;
public class InstanceTagPoolConfig extends BaseJsonConfig {
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConfig.java
index ff5dc38..817bcd5 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConfig.java
@@ -24,7 +24,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
-import org.apache.pinot.spi.utils.DataSize;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.pinot.spi.utils.EqualityUtils;
import org.apache.pinot.spi.utils.TimeUtils;
import org.slf4j.Logger;
@@ -141,8 +141,8 @@ public class StreamConfig {
try {
connectionTimeoutMillis = Long.parseLong(connectionTimeoutValue);
} catch (Exception e) {
- LOGGER.warn("Caught exception while parsing the connection timeout, defaulting to {} ms",
- DEFAULT_STREAM_CONNECTION_TIMEOUT_MILLIS, e);
+ LOGGER.warn("Invalid config {}: {}, defaulting to: {}", connectionTimeoutKey, connectionTimeoutValue,
+ DEFAULT_STREAM_CONNECTION_TIMEOUT_MILLIS);
}
}
_connectionTimeoutMillis = connectionTimeoutMillis;
@@ -155,8 +155,8 @@ public class StreamConfig {
try {
fetchTimeoutMillis = Integer.parseInt(fetchTimeoutValue);
} catch (Exception e) {
- LOGGER.warn("Caught exception while parsing the fetch timeout, defaulting to {} ms",
- DEFAULT_STREAM_FETCH_TIMEOUT_MILLIS, e);
+ LOGGER.warn("Invalid config {}: {}, defaulting to: {}", fetchTimeoutKey, fetchTimeoutValue,
+ DEFAULT_STREAM_CONNECTION_TIMEOUT_MILLIS);
}
}
_fetchTimeoutMillis = fetchTimeoutMillis;
@@ -167,7 +167,12 @@ public class StreamConfig {
long flushDesiredSize = -1;
String flushSegmentDesiredSizeValue = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE);
if (flushSegmentDesiredSizeValue != null) {
- flushDesiredSize = DataSize.toBytes(flushSegmentDesiredSizeValue);
+ try {
+ flushDesiredSize = DataSizeUtils.toBytes(flushSegmentDesiredSizeValue);
+ } catch (Exception e) {
+ LOGGER.warn("Invalid config {}: {}, defaulting to: {}", StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE,
+ flushSegmentDesiredSizeValue, DataSizeUtils.fromBytes(DEFAULT_FLUSH_SEGMENT_DESIRED_SIZE_BYTES));
+ }
}
if (flushDesiredSize > 0) {
_flushSegmentDesiredSizeBytes = flushDesiredSize;
@@ -181,9 +186,9 @@ public class StreamConfig {
try {
autotuneInitialRows = Integer.parseInt(initialRowsValue);
} catch (Exception e) {
- LOGGER.warn("Caught exception while parsing {}:{}, defaulting to {}",
+ LOGGER.warn("Invalid config {}: {}, defaulting to: {}",
StreamConfigProperties.SEGMENT_FLUSH_AUTOTUNE_INITIAL_ROWS, initialRowsValue,
- DEFAULT_FLUSH_AUTOTUNE_INITIAL_ROWS, e);
+ DEFAULT_FLUSH_AUTOTUNE_INITIAL_ROWS);
}
}
_flushAutotuneInitialRows = autotuneInitialRows > 0 ? autotuneInitialRows : DEFAULT_FLUSH_AUTOTUNE_INITIAL_ROWS;
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSize.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSize.java
deleted file mode 100644
index d74baf2..0000000
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSize.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * 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.pinot.spi.utils;
-
-import java.math.BigDecimal;
-import java.text.DecimalFormat;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.annotation.Nullable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Utility to convert human readable datsize strings like '4567G', '128M'
- * to bytes. Note that this follows the convention of unix utilities like du and
- * ls with -h option.
- */
-public class DataSize {
- private static final Logger LOGGER = LoggerFactory.getLogger(DataSize.class);
-
- static final Pattern STORAGE_VAL_PATTERN = Pattern.compile("([\\d.]+)([TGMK])?$", Pattern.CASE_INSENSITIVE);
- private static final DecimalFormat DECIMAL_FORMAT = new DecimalFormat("0.##");
-
- static final Map<String, Long> MULTIPLIER;
-
- static {
- MULTIPLIER = new HashMap<>(4);
- MULTIPLIER.put("T", 1024L * 1024 * 1024 * 1024L);
- MULTIPLIER.put("G", 1024L * 1024 * 1024L);
- MULTIPLIER.put("M", 1024 * 1024L);
- MULTIPLIER.put("K", 1024L);
- MULTIPLIER.put("B", 1L);
- }
-
- /**
- * Convert human readable datasize to bytes
- * @param val string to parse
- * @return returns -1 in case of invalid value
- */
- public static long toBytes(@Nullable String val) {
- if (val == null) {
- return -1;
- }
-
- Matcher matcher = STORAGE_VAL_PATTERN.matcher(val);
- if (!matcher.matches()) {
- return -1;
- }
- String number = matcher.group(1);
- String unit = matcher.group(2);
- if (unit == null) {
- unit = "B";
- }
- long multiplier = MULTIPLIER.get(unit.toUpperCase());
- BigDecimal bytes = new BigDecimal(number);
- return bytes.multiply(BigDecimal.valueOf(multiplier)).longValue();
- }
-
- public static String fromBytes(long bytes) {
- if (bytes < MULTIPLIER.get("K")) {
- return String.valueOf(bytes) + "B";
- } else if (bytes < MULTIPLIER.get("M")) {
- return DECIMAL_FORMAT.format(((double) bytes) / MULTIPLIER.get("K")) + "KB";
- } else if (bytes < MULTIPLIER.get("G")) {
- return DECIMAL_FORMAT.format(((double) bytes) / MULTIPLIER.get("M")) + "MB";
- } else if (bytes < MULTIPLIER.get("T")) {
- return DECIMAL_FORMAT.format(((double) bytes) / MULTIPLIER.get("G")) + "GB";
- } else {
- return DECIMAL_FORMAT.format(((double) bytes) / MULTIPLIER.get("T")) + "TB";
- }
- }
-}
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSizeUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSizeUtils.java
new file mode 100644
index 0000000..aa283c6
--- /dev/null
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/DataSizeUtils.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.pinot.spi.utils;
+
+import com.google.common.base.Preconditions;
+import java.text.DecimalFormat;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
... 1121 lines suppressed ...
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org