This is an automated email from the ASF dual-hosted git repository.
nehapawar 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 7c8f47d TableConfigs APIs (#6840)
7c8f47d is described below
commit 7c8f47dcd6e262886470c613bf7d0542543a5356
Author: Neha Pawar <[email protected]>
AuthorDate: Tue Apr 27 21:28:09 2021 -0700
TableConfigs APIs (#6840)
Introducing new tableConfigs APIs. These serve as a way to operate on the
entire Pinot config as 1 unit, consisting of Pinot schema, realtime table
config and offline table config
---
.../broker/broker/HelixBrokerStarterTest.java | 4 +-
.../controller/api/access/AccessControlUtils.java | 2 +-
.../ControllerApplicationException.java | 2 +-
.../api/exception/InvalidTableConfigException.java | 33 ++
.../api/exception/TableAlreadyExistsException.java | 29 ++
.../api/resources/PinotBrokerRestletResource.java | 1 +
.../api/resources/PinotClusterConfigs.java | 1 +
.../resources/PinotIngestionRestletResource.java | 1 +
.../PinotInstanceAssignmentRestletResource.java | 1 +
.../resources/PinotInstanceRestletResource.java | 1 +
.../PinotLeadControllerRestletResource.java | 1 +
.../api/resources/PinotSchemaRestletResource.java | 1 +
.../api/resources/PinotSegmentRestletResource.java | 1 +
.../PinotSegmentUploadDownloadRestletResource.java | 1 +
.../api/resources/PinotTableIndexingConfigs.java | 1 +
.../api/resources/PinotTableMetadataConfigs.java | 1 +
.../api/resources/PinotTableRestletResource.java | 214 ++------
.../controller/api/resources/PinotTableSchema.java | 1 +
.../api/resources/PinotTableSegmentConfigs.java | 1 +
.../api/resources/PinotTableTenantConfigs.java | 1 +
.../api/resources/PinotTenantRestletResource.java | 1 +
.../api/resources/TableConfigsRestletResource.java | 399 +++++++++++++++
.../pinot/controller/api/resources/TableSize.java | 1 +
.../pinot/controller/api/resources/TableViews.java | 1 +
.../api/resources/ZookeeperResource.java | 1 +
.../controller/api/upload/SegmentValidator.java | 2 +-
.../pinot/controller/api/upload/ZKOperator.java | 2 +-
.../helix/ControllerRequestURLBuilder.java | 24 +
.../helix/core/PinotHelixResourceManager.java | 28 +-
.../helix/core/PinotTableIdealStateBuilder.java | 3 +-
.../api/TableConfigsRestletResourceTest.java | 553 +++++++++++++++++++++
...otSegmentUploadDownloadRestletResourceTest.java | 3 +-
.../PinotHelixResourceManagerStatelessTest.java | 4 +-
pinot-segment-local/pom.xml | 6 +-
.../segment/local/utils}/ReplicationUtils.java | 0
.../segment/local/utils/TableConfigUtils.java | 159 +++++-
.../org/apache/pinot/spi/config/TableConfigs.java | 80 +++
37 files changed, 1352 insertions(+), 213 deletions(-)
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 0d97e20..addb8bc 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
@@ -31,8 +31,8 @@ import
org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
import org.apache.pinot.common.request.BrokerRequest;
import org.apache.pinot.common.utils.ZkStarter;
import org.apache.pinot.common.utils.config.TagNameUtils;
+import org.apache.pinot.controller.api.exception.InvalidTableConfigException;
import org.apache.pinot.controller.helix.ControllerTest;
-import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.utils.SegmentMetadataMockUtils;
import org.apache.pinot.pql.parsers.Pql2Compiler;
import org.apache.pinot.spi.config.table.TableConfig;
@@ -168,7 +168,7 @@ public class HelixBrokerStarterTest extends ControllerTest {
try {
_helixResourceManager.addTable(newTableConfig);
Assert.fail("Table creation should fail as testBroker does not exist");
- } catch (PinotHelixResourceManager.InvalidTableConfigException e) {
+ } catch (InvalidTableConfigException e) {
// expected
}
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
index 661289a..95010c4 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
@@ -22,7 +22,7 @@ package org.apache.pinot.controller.api.access;
import java.util.Optional;
import javax.ws.rs.core.HttpHeaders;
import javax.ws.rs.core.Response;
-import
org.apache.pinot.controller.api.resources.ControllerApplicationException;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
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/ControllerApplicationException.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/ControllerApplicationException.java
similarity index 97%
rename from
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ControllerApplicationException.java
rename to
pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/ControllerApplicationException.java
index e559551..1544abb 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ControllerApplicationException.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/ControllerApplicationException.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.controller.api.resources;
+package org.apache.pinot.controller.api.exception;
import javax.annotation.Nullable;
import javax.ws.rs.WebApplicationException;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/InvalidTableConfigException.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/InvalidTableConfigException.java
new file mode 100644
index 0000000..ad4dd19
--- /dev/null
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/InvalidTableConfigException.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.controller.api.exception;
+
+public class InvalidTableConfigException extends RuntimeException {
+ public InvalidTableConfigException(String message) {
+ super(message);
+ }
+
+ public InvalidTableConfigException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public InvalidTableConfigException(Throwable cause) {
+ super(cause);
+ }
+}
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/TableAlreadyExistsException.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/TableAlreadyExistsException.java
new file mode 100644
index 0000000..549274e
--- /dev/null
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/exception/TableAlreadyExistsException.java
@@ -0,0 +1,29 @@
+/**
+ * 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.controller.api.exception;
+
+public class TableAlreadyExistsException extends RuntimeException {
+ public TableAlreadyExistsException(String message) {
+ super(message);
+ }
+
+ public TableAlreadyExistsException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java
index 052e394..5395e9c 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java
@@ -44,6 +44,7 @@ import org.apache.helix.model.InstanceConfig;
import org.apache.pinot.common.exception.TableNotFoundException;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.utils.CommonConstants;
import org.slf4j.Logger;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotClusterConfigs.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotClusterConfigs.java
index e6f428b..1437f73 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotClusterConfigs.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotClusterConfigs.java
@@ -45,6 +45,7 @@ import org.apache.helix.model.HelixConfigScope;
import org.apache.helix.model.builder.HelixConfigScopeBuilder;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.utils.JsonUtils;
import org.slf4j.Logger;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotIngestionRestletResource.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotIngestionRestletResource.java
index 00f1e68..9d867f8 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotIngestionRestletResource.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotIngestionRestletResource.java
@@ -40,6 +40,7 @@ import javax.ws.rs.core.Response;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.util.FileIngestionHelper;
import org.apache.pinot.controller.util.FileIngestionHelper.DataPayload;
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 9d9afd1..9e63014 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
@@ -46,6 +46,7 @@ import org.apache.pinot.common.assignment.InstancePartitions;
import org.apache.pinot.common.assignment.InstancePartitionsUtils;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import
org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssignmentDriver;
import org.apache.pinot.spi.config.table.TableConfig;
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 00711e6..71ff652 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
@@ -42,6 +42,7 @@ import org.apache.helix.model.InstanceConfig;
import org.apache.pinot.common.utils.config.InstanceUtils;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
import org.apache.pinot.spi.config.instance.Instance;
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 247c6b1..6a75d77 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
@@ -41,6 +41,7 @@ import org.apache.helix.HelixManager;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.MasterSlaveSMD;
import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.utils.CommonConstants.Helix;
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
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 9e829f6..11f150c 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
@@ -54,6 +54,7 @@ import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
import org.apache.pinot.controller.api.events.MetadataEventNotifierFactory;
import org.apache.pinot.controller.api.events.SchemaEventType;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.segment.local.utils.SchemaUtils;
import org.apache.pinot.spi.config.table.TableConfig;
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 9237ec8..714a7ad 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
@@ -61,6 +61,7 @@ import org.apache.pinot.common.utils.URIUtils;
import org.apache.pinot.controller.ControllerConf;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
import org.apache.pinot.controller.util.ConsumingSegmentInfoReader;
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 1c260ef..35a3e07 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
@@ -72,6 +72,7 @@ import org.apache.pinot.controller.api.access.AccessControl;
import org.apache.pinot.controller.api.access.AccessControlFactory;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.api.upload.SegmentValidator;
import org.apache.pinot.controller.api.upload.ZKOperator;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
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 f5ad68a..385931b 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
@@ -32,6 +32,7 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.segment.local.utils.TableConfigUtils;
import org.apache.pinot.spi.config.table.TableConfig;
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 639d5dd..46d8b79 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
@@ -31,6 +31,7 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.segment.local.utils.TableConfigUtils;
import org.apache.pinot.spi.config.table.TableConfig;
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 b779b1f..9b88e62 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
@@ -20,7 +20,6 @@ package org.apache.pinot.controller.api.resources;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import io.swagger.annotations.Api;
import io.swagger.annotations.ApiOperation;
@@ -29,9 +28,7 @@ import java.io.IOException;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import javax.inject.Inject;
import javax.ws.rs.Consumes;
@@ -50,7 +47,6 @@ 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.tuner.TableConfigTunerRegistry;
import org.apache.pinot.common.exception.SchemaNotFoundException;
import org.apache.pinot.common.exception.TableNotFoundException;
import org.apache.pinot.common.metrics.ControllerMeter;
@@ -60,28 +56,21 @@ import
org.apache.pinot.controller.api.access.AccessControlFactory;
import org.apache.pinot.controller.api.access.AccessControlUtils;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
+import org.apache.pinot.controller.api.exception.InvalidTableConfigException;
+import org.apache.pinot.controller.api.exception.TableAlreadyExistsException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
-import org.apache.pinot.controller.helix.core.minion.PinotTaskManager;
import
org.apache.pinot.controller.helix.core.rebalance.RebalanceConfigConstants;
import org.apache.pinot.controller.helix.core.rebalance.RebalanceResult;
import org.apache.pinot.controller.recommender.RecommenderDriver;
-import org.apache.pinot.core.common.MinionConstants;
-import org.apache.pinot.core.util.ReplicationUtils;
import org.apache.pinot.segment.local.utils.TableConfigUtils;
-import org.apache.pinot.spi.config.table.QuotaConfig;
-import org.apache.pinot.spi.config.table.SegmentsValidationAndRetentionConfig;
import org.apache.pinot.spi.config.table.TableConfig;
import org.apache.pinot.spi.config.table.TableStats;
-import org.apache.pinot.spi.config.table.TableTaskConfig;
import org.apache.pinot.spi.config.table.TableType;
-import org.apache.pinot.spi.config.table.TunerConfig;
-import org.apache.pinot.spi.config.table.tuner.TableConfigTuner;
import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.glassfish.grizzly.http.server.Request;
-import org.quartz.CronScheduleBuilder;
import org.slf4j.LoggerFactory;
@@ -149,12 +138,7 @@ public class PinotTableRestletResource {
Schema schema =
_pinotHelixResourceManager.getSchemaForTableConfig(tableConfig);
- TunerConfig tunerConfig = tableConfig.getTunerConfig();
- if (tunerConfig != null && tunerConfig.getName() != null &&
!tunerConfig.getName().isEmpty()) {
- TableConfigTuner tuner =
TableConfigTunerRegistry.getTuner(tunerConfig.getName());
- tuner.init(tunerConfig, schema);
- tableConfig = tuner.apply(tableConfig);
- }
+ TableConfigUtils.applyTunerConfig(tableConfig, schema);
// TableConfigUtils.validate(...) is used across table create/update.
TableConfigUtils.validate(tableConfig, schema);
@@ -165,19 +149,23 @@ public class PinotTableRestletResource {
throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.BAD_REQUEST, e);
}
try {
- ensureMinReplicas(tableConfig);
- ensureStorageQuotaConstraints(tableConfig);
- verifyTableConfigs(tableConfig);
+ try {
+ TableConfigUtils.ensureMinReplicas(tableConfig,
_controllerConf.getDefaultTableMinReplicas());
+ TableConfigUtils.ensureStorageQuotaConstraints(tableConfig,
_controllerConf.getDimTableMaxSize());
+
checkHybridTableConfig(TableNameBuilder.extractRawTableName(tableName),
tableConfig);
+ } catch (Exception e) {
+ throw new InvalidTableConfigException(e);
+ }
_pinotHelixResourceManager.addTable(tableConfig);
// TODO: validate that table was created successfully
// (in realtime case, metadata might not have been created but would be
created successfully in the next run of the validation manager)
return new SuccessResponse("Table " + tableName + " succesfully added");
} catch (Exception e) {
_controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR,
1L);
- if (e instanceof PinotHelixResourceManager.InvalidTableConfigException) {
+ if (e instanceof InvalidTableConfigException) {
String errStr = String.format("Invalid table config for table %s: %s",
tableName, e.getMessage());
throw new ControllerApplicationException(LOGGER, errStr,
Response.Status.BAD_REQUEST, e);
- } else if (e instanceof
PinotHelixResourceManager.TableAlreadyExistsException) {
+ } else if (e instanceof TableAlreadyExistsException) {
throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.CONFLICT, e);
} else {
throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.INTERNAL_SERVER_ERROR, e);
@@ -397,11 +385,15 @@ public class PinotTableRestletResource {
Response.Status.NOT_FOUND);
}
- ensureMinReplicas(tableConfig);
- ensureStorageQuotaConstraints(tableConfig);
- verifyTableConfigs(tableConfig);
+ try {
+ TableConfigUtils.ensureMinReplicas(tableConfig,
_controllerConf.getDefaultTableMinReplicas());
+ TableConfigUtils.ensureStorageQuotaConstraints(tableConfig,
_controllerConf.getDimTableMaxSize());
+
checkHybridTableConfig(TableNameBuilder.extractRawTableName(tableName),
tableConfig);
+ } catch (Exception e) {
+ throw new InvalidTableConfigException(e);
+ }
_pinotHelixResourceManager.updateTableConfig(tableConfig);
- } catch (PinotHelixResourceManager.InvalidTableConfigException e) {
+ } catch (InvalidTableConfigException e) {
String errStr = String.format("Failed to update configuration for %s due
to: %s", tableName, e.getMessage());
_controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR,
1L);
throw new ControllerApplicationException(LOGGER, errStr,
Response.Status.BAD_REQUEST, e);
@@ -430,12 +422,14 @@ public class PinotTableRestletResource {
return validateConfig(tableConfig,
_pinotHelixResourceManager.getSchemaForTableConfig(tableConfig));
}
+ @Deprecated
@POST
@Path("/tables/validateTableAndSchema")
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@ApiOperation(value = "Validate table config for a table along with
specified schema", notes =
- "Validate given table config and schema. If specified schema is null,
attempt to retrieve schema using the "
+ "Deprecated. Use /tableConfigs/validate instead."
+ + "Validate given table config and schema. If specified schema is
null, attempt to retrieve schema using the "
+ "table name. This API returns the table config that matches the
one you get from 'GET /tables/{tableName}'."
+ " This allows us to validate table config before apply.")
public String validateTableAndSchema(TableAndSchemaConfig tableSchemaConfig)
{
@@ -467,151 +461,6 @@ public class PinotTableRestletResource {
}
}
- private void ensureMinReplicas(TableConfig tableConfig) {
- // For self-serviced cluster, ensure that the tables are created with at
least min replication factor irrespective
- // of table configuration value
- SegmentsValidationAndRetentionConfig segmentsConfig =
tableConfig.getValidationConfig();
- int configMinReplication = _controllerConf.getDefaultTableMinReplicas();
- boolean verifyReplicasPerPartition;
- boolean verifyReplication;
-
- try {
- verifyReplicasPerPartition =
ReplicationUtils.useReplicasPerPartition(tableConfig);
- verifyReplication = ReplicationUtils.useReplication(tableConfig);
- } catch (Exception e) {
- String errorMsg = String.format("Invalid tableIndexConfig or
streamConfig: %s", e.getMessage());
- throw new
PinotHelixResourceManager.InvalidTableConfigException(errorMsg, e);
- }
-
- if (verifyReplication) {
- int requestReplication;
- try {
- requestReplication = segmentsConfig.getReplicationNumber();
- if (requestReplication < configMinReplication) {
- LOGGER.info("Creating table with minimum replication factor of: {}
instead of requested replication: {}",
- configMinReplication, requestReplication);
- segmentsConfig.setReplication(String.valueOf(configMinReplication));
- }
- } catch (NumberFormatException e) {
- throw new
PinotHelixResourceManager.InvalidTableConfigException("Invalid replication
number", e);
- }
- }
-
- if (verifyReplicasPerPartition) {
- String replicasPerPartitionStr =
segmentsConfig.getReplicasPerPartition();
- if (replicasPerPartitionStr == null) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(
- "Field replicasPerPartition needs to be specified");
- }
- try {
- int replicasPerPartition = Integer.valueOf(replicasPerPartitionStr);
- if (replicasPerPartition < configMinReplication) {
- LOGGER.info(
- "Creating table with minimum replicasPerPartition of: {} instead
of requested replicasPerPartition: {}",
- configMinReplication, replicasPerPartition);
-
segmentsConfig.setReplicasPerPartition(String.valueOf(configMinReplication));
- }
- } catch (NumberFormatException e) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(
- "Invalid value for replicasPerPartition: '" +
replicasPerPartitionStr + "'", e);
- }
- }
- }
-
- private void ensureStorageQuotaConstraints(TableConfig tableConfig) {
- // Dim tables must adhere to cluster level storage size limits
- if (tableConfig.isDimTable()) {
- QuotaConfig quotaConfig = tableConfig.getQuotaConfig();
- String maxAllowedSize = _controllerConf.getDimTableMaxSize();
- long maxAllowedSizeInBytes = DataSizeUtils.toBytes(maxAllowedSize);
-
- if (quotaConfig == null) {
- // set a default storage quota
- tableConfig.setQuotaConfig(new QuotaConfig(maxAllowedSize, null));
- LOGGER.info("Assigning default storage quota ({}) for dimension table:
{}", maxAllowedSize,
- tableConfig.getTableName());
- } else {
- if (quotaConfig.getStorage() == null) {
- // set a default storage quota and keep the RPS value
- tableConfig.setQuotaConfig(new QuotaConfig(maxAllowedSize,
quotaConfig.getMaxQueriesPerSecond()));
- LOGGER.info("Assigning default storage quota ({}) for dimension
table: {}", maxAllowedSize,
- tableConfig.getTableName());
- } else {
- if (quotaConfig.getStorageInBytes() > maxAllowedSizeInBytes) {
- throw new
PinotHelixResourceManager.InvalidTableConfigException(String
- .format("Invalid storage quota: %d, max allowed size: %d",
quotaConfig.getStorageInBytes(),
- maxAllowedSizeInBytes));
- }
- }
- }
- }
- }
-
- /**
- * Verify table configs if it's a hybrid table, i.e. having both offline and
real-time sub-tables.
- */
- private void verifyTableConfigs(TableConfig newTableConfig) {
- String rawTableName =
TableNameBuilder.extractRawTableName(newTableConfig.getTableName());
- LOGGER.info("Validating table configs for Table: {}", rawTableName);
-
- TableConfig tableConfigToCompare = null;
- if (newTableConfig.getTableType() == TableType.REALTIME) {
- if (_pinotHelixResourceManager.hasOfflineTable(rawTableName)) {
- tableConfigToCompare =
_pinotHelixResourceManager.getOfflineTableConfig(rawTableName);
- }
- } else {
- if (_pinotHelixResourceManager.hasRealtimeTable(rawTableName)) {
- tableConfigToCompare =
_pinotHelixResourceManager.getRealtimeTableConfig(rawTableName);
- }
- }
-
- // Check if task schedule is valid.
- TableTaskConfig taskConfig = newTableConfig.getTaskConfig();
- if (taskConfig != null &&
taskConfig.isTaskTypeEnabled(MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE))
{
- Map<String, String> taskTypeConfig =
-
taskConfig.getConfigsForTaskType(MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE);
- if (taskTypeConfig != null &&
taskTypeConfig.containsKey(PinotTaskManager.SCHEDULE_KEY)) {
- String cronExprStr = taskTypeConfig.get(PinotTaskManager.SCHEDULE_KEY);
- try {
- CronScheduleBuilder.cronSchedule(cronExprStr);
- } catch (Exception e) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(
- String.format("SegmentGenerationAndPushTask contains an invalid
cron schedule: %s", cronExprStr), e);
- }
- }
- }
-
- // Check if it is a hybrid table or not. If not, there's no need to verify
anything.
- if (tableConfigToCompare == null) {
- LOGGER.info(
- "Table: {} is not a hybrid table. Skipping consistency check across
realtime and offline parts of the table.",
- rawTableName);
- return;
- }
-
- SegmentsValidationAndRetentionConfig newSegmentConfig =
newTableConfig.getValidationConfig();
- SegmentsValidationAndRetentionConfig SegmentConfigToCompare =
tableConfigToCompare.getValidationConfig();
-
- String newTimeColumnName = newSegmentConfig.getTimeColumnName();
- String existingTimeColumnName = SegmentConfigToCompare.getTimeColumnName();
- if (!Objects.equal(existingTimeColumnName, newTimeColumnName)) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(String
- .format("Time column names are different! Existing time column name:
%s. New time column name: %s",
- existingTimeColumnName, newTimeColumnName));
- }
-
- TimeUnit existingTimeColumnType = SegmentConfigToCompare.getTimeType();
- TimeUnit newTimeColumnType = newSegmentConfig.getTimeType();
- if (existingTimeColumnType != newTimeColumnType) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(String
- .format("Time column types are different! Existing time column type:
%s. New time column type: %s",
- existingTimeColumnType, newTimeColumnType));
- }
-
- // TODO: validate time unit size but now there's no metadata for that in
table config.
- LOGGER.info("Finished validating tables config for Table: {}",
rawTableName);
- }
-
@POST
@Produces(MediaType.APPLICATION_JSON)
@Authenticate(AccessType.UPDATE)
@@ -722,4 +571,19 @@ public class PinotTableRestletResource {
}
return TableNameBuilder.forType(tableType).tableNameWithType(tableName);
}
+
+ private void checkHybridTableConfig(String rawTableName, TableConfig
tableConfig) {
+ if (tableConfig.getTableType() == TableType.REALTIME) {
+ if (_pinotHelixResourceManager.hasOfflineTable(rawTableName)) {
+ TableConfigUtils
+ .verifyHybridTableConfigs(rawTableName,
_pinotHelixResourceManager.getOfflineTableConfig(rawTableName),
+ tableConfig);
+ }
+ } else {
+ if (_pinotHelixResourceManager.hasRealtimeTable(rawTableName)) {
+ TableConfigUtils.verifyHybridTableConfigs(rawTableName, tableConfig,
+ _pinotHelixResourceManager.getRealtimeTableConfig(rawTableName));
+ }
+ }
+ }
}
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSchema.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSchema.java
index 319f142..c10f66d 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSchema.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableSchema.java
@@ -30,6 +30,7 @@ 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.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.data.Schema;
import org.slf4j.Logger;
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 6000f66..23df1d6 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
@@ -33,6 +33,7 @@ import javax.ws.rs.core.Response;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.segment.local.utils.TableConfigUtils;
import org.apache.pinot.spi.config.table.TableConfig;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableTenantConfigs.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableTenantConfigs.java
index d9e4086..5579ce0 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableTenantConfigs.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableTenantConfigs.java
@@ -33,6 +33,7 @@ import javax.ws.rs.core.Response;
import org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
import org.slf4j.Logger;
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 95d1759..4d304bb 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
@@ -48,6 +48,7 @@ import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.helix.core.PinotResourceManagerResponse;
import org.apache.pinot.spi.config.table.TableConfig;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java
new file mode 100644
index 0000000..87f7887
--- /dev/null
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java
@@ -0,0 +1,399 @@
+/**
+ * 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.controller.api.resources;
+
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.google.common.base.Preconditions;
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiParam;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.common.metrics.ControllerMeter;
+import org.apache.pinot.common.metrics.ControllerMetrics;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.controller.api.access.AccessControl;
+import org.apache.pinot.controller.api.access.AccessControlFactory;
+import org.apache.pinot.controller.api.access.AccessControlUtils;
+import org.apache.pinot.controller.api.access.AccessType;
+import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
+import org.apache.pinot.controller.api.exception.InvalidTableConfigException;
+import org.apache.pinot.controller.api.exception.TableAlreadyExistsException;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.segment.local.utils.SchemaUtils;
+import org.apache.pinot.segment.local.utils.TableConfigUtils;
+import org.apache.pinot.spi.config.TableConfigs;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.glassfish.grizzly.http.server.Request;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Endpoints for CRUD of {@link TableConfigs}.
+ * {@link TableConfigs} is a group of the offline table config, realtime table
config and schema for the same tableName.
+ */
+@Api(tags = Constants.TABLE_TAG)
+@Path("/")
+public class TableConfigsRestletResource {
+
+ public static org.slf4j.Logger LOGGER =
LoggerFactory.getLogger(TableConfigsRestletResource.class);
+
+ @Inject
+ PinotHelixResourceManager _pinotHelixResourceManager;
+
+ @Inject
+ ControllerConf _controllerConf;
+
+ @Inject
+ ControllerMetrics _controllerMetrics;
+
+ @Inject
+ AccessControlFactory _accessControlFactory;
+ AccessControlUtils _accessControlUtils = new AccessControlUtils();
+
+ /**
+ * List all {@link TableConfigs}, where each is a group of the offline table
config, realtime table config and schema for the same tableName.
+ * This is equivalent to a list of all raw table names
+ */
+ @GET
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("/tableConfigs")
+ @Authenticate(AccessType.READ)
+ @ApiOperation(value = "Lists all TableConfigs in cluster", notes = "Lists
all TableConfigs in cluster")
+ public String listConfigs() {
+ try {
+ List<String> rawTableNames =
_pinotHelixResourceManager.getAllRawTables();
+ Collections.sort(rawTableNames);
+
+ ArrayNode configsList = JsonUtils.newArrayNode();
+ for (String rawTableName : rawTableNames) {
+ configsList.add(rawTableName);
+ }
+ return configsList.toString();
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.INTERNAL_SERVER_ERROR, e);
+ }
+ }
+
+ /**
+ * Gets the {@link TableConfigs} for the provided raw tableName, by fetching
the offline table config for tableName_OFFLINE,
+ * realtime table config for tableName_REALTIME and schema for tableName
+ */
+ @GET
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("/tableConfigs/{tableName}")
+ @Authenticate(AccessType.READ)
+ @ApiOperation(value = "Get the TableConfigs for a given raw tableName",
notes = "Get the TableConfigs for a given raw tableName")
+ public String getConfig(
+ @ApiParam(value = "Raw table name", required = true)
@PathParam("tableName") String tableName) {
+
+ try {
+ Schema schema = _pinotHelixResourceManager.getSchema(tableName);
+ TableConfig offlineTableConfig =
_pinotHelixResourceManager.getOfflineTableConfig(tableName);
+ TableConfig realtimeTableConfig =
_pinotHelixResourceManager.getRealtimeTableConfig(tableName);
+ TableConfigs config = new TableConfigs(tableName, schema,
offlineTableConfig, realtimeTableConfig);
+ return config.toJsonString();
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.INTERNAL_SERVER_ERROR, e);
+ }
+ }
+
+ /**
+ * Creates a {@link TableConfigs} using the <code>tableConfigsStr</code>, by
creating the schema,
+ * followed by the realtime tableConfig and offline tableConfig as
applicable, from the {@link TableConfigs}.
+ * Validates the configs before applying.
+ */
+ @POST
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("/tableConfigs")
+ @ApiOperation(value = "Add the TableConfigs using the tableConfigsStr json",
notes = "Add the TableConfigs using the tableConfigsStr json")
+ public SuccessResponse addConfig(String tableConfigsStr, @Context
HttpHeaders httpHeaders, @Context Request request) {
+ TableConfigs tableConfigs;
+ try {
+ tableConfigs = JsonUtils.stringToObject(tableConfigsStr,
TableConfigs.class);
+ validateConfig(tableConfigs);
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, String.format("Invalid
TableConfigs. %s", e.getMessage()),
+ Response.Status.BAD_REQUEST, e);
+ }
+
+ String rawTableName = tableConfigs.getTableName();
+ if (_pinotHelixResourceManager.hasOfflineTable(rawTableName) ||
_pinotHelixResourceManager
+ .hasRealtimeTable(rawTableName) ||
_pinotHelixResourceManager.getSchema(rawTableName) != null) {
+ throw new ControllerApplicationException(LOGGER,
+ String.format("TableConfigs: %s already exists. Use PUT to update
existing config", rawTableName),
+ Response.Status.BAD_REQUEST);
+ }
+
+ TableConfig offlineTableConfig = tableConfigs.getOffline();
+ TableConfig realtimeTableConfig = tableConfigs.getRealtime();
+ Schema schema = tableConfigs.getSchema();
+
+ try {
+ String endpointUrl = request.getRequestURL().toString();
+ AccessControl accessControl = _accessControlFactory.create();
+ _accessControlUtils
+ .validatePermission(schema.getSchemaName(), AccessType.CREATE,
httpHeaders, endpointUrl, accessControl);
+
+ if (offlineTableConfig != null) {
+ tuneConfig(offlineTableConfig, schema);
+ _accessControlUtils
+ .validatePermission(offlineTableConfig.getTableName(),
AccessType.CREATE, httpHeaders, endpointUrl,
+ accessControl);
+ }
+ if (realtimeTableConfig != null) {
+ tuneConfig(realtimeTableConfig, schema);
+ _accessControlUtils
+ .validatePermission(realtimeTableConfig.getTableName(),
AccessType.CREATE, httpHeaders, endpointUrl,
+ accessControl);
+ }
+
+ try {
+ _pinotHelixResourceManager.addSchema(schema, false);
+ LOGGER.info("Added schema: {}", schema.getSchemaName());
+ if (offlineTableConfig != null) {
+ _pinotHelixResourceManager.addTable(offlineTableConfig);
+ LOGGER.info("Added offline table config: {}",
offlineTableConfig.getTableName());
+ }
+ if (realtimeTableConfig != null) {
+ _pinotHelixResourceManager.addTable(realtimeTableConfig);
+ LOGGER.info("Added realtime table config: {}",
realtimeTableConfig.getTableName());
+ }
+ } catch (Exception e) {
+ // In case of exception when adding any of the above configs, revert
all configs added
+ // Invoke delete on tables whether they exist or not, to account for
metadata/segments etc.
+ _pinotHelixResourceManager.deleteRealtimeTable(rawTableName);
+ _pinotHelixResourceManager.deleteOfflineTable(rawTableName);
+ _pinotHelixResourceManager.deleteSchema(schema);
+ throw e;
+ }
+
+ return new SuccessResponse("TableConfigs " + tableConfigs.getTableName()
+ " successfully added");
+ } catch (Exception e) {
+
_controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR,
1L);
+ if (e instanceof InvalidTableConfigException) {
+ throw new ControllerApplicationException(LOGGER,
+ String.format("Invalid TableConfigs: %s",
tableConfigs.getTableName()), Response.Status.BAD_REQUEST, e);
+ } else if (e instanceof TableAlreadyExistsException) {
+ throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.CONFLICT, e);
+ } else {
+ throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.INTERNAL_SERVER_ERROR, e);
+ }
+ }
+ }
+
+ /**
+ * Deletes the {@link TableConfigs} by deleting the schema tableName, the
offline table config for tableName_OFFLINE and
+ * the realtime table config for tableName_REALTIME
+ */
+ @DELETE
+ @Path("/tableConfigs/{tableName}")
+ @Authenticate(AccessType.DELETE)
+ @Produces(MediaType.APPLICATION_JSON)
+ @ApiOperation(value = "Delete the TableConfigs", notes = "Delete the
TableConfigs")
+ public SuccessResponse deleteConfig(
+ @ApiParam(value = "TableConfigs name i.e. raw table name", required =
true) @PathParam("tableName") String tableName) {
+
+ try {
+ boolean tableExists = false;
+ if (_pinotHelixResourceManager.hasRealtimeTable(tableName) ||
_pinotHelixResourceManager
+ .hasOfflineTable(tableName)) {
+ tableExists = true;
+ }
+ // Delete whether tables exist or not
+ _pinotHelixResourceManager.deleteRealtimeTable(tableName);
+ LOGGER.info("Deleted realtime table: {}", tableName);
+ _pinotHelixResourceManager.deleteOfflineTable(tableName);
+ LOGGER.info("Deleted offline table: {}", tableName);
+ Schema schema = _pinotHelixResourceManager.getSchema(tableName);
+ if (schema != null) {
+ _pinotHelixResourceManager.deleteSchema(schema);
+ LOGGER.info("Deleted schema: {}", tableName);
+ }
+ if (tableExists || schema != null) {
+ return new SuccessResponse("Deleted TableConfigs: " + tableName);
+ } else {
+ return new SuccessResponse(
+ "TableConfigs: " + tableName + " don't exist. Invoked delete
anyway to clean stale metadata/segments");
+ }
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, e.getMessage(),
Response.Status.INTERNAL_SERVER_ERROR, e);
+ }
+ }
+
+ /**
+ * Updated the {@link TableConfigs} by updating the schema tableName,
+ * then updating the offline tableConfig or creating a new one if it doesn't
already exist in the cluster,
+ * then updating the realtime tableConfig or creating a new one if it
doesn't already exist in the cluster.
+ */
+ @PUT
+ @Path("/tableConfigs/{tableName}")
+ @Authenticate(AccessType.UPDATE)
+ @Produces(MediaType.APPLICATION_JSON)
+ @ApiOperation(value = "Update the TableConfigs provided by the
tableConfigsStr json", notes = "Update the TableConfigs provided by the
tableConfigsStr json")
+ public SuccessResponse updateConfig(
+ @ApiParam(value = "TableConfigs name i.e. raw table name", required =
true) @PathParam("tableName") String tableName,
+ @ApiParam(value = "Reload the table if the new schema is backward
compatible") @DefaultValue("false") @QueryParam("reload") boolean reload,
+ String tableConfigsStr)
+ throws Exception {
+ TableConfigs tableConfigs;
+ try {
+ tableConfigs = JsonUtils.stringToObject(tableConfigsStr,
TableConfigs.class);
+ Preconditions.checkState(tableConfigs.getTableName().equals(tableName),
+ "'tableName' in TableConfigs: %s must match provided tableName: %s",
tableConfigs.getTableName(), tableName);
+ validateConfig(tableConfigs);
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER, String.format("Invalid
TableConfigs: %s", tableName),
+ Response.Status.BAD_REQUEST, e);
+ }
+
+ if (!_pinotHelixResourceManager.hasOfflineTable(tableName) &&
!_pinotHelixResourceManager
+ .hasRealtimeTable(tableName)) {
+ throw new ControllerApplicationException(LOGGER,
+ String.format("TableConfigs: %s does not exist. Use POST to create
it first.", tableName),
+ Response.Status.BAD_REQUEST);
+ }
+
+ TableConfig offlineTableConfig = tableConfigs.getOffline();
+ TableConfig realtimeTableConfig = tableConfigs.getRealtime();
+ Schema schema = tableConfigs.getSchema();
+
+ try {
+ _pinotHelixResourceManager.updateSchema(schema, reload);
+ LOGGER.info("Updated schema: {}", tableName);
+
+ if (offlineTableConfig != null) {
+ tuneConfig(offlineTableConfig, schema);
+ if (_pinotHelixResourceManager.hasOfflineTable(tableName)) {
+ _pinotHelixResourceManager.updateTableConfig(offlineTableConfig);
+ LOGGER.info("Updated offline table config: {}", tableName);
+ } else {
+ _pinotHelixResourceManager.addTable(offlineTableConfig);
+ LOGGER.info("Created offline table config: {}", tableName);
+ }
+ if (realtimeTableConfig != null) {
+ tuneConfig(realtimeTableConfig, schema);
+ if (_pinotHelixResourceManager.hasRealtimeTable(tableName)) {
+ _pinotHelixResourceManager.updateTableConfig(realtimeTableConfig);
+ LOGGER.info("Updated realtime table config: {}", tableName);
+ } else {
+ _pinotHelixResourceManager.addTable(realtimeTableConfig);
+ LOGGER.info("Created realtime table config: {}", tableName);
+ }
+ }
+ }
+ } catch (InvalidTableConfigException e) {
+
_controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR,
1L);
+ throw new ControllerApplicationException(LOGGER,
+ String.format("Invalid TableConfigs for: %s, %s", tableName,
e.getMessage()), Response.Status.BAD_REQUEST, e);
+ } catch (Exception e) {
+
_controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR,
1L);
+ throw new ControllerApplicationException(LOGGER,
+ String.format("Failed to update TableConfigs for: %s, %s",
tableName, e.getMessage()),
+ Response.Status.INTERNAL_SERVER_ERROR, e);
+ }
+
+ return new SuccessResponse("TableConfigs updated for " + tableName);
+ }
+
+ /**
+ * Validates the {@link TableConfigs} as provided in the tableConfigsStr
json, by validating the schema,
+ * the realtime table config and the offline table config
+ */
+ @POST
+ @Path("/tableConfigs/validate")
+ @Produces(MediaType.APPLICATION_JSON)
+ @ApiOperation(value = "Validate the TableConfigs", notes = "Validate the
TableConfigs")
+ public String validateConfig(String tableConfigsStr) {
+ TableConfigs tableConfigs;
+ try {
+ tableConfigs = JsonUtils.stringToObject(tableConfigsStr,
TableConfigs.class);
+ } catch (IOException e) {
+ throw new ControllerApplicationException(LOGGER,
+ String.format("Invalid TableConfigs json string: %s",
tableConfigsStr), Response.Status.BAD_REQUEST, e);
+ }
+ return validateConfig(tableConfigs);
+ }
+
+ private void tuneConfig(TableConfig tableConfig, Schema schema) {
+ TableConfigUtils.applyTunerConfig(tableConfig, schema);
+ TableConfigUtils.ensureMinReplicas(tableConfig,
_controllerConf.getDefaultTableMinReplicas());
+ TableConfigUtils.ensureStorageQuotaConstraints(tableConfig,
_controllerConf.getDimTableMaxSize());
+ }
+
+ private String validateConfig(TableConfigs tableConfigs) {
+ String rawTableName = tableConfigs.getTableName();
+ TableConfig offlineTableConfig = tableConfigs.getOffline();
+ TableConfig realtimeTableConfig = tableConfigs.getRealtime();
+ Schema schema = tableConfigs.getSchema();
+ try {
+ Preconditions.checkState(offlineTableConfig != null ||
realtimeTableConfig != null,
+ "Must provide at least one of 'realtime' or 'offline' table configs
for adding TableConfigs: %s",
+ rawTableName);
+ Preconditions.checkState(schema != null, "Must provide 'schema' for
adding TableConfigs: %s", rawTableName);
+ Preconditions.checkState(!rawTableName.isEmpty(), "'tableName' cannot be
empty in TableConfigs");
+
+ Preconditions.checkState(rawTableName.equals(schema.getSchemaName()),
+ "'tableName': %s must be equal to 'schemaName' from 'schema': %s",
rawTableName, schema.getSchemaName());
+ SchemaUtils.validate(schema);
+
+ if (offlineTableConfig != null) {
+ String offlineRawTableName =
TableNameBuilder.extractRawTableName(offlineTableConfig.getTableName());
+ Preconditions.checkState(offlineRawTableName.equals(rawTableName),
+ "Name in 'offline' table config: %s must be equal to 'tableName':
%s", offlineRawTableName, rawTableName);
+ TableConfigUtils.validateTableName(offlineTableConfig);
+ TableConfigUtils.validate(offlineTableConfig, schema);
+ }
+ if (realtimeTableConfig != null) {
+ String realtimeRawTableName =
TableNameBuilder.extractRawTableName(realtimeTableConfig.getTableName());
+ Preconditions.checkState(realtimeRawTableName.equals(rawTableName),
+ "Name in 'realtime' table config: %s must be equal to 'tableName':
%s", realtimeRawTableName, rawTableName);
+ TableConfigUtils.validateTableName(realtimeTableConfig);
+ TableConfigUtils.validate(realtimeTableConfig, schema);
+ }
+ if (offlineTableConfig != null && realtimeTableConfig != null) {
+ TableConfigUtils.verifyHybridTableConfigs(rawTableName,
offlineTableConfig, realtimeTableConfig);
+ }
+
+ return tableConfigs.toJsonString();
+ } catch (Exception e) {
+ throw new ControllerApplicationException(LOGGER,
+ String.format("Invalid TableConfigs: %s. %s", rawTableName,
e.getMessage()), Response.Status.BAD_REQUEST, e);
+ }
+ }
+}
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableSize.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableSize.java
index eaa0b03..d13a52c 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableSize.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableSize.java
@@ -36,6 +36,7 @@ import javax.ws.rs.core.Response;
import org.apache.commons.httpclient.HttpConnectionManager;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.ControllerConf;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.util.TableSizeReader;
import org.slf4j.Logger;
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 e5f519f..594e198 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,6 +35,7 @@ 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.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.config.table.TableType;
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
diff --git
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ZookeeperResource.java
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ZookeeperResource.java
index a428985..ad6661f 100644
---
a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ZookeeperResource.java
+++
b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ZookeeperResource.java
@@ -42,6 +42,7 @@ import org.apache.helix.ZNRecord;
import org.apache.helix.manager.zk.ZNRecordSerializer;
import org.apache.pinot.controller.api.access.AccessType;
import org.apache.pinot.controller.api.access.Authenticate;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.spi.utils.JsonUtils;
import org.apache.zookeeper.data.Stat;
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 6dca028..8741d82 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
@@ -27,7 +27,7 @@ import
org.apache.pinot.common.exception.InvalidConfigException;
import org.apache.pinot.common.metadata.ZKMetadataProvider;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.controller.ControllerConf;
-import
org.apache.pinot.controller.api.resources.ControllerApplicationException;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.controller.util.TableSizeReader;
import org.apache.pinot.controller.validation.StorageQuotaChecker;
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 e1a9a0b..0c6ad65 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
@@ -29,7 +29,7 @@ import org.apache.pinot.common.metrics.ControllerMeter;
import org.apache.pinot.common.metrics.ControllerMetrics;
import org.apache.pinot.common.utils.FileUploadDownloadClient;
import org.apache.pinot.controller.ControllerConf;
-import
org.apache.pinot.controller.api.resources.ControllerApplicationException;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
import org.apache.pinot.segment.spi.SegmentMetadata;
import org.apache.pinot.spi.filesystem.PinotFS;
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 013c65b..cea7582 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
@@ -231,6 +231,30 @@ public class ControllerRequestURLBuilder {
return StringUtil.join("/", _baseUrl, "schemas", schemaName);
}
+ public String forTableConfigsCreate() {
+ return StringUtil.join("/", _baseUrl, "tableConfigs");
+ }
+
+ public String forTableConfigsGet(String configName) {
+ return StringUtil.join("/", _baseUrl, "tableConfigs", configName);
+ }
+
+ public String forTableConfigsList() {
+ return StringUtil.join("/", _baseUrl, "tableConfigs");
+ }
+
+ public String forTableConfigsUpdate(String configName) {
+ return StringUtil.join("/", _baseUrl, "tableConfigs", configName);
+ }
+
+ public String forTableConfigsDelete(String configName) {
+ return StringUtil.join("/", _baseUrl, "tableConfigs", configName);
+ }
+
+ public String forTableConfigsValidate() {
+ return StringUtil.join("/", _baseUrl, "tableConfigs", "validate");
+ }
+
public String forSegmentDownload(String tableNameWithType, String
segmentName) {
return URIUtils.constructDownloadUrl(_baseUrl, tableNameWithType,
segmentName);
}
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 42c83b0..8441d12 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
@@ -98,7 +98,9 @@ import org.apache.pinot.common.utils.helix.HelixHelper;
import
org.apache.pinot.common.utils.helix.PinotHelixPropertyStoreZnRecordProvider;
import org.apache.pinot.common.utils.helix.TableCache;
import org.apache.pinot.controller.ControllerConf;
-import
org.apache.pinot.controller.api.resources.ControllerApplicationException;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
+import org.apache.pinot.controller.api.exception.InvalidTableConfigException;
+import org.apache.pinot.controller.api.exception.TableAlreadyExistsException;
import org.apache.pinot.controller.api.resources.StateType;
import
org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssignmentDriver;
import
org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
@@ -1309,30 +1311,6 @@ public class PinotHelixResourceManager {
return _helixDataAccessor.getBaseDataAccessor().getStat(path, -1);
}
- public static class InvalidTableConfigException extends RuntimeException {
- public InvalidTableConfigException(String message) {
- super(message);
- }
-
- public InvalidTableConfigException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public InvalidTableConfigException(Throwable cause) {
- super(cause);
- }
- }
-
- public static class TableAlreadyExistsException extends RuntimeException {
- public TableAlreadyExistsException(String message) {
- super(message);
- }
-
- public TableAlreadyExistsException(String message, Throwable cause) {
- super(message, cause);
- }
- }
-
public void
registerPinotLLCRealtimeSegmentManager(PinotLLCRealtimeSegmentManager
pinotLLCRealtimeSegmentManager) {
_pinotLLCRealtimeSegmentManager = pinotLLCRealtimeSegmentManager;
}
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 28a9bd3..73b8e5b 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
@@ -30,6 +30,7 @@ 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.controller.api.exception.InvalidTableConfigException;
import
org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
import org.apache.pinot.spi.config.table.TableConfig;
import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus;
@@ -111,7 +112,7 @@ public class PinotTableIdealStateBuilder {
try {
nReplicas = Integer.valueOf(replicasPerPartitionStr);
} catch (NumberFormatException e) {
- throw new PinotHelixResourceManager.InvalidTableConfigException(
+ throw new InvalidTableConfigException(
"Invalid value for replicasPerPartition, expected a number: " +
replicasPerPartitionStr, e);
}
if (idealState == null) {
diff --git
a/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableConfigsRestletResourceTest.java
b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableConfigsRestletResourceTest.java
new file mode 100644
index 0000000..6b6ca49
--- /dev/null
+++
b/pinot-controller/src/test/java/org/apache/pinot/controller/api/TableConfigsRestletResourceTest.java
@@ -0,0 +1,553 @@
+/**
+ * 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.controller.api;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.List;
+import org.apache.pinot.controller.ControllerTestUtils;
+import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
+import org.apache.pinot.spi.config.TableConfigs;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.TunerConfig;
+import org.apache.pinot.spi.data.DateTimeFieldSpec;
+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.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;
+import org.testng.annotations.Test;
+import org.testng.collections.Lists;
+
+
+/**
+ * Tests for CRUD APIs of {@link TableConfigs}
+ */
+public class TableConfigsRestletResourceTest {
+
+ private String _createTableConfigsUrl;
+
+ @BeforeClass
+ public void setUp()
+ throws Exception {
+ ControllerTestUtils.setupClusterAndValidate();
+ _createTableConfigsUrl =
ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsCreate();
+ }
+
+ private Schema getSchema(String tableName) {
+ return ControllerTestUtils.createDummySchema(tableName);
+ }
+
+ private Schema getDimSchema(String tableName) {
+ Schema schema = ControllerTestUtils.createDummySchema(tableName);
+
schema.setPrimaryKeyColumns(Lists.newArrayList(schema.getDimensionNames().get(0)));
+ return schema;
+ }
+
+ private TableConfigBuilder getBaseTableConfigBuilder(String tableName,
TableType tableType) {
+ if (tableType == TableType.OFFLINE) {
+ return new
TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setTimeColumnName("timeColumn")
+ .setRetentionTimeUnit("DAYS").setRetentionTimeValue("50");
+ } else {
+ StreamConfig streamConfig =
FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs();
+ return new
TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setTimeColumnName("timeColumn")
+ .setRetentionTimeUnit("DAYS").setLLC(true).setRetentionTimeValue("5")
+ .setStreamConfigs(streamConfig.getStreamConfigsMap());
+ }
+ }
+
+ private TableConfig getOfflineTableConfig(String tableName) {
+ return getBaseTableConfigBuilder(tableName, TableType.OFFLINE).build();
+ }
+
+ private TableConfig getRealtimeTableConfig(String tableName) {
+ return getBaseTableConfigBuilder(tableName, TableType.REALTIME).build();
+ }
+
+ private TableConfig getOfflineTunerTableConfig(String tableName) {
+ return getBaseTableConfigBuilder(tableName, TableType.OFFLINE)
+ .setTunerConfig(new TunerConfig("realtimeAutoIndexTuner",
null)).build();
+ }
+
+ private TableConfig getRealtimeTunerTableConfig(String tableName) {
+ return getBaseTableConfigBuilder(tableName, TableType.REALTIME)
+ .setTunerConfig(new TunerConfig("realtimeAutoIndexTuner",
null)).build();
+ }
+
+ private TableConfig getOfflineDimTableConfig(String tableName) {
+ return getBaseTableConfigBuilder(tableName,
TableType.OFFLINE).setIsDimTable(true).build();
+ }
+
+ @Test
+ public void testValidateConfig()
+ throws IOException {
+
+ String validateConfigUrl =
ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsValidate();
+
+ String tableName = "testValidate";
+ TableConfig offlineTableConfig = getOfflineTableConfig(tableName);
+ TableConfig realtimeTableConfig = getRealtimeTableConfig(tableName);
+ Schema schema = getSchema(tableName);
+ TableConfigs tableConfigs;
+
+ // invalid json
+ try {
+ tableConfigs = new TableConfigs(tableName, schema, offlineTableConfig,
realtimeTableConfig);
+ ControllerTestUtils
+ .sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString().replace("\"offline\"", "offline\""));
+ Assert.fail("Creation of a TableConfigs with invalid json string should
have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // null table configs
+ try {
+ tableConfigs = new TableConfigs(tableName, schema, null, null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail(
+ "Creation of an TableConfigs with null table offline tableConfig and
realtime tableConfig should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // null schema
+ try {
+ tableConfigs = new TableConfigs(tableName, null, offlineTableConfig,
null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with null schema should have
failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // empty config name
+ try {
+ tableConfigs = new TableConfigs("", schema, offlineTableConfig,
realtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with empty config name should
have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // schema name doesn't match config name
+ try {
+ tableConfigs = new TableConfigs(tableName, getSchema("differentName"),
offlineTableConfig, realtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with schema name different than
tableName should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // schema validation fails
+ try {
+ Schema schemaWithBlankSpace = getSchema(tableName);
+ schemaWithBlankSpace.addField(new MetricFieldSpec("blank space",
FieldSpec.DataType.LONG));
+ tableConfigs = new TableConfigs(tableName, schemaWithBlankSpace,
offlineTableConfig, realtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with blank space in column
should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // offline table name doesn't match config name
+ try {
+ tableConfigs = new TableConfigs(tableName, schema,
getOfflineTableConfig("differentName"), null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with offline table name
different than tableName should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // table name validation fails
+ try {
+ tableConfigs =
+ new TableConfigs("blank space", getSchema("blank space"),
getOfflineTableConfig("blank space"), null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with blank space in table name
should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // table validation fails
+ try {
+ TableConfig invalidTableConfig = getOfflineTableConfig(tableName);
+
invalidTableConfig.getIndexingConfig().setInvertedIndexColumns(Lists.newArrayList("nonExistent"));
+ tableConfigs = new TableConfigs(tableName, schema, invalidTableConfig,
null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with invalid table config
should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // realtime table name doesn't match config name
+ try {
+ tableConfigs = new TableConfigs(tableName, schema, null,
getRealtimeTableConfig("differentName"));
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with realtime table name
different than tableName should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // table name validation fails
+ try {
+ tableConfigs =
+ new TableConfigs("blank space", getSchema("blank space"), null,
getRealtimeTableConfig("blank space"));
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with blank space in table name
should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // table validation fails
+ try {
+ TableConfig invalidTableConfig = getRealtimeTableConfig(tableName);
+
invalidTableConfig.getIndexingConfig().setInvertedIndexColumns(Lists.newArrayList("nonExistent"));
+ tableConfigs = new TableConfigs(tableName, schema, null,
invalidTableConfig);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Creation of an TableConfigs with invalid table config
should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // hybrid config consistency check fails
+ try {
+ Schema twoTimeColumns = getSchema(tableName);
+ twoTimeColumns
+ .addField(new DateTimeFieldSpec("time1", FieldSpec.DataType.LONG,
"1:MILLISECONDS:EPOCH", "1:MILLISECONDS"));
+ twoTimeColumns
+ .addField(new DateTimeFieldSpec("time2", FieldSpec.DataType.LONG,
"1:MILLISECONDS:EPOCH", "1:MILLISECONDS"));
+ TableConfig offlineTableConfig1 = getOfflineTableConfig(tableName);
+ offlineTableConfig1.getValidationConfig().setTimeColumnName("time1");
+ TableConfig realtimeTableConfig1 = getRealtimeTableConfig(tableName);
+ realtimeTableConfig1.getValidationConfig().setTimeColumnName("time2");
+ tableConfigs = new TableConfigs(tableName, twoTimeColumns,
offlineTableConfig1, realtimeTableConfig1);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail(
+ "Creation of an TableConfigs with inconsistencies across offline and
realtime table config should have failed");
+ } catch (Exception e) {
+ // expected
+ }
+
+ // successfully created with all 3 configs
+ String tableName1 = "testValidate1";
+ tableConfigs = new TableConfigs(tableName1, getSchema(tableName1),
getOfflineTableConfig(tableName1),
+ getRealtimeTableConfig(tableName1));
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+
+ // successfully create with offline config
+ String tableName2 = "testValidate2";
+ tableConfigs = new TableConfigs(tableName2, getSchema(tableName2),
getOfflineTableConfig(tableName2), null);
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+
+ // successfully create with realtime config
+ String tableName3 = "testValidate3";
+ tableConfigs = new TableConfigs(tableName3, getSchema(tableName3), null,
getRealtimeTableConfig(tableName3));
+ ControllerTestUtils.sendPostRequest(validateConfigUrl,
tableConfigs.toPrettyJsonString());
+
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName1));
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName2));
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName3));
+ }
+
+ /**
+ * Tests for creation of TableConfigs
+ */
+ @Test
+ public void testCreateConfig()
+ throws IOException {
+ String tableName = "testCreate";
+ TableConfig offlineTableConfig = getOfflineTableConfig(tableName);
+ TableConfig realtimeTableConfig = getRealtimeTableConfig(tableName);
+ Schema schema = getSchema(tableName);
+ TableConfigs tableConfigs = new TableConfigs(tableName, schema,
offlineTableConfig, realtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ String response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ TableConfigs tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+ Assert.assertEquals(tableConfigsResponse.getOffline().getTableName(),
offlineTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getRealtime().getTableName(),
realtimeTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getSchema().getSchemaName(),
schema.getSchemaName());
+
+ // test POST of existing configs fails
+ try {
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ Assert.fail("Should fail for trying to add existing config");
+ } catch (Exception e) {
+ // expected
+ }
+
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+
+ // replica check
+ tableName = "testCreateReplicas";
+ TableConfig replicaTestOfflineTableConfig =
getOfflineTableConfig(tableName);
+ TableConfig replicaTestRealtimeTableConfig =
getRealtimeTableConfig(tableName);
+ replicaTestOfflineTableConfig.getValidationConfig().setReplication("1");
+
replicaTestRealtimeTableConfig.getValidationConfig().setReplicasPerPartition("1");
+ tableConfigs = new TableConfigs(tableName, getSchema(tableName),
replicaTestOfflineTableConfig,
+ replicaTestRealtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+
Assert.assertEquals(tableConfigsResponse.getOffline().getValidationConfig().getReplicationNumber(),
+ ControllerTestUtils.MIN_NUM_REPLICAS);
+
Assert.assertEquals(tableConfigsResponse.getRealtime().getValidationConfig().getReplicasPerPartitionNumber(),
+ ControllerTestUtils.MIN_NUM_REPLICAS);
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+
+ // quota check
+ tableName = "testCreateQuota";
+ TableConfig offlineDimTableConfig = getOfflineDimTableConfig(tableName);
+ Schema dimSchema = getDimSchema(tableName);
+ tableConfigs = new TableConfigs(tableName, dimSchema,
offlineDimTableConfig, null);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableName, tableConfigsResponse.getTableName());
+
Assert.assertEquals(tableConfigsResponse.getOffline().getQuotaConfig().getStorage(),
+ ControllerTestUtils.getControllerConfig().getDimTableMaxSize());
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+
+ // tuner config
+ tableName = "testTunerConfig";
+ TableConfig offlineTunerTableConfig =
getOfflineTunerTableConfig(tableName);
+ TableConfig realtimeTunerTableConfig =
getRealtimeTunerTableConfig(tableName);
+ tableConfigs = new TableConfigs(tableName, getSchema(tableName),
offlineTunerTableConfig, realtimeTunerTableConfig);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableName, tableConfigsResponse.getTableName());
+
Assert.assertTrue(tableConfigsResponse.getOffline().getIndexingConfig().getInvertedIndexColumns()
+ .containsAll(schema.getDimensionNames()));
+
Assert.assertTrue(tableConfigsResponse.getOffline().getIndexingConfig().getNoDictionaryColumns()
+ .containsAll(schema.getMetricNames()));
+
Assert.assertTrue(tableConfigsResponse.getRealtime().getIndexingConfig().getInvertedIndexColumns()
+ .containsAll(schema.getDimensionNames()));
+
Assert.assertTrue(tableConfigsResponse.getRealtime().getIndexingConfig().getNoDictionaryColumns()
+ .containsAll(schema.getMetricNames()));
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+ }
+
+ @Test
+ public void testListConfigs()
+ throws IOException {
+ // create with 1 config
+ String tableName1 = "testList1";
+ TableConfig offlineTableConfig = getOfflineTableConfig(tableName1);
+ TableConfig realtimeTableConfig = getRealtimeTableConfig(tableName1);
+ Schema schema = getSchema(tableName1);
+ TableConfigs tableConfigs = new TableConfigs(tableName1, schema,
offlineTableConfig, null);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+
+ // list
+ String getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ List<String> configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 1);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet(tableName1)));
+
+ // update to 2
+ tableConfigs = new TableConfigs(tableName1, schema, offlineTableConfig,
realtimeTableConfig);
+ ControllerTestUtils
+
.sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsUpdate(tableName1),
+ tableConfigs.toPrettyJsonString());
+
+ // list
+ getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 1);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet("testList1")));
+
+ // create new
+ String tableName2 = "testList2";
+ offlineTableConfig = getOfflineTableConfig(tableName2);
+ schema = getSchema(tableName2);
+ tableConfigs = new TableConfigs(tableName2, schema, offlineTableConfig,
null);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+
+ // list
+ getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 2);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet(tableName1,
tableName2)));
+
+ // delete 1
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName2));
+
+ // list 1
+ getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 1);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet(tableName1)));
+
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName1));
+ }
+
+ @Test
+ public void testUpdateConfig()
+ throws IOException {
+
+ // create with 1
+ String tableName = "testUpdate1";
+ TableConfig offlineTableConfig = getOfflineTableConfig(tableName);
+ TableConfig realtimeTableConfig = getRealtimeTableConfig(tableName);
+ Schema schema = getSchema(tableName);
+ TableConfigs tableConfigs = new TableConfigs(tableName, schema,
offlineTableConfig, null);
+ // PUT before POST should fail
+ try {
+ ControllerTestUtils
+
.sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsUpdate(tableName),
+ tableConfigs.toPrettyJsonString());
+ Assert.fail("Should fail for trying to PUT config before creating via
POST");
+ } catch (Exception e) {
+ // expected
+ }
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ String response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ TableConfigs tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+ Assert.assertEquals(tableConfigsResponse.getOffline().getTableName(),
offlineTableConfig.getTableName());
+ Assert.assertNull(tableConfigs.getRealtime());
+ Assert.assertEquals(tableConfigsResponse.getSchema().getSchemaName(),
schema.getSchemaName());
+
+ // list
+ String getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ List<String> configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 1);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet(tableName)));
+
+ // update to 2
+ tableConfigs = new TableConfigs(tableName,
tableConfigsResponse.getSchema(), tableConfigsResponse.getOffline(),
+ realtimeTableConfig);
+ ControllerTestUtils
+
.sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsUpdate(tableName),
+ tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+ Assert.assertEquals(tableConfigsResponse.getOffline().getTableName(),
offlineTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getRealtime().getTableName(),
realtimeTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getSchema().getSchemaName(),
schema.getSchemaName());
+
+ // list
+ getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 1);
+ Assert.assertTrue(configs.containsAll(Sets.newHashSet(tableName)));
+
+ // update existing config
+ schema.addField(new MetricFieldSpec("newMetric", FieldSpec.DataType.LONG));
+ tableConfigs = new TableConfigs(tableName, schema,
tableConfigsResponse.getOffline(), tableConfigs.getRealtime());
+ ControllerTestUtils
+
.sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsUpdate(tableName),
+ tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+ Assert.assertEquals(tableConfigsResponse.getOffline().getTableName(),
offlineTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getRealtime().getTableName(),
realtimeTableConfig.getTableName());
+ Assert.assertEquals(tableConfigsResponse.getSchema().getSchemaName(),
schema.getSchemaName());
+
Assert.assertTrue(tableConfigsResponse.getSchema().getMetricNames().contains("newMetric"));
+
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+ }
+
+ @Test
+ public void testDeleteConfig()
+ throws Exception {
+ // create with 1 config
+ String tableName = "testDelete1";
+ TableConfig offlineTableConfig = getOfflineTableConfig(tableName);
+ Schema schema = getSchema(tableName);
+ TableConfigs tableConfigs = new TableConfigs(tableName, schema,
offlineTableConfig, null);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ String response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ TableConfigs tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+
+ // delete & check
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+ String getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ List<String> configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 0);
+
+ tableName = "testDelete2";
+ offlineTableConfig = getOfflineTableConfig(tableName);
+ TableConfig realtimeTableConfig = getRealtimeTableConfig(tableName);
+ schema = getSchema(tableName);
+ tableConfigs = new TableConfigs(tableName, schema, offlineTableConfig,
realtimeTableConfig);
+ ControllerTestUtils.sendPostRequest(_createTableConfigsUrl,
tableConfigs.toPrettyJsonString());
+ response = ControllerTestUtils
+
.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsGet(tableName));
+ tableConfigsResponse = JsonUtils.stringToObject(response,
TableConfigs.class);
+ Assert.assertEquals(tableConfigsResponse.getTableName(), tableName);
+
+ // delete & check
+ ControllerTestUtils
+
.sendDeleteRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsDelete(tableName));
+ getResponse =
+
ControllerTestUtils.sendGetRequest(ControllerTestUtils.getControllerRequestURLBuilder().forTableConfigsList());
+ configs = JsonUtils.stringToObject(getResponse, new
TypeReference<List<String>>() {
+ });
+ Assert.assertEquals(configs.size(), 0);
+ }
+
+ @AfterClass
+ public void tearDown() {
+ ControllerTestUtils.cleanup();
+ }
+}
diff --git
a/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResourceTest.java
b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResourceTest.java
index 1718186..e6fafef 100644
---
a/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResourceTest.java
+++
b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResourceTest.java
@@ -22,6 +22,7 @@ import java.io.File;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.lang3.tuple.Pair;
+import
org.apache.pinot.controller.api.exception.ControllerApplicationException;
import org.apache.pinot.spi.crypt.NoOpPinotCrypter;
import org.apache.pinot.spi.crypt.PinotCrypterFactory;
import org.apache.pinot.spi.env.PinotConfiguration;
@@ -124,4 +125,4 @@ public class PinotSegmentUploadDownloadRestletResourceTest {
assertNull(encryptionInfo.getLeft());
assertEquals(_decryptedFile, encryptionInfo.getRight());
}
-}
\ No newline at end of file
+}
diff --git
a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerStatelessTest.java
b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerStatelessTest.java
index 71fbedd..c0daad8 100644
---
a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerStatelessTest.java
+++
b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManagerStatelessTest.java
@@ -21,6 +21,7 @@ package org.apache.pinot.controller.helix.core;
import java.util.Map;
import org.apache.pinot.common.utils.config.TagNameUtils;
import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.controller.api.exception.InvalidTableConfigException;
import org.apache.pinot.controller.helix.ControllerTest;
import org.apache.pinot.spi.config.table.TableConfig;
import org.apache.pinot.spi.config.table.TableType;
@@ -36,9 +37,6 @@ import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
-import static
org.apache.pinot.controller.helix.core.PinotHelixResourceManager.InvalidTableConfigException;
-
-
public class PinotHelixResourceManagerStatelessTest extends ControllerTest {
private static final int BASE_SERVER_ADMIN_PORT = 10000;
private static final int NUM_INSTANCES = 5;
diff --git a/pinot-segment-local/pom.xml b/pinot-segment-local/pom.xml
index 809a1f6..fba6f1f 100644
--- a/pinot-segment-local/pom.xml
+++ b/pinot-segment-local/pom.xml
@@ -102,6 +102,10 @@
<groupId>com.clearspring.analytics</groupId>
<artifactId>stream</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.quartz-scheduler</groupId>
+ <artifactId>quartz</artifactId>
+ </dependency>
<!-- Test dependencies -->
<dependency>
@@ -153,4 +157,4 @@
<scope>test</scope>
</dependency>
</dependencies>
-</project>
\ No newline at end of file
+</project>
diff --git
a/pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java
b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ReplicationUtils.java
similarity index 100%
rename from
pinot-core/src/main/java/org/apache/pinot/core/util/ReplicationUtils.java
rename to
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ReplicationUtils.java
diff --git
a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
index 3bac169..1365cfe 100644
---
a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
+++
b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
@@ -18,6 +18,7 @@
*/
package org.apache.pinot.segment.local.utils;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ArrayListMultimap;
import java.util.HashSet;
@@ -29,32 +30,42 @@ import java.util.stream.Collectors;
import javax.annotation.Nullable;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.collections.MapUtils;
+import org.apache.pinot.common.config.tuner.TableConfigTunerRegistry;
import org.apache.pinot.common.tier.TierFactory;
import org.apache.pinot.common.utils.config.TagNameUtils;
+import org.apache.pinot.core.util.ReplicationUtils;
import org.apache.pinot.segment.local.function.FunctionEvaluator;
import org.apache.pinot.segment.local.function.FunctionEvaluatorFactory;
import
org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair;
import org.apache.pinot.spi.config.table.FieldConfig;
import org.apache.pinot.spi.config.table.IndexingConfig;
+import org.apache.pinot.spi.config.table.QuotaConfig;
import org.apache.pinot.spi.config.table.RoutingConfig;
import org.apache.pinot.spi.config.table.SegmentsValidationAndRetentionConfig;
import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
import org.apache.pinot.spi.config.table.TableType;
import org.apache.pinot.spi.config.table.TierConfig;
+import org.apache.pinot.spi.config.table.TunerConfig;
import org.apache.pinot.spi.config.table.UpsertConfig;
import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
import org.apache.pinot.spi.config.table.ingestion.FilterConfig;
import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
import org.apache.pinot.spi.config.table.ingestion.TransformConfig;
+import org.apache.pinot.spi.config.table.tuner.TableConfigTuner;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.FieldSpec.DataType;
import org.apache.pinot.spi.data.Schema;
import org.apache.pinot.spi.ingestion.batch.BatchConfig;
import org.apache.pinot.spi.stream.StreamConfig;
import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.DataSizeUtils;
import org.apache.pinot.spi.utils.IngestionConfigUtils;
import org.apache.pinot.spi.utils.TimeUtils;
+import org.quartz.CronScheduleBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
@@ -63,6 +74,10 @@ import org.apache.pinot.spi.utils.TimeUtils;
*/
public final class TableConfigUtils {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(TableConfigUtils.class);
+ private static final String SEGMENT_GENERATION_AND_PUSH_TASK_TYPE =
"SegmentGenerationAndPushTask";
+ private static final String SCHEDULE_KEY = "schedule";
+
private TableConfigUtils() {
}
@@ -88,6 +103,7 @@ public final class TableConfigUtils {
validateIndexingConfig(tableConfig.getIndexingConfig(), schema);
validateFieldConfigList(tableConfig.getFieldConfigList(),
tableConfig.getIndexingConfig(), schema);
validateUpsertConfig(tableConfig, schema);
+ validateTaskConfigs(tableConfig);
}
/**
@@ -108,7 +124,7 @@ public final class TableConfigUtils {
* - Valid segmentPushType
* - Valid retentionTimeUnit
*/
- public static void validateRetentionConfig(TableConfig tableConfig) {
+ private static void validateRetentionConfig(TableConfig tableConfig) {
SegmentsValidationAndRetentionConfig segmentsConfig =
tableConfig.getValidationConfig();
String tableName = tableConfig.getTableName();
@@ -191,6 +207,7 @@ public final class TableConfigUtils {
* 5. checks for source fields used in destination columns
* 6. ingestion type for dimension tables
*/
+ @VisibleForTesting
public static void validateIngestionConfig(TableConfig tableConfig,
@Nullable Schema schema) {
IngestionConfig ingestionConfig = tableConfig.getIngestionConfig();
@@ -283,6 +300,22 @@ public final class TableConfigUtils {
}
}
+ private static void validateTaskConfigs(TableConfig tableConfig) {
+ TableTaskConfig taskConfig = tableConfig.getTaskConfig();
+ if (taskConfig != null &&
taskConfig.isTaskTypeEnabled(SEGMENT_GENERATION_AND_PUSH_TASK_TYPE)) {
+ Map<String, String> taskTypeConfig =
taskConfig.getConfigsForTaskType(SEGMENT_GENERATION_AND_PUSH_TASK_TYPE);
+ if (taskTypeConfig != null && taskTypeConfig.containsKey(SCHEDULE_KEY)) {
+ String cronExprStr = taskTypeConfig.get(SCHEDULE_KEY);
+ try {
+ CronScheduleBuilder.cronSchedule(cronExprStr);
+ } catch (Exception e) {
+ throw new IllegalStateException(
+ String.format("SegmentGenerationAndPushTask contains an invalid
cron schedule: %s", cronExprStr), e);
+ }
+ }
+ }
+ }
+
/**
* Validates the upsert-related configurations
* - check table type is realtime
@@ -290,6 +323,7 @@ public final class TableConfigUtils {
* - strict replica-group is configured for routing type
* - consumer type must be low-level
*/
+ @VisibleForTesting
public static void validateUpsertConfig(TableConfig tableConfig, Schema
schema) {
if (tableConfig.getUpsertMode() == UpsertConfig.Mode.NONE) {
return;
@@ -581,4 +615,127 @@ public final class TableConfigUtils {
}
return null;
}
+
+ /**
+ * Apply TunerConfig to the tableConfig
+ */
+ public static void applyTunerConfig(TableConfig tableConfig, Schema schema) {
+ TunerConfig tunerConfig = tableConfig.getTunerConfig();
+ if (tunerConfig != null && tunerConfig.getName() != null &&
!tunerConfig.getName().isEmpty()) {
+ TableConfigTuner tuner =
TableConfigTunerRegistry.getTuner(tunerConfig.getName());
+ tuner.init(tunerConfig, schema);
+ tuner.apply(tableConfig);
+ }
+ }
+
+ /**
+ * Ensure that the table config has the minimum number of replicas set as
per cluster configs.
+ * If is doesn't, set the required amount of replication in the table config
+ */
+ public static void ensureMinReplicas(TableConfig tableConfig, int
defaultTableMinReplicas) {
+ // For self-serviced cluster, ensure that the tables are created with at
least min replication factor irrespective
+ // of table configuration value
+ SegmentsValidationAndRetentionConfig segmentsConfig =
tableConfig.getValidationConfig();
+ boolean verifyReplicasPerPartition;
+ boolean verifyReplication;
+
+ try {
+ verifyReplicasPerPartition =
ReplicationUtils.useReplicasPerPartition(tableConfig);
+ verifyReplication = ReplicationUtils.useReplication(tableConfig);
+ } catch (Exception e) {
+ throw new IllegalStateException(String.format("Invalid tableIndexConfig
or streamConfig: %s", e.getMessage()), e);
+ }
+
+ if (verifyReplication) {
+ int requestReplication;
+ try {
+ requestReplication = segmentsConfig.getReplicationNumber();
+ if (requestReplication < defaultTableMinReplicas) {
+ LOGGER.info("Creating table with minimum replication factor of: {}
instead of requested replication: {}",
+ defaultTableMinReplicas, requestReplication);
+
segmentsConfig.setReplication(String.valueOf(defaultTableMinReplicas));
+ }
+ } catch (NumberFormatException e) {
+ throw new IllegalStateException("Invalid replication number", e);
+ }
+ }
+
+ if (verifyReplicasPerPartition) {
+ String replicasPerPartitionStr =
segmentsConfig.getReplicasPerPartition();
+ if (replicasPerPartitionStr == null) {
+ throw new IllegalStateException("Field replicasPerPartition needs to
be specified");
+ }
+ try {
+ int replicasPerPartition = Integer.parseInt(replicasPerPartitionStr);
+ if (replicasPerPartition < defaultTableMinReplicas) {
+ LOGGER.info(
+ "Creating table with minimum replicasPerPartition of: {} instead
of requested replicasPerPartition: {}",
+ defaultTableMinReplicas, replicasPerPartition);
+
segmentsConfig.setReplicasPerPartition(String.valueOf(defaultTableMinReplicas));
+ }
+ } catch (NumberFormatException e) {
+ throw new IllegalStateException("Invalid value for
replicasPerPartition: '" + replicasPerPartitionStr + "'", e);
+ }
+ }
+ }
+
+ /**
+ * Ensure the table config has storage quota set as per cluster configs.
+ * If it doesn't, set the quota config into the table config
+ */
+ public static void ensureStorageQuotaConstraints(TableConfig tableConfig,
String maxAllowedSize) {
+ // Dim tables must adhere to cluster level storage size limits
+ if (tableConfig.isDimTable()) {
+ QuotaConfig quotaConfig = tableConfig.getQuotaConfig();
+ long maxAllowedSizeInBytes = DataSizeUtils.toBytes(maxAllowedSize);
+
+ if (quotaConfig == null) {
+ // set a default storage quota
+ tableConfig.setQuotaConfig(new QuotaConfig(maxAllowedSize, null));
+ LOGGER.info("Assigning default storage quota ({}) for dimension table:
{}", maxAllowedSize,
+ tableConfig.getTableName());
+ } else {
+ if (quotaConfig.getStorage() == null) {
+ // set a default storage quota and keep the RPS value
+ tableConfig.setQuotaConfig(new QuotaConfig(maxAllowedSize,
quotaConfig.getMaxQueriesPerSecond()));
+ LOGGER.info("Assigning default storage quota ({}) for dimension
table: {}", maxAllowedSize,
+ tableConfig.getTableName());
+ } else {
+ if (quotaConfig.getStorageInBytes() > maxAllowedSizeInBytes) {
+ throw new IllegalStateException(String
+ .format("Invalid storage quota: %d, max allowed size: %d",
quotaConfig.getStorageInBytes(),
+ maxAllowedSizeInBytes));
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Consistency checks across the offline and realtime counterparts of a
hybrid table
+ */
+ public static void verifyHybridTableConfigs(String rawTableName, TableConfig
offlineTableConfig,
+ TableConfig realtimeTableConfig) {
+ Preconditions
+ .checkNotNull(offlineTableConfig, "Found null offline table config in
hybrid table check for table: %s",
+ rawTableName);
+ Preconditions
+ .checkNotNull(realtimeTableConfig, "Found null realtime table config
in hybrid table check for table: %s",
+ rawTableName);
+ LOGGER.info("Validating realtime and offline configs for the hybrid table:
{}", rawTableName);
+ SegmentsValidationAndRetentionConfig offlineSegmentConfig =
offlineTableConfig.getValidationConfig();
+ SegmentsValidationAndRetentionConfig realtimeSegmentConfig =
realtimeTableConfig.getValidationConfig();
+ String offlineTimeColumnName = offlineSegmentConfig.getTimeColumnName();
+ String realtimeTimeColumnName = realtimeSegmentConfig.getTimeColumnName();
+ if (offlineTimeColumnName == null || realtimeTimeColumnName == null) {
+ throw new IllegalStateException(String.format(
+ "'timeColumnName' cannot be null for table: %s! Offline time column
name: %s. Realtime time column name: %s",
+ rawTableName, offlineTimeColumnName, realtimeTimeColumnName));
+ }
+ if (!offlineTimeColumnName.equals(realtimeTimeColumnName)) {
+ throw new IllegalStateException(String.format(
+ "Time column names are different for table: %s! Offline time column
name: %s. Realtime time column name: %s",
+ rawTableName, offlineTimeColumnName, realtimeTimeColumnName));
+ }
+ }
}
diff --git
a/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfigs.java
b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfigs.java
new file mode 100644
index 0000000..02216ac
--- /dev/null
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/TableConfigs.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.spi.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import javax.annotation.Nullable;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+
+
+/**
+ * Wrapper for all configs of a table, which include the offline table config,
realtime table config and schema.
+ * This helps look at and operate on the pinot table configs as a whole unit.
+ */
+public class TableConfigs extends BaseJsonConfig {
+ private final String _tableName;
+ private final Schema _schema;
+ private final TableConfig _offline;
+ private final TableConfig _realtime;
+
+ @JsonCreator
+ public TableConfigs(@JsonProperty(value = "tableName", required = true)
String tableName,
+ @JsonProperty(value = "schema", required = true) Schema schema,
+ @JsonProperty(value = "offline") @Nullable TableConfig offline,
+ @JsonProperty(value = "realtime") @Nullable TableConfig realtime) {
+ Preconditions.checkState(StringUtils.isNotBlank(tableName), "'tableName'
cannot be null or empty in TableConfigs");
+ Preconditions.checkNotNull(schema, "'schema' cannot be null in
TableConfigs");
+ _tableName = tableName;
+ _schema = schema;
+ _offline = offline;
+ _realtime = realtime;
+ }
+
+ public String getTableName() {
+ return _tableName;
+ }
+
+ public Schema getSchema() {
+ return _schema;
+ }
+
+ @Nullable
+ public TableConfig getOffline() {
+ return _offline;
+ }
+
+ @Nullable
+ public TableConfig getRealtime() {
+ return _realtime;
+ }
+
+ public String toPrettyJsonString() {
+ try {
+ return JsonUtils.objectToPrettyString(this);
+ } catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]