diff --git a/pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/README.md b/pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/README.md index 270397ec5335..2f858b3e9ff3 100644 --- a/pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/README.md +++ b/pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/README.md @@ -147,6 +147,68 @@ String status = tableClient.getTableStatus("myTable"); String rebalanceResult = tableClient.rebalanceTable("myTable", true, "default", 1); ``` +### Creating Tables With Modern Config Fields + +Table creation endpoints reject deprecated table-config properties when they are explicitly present in the request +payload. Existing tables can still be updated with legacy fields for backward compatibility, but new create requests +must use the current config layout. + +Common migrations: + +- `segmentsConfig.segmentPushType` -> `ingestionConfig.batchIngestionConfig.segmentIngestionType` +- `segmentsConfig.segmentPushFrequency` -> `ingestionConfig.batchIngestionConfig.segmentIngestionFrequency` +- `tableIndexConfig.streamConfigs` -> `ingestionConfig.streamIngestionConfig.streamConfigMaps` +- `fieldConfigList[].indexType` -> `fieldConfigList[].indexTypes` + +Sample REALTIME table config for create: + +```json +{ + "tableName": "events", + "tableType": "REALTIME", + "segmentsConfig": { + "timeColumnName": "ts", + "replication": "1" + }, + "tenants": { + "broker": "DefaultTenant", + "server": "DefaultTenant" + }, + "tableIndexConfig": { + "loadMode": "MMAP" + }, + "fieldConfigList": [ + { + "name": "userId", + "encodingType": "DICTIONARY", + "indexTypes": [ + "INVERTED" + ] + } + ], + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND", + "segmentIngestionFrequency": "DAILY" + }, + "streamIngestionConfig": { + "streamConfigMaps": [ + { + "streamType": "kafka", + "stream.kafka.topic.name": "events", + "stream.kafka.consumer.type": "lowlevel", + "stream.kafka.decoder.class.name": "org.apache.pinot.plugin.inputformat.json.JSONMessageDecoder" + } + ] + } + }, + "metadata": {} +} +``` + +If a create request still sends deprecated fields, the controller returns `400 BAD_REQUEST` with the offending JSON path +and the replacement field to use. + ### Schema Operations ```java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java index 00db262b8d4a..3b829bd3c3ae 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java @@ -519,6 +519,16 @@ public static TableConfig getTableConfig(ZkHelixPropertyStore property AccessOption.PERSISTENT), replaceVariables, applyDecorator); } + /// Returns the raw [ZNRecord] for a stored table config without deserializing it. Use this when callers need the + /// byte-faithful view of what was last written to ZK (e.g. update-time deprecation diffing). Returns null when + /// the table does not exist. + @Nullable + public static ZNRecord getTableConfigZNRecord(ZkHelixPropertyStore propertyStore, + String tableNameWithType) { + return propertyStore.get(constructPropertyStorePathForResourceConfig(tableNameWithType), null, + AccessOption.PERSISTENT); + } + @Nullable public static ImmutablePair getTableConfigWithStat(ZkHelixPropertyStore propertyStore, String tableNameWithType) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtils.java index a24583891fd7..a29cc609504e 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtils.java @@ -20,6 +20,9 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Preconditions; import java.io.IOException; import java.util.HashMap; @@ -293,4 +296,36 @@ public static ZNRecord toZNRecord(TableConfig tableConfig) znRecord.setSimpleFields(simpleFields); return znRecord; } + + /// Reconstructs the table-config JSON tree from a stored [ZNRecord], preserving every key originally written to + /// ZK. Unlike `TableConfig.toJsonNode()`, this method does not round-trip through the Java bean and therefore + /// does not strip fields that the bean's getters mark with `@JsonIgnore` or `@JsonInclude(NON_DEFAULT)`. It is + /// intended for code paths (e.g. update-time deprecation diffing) that need to compare an incoming request + /// against the exact bytes that were stored. + /// + /// @param znRecord the raw ZNRecord read from the property store + /// @return a JsonNode equivalent to what the user originally PUT/POST-ed for the table, or `null` if the input is + /// `null` + public static JsonNode toRawJsonNode(ZNRecord znRecord) { + if (znRecord == null) { + return null; + } + ObjectNode root = JsonNodeFactory.instance.objectNode(); + Map simpleFields = znRecord.getSimpleFields(); + for (Map.Entry entry : simpleFields.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + // Try to parse as a JSON node first (most fields hold JSON-encoded sub-objects). Fall back to a string node + // for primitive simple fields (e.g. tableType, isDimTable). + try { + root.set(key, JsonUtils.stringToJsonNode(value)); + } catch (IOException e) { + root.put(key, value); + } + } + if (!root.has(TableConfig.TABLE_NAME_KEY)) { + root.put(TableConfig.TABLE_NAME_KEY, znRecord.getId()); + } + return root; + } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtilsTest.java index 796f26b02db3..5f39b8855a1d 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeUtilsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.common.utils.config; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.collect.Lists; import java.io.IOException; @@ -26,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.tier.TierFactory; import org.apache.pinot.spi.config.table.CompletionConfig; import org.apache.pinot.spi.config.table.DedupConfig; @@ -607,4 +609,31 @@ private void checkTableConfigWithDedupConfigWithTTL(TableConfig tableConfig) { assertEquals(dedupConfig.getMetadataTTL(), 10); assertEquals(dedupConfig.getDedupTimeColumn(), "dedupTimeColumn"); } + + /** + * Confirms that {@code toRawJsonNode} preserves keys that are stripped by the deserialize/re-serialize round-trip + * (e.g. {@code @JsonIgnore} or {@code @JsonInclude(NON_DEFAULT)} getters). The deprecation diff in + * {@code DeprecatedTableConfigValidationUtils} relies on this. + */ + @Test + public void testToRawJsonNodePreservesStrippedKeys() { + ZNRecord znRecord = new ZNRecord("myTable_OFFLINE"); + znRecord.setSimpleField(TableConfig.TABLE_NAME_KEY, "myTable_OFFLINE"); + znRecord.setSimpleField(TableConfig.TABLE_TYPE_KEY, "OFFLINE"); + // The fieldConfigList stored in ZK still contains the legacy 'indexType' key (written before @JsonIgnore). + znRecord.setSimpleField(TableConfig.FIELD_CONFIG_LIST_KEY, + "[{\"name\":\"c1\",\"indexType\":\"INVERTED\",\"indexTypes\":[\"INVERTED\"]}]"); + + JsonNode raw = TableConfigSerDeUtils.toRawJsonNode(znRecord); + assertNotNull(raw); + JsonNode legacy = raw.get(TableConfig.FIELD_CONFIG_LIST_KEY); + assertNotNull(legacy); + assertTrue(legacy.isArray()); + assertEquals(legacy.get(0).get("indexType").asText(), "INVERTED"); + } + + @Test + public void testToRawJsonNodeHandlesNull() { + assertNull(TableConfigSerDeUtils.toRawJsonNode(null)); + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java index ab5eff708060..4b71fac5f698 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java @@ -18,17 +18,33 @@ */ package org.apache.pinot.controller.api.resources; +import com.fasterxml.jackson.annotation.JsonInclude; +import java.util.List; import java.util.Map; public final class ConfigSuccessResponse extends SuccessResponse { private final Map _unrecognizedProperties; + private final List _deprecationWarnings; public ConfigSuccessResponse(String status, Map unrecognizedProperties) { + this(status, unrecognizedProperties, List.of()); + } + + public ConfigSuccessResponse(String status, Map unrecognizedProperties, + List deprecationWarnings) { super(status); - _unrecognizedProperties = unrecognizedProperties; + _unrecognizedProperties = unrecognizedProperties == null ? Map.of() : unrecognizedProperties; + _deprecationWarnings = deprecationWarnings == null ? List.of() : deprecationWarnings; } public Map getUnrecognizedProperties() { return _unrecognizedProperties; } + + /// `@JsonInclude(NON_EMPTY)` is on the getter so the empty-list case is elided from the response. Older clients + /// that strict-parse this DTO continue to see the original (pre-deprecationWarnings) shape when no warnings fire. + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public List getDeprecationWarnings() { + return _deprecationWarnings; + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtils.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtils.java new file mode 100644 index 000000000000..da0c2928a656 --- /dev/null +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtils.java @@ -0,0 +1,515 @@ +/** + * 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.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.common.version.PinotVersion; +import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/// Validates raw table-config JSON for deprecated properties on create and update paths. +/// +/// Rules are derived at class-load time from [DeprecatedConfig] annotations on getters reachable from [TableConfig]. +/// The current soft-launch policy reports every parseable annotation `since` value as [Severity#WARNING] so existing +/// callers continue to work while seeing migration guidance. Only an unparseable annotation `since` value classifies +/// as [Severity#ERROR], since that case is a code-side bug rather than user-supplied data. +/// +/// The validator operates on raw JSON rather than the deserialized [TableConfig] so it can detect explicitly +/// provided deprecated keys even when they carry default or false-y values that Jackson would otherwise elide. +public final class DeprecatedTableConfigValidationUtils { + private DeprecatedTableConfigValidationUtils() { + } + + private static final Logger LOGGER = LoggerFactory.getLogger(DeprecatedTableConfigValidationUtils.class); + private static final String WILDCARD = "*"; + private static final String FIELD_CONFIG_INDEX_TYPES_KEY = "indexTypes"; + private static final List FIELD_CONFIG_INDEX_TYPE_PATH = + List.of(TableConfig.FIELD_CONFIG_LIST_KEY, WILDCARD, "indexType"); + // CURRENT_MAJOR_MINOR must be initialized before RULES because discoverRules() -> classifySeverity() reads it. + private static final String CURRENT_MAJOR_MINOR = currentMajorMinor(); + private static final List RULES = discoverRules(); + + public enum Severity { + WARNING, ERROR + } + + /// Result of validating a table-config JSON, exposing errors and warnings separately so callers can decide + /// whether to reject the request or surface non-fatal warnings to the user. + public static final class Result { + private final List _errors; + private final List _warnings; + + private Result(List errors, List warnings) { + _errors = List.copyOf(errors); + _warnings = List.copyOf(warnings); + } + + public List getErrors() { + return _errors; + } + + public List getWarnings() { + return _warnings; + } + + public boolean hasErrors() { + return !_errors.isEmpty(); + } + + public boolean hasWarnings() { + return !_warnings.isEmpty(); + } + } + + /// Validates a table-config JSON document. When `oldTableConfigJson` is non-null, only paths whose value newly + /// appears in `newTableConfigJson` or whose value differs from the corresponding path in `oldTableConfigJson` are + /// reported, so re-submitting an unchanged legacy field on an update is a no-op. When `oldTableConfigJson` is null + /// (creation path), every present deprecated path is reported. + /// + /// @param newTableConfigJson the incoming table config to validate; must not be `null` + /// @param oldTableConfigJson the currently-stored table config to diff against, or `null` for create paths + /// @param rootPathPrefix optional prefix for emitted paths (e.g. `"realtime"` when validating a sub-section) + public static Result validate(JsonNode newTableConfigJson, @Nullable JsonNode oldTableConfigJson, + @Nullable String rootPathPrefix) { + Objects.requireNonNull(newTableConfigJson, "newTableConfigJson"); + List errors = new ArrayList<>(); + List warnings = new ArrayList<>(); + String prefix = rootPathPrefix == null ? "" : rootPathPrefix; + for (DeprecatedConfigRule rule : RULES) { + rule.collect(newTableConfigJson, oldTableConfigJson, prefix, errors, warnings); + } + return new Result(errors, warnings); + } + + /// Validates a freshly-submitted table config (no prior stored value). On any error the method throws + /// [IllegalArgumentException]; warnings are returned so the caller can surface them in the response. + public static List validateOnCreate(JsonNode newTableConfigJson, @Nullable String rootPathPrefix) { + Result result = validate(newTableConfigJson, null, rootPathPrefix); + if (result.hasErrors()) { + throw new IllegalArgumentException("Deprecated table config properties are not allowed on table creation: " + + String.join("; ", result.getErrors())); + } + if (result.hasWarnings()) { + LOGGER.warn("Deprecated table config properties on creation: {}", String.join("; ", result.getWarnings())); + } + return result.getWarnings(); + } + + /// Validates an updated table config against its currently-stored counterpart. Only newly-introduced or + /// value-changed deprecated paths are reported, so legacy values that were already present do not block updates. + /// On any error the method throws [IllegalArgumentException]; warnings are returned for the caller to surface. + /// + /// Callers must ensure the table exists before invoking this method; pass the stored config JSON (never `null`) + /// for `oldTableConfigJson`. For paths where no stored counterpart exists, use [#validateOnCreate] instead. + public static List validateOnUpdate(JsonNode newTableConfigJson, JsonNode oldTableConfigJson, + @Nullable String rootPathPrefix) { + Objects.requireNonNull(oldTableConfigJson, "oldTableConfigJson; use validateOnCreate for create paths"); + Result result = validate(newTableConfigJson, oldTableConfigJson, rootPathPrefix); + if (result.hasErrors()) { + throw new IllegalArgumentException("Newly introduced deprecated table config properties are not allowed: " + + String.join("; ", result.getErrors())); + } + if (result.hasWarnings()) { + LOGGER.warn("Newly introduced deprecated table config properties on update: {}", + String.join("; ", result.getWarnings())); + } + return result.getWarnings(); + } + + static List rulesForTesting() { + return Collections.unmodifiableList(RULES); + } + + private static List discoverRules() { + List rules = new ArrayList<>(); + walk(TableConfig.class, new ArrayList<>(), new HashSet<>(), rules); + return Collections.unmodifiableList(rules); + } + + private static void walk(Type type, List currentPath, Set> visiting, + List rules) { + Class rawClass = rawClass(type); + if (rawClass == null) { + return; + } + if (Map.class.isAssignableFrom(rawClass)) { + Type valueType = typeArg(type, 1); + if (valueType != null) { + currentPath.add(WILDCARD); + walk(valueType, currentPath, visiting, rules); + currentPath.remove(currentPath.size() - 1); + } + return; + } + if (Collection.class.isAssignableFrom(rawClass)) { + Type elemType = typeArg(type, 0); + if (elemType != null) { + currentPath.add(WILDCARD); + walk(elemType, currentPath, visiting, rules); + currentPath.remove(currentPath.size() - 1); + } + return; + } + if (rawClass.isArray()) { + currentPath.add(WILDCARD); + walk(rawClass.getComponentType(), currentPath, visiting, rules); + currentPath.remove(currentPath.size() - 1); + return; + } + if (!isConfigBean(rawClass) || !visiting.add(rawClass)) { + return; + } + try { + for (Method method : rawClass.getMethods()) { + if (!isJsonAccessor(method)) { + continue; + } + String propertyName = jsonPropertyName(method); + if (propertyName == null) { + continue; + } + currentPath.add(propertyName); + DeprecatedConfig anno = method.getAnnotation(DeprecatedConfig.class); + if (anno != null) { + rules.add(new DeprecatedConfigRule(List.copyOf(currentPath), anno.replacement(), anno.since(), + classifySeverity(anno.since()))); + } + walk(method.getGenericReturnType(), currentPath, visiting, rules); + currentPath.remove(currentPath.size() - 1); + } + } finally { + visiting.remove(rawClass); + } + } + + private static boolean isConfigBean(Class rawClass) { + return BaseJsonConfig.class.isAssignableFrom(rawClass) && rawClass != BaseJsonConfig.class; + } + + private static boolean isJsonAccessor(Method method) { + if (method.getParameterCount() != 0 || method.getDeclaringClass() == Object.class + || Modifier.isStatic(method.getModifiers())) { + return false; + } + Class returnType = method.getReturnType(); + if (returnType == void.class) { + return false; + } + String name = method.getName(); + if (name.startsWith("get") && name.length() > 3) { + return true; + } + return name.startsWith("is") && name.length() > 2 && (returnType == boolean.class || returnType == Boolean.class); + } + + @Nullable + private static String jsonPropertyName(Method method) { + JsonProperty jsonProperty = method.getAnnotation(JsonProperty.class); + if (jsonProperty != null && !jsonProperty.value().isEmpty()) { + return jsonProperty.value(); + } + String name = method.getName(); + String stripped; + if (name.startsWith("get")) { + stripped = name.substring(3); + } else if (name.startsWith("is")) { + stripped = name.substring(2); + } else { + return null; + } + if (stripped.isEmpty()) { + return null; + } + return Character.toLowerCase(stripped.charAt(0)) + stripped.substring(1); + } + + @Nullable + private static Class rawClass(Type type) { + if (type instanceof Class) { + return (Class) type; + } + if (type instanceof ParameterizedType) { + Type rawType = ((ParameterizedType) type).getRawType(); + if (rawType instanceof Class) { + return (Class) rawType; + } + } + return null; + } + + @Nullable + private static Type typeArg(Type type, int index) { + if (type instanceof ParameterizedType) { + Type[] args = ((ParameterizedType) type).getActualTypeArguments(); + if (index < args.length) { + return args[index]; + } + } + return null; + } + + @Nullable + private static String currentMajorMinor() { + String version = PinotVersion.VERSION; + if (version == null || PinotVersion.UNKNOWN.equals(version)) { + return null; + } + return majorMinor(version); + } + + /// Returns the `major.minor` prefix of a version string, stripping any pre-release qualifier (e.g. `-SNAPSHOT`) + /// and ignoring the patch component. Returns `null` if the input does not parse. + @Nullable + static String majorMinor(@Nullable String version) { + if (version == null) { + return null; + } + String trimmed = version.trim(); + int qualifierIdx = trimmed.indexOf('-'); + if (qualifierIdx >= 0) { + trimmed = trimmed.substring(0, qualifierIdx); + } + String[] parts = trimmed.split("\\."); + if (parts.length < 2) { + return null; + } + if (!isNumeric(parts[0]) || !isNumeric(parts[1])) { + return null; + } + return parts[0] + "." + parts[1]; + } + + private static boolean isNumeric(String s) { + if (s.isEmpty()) { + return false; + } + for (int i = 0; i < s.length(); i++) { + if (!Character.isDigit(s.charAt(i))) { + return false; + } + } + return true; + } + + /// Decides the severity for a violation based on the annotation's `since` value. The current Pinot release's + /// deprecations are warnings (one-release grace period); older deprecations escalate to errors. + /// + /// When the current Pinot version cannot be determined (e.g. running from an IDE / shaded jar where + /// `pinot-version.properties` was not maven-filtered), severity falls back to [Severity#WARNING] so a + /// misconfigured deployment does not silently start rejecting every previously-valid table config. An + /// unparseable `since` value on the annotation itself still classifies as [Severity#ERROR] because that case + /// reflects a code-side bug rather than an environment-side one. + static Severity classifySeverity(String since) { + return classifySeverity(since, CURRENT_MAJOR_MINOR); + } + + /// Test seam for [#classifySeverity(String)] that takes the current major.minor version explicitly so the + /// version-unknown fallback branch can be unit-tested without manipulating the classloader-scoped + /// `pinot-version.properties` resource. + /// + /// Initial soft-launch policy: every parseable `since` returns WARNING so existing callers that already use + /// deprecated keys (TableConfigBuilder setters, integration test bases, downstream automations) keep working. + /// Only an unparseable `since` (a code-side bug in the annotation) classifies as ERROR. A follow-up PR can + /// promote older deprecations to ERROR after the codebase migrates off them. + static Severity classifySeverity(String since, @Nullable String currentMajorMinor) { + String sinceMajorMinor = majorMinor(since); + if (sinceMajorMinor == null) { + return Severity.ERROR; + } + return Severity.WARNING; + } + + static final class DeprecatedConfigRule { + private final List _pathSegments; + private final String _replacement; + private final String _since; + private final Severity _severity; + + DeprecatedConfigRule(List pathSegments, String replacement, String since, Severity severity) { + _pathSegments = pathSegments; + _replacement = replacement; + _since = since; + _severity = severity; + } + + List pathSegments() { + return _pathSegments; + } + + Severity severity() { + return _severity; + } + + void collect(JsonNode newRoot, @Nullable JsonNode oldRoot, String pathPrefix, List errors, + List warnings) { + List matches = new ArrayList<>(); + collectMatches(newRoot, oldRoot, null, 0, pathPrefix, matches); + for (MatchedPath match : matches) { + if (oldRoot != null) { + // Update path. Silently drop matches that are unchanged from the stored config, OR that are absent in + // the stored config but whose new value is the type's Java default. The default-skip handles the case + // where many deprecated getters carry @JsonInclude(NON_DEFAULT): a previous create with + // `enableSnapshot: false` is stripped at ZK write time, so on PUT the diff would otherwise see `false` + // as "newly introduced" and reject an unchanged config. Crucially, the default-skip applies ONLY when + // the stored config did not contain the path — a deliberate flip of an existing non-default value back + // to the default (e.g. `true` → `false`) is still reported, matching validateOnUpdate's contract that + // value-changed deprecated paths must be flagged. + if (match._oldValue != null) { + if (Objects.equals(match._newValue, match._oldValue)) { + continue; + } + } else if (isModernFieldConfigIndexTypeEquivalent(match) || isJacksonDefault(match._newValue)) { + continue; + } + } + String message = "'" + match._path + "' is deprecated since " + _since + ". " + _replacement; + if (_severity == Severity.ERROR) { + errors.add(message); + } else { + warnings.add(message); + } + } + } + + private boolean isModernFieldConfigIndexTypeEquivalent(MatchedPath match) { + if (!_pathSegments.equals(FIELD_CONFIG_INDEX_TYPE_PATH) || match._oldParent == null + || !match._oldParent.isObject()) { + return false; + } + JsonNode oldIndexTypes = match._oldParent.get(FIELD_CONFIG_INDEX_TYPES_KEY); + return oldIndexTypes != null && oldIndexTypes.isArray() && oldIndexTypes.size() == 1 + && Objects.equals(oldIndexTypes.get(0), match._newValue); + } + + /// Returns true when the JSON value matches the Java zero-value for its type (`false` for booleans, `0` for + /// numerics, empty for strings/arrays/objects, and explicit JSON `null`). This is an approximation of + /// Jackson's `@JsonInclude(NON_DEFAULT)` semantics: the real Jackson behaviour consults the bean's actual + /// initialised default by instantiating it, but every deprecated getter currently annotated with + /// `@DeprecatedConfig` uses a type-zero default, so the approximation is exact for today's rules. If a future + /// `@DeprecatedConfig` annotation is added on a field whose bean default is non-zero (e.g. an enum that + /// defaults to a non-null value), update this helper to consult the bean default via reflection. + /// + /// Numeric types are compared via integer paths where possible to avoid the `double`-coercion edge cases + /// (lossy conversion for `BigDecimal`/`BigInteger`, IEEE-754 `-0.0 == 0.0d`, etc.). Today no annotated getter + /// returns a floating-point or arbitrary-precision type, but the dispatch is robust enough to extend to those + /// cases in the future. + private static boolean isJacksonDefault(@Nullable JsonNode node) { + if (node == null || node.isMissingNode() || node.isNull()) { + return true; + } + if (node.isBoolean()) { + return !node.booleanValue(); + } + if (node.isShort() || node.isInt() || node.isLong()) { + return node.longValue() == 0L; + } + if (node.isBigInteger()) { + return BigInteger.ZERO.equals(node.bigIntegerValue()); + } + if (node.isBigDecimal()) { + return BigDecimal.ZERO.compareTo(node.decimalValue()) == 0; + } + if (node.isFloatingPointNumber()) { + // Use Double.compare so -0.0 is treated as non-default (it differs from the Java zero-value of 0.0d). + return Double.compare(node.doubleValue(), 0.0d) == 0; + } + if (node.isTextual()) { + // String-returning deprecated getters today (e.g. segmentPushType) all initialise to null, not "". Match + // Jackson's actual NON_DEFAULT contract: only the explicit-null case (handled above) is "default"; + // empty-string is a real user-supplied value and must be flagged on update. + return false; + } + if (node.isArray() || node.isObject()) { + return node.isEmpty(); + } + // BinaryNode / POJONode never arise from text JSON parsed by Jackson; treat as non-default conservatively. + return false; + } + + private void collectMatches(@Nullable JsonNode newNode, @Nullable JsonNode oldNode, + @Nullable JsonNode oldParentNode, int idx, String currentPath, List matches) { + if (newNode == null || newNode.isMissingNode()) { + return; + } + if (idx == _pathSegments.size()) { + matches.add(new MatchedPath(currentPath, newNode, oldNode, oldParentNode)); + return; + } + String segment = _pathSegments.get(idx); + if (WILDCARD.equals(segment)) { + if (newNode.isArray()) { + for (int i = 0; i < newNode.size(); i++) { + JsonNode oldElem = (oldNode != null && oldNode.isArray() && i < oldNode.size()) ? oldNode.get(i) : null; + collectMatches(newNode.get(i), oldElem, oldNode, idx + 1, currentPath + "[" + i + "]", matches); + } + } else if (newNode.isObject()) { + newNode.fieldNames().forEachRemaining(field -> { + JsonNode oldChild = (oldNode != null && oldNode.isObject()) ? oldNode.get(field) : null; + collectMatches(newNode.get(field), oldChild, oldNode, idx + 1, append(currentPath, field), matches); + }); + } + return; + } + if (newNode.has(segment)) { + JsonNode oldChild = (oldNode != null && oldNode.isObject()) ? oldNode.get(segment) : null; + collectMatches(newNode.get(segment), oldChild, oldNode, idx + 1, append(currentPath, segment), matches); + } + } + + private static String append(String currentPath, String segment) { + return currentPath.isEmpty() ? segment : currentPath + "." + segment; + } + } + + private static final class MatchedPath { + final String _path; + final JsonNode _newValue; + @Nullable + final JsonNode _oldValue; + @Nullable + final JsonNode _oldParent; + + MatchedPath(String path, JsonNode newValue, @Nullable JsonNode oldValue, @Nullable JsonNode oldParent) { + _path = path; + _newValue = newValue; + _oldValue = oldValue; + _oldParent = oldParent; + } + } +} 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 91985f5997ea..72a84cead708 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 @@ -93,6 +93,7 @@ import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.LogicalTableConfigUtils; import org.apache.pinot.common.utils.SimpleHttpResponse; +import org.apache.pinot.common.utils.config.TableConfigSerDeUtils; import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.controller.ControllerConf; @@ -231,6 +232,7 @@ public ConfigSuccessResponse addTable(String tableConfigStr, TableConfig tableConfig; String tableNameWithType; Schema schema; + List deprecationWarnings; try { tableConfigAndUnrecognizedProperties = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigStr, TableConfig.class); @@ -252,6 +254,8 @@ public ConfigSuccessResponse addTable(String tableConfigStr, schema = _pinotHelixResourceManager.getTableSchema(tableNameWithType); Preconditions.checkState(schema != null, "Failed to find schema for table: %s", tableNameWithType); + deprecationWarnings = DeprecatedTableConfigValidationUtils.validateOnCreate( + JsonUtils.stringToJsonNode(tableConfigStr), null); TableConfigTunerUtils.applyTunerConfigs(_pinotHelixResourceManager, tableConfig, schema, Collections.emptyMap()); TableConfigValidationUtils.validateTableConfig( @@ -271,7 +275,7 @@ public ConfigSuccessResponse addTable(String tableConfigStr, // the validation manager) LOGGER.info("Successfully added table: {} with config: {}", tableNameWithType, tableConfig); return new ConfigSuccessResponse("Table " + tableNameWithType + " successfully added", - tableConfigAndUnrecognizedProperties.getRight()); + tableConfigAndUnrecognizedProperties.getRight(), deprecationWarnings); } catch (Exception e) { _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR, 1L); if (e instanceof InvalidTableConfigException) { @@ -346,6 +350,11 @@ public CopyTableResponse copyTable( ObjectNode realtimeTableConfigNode = (ObjectNode) tableConfigNode.get(TableType.REALTIME.name()); tweakRealtimeTableConfig(realtimeTableConfigNode, copyTablePayload); + try { + DeprecatedTableConfigValidationUtils.validateOnCreate(realtimeTableConfigNode, null); + } catch (IllegalArgumentException e) { + throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST, e); + } TableConfig realtimeTableConfig = JsonUtils.jsonNodeToObject(realtimeTableConfigNode, TableConfig.class); if (realtimeTableConfig.getUpsertConfig() != null) { throw new IllegalStateException("upsert table copy not supported"); @@ -777,12 +786,17 @@ public ConfigSuccessResponse updateTableConfig( throws Exception { Pair> tableConfigAndUnrecognizedProperties; TableConfig tableConfig; + JsonNode newTableConfigJson; String tableNameWithType; Schema schema; + List deprecationWarnings = List.of(); try { tableConfigAndUnrecognizedProperties = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigString, TableConfig.class); tableConfig = tableConfigAndUnrecognizedProperties.getLeft(); + // Capture the raw JsonNode view alongside the typed view so the deprecation diff below does not re-parse + // the same input. + newTableConfigJson = JsonUtils.stringToJsonNode(tableConfigString); tableNameWithType = DatabaseUtils.translateTableName(tableConfig.getTableName(), headers); tableConfig.setTableName(tableNameWithType); String tableNameFromPath = DatabaseUtils.translateTableName( @@ -802,11 +816,33 @@ public ConfigSuccessResponse updateTableConfig( throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e); } + // Existence check runs before the deprecation diff so a PUT to a missing table reports 404 rather than a + // misleading 400 about deprecated keys. + if (!_pinotHelixResourceManager.hasTable(tableNameWithType)) { + throw new ControllerApplicationException(LOGGER, "Table " + tableNameWithType + " does not exist", + Response.Status.NOT_FOUND); + } + + // Deprecation diff is computed against the byte-faithful stored ZNRecord. Runs outside the BAD_REQUEST catch + // so ZK transient failures propagate as 5xx rather than masquerading as client-side validation errors. + // Best-effort: read and write are not atomic, so a concurrent legal write between the read here and the write + // below can leave deprecated keys in the stored config without firing a deprecation error. + try { + JsonNode oldTableConfigJson = TableConfigSerDeUtils.toRawJsonNode( + ZKMetadataProvider.getTableConfigZNRecord(_pinotHelixResourceManager.getPropertyStore(), + tableNameWithType)); + deprecationWarnings = DeprecatedTableConfigValidationUtils.validateOnUpdate( + newTableConfigJson, oldTableConfigJson, null); + } catch (IllegalArgumentException e) { + String msg = String.format("Invalid table config: %s with error: %s", tableName, e.getMessage()); + throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e); + } catch (RuntimeException e) { + _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR, 1L); + LOGGER.warn("Failed to compute deprecation diff for update of table: {}", tableNameWithType, e); + throw e; + } + try { - if (!_pinotHelixResourceManager.hasTable(tableNameWithType)) { - throw new ControllerApplicationException(LOGGER, "Table " + tableNameWithType + " does not exist", - Response.Status.NOT_FOUND); - } _pinotHelixResourceManager.updateTableConfig(tableConfig, force); } catch (TableConfigBackwardIncompatibleException e) { String errStr = String.format("Failed to update configuration for %s due to: %s", tableName, e.getMessage()); @@ -822,7 +858,7 @@ public ConfigSuccessResponse updateTableConfig( } LOGGER.info("Successfully updated table: {} with new config: {}", tableNameWithType, tableConfig); return new ConfigSuccessResponse("Table config updated for " + tableName, - tableConfigAndUnrecognizedProperties.getRight()); + tableConfigAndUnrecognizedProperties.getRight(), deprecationWarnings); } @POST @@ -837,9 +873,11 @@ public ObjectNode checkTableConfig(String tableConfigStr, @QueryParam("validationTypesToSkip") @Nullable String typesToSkip, @Context HttpHeaders httpHeaders, @Context Request request) { Pair> tableConfigAndUnrecognizedProperties; + JsonNode tableConfigJson; try { tableConfigAndUnrecognizedProperties = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigStr, TableConfig.class); + tableConfigJson = JsonUtils.stringToJsonNode(tableConfigStr); } catch (IOException e) { String msg = String.format("Invalid table config json string: %s. Reason: %s", tableConfigStr, e.getMessage()); throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e); @@ -851,10 +889,34 @@ public ObjectNode checkTableConfig(String tableConfigStr, // validate permission ResourceUtils.checkPermissionAndAccess(tableNameWithType, request, httpHeaders, AccessType.READ, Actions.Table.VALIDATE_TABLE_CONFIGS, _accessControlFactory, LOGGER); + JsonNode oldTableConfigJson; + try { + oldTableConfigJson = TableConfigSerDeUtils.toRawJsonNode( + ZKMetadataProvider.getTableConfigZNRecord(_pinotHelixResourceManager.getPropertyStore(), + tableNameWithType)); + } catch (RuntimeException e) { + LOGGER.warn("Failed to read stored config for validate of table: {}", tableNameWithType, e); + throw e; + } + List deprecationWarnings; + try { + if (oldTableConfigJson == null) { + deprecationWarnings = DeprecatedTableConfigValidationUtils.validateOnCreate(tableConfigJson, null); + } else { + deprecationWarnings = DeprecatedTableConfigValidationUtils.validateOnUpdate(tableConfigJson, + oldTableConfigJson, null); + } + } catch (IllegalArgumentException e) { + String msg = String.format("Invalid table config: %s. %s", tableNameWithType, e.getMessage()); + throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e); + } ObjectNode validationResponse = validateConfig(tableConfig, typesToSkip); validationResponse.set("unrecognizedProperties", JsonUtils.objectToJsonNode(tableConfigAndUnrecognizedProperties.getRight())); + if (!deprecationWarnings.isEmpty()) { + validationResponse.set("deprecationWarnings", JsonUtils.objectToJsonNode(deprecationWarnings)); + } return validationResponse; } 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 index 87f27e0d38c1..9b1eefe5a752 100644 --- 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 @@ -18,6 +18,7 @@ */ package org.apache.pinot.controller.api.resources; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Preconditions; @@ -29,6 +30,7 @@ import io.swagger.annotations.SecurityDefinition; import io.swagger.annotations.SwaggerDefinition; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -56,6 +58,7 @@ import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.LogicalTableConfigUtils; +import org.apache.pinot.common.utils.config.TableConfigSerDeUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.access.AccessControl; import org.apache.pinot.controller.api.access.AccessControlFactory; @@ -79,6 +82,7 @@ import org.apache.pinot.spi.config.TableConfigs; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableConfigValidatorRegistry; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.LogicalTableConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.utils.JsonUtils; @@ -209,7 +213,7 @@ public ConfigSuccessResponse addConfig( tableConfigsAndUnrecognizedProps = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class); } catch (Exception e) { - throw new ControllerApplicationException(LOGGER, String.format("Invalid TableConfigs. %s", e.getMessage()), + throw new ControllerApplicationException(LOGGER, "Invalid TableConfigs. " + e.getMessage(), Response.Status.BAD_REQUEST, e); } TableConfigs tableConfigs = tableConfigsAndUnrecognizedProps.getLeft(); @@ -222,8 +226,22 @@ public ConfigSuccessResponse addConfig( Response.Status.BAD_REQUEST); } - validateConfig(tableConfigs, databaseName, typesToSkip); - tableConfigs.setTableName(rawTableName); + List deprecationWarnings; + try { + deprecationWarnings = validateNoDeprecatedConfigs(tableConfigs, JsonUtils.stringToJsonNode(tableConfigsStr), + databaseName); + validateConfig(tableConfigs, databaseName, typesToSkip); + tableConfigs.setTableName(rawTableName); + } catch (ControllerApplicationException e) { + throw e; + } catch (IOException | IllegalArgumentException | IllegalStateException e) { + throw new ControllerApplicationException(LOGGER, "Invalid TableConfigs. " + e.getMessage(), + Response.Status.BAD_REQUEST, e); + } catch (RuntimeException e) { + _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR, 1L); + LOGGER.warn("Failed to validate TableConfigs for create of table: {}", rawTableName, e); + throw e; + } TableConfig offlineTableConfig = tableConfigs.getOffline(); TableConfig realtimeTableConfig = tableConfigs.getRealtime(); @@ -272,7 +290,7 @@ public ConfigSuccessResponse addConfig( } return new ConfigSuccessResponse("TableConfigs " + rawTableName + " successfully added", - tableConfigsAndUnrecognizedProps.getRight()); + tableConfigsAndUnrecognizedProps.getRight(), deprecationWarnings); } catch (Exception e) { _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR, 1L); if (e instanceof InvalidTableConfigException) { @@ -378,25 +396,39 @@ public ConfigSuccessResponse updateConfig( tableName = DatabaseUtils.translateTableName(tableName, databaseName); Pair> tableConfigsAndUnrecognizedProps; TableConfigs tableConfigs; + List deprecationWarnings; + + // Existence check runs before deprecation validation so a PUT to a missing TableConfigs reports the actual + // problem (table does not exist) instead of a misleading "deprecated property" 400 from create-mode fallback. + 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); + } + try { tableConfigsAndUnrecognizedProps = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class); tableConfigs = tableConfigsAndUnrecognizedProps.getLeft(); + deprecationWarnings = validateNoDeprecatedConfigs(tableConfigs, JsonUtils.stringToJsonNode(tableConfigsStr), + databaseName); validateConfig(tableConfigs, databaseName, typesToSkip); Preconditions.checkState( DatabaseUtils.translateTableName(tableConfigs.getTableName(), databaseName).equals(tableName), "'tableName' in TableConfigs: %s must match provided tableName: %s", tableConfigs.getTableName(), tableName); tableConfigs.setTableName(tableName); - } catch (Exception e) { - throw new ControllerApplicationException(LOGGER, String.format("Invalid TableConfigs: %s. Reason: %s", tableName, - e.getMessage()), 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); + } catch (ControllerApplicationException e) { + throw e; + } catch (IOException | IllegalArgumentException | IllegalStateException e) { + // Narrowed to client-input exception types so transient ZK / Helix errors propagate as 5xx instead of being + // mis-reported as 400. + throw new ControllerApplicationException(LOGGER, "Invalid TableConfigs: " + tableName + ". " + e.getMessage(), + Response.Status.BAD_REQUEST, e); + } catch (RuntimeException e) { + _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR, 1L); + LOGGER.warn("Failed to validate TableConfigs for update of table: {}", tableName, e); + throw e; } TableConfig offlineTableConfig = tableConfigs.getOffline(); @@ -443,7 +475,7 @@ public ConfigSuccessResponse updateConfig( } return new ConfigSuccessResponse("TableConfigs updated for " + tableName, - tableConfigsAndUnrecognizedProps.getRight()); + tableConfigsAndUnrecognizedProps.getRight(), deprecationWarnings); } /** @@ -461,9 +493,11 @@ public String validateConfig(String tableConfigsStr, @QueryParam("validationTypesToSkip") @Nullable String typesToSkip, @Context HttpHeaders httpHeaders, @Context Request request) { Pair> tableConfigsAndUnrecognizedProps; + JsonNode tableConfigsJson; try { tableConfigsAndUnrecognizedProps = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class); + tableConfigsJson = JsonUtils.stringToJsonNode(tableConfigsStr); } catch (IOException e) { throw new ControllerApplicationException(LOGGER, String.format("Invalid TableConfigs json string: %s. Reason: %s", tableConfigsStr, e.getMessage()), @@ -471,9 +505,23 @@ public String validateConfig(String tableConfigsStr, } String databaseName = DatabaseUtils.extractDatabaseFromHttpHeaders(httpHeaders); TableConfigs tableConfigs = tableConfigsAndUnrecognizedProps.getLeft(); - validateConfig(tableConfigs, databaseName, typesToSkip); - String rawTableName = DatabaseUtils.translateTableName(tableConfigs.getTableName(), databaseName); - tableConfigs.setTableName(rawTableName); + String rawTableName; + List deprecationWarnings; + try { + deprecationWarnings = validateNoDeprecatedConfigs(tableConfigs, tableConfigsJson, databaseName); + validateConfig(tableConfigs, databaseName, typesToSkip); + rawTableName = DatabaseUtils.translateTableName(tableConfigs.getTableName(), databaseName); + tableConfigs.setTableName(rawTableName); + } catch (ControllerApplicationException e) { + // Preserve the upstream status (e.g. 500 from validateConfig); don't downgrade to 400. + throw e; + } catch (IllegalArgumentException | IllegalStateException e) { + throw new ControllerApplicationException(LOGGER, "Invalid TableConfigs. " + e.getMessage(), + Response.Status.BAD_REQUEST, e); + } catch (RuntimeException e) { + LOGGER.warn("Failed to validate TableConfigs for: {}", tableConfigs.getTableName(), e); + throw e; + } // validate permission String endpointUrl = request.getRequestURL().toString(); @@ -485,6 +533,9 @@ public String validateConfig(String tableConfigsStr, ObjectNode response = JsonUtils.objectToJsonNode(tableConfigs).deepCopy(); response.set("unrecognizedProperties", JsonUtils.objectToJsonNode(tableConfigsAndUnrecognizedProps.getRight())); + if (!deprecationWarnings.isEmpty()) { + response.set("deprecationWarnings", JsonUtils.objectToJsonNode(deprecationWarnings)); + } return response.toString(); } @@ -567,4 +618,57 @@ private void validateConfig(TableConfigs tableConfigs, String database, @Nullabl String.format("Invalid TableConfigs: %s. %s", rawTableName, e.getMessage()), Response.Status.BAD_REQUEST, e); } } + + /** + * Validates the offline and realtime sub-configs for deprecated properties. For each sub-type, if a stored + * table config already exists for {@code rawTableName} the validation runs in update mode (diffing against the + * stored config), otherwise it runs in create mode. Aggregated warnings from all sub-types are returned so the + * caller can include them in the response. + */ + private List validateNoDeprecatedConfigs(TableConfigs tableConfigs, JsonNode tableConfigsJson, + String database) { + if (tableConfigs.getTableName() == null) { + return List.of(); + } + String rawTableName = DatabaseUtils.translateTableName(tableConfigs.getTableName(), database); + List warnings = new ArrayList<>(); + JsonNode offlineTableConfigJson = subConfigJson(tableConfigsJson, TableType.OFFLINE); + if (offlineTableConfigJson != null) { + JsonNode existingJson = readStoredTableConfigJson(TableNameBuilder.OFFLINE.tableNameWithType(rawTableName)); + String prefix = TableType.OFFLINE.name().toLowerCase(); + if (existingJson == null) { + warnings.addAll(DeprecatedTableConfigValidationUtils.validateOnCreate(offlineTableConfigJson, prefix)); + } else { + warnings.addAll(DeprecatedTableConfigValidationUtils.validateOnUpdate(offlineTableConfigJson, existingJson, + prefix)); + } + } + JsonNode realtimeTableConfigJson = subConfigJson(tableConfigsJson, TableType.REALTIME); + if (realtimeTableConfigJson != null) { + JsonNode existingJson = readStoredTableConfigJson(TableNameBuilder.REALTIME.tableNameWithType(rawTableName)); + String prefix = TableType.REALTIME.name().toLowerCase(); + if (existingJson == null) { + warnings.addAll(DeprecatedTableConfigValidationUtils.validateOnCreate(realtimeTableConfigJson, prefix)); + } else { + warnings.addAll(DeprecatedTableConfigValidationUtils.validateOnUpdate(realtimeTableConfigJson, existingJson, + prefix)); + } + } + return warnings; + } + + @Nullable + private JsonNode readStoredTableConfigJson(String tableNameWithType) { + return TableConfigSerDeUtils.toRawJsonNode( + ZKMetadataProvider.getTableConfigZNRecord(_pinotHelixResourceManager.getPropertyStore(), tableNameWithType)); + } + + @Nullable + private static JsonNode subConfigJson(JsonNode tableConfigsJson, TableType type) { + // Mirror what Jackson populates on the deserialized POJO: TableConfigs uses @JsonProperty("offline") / + // @JsonProperty("realtime"), so any uppercase variant in the raw user JSON is already silently ignored at + // deserialization time. We deliberately use the same lookup here so the deprecation pass agrees with what + // ends up in the stored config. + return tableConfigsJson.get(type.name().toLowerCase()); + } } diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java index cb29afe3d209..5a5c045ae117 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java @@ -84,12 +84,13 @@ import static org.testng.Assert.fail; -/** - * Test for table creation - */ +/// Test for table creation. public class PinotTableRestletResourceTest extends ControllerTest { private static final String OFFLINE_TABLE_NAME = "testOfflineTable"; private static final String REALTIME_TABLE_NAME = "testRealtimeTable"; + private static final String STREAM_CONFIGS_DEPRECATION_WARNING = + "'tableIndexConfig.streamConfigs' is deprecated since 0.7.1. Use " + + "'ingestionConfig.streamIngestionConfig.streamConfigMaps' instead."; private final TableConfigBuilder _offlineBuilder = getOfflineTableBuilder(OFFLINE_TABLE_NAME); private final TableConfigBuilder _realtimeBuilder = getRealtimeTableBuilder(REALTIME_TABLE_NAME); @@ -134,6 +135,11 @@ private String createTable(String tableConfigJson) } } + private static String successfulRealtimeTableCreationResponse(String tableNameWithType) { + return "{\"unrecognizedProperties\":{},\"deprecationWarnings\":[\"" + STREAM_CONFIGS_DEPRECATION_WARNING + + "\"],\"status\":\"Table " + tableNameWithType + " successfully added\"}"; + } + private String updateTable(String tableName, String tableConfigJson) throws IOException { try { @@ -354,6 +360,86 @@ public void testCreateTable() createTable(realtimeTableConfigString); } + @Test + public void testReportsDeprecatedConfigOnCreateAndOnUpdateAsWarning() + throws Exception { + // Soft-launch policy: deprecated keys are reported via the deprecationWarnings field on the success + // response, not rejected with a 400. + TableConfig offlineTableConfig = _offlineBuilder.setTableName(OFFLINE_TABLE_NAME).build(); + ObjectNode createTableJson = (ObjectNode) JsonUtils.stringToJsonNode(offlineTableConfig.toJsonString()); + createTableJson.with("segmentsConfig").put("replicasPerPartition", "APPEND"); + + JsonNode createResponse = JsonUtils.stringToJsonNode(createTable(createTableJson.toString())); + JsonNode createWarnings = createResponse.path("deprecationWarnings"); + assertTrue(createWarnings.isArray() && createWarnings.size() > 0, + "expected deprecationWarnings on create response: " + createResponse); + assertTrue(createWarnings.toString().contains("segmentsConfig.replicasPerPartition"), + createWarnings.toString()); + + // Update that introduces a previously-absent deprecated property: also reported as a warning. + String rawTableName = "deprecated_update_table"; + DEFAULT_INSTANCE.addDummySchema(rawTableName); + TableConfig existingTableConfig = getOfflineTableBuilder(rawTableName).build(); + createTable(existingTableConfig.toJsonString()); + + ObjectNode updateTableJson = (ObjectNode) JsonUtils.stringToJsonNode(existingTableConfig.toJsonString()); + updateTableJson.with("segmentsConfig").put("replicasPerPartition", "APPEND"); + JsonNode updateResponse = JsonUtils.stringToJsonNode( + updateTable(existingTableConfig.getTableName(), updateTableJson.toString())); + JsonNode updateWarnings = updateResponse.path("deprecationWarnings"); + assertTrue(updateWarnings.isArray() && updateWarnings.size() > 0, + "expected deprecationWarnings on update response: " + updateResponse); + assertTrue(updateWarnings.toString().contains("segmentsConfig.replicasPerPartition"), + updateWarnings.toString()); + } + + @Test + public void testUpdateMissingTableReports404NotDeprecationError() + throws Exception { + // PUT to a missing table whose body contains a deprecated key must report 404 (table does not exist) rather + // than a misleading 400 about a deprecated property — the existence check runs before the deprecation diff. + String rawTableName = "missing_update_with_legacy_key"; + DEFAULT_INSTANCE.addDummySchema(rawTableName); + TableConfig tableConfig = getOfflineTableBuilder(rawTableName).build(); + ObjectNode updateTableJson = (ObjectNode) JsonUtils.stringToJsonNode(tableConfig.toJsonString()); + updateTableJson.with("segmentsConfig").put("replicasPerPartition", "APPEND"); + IOException e = expectThrows(IOException.class, + () -> updateTable(tableConfig.getTableName(), updateTableJson.toString())); + String message = e.getMessage() != null ? e.getMessage() : ""; + assertTrue(message.contains("\"code\":404") || message.contains("status: 404") + || message.contains("status code: 404"), + "Expected 404 status, got: " + message); + assertTrue(message.contains("does not exist"), + "Expected 'does not exist' to take precedence over deprecation, got: " + message); + assertTrue(!message.contains("Newly introduced deprecated"), + "404 must take precedence over deprecation diff, got: " + message); + } + + @Test + public void testUpdateAllowsUnchangedLegacyDeprecatedConfig() + throws Exception { + // Simulate a table whose stored config already contains a deprecated property (e.g. created on an older + // version). On update, re-submitting the same legacy value must NOT trigger validation; only newly introduced + // or value-changed deprecated paths are flagged. + String rawTableName = "deprecated_legacy_table"; + DEFAULT_INSTANCE.addDummySchema(rawTableName); + TableConfig existingTableConfig = getOfflineTableBuilder(rawTableName).build(); + createTable(existingTableConfig.toJsonString()); + + // Inject a deprecated value directly into ZK to mimic a pre-existing legacy config. + String tableNameWithType = TableNameBuilder.OFFLINE.tableNameWithType(rawTableName); + TableConfig stored = DEFAULT_INSTANCE.getHelixResourceManager().getTableConfig(tableNameWithType); + stored.getValidationConfig().setReplicasPerPartition("3"); + DEFAULT_INSTANCE.getHelixResourceManager().setExistingTableConfig(stored); + + // Re-submit the same legacy value. The diff against the stored config means this is treated as unchanged and + // should pass. + ObjectNode updateTableJson = (ObjectNode) JsonUtils.stringToJsonNode(stored.toJsonString()); + JsonNode response = JsonUtils.stringToJsonNode( + updateTable(existingTableConfig.getTableName(), updateTableJson.toString())); + assertEquals(response.get("status").asText(), "Table config updated for " + existingTableConfig.getTableName()); + } + @Test public void testTableCronSchedule() throws IOException { @@ -664,8 +750,7 @@ public void testDeleteTable() DEFAULT_INSTANCE.addDummySchema("table0"); TableConfig realtimeTableConfig = _realtimeBuilder.setTableName("table0").build(); String creationResponse = createTable(realtimeTableConfig.toJsonString()); - assertEquals(creationResponse, - "{\"unrecognizedProperties\":{},\"status\":\"Table table0_REALTIME successfully added\"}"); + assertEquals(creationResponse, successfulRealtimeTableCreationResponse("table0_REALTIME")); // Delete realtime table using REALTIME suffix. String deleteResponse = sendDeleteRequest( @@ -687,8 +772,7 @@ public void testDeleteTable() DEFAULT_INSTANCE.addDummySchema("table1"); TableConfig rtConfig1 = _realtimeBuilder.setTableName("table1").build(); creationResponse = createTable(rtConfig1.toJsonString()); - assertEquals(creationResponse, - "{\"unrecognizedProperties\":{},\"status\":\"Table table1_REALTIME successfully added\"}"); + assertEquals(creationResponse, successfulRealtimeTableCreationResponse("table1_REALTIME")); TableConfig offlineConfig1 = _offlineBuilder.setTableName("table1").build(); creationResponse = createTable(offlineConfig1.toJsonString()); @@ -703,8 +787,7 @@ public void testDeleteTable() DEFAULT_INSTANCE.addDummySchema("table2"); TableConfig rtConfig2 = _realtimeBuilder.setTableName("table2").build(); creationResponse = createTable(rtConfig2.toJsonString()); - assertEquals(creationResponse, - "{\"unrecognizedProperties\":{},\"status\":\"Table table2_REALTIME successfully added\"}"); + assertEquals(creationResponse, successfulRealtimeTableCreationResponse("table2_REALTIME")); TableConfig offlineConfig2 = _offlineBuilder.setTableName("table2").build(); creationResponse = createTable(offlineConfig2.toJsonString()); @@ -741,8 +824,7 @@ public void testDeleteTable() DEFAULT_INSTANCE.addDummySchema("table3"); TableConfig rtConfig3 = _realtimeBuilder.setTableName("table3").build(); creationResponse = createTable(rtConfig3.toJsonString()); - assertEquals(creationResponse, - "{\"unrecognizedProperties\":{},\"status\":\"Table table3_REALTIME successfully added\"}"); + assertEquals(creationResponse, successfulRealtimeTableCreationResponse("table3_REALTIME")); TableConfig offlineConfig3 = _offlineBuilder.setTableName("table3").build(); creationResponse = createTable(offlineConfig3.toJsonString()); @@ -811,8 +893,7 @@ public void testCheckTableState() DEFAULT_INSTANCE.addDummySchema(tableName); TableConfig realtimeTableConfig = _realtimeBuilder.setTableName(tableName).build(); String creationResponse = createTable(realtimeTableConfig.toJsonString()); - assertEquals(creationResponse, - "{\"unrecognizedProperties\":{},\"status\":\"Table testTable_REALTIME successfully added\"}"); + assertEquals(creationResponse, successfulRealtimeTableCreationResponse("testTable_REALTIME")); // Create a valid OFFLINE table TableConfig offlineTableConfig = _offlineBuilder.setTableName(tableName).build(); @@ -903,8 +984,9 @@ public void testUnrecognizedProperties() String creationResponse = createTable(jsonNode.toString()); assertEquals(creationResponse, - "{\"unrecognizedProperties\":{\"/illegalKey1\":1,\"/illegalKey2/illegalKey3\":2},\"status\":\"Table " - + "valid_table_name_extra_props_REALTIME successfully added\"}"); + "{\"unrecognizedProperties\":{\"/illegalKey1\":1,\"/illegalKey2/illegalKey3\":2},\"deprecationWarnings\":[\"" + + STREAM_CONFIGS_DEPRECATION_WARNING + "\"],\"status\":\"Table valid_table_name_extra_props_REALTIME " + + "successfully added\"}"); // update table with unrecognizedProperties String updateResponse = @@ -922,24 +1004,23 @@ public void testUnrecognizedProperties() "unrecognizedProperties\":{\"/illegalKey1\":1," + "\"/illegalKey2/illegalKey3\":2}}")); } - /** - * Validates the behavior of the system when creating or updating tables with invalid replication factors. - * This method tests both REALTIME and OFFLINE table configurations. - * - * The method performs the following steps: - * 1. Attempts to create a REALTIME table with an invalid replication factor of 5, which exceeds the number of - * available instances. The creation is expected to fail, and the test verifies that the exception message - * contains the expected error. - * 2. Attempts to create an OFFLINE table with the same invalid replication factor. The creation is expected to - * fail, and the test verifies that the exception message contains the expected error. - * 3. Creates REALTIME and OFFLINE tables with a valid replication factor of 1 to establish a baseline for further - * testing. These creations are expected to succeed. - * 4. Attempts to update the replication factor of the previously created REALTIME and OFFLINE tables to the - * invalid value of 5. These updates are expected to fail, and the test verifies that the appropriate error - * messages are returned. - * - * @throws Exception if any error occurs during the validation process - */ + /// Validates the behavior of the system when creating or updating tables with invalid replication factors. This + /// method tests both REALTIME and OFFLINE table configurations. + /// + /// The method performs the following steps: + /// + /// 1. Attempts to create a REALTIME table with an invalid replication factor of 5, which exceeds the number of + /// available instances. The creation is expected to fail, and the test verifies that the exception message + /// contains the expected error. + /// 2. Attempts to create an OFFLINE table with the same invalid replication factor. The creation is expected to + /// fail, and the test verifies that the exception message contains the expected error. + /// 3. Creates REALTIME and OFFLINE tables with a valid replication factor of 1 to establish a baseline for further + /// testing. These creations are expected to succeed. + /// 4. Attempts to update the replication factor of the previously created REALTIME and OFFLINE tables to the + /// invalid value of 5. These updates are expected to fail, and the test verifies that the appropriate error + /// messages are returned. + /// + /// @throws Exception if any error occurs during the validation process @Test public void validateInvalidTableReplication() throws Exception { @@ -955,20 +1036,19 @@ public void validateInvalidTableReplication() validateTableUpdateReplicationToInvalidValue(rawTableName, TableType.OFFLINE); } - /** - * Validates the behavior of the system when creating or updating tables with invalid replica group configurations. - * This method tests the REALTIME table configuration. - * - * The method performs the following steps: - * 1. Attempts to create a REALTIME table with an invalid replica group configuration. The creation is expected to - * fail, and the test verifies that the exception message contains the expected error. - * 2. Creates a new REALTIME table with a valid replica group configuration to establish a baseline for further - * testing. This creation is expected to succeed. - * 3. Attempts to update the replica group configuration of the previously created REALTIME table to an invalid - * value. The update is expected to fail, and the test verifies that the appropriate error message is returned. - * - * @throws Exception if any error occurs during the validation process - */ + /// Validates the behavior of the system when creating or updating tables with invalid replica group configurations. + /// This method tests the REALTIME table configuration. + /// + /// The method performs the following steps: + /// + /// 1. Attempts to create a REALTIME table with an invalid replica group configuration. The creation is expected to + /// fail, and the test verifies that the exception message contains the expected error. + /// 2. Creates a new REALTIME table with a valid replica group configuration to establish a baseline for further + /// testing. This creation is expected to succeed. + /// 3. Attempts to update the replica group configuration of the previously created REALTIME table to an invalid + /// value. The update is expected to fail, and the test verifies that the appropriate error message is returned. + /// + /// @throws Exception if any error occurs during the validation process @Test public void validateInvalidReplicaGroupConfig() throws Exception { @@ -1080,9 +1160,7 @@ public void testGetNonExistentTableConfig() assertTrue(msg.contains("Table nonExistentTable_REALTIME does not exist"), msg); } - /** - * Updating existing REALTIME table with invalid replication factor should throw exception. - */ + /// Updating existing REALTIME table with invalid replication factor should throw exception. private void validateTableUpdateReplicationToInvalidValue(String rawTableName, TableType tableType) { String tableNameWithType = TableNameBuilder.forType(tableType).tableNameWithType(rawTableName); TableConfig tableConfig = (tableType == TableType.REALTIME @@ -1114,9 +1192,7 @@ private void createTableWithValidReplication(String rawTableName, TableType tabl } } - /** - * When table is created with invalid replication factor, it should throw exception. - */ + /// When table is created with invalid replication factor, it should throw exception. private void validateTableCreationWithInvalidReplication(String rawTableName, TableType tableType) throws IOException { String tableNameWithType = TableNameBuilder.forType(tableType).tableNameWithType(rawTableName); @@ -1142,7 +1218,7 @@ private static InstanceAssignmentConfig getInstanceAssignmentConfig(String tag, constraints.add("constraints1"); InstanceConstraintConfig instanceConstraintConfig = new InstanceConstraintConfig(constraints); InstanceReplicaGroupPartitionConfig instanceReplicaGroupPartitionConfig = - new InstanceReplicaGroupPartitionConfig(true, 1, numReplicaGroups, numInstancesPerReplicaGroup, 1, 1, true, + new InstanceReplicaGroupPartitionConfig(true, 1, numReplicaGroups, numInstancesPerReplicaGroup, 1, 1, false, null); return new InstanceAssignmentConfig(instanceTagPoolConfig, instanceConstraintConfig, instanceReplicaGroupPartitionConfig, 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 index 75552355c608..4025bfe44af9 100644 --- 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 @@ -391,6 +391,71 @@ public void testCreateConfig() adminClient.getTableClient().deleteTableConfigs(tableName, null); } + @Test + public void testCreateConfigReportsDeprecatedTablePropertiesAsWarning() + throws Exception { + PinotAdminClient adminClient = getOrCreateAdminClient(); + String tableName = "testDeprecatedCreate"; + TableConfigs tableConfigs = + new TableConfigs(tableName, createDummySchema(tableName), createOfflineTableConfig(tableName), + createRealtimeTableConfig(tableName)); + ObjectNode tableConfigsJson = (ObjectNode) JsonUtils.stringToJsonNode(tableConfigs.toPrettyJsonString()); + ((ObjectNode) tableConfigsJson.get(TableType.REALTIME.name().toLowerCase()).get("segmentsConfig")) + .put("replicasPerPartition", "3"); + + try { + String response = + adminClient.getTableClient().createTableConfigs(tableConfigsJson.toPrettyString(), null, null); + Assert.assertTrue(response.contains("realtime.segmentsConfig.replicasPerPartition"), + "Expected deprecation warning in response: " + response); + } finally { + adminClient.getTableClient().deleteTableConfigs(tableName, null); + } + } + + @Test + public void testUpdateConfigReportsNewlyIntroducedDeprecatedPropertyAsWarning() + throws Exception { + PinotAdminClient adminClient = getOrCreateAdminClient(); + String tableName = "testDeprecatedUpdateNew"; + TableConfigs tableConfigs = + new TableConfigs(tableName, createDummySchema(tableName), createOfflineTableConfig(tableName), + createRealtimeTableConfig(tableName)); + adminClient.getTableClient().createTableConfigs(tableConfigs.toPrettyJsonString(), null, null); + try { + ObjectNode tableConfigsJson = (ObjectNode) JsonUtils.stringToJsonNode(tableConfigs.toPrettyJsonString()); + ((ObjectNode) tableConfigsJson.get(TableType.REALTIME.name().toLowerCase()).get("segmentsConfig")) + .put("replicasPerPartition", "3"); + String response = adminClient.getTableClient() + .updateTableConfigs(tableName, tableConfigsJson.toPrettyString(), null, false, false); + Assert.assertTrue(response.contains("realtime.segmentsConfig.replicasPerPartition"), + "Expected deprecation warning in update response: " + response); + } finally { + adminClient.getTableClient().deleteTableConfigs(tableName, null); + } + } + + @Test + public void testUpdateMissingTableConfigsReportsNotExistsNotDeprecation() + throws Exception { + PinotAdminClient adminClient = getOrCreateAdminClient(); + String tableName = "testMissingUpdate"; + TableConfigs tableConfigs = + new TableConfigs(tableName, createDummySchema(tableName), createOfflineTableConfig(tableName), + createRealtimeTableConfig(tableName)); + ObjectNode tableConfigsJson = (ObjectNode) JsonUtils.stringToJsonNode(tableConfigs.toPrettyJsonString()); + ((ObjectNode) tableConfigsJson.get(TableType.REALTIME.name().toLowerCase()).get("segmentsConfig")) + .put("replicasPerPartition", "3"); + try { + adminClient.getTableClient() + .updateTableConfigs(tableName, tableConfigsJson.toPrettyString(), null, false, false); + fail("Update of a non-existent TableConfigs should have failed"); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("does not exist"), + "Expected 'does not exist' error to take precedence over deprecation, but got: " + e.getMessage()); + } + } + @Test public void testListConfigs() throws Exception { diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponseTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponseTest.java new file mode 100644 index 000000000000..08fd49386576 --- /dev/null +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponseTest.java @@ -0,0 +1,62 @@ +/** + * 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.JsonNode; +import java.util.List; +import java.util.Map; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + + +/// Round-trip tests for the wire format of [ConfigSuccessResponse]. The DTO gained a `deprecationWarnings` field +/// that must (a) be elided from the JSON when empty so older clients that strict-parse the response continue to +/// see the original shape, and (b) appear when non-empty so callers can surface controller-side warnings. +public class ConfigSuccessResponseTest { + + @Test + public void testEmptyDeprecationWarningsAreOmittedFromJson() { + // The new `deprecationWarnings` field carries `@JsonInclude(NON_EMPTY)` on its getter so existing clients + // (including older Java client versions that strict-parse this DTO) keep observing the same JSON shape when no + // warnings fire. + ConfigSuccessResponse response = new ConfigSuccessResponse("ok", Map.of()); + JsonNode json = JsonUtils.objectToJsonNode(response); + assertFalse(json.has("deprecationWarnings"), "expected deprecationWarnings to be elided when empty: " + json); + } + + @Test + public void testNonEmptyDeprecationWarningsAppearInJson() { + ConfigSuccessResponse response = new ConfigSuccessResponse("ok", Map.of(), + List.of("'segmentsConfig.segmentPushType' is deprecated since 0.8.0. Use 'ingestionConfig...' instead.")); + JsonNode json = JsonUtils.objectToJsonNode(response); + assertTrue(json.has("deprecationWarnings")); + assertEquals(json.get("deprecationWarnings").size(), 1); + } + + @Test + public void testNullCollectionsNormalisedToEmpty() { + ConfigSuccessResponse response = new ConfigSuccessResponse("ok", null, null); + assertTrue(response.getUnrecognizedProperties().isEmpty()); + assertTrue(response.getDeprecationWarnings().isEmpty()); + } +} diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtilsTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtilsTest.java new file mode 100644 index 000000000000..d8e0cab580b7 --- /dev/null +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/resources/DeprecatedTableConfigValidationUtilsTest.java @@ -0,0 +1,370 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.controller.api.resources; + +import com.fasterxml.jackson.databind.JsonNode; +import java.util.List; +import org.apache.pinot.controller.api.resources.DeprecatedTableConfigValidationUtils.DeprecatedConfigRule; +import org.apache.pinot.controller.api.resources.DeprecatedTableConfigValidationUtils.Result; +import org.apache.pinot.controller.api.resources.DeprecatedTableConfigValidationUtils.Severity; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + + +public class DeprecatedTableConfigValidationUtilsTest { + + @Test + public void testReportsDeprecatedConfigsOnCreateAsWarnings() + throws Exception { + // Soft-launch policy: every parseable @DeprecatedConfig annotation is reported as a WARNING on create so + // legacy callers (TableConfigBuilder setters, integration test bases, downstream automations) keep working. + // A follow-up PR can promote to ERROR after the codebase migrates off these paths. + JsonNode tableConfigJson = JsonUtils.stringToJsonNode("{" + + "\"segmentsConfig\":{\"replicasPerPartition\":\"APPEND\",\"minimizeDataMovement\":false}," + + "\"fieldConfigList\":[{\"name\":\"c1\",\"indexType\":\"INVERTED\"}]," + + "\"instanceAssignmentConfigMap\":{\"CONSUMING\":{\"replicaGroupPartitionConfig\":" + + "{\"minimizeDataMovement\":false}}}}"); + + java.util.List warnings = + DeprecatedTableConfigValidationUtils.validateOnCreate(tableConfigJson, "realtime"); + assertTrue(warnings.stream().anyMatch(w -> w.contains("realtime.segmentsConfig.replicasPerPartition")), warnings + .toString()); + assertTrue(warnings.stream().anyMatch(w -> w.contains("realtime.segmentsConfig.minimizeDataMovement")), warnings + .toString()); + assertTrue(warnings.stream().anyMatch(w -> w.contains("realtime.fieldConfigList[0].indexType")), warnings + .toString()); + assertTrue(warnings.stream().anyMatch(w -> w.contains( + "realtime.instanceAssignmentConfigMap.CONSUMING.replicaGroupPartitionConfig.minimizeDataMovement")), + warnings.toString()); + } + + @Test + public void testCurrentVersionDeprecationIsWarningNotError() + throws Exception { + // tableIndexConfig.createInvertedIndexDuringSegmentGeneration is annotated with since=1.6.0, matching the + // current Pinot release line. It is reported as a warning (matches the soft-launch policy where every + // parseable `since` classifies as a warning). + JsonNode tableConfigJson = JsonUtils.stringToJsonNode( + "{\"tableIndexConfig\":{\"createInvertedIndexDuringSegmentGeneration\":false}}"); + + java.util.List warnings = DeprecatedTableConfigValidationUtils.validateOnCreate(tableConfigJson, null); + assertTrue(warnings.stream().anyMatch( + w -> w.contains("tableIndexConfig.createInvertedIndexDuringSegmentGeneration")), + "expected warning for current-version deprecation, got: " + warnings); + } + + @Test + public void testAllowsModernConfigsOnCreate() + throws Exception { + JsonNode tableConfigJson = JsonUtils.stringToJsonNode("{" + + "\"segmentsConfig\":{\"replication\":\"1\"}," + + "\"fieldConfigList\":[{\"name\":\"c1\",\"indexTypes\":[\"INVERTED\"]}]," + + "\"ingestionConfig\":{\"batchIngestionConfig\":{\"segmentIngestionType\":\"APPEND\"}," + + "\"streamIngestionConfig\":{\"streamConfigMaps\":[{\"streamType\":\"kafka\"}]}}}"); + + DeprecatedTableConfigValidationUtils.validateOnCreate(tableConfigJson, null); + } + + @Test + public void testUpdateAllowsUnchangedLegacyDeprecatedValue() + throws Exception { + // Legacy config is already stored with replicasPerPartition=APPEND. Re-submitting the same value must NOT trigger + // an error: the diff sees the value as unchanged. + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replicasPerPartition\":\"APPEND\"}}"); + JsonNode newJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replicasPerPartition\":\"APPEND\"}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertFalse(result.hasErrors(), "errors=" + result.getErrors()); + assertFalse(result.hasWarnings(), "warnings=" + result.getWarnings()); + } + + @Test + public void testUpdateReportsValueChangeOnDeprecatedFieldAsWarning() + throws Exception { + // Legacy config had replicasPerPartition=APPEND. The update changes it to REFRESH — the diff treats this as + // a new write to a deprecated key and reports a warning under the soft-launch policy. + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replicasPerPartition\":\"APPEND\"}}"); + JsonNode newJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replicasPerPartition\":\"REFRESH\"}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings(), "expected warning on changed deprecated value"); + assertTrue(result.getWarnings().get(0).contains("segmentsConfig.replicasPerPartition"), + result.getWarnings().toString()); + } + + @Test + public void testUpdateAllowsReSubmittedDefaultValueWhenAbsentFromStored() + throws Exception { + // Many deprecated booleans carry @JsonInclude(NON_DEFAULT). A previous create with `enableSnapshot: false` + // (the type default) is stripped at ZK write time, so the stored config has no `enableSnapshot` key. On PUT, + // the diff sees the path as missing in the stored config but present in the new submission with the type + // default — the validator must treat this as a no-op so users can re-submit cached configs unchanged. + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{}}"); + JsonNode newJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{\"enableSnapshot\":false}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertFalse(result.hasErrors(), "errors=" + result.getErrors()); + assertFalse(result.hasWarnings(), "warnings=" + result.getWarnings()); + } + + @Test + public void testUpdateAllowsLegacyIndexTypeWhenStoredAsSingletonIndexTypes() + throws Exception { + // FieldConfig#getIndexType is @JsonIgnore, so a create with legacy `indexType` is stored as singleton + // `indexTypes`. Re-submitting the same legacy payload on update must remain idempotent. + JsonNode oldJson = + JsonUtils.stringToJsonNode("{\"fieldConfigList\":[{\"name\":\"c1\",\"indexTypes\":[\"INVERTED\"]}]}"); + JsonNode newJson = + JsonUtils.stringToJsonNode("{\"fieldConfigList\":[{\"name\":\"c1\",\"indexType\":\"INVERTED\"}]}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertFalse(result.hasErrors(), "errors=" + result.getErrors()); + assertFalse(result.hasWarnings(), "warnings=" + result.getWarnings()); + } + + @Test + public void testUpdateReportsLegacyIndexTypeWhenStoredIndexTypesAreNotEquivalent() + throws Exception { + // The compatibility shortcut only applies to a singleton `indexTypes` value that exactly matches `indexType`. + // Multi-index stored configs cannot be represented by legacy `indexType`, so the deprecated write is reported. + JsonNode oldJson = JsonUtils.stringToJsonNode( + "{\"fieldConfigList\":[{\"name\":\"c1\",\"indexTypes\":[\"INVERTED\",\"RANGE\"]}]}"); + JsonNode newJson = + JsonUtils.stringToJsonNode("{\"fieldConfigList\":[{\"name\":\"c1\",\"indexType\":\"INVERTED\"}]}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings(), "expected warning on non-equivalent legacy indexType"); + assertTrue(result.getWarnings().get(0).contains("fieldConfigList[0].indexType"), + result.getWarnings().toString()); + } + + @Test + public void testUpdateTreatsExplicitJsonNullStoredValueAsPresent() + throws Exception { + // The stored JSON has an explicit `null` value for a deprecated path. The new submission flips it to a + // non-default `true`. Both differ, so the diff must report it (the default-skip applies only when the path + // was *absent* in the stored config, not when it was present-but-null). + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{\"enableSnapshot\":null}}"); + JsonNode newJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{\"enableSnapshot\":true}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings(), "expected warning on flip from null to true"); + assertTrue(result.getWarnings().get(0).contains("upsertConfig.enableSnapshot"), + result.getWarnings().toString()); + } + + @Test + public void testUpdateReportsDeliberateFlipFromNonDefaultToDefaultAsWarning() + throws Exception { + // The default-skip optimisation must not silently swallow a deliberate value flip on an existing field. + // Stored config has `enableSnapshot: true`; user submits `enableSnapshot: false` — this is a value change on + // a deprecated path and is reported as a warning under the soft-launch policy. + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{\"enableSnapshot\":true}}"); + JsonNode newJson = JsonUtils.stringToJsonNode("{\"upsertConfig\":{\"enableSnapshot\":false}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings(), "expected warning on deliberate flip true → false"); + assertTrue(result.getWarnings().get(0).contains("upsertConfig.enableSnapshot"), + result.getWarnings().toString()); + } + + @Test + public void testUpdateReportsEmptyStringValueForNullDefaultField() + throws Exception { + // String-returning deprecated getters initialise to null (the Java default), not "". A user submitting + // "replicasPerPartition":"" on update — when the stored config lacks the key — is supplying a real value that + // Jackson would NOT elide under NON_DEFAULT, and is flagged as a warning. Locks the textual branch of + // isJacksonDefault returning false (rather than treating empty string as default). + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replication\":\"1\"}}"); + JsonNode newJson = JsonUtils.stringToJsonNode( + "{\"segmentsConfig\":{\"replication\":\"1\",\"replicasPerPartition\":\"\"}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings(), "expected warning on empty-string value for deprecated path"); + assertTrue(result.getWarnings().get(0).contains("segmentsConfig.replicasPerPartition"), + result.getWarnings().toString()); + } + + @Test + public void testUpdateReportsNewlyIntroducedDeprecatedField() + throws Exception { + // Legacy config did not contain the deprecated field. Adding it on update fires a warning under the + // soft-launch policy (parseable since classifies as warning). + JsonNode oldJson = JsonUtils.stringToJsonNode("{\"segmentsConfig\":{\"replication\":\"1\"}}"); + JsonNode newJson = JsonUtils.stringToJsonNode( + "{\"segmentsConfig\":{\"replication\":\"1\",\"replicasPerPartition\":\"APPEND\"}}"); + + Result result = DeprecatedTableConfigValidationUtils.validate(newJson, oldJson, null); + assertTrue(result.hasWarnings()); + assertTrue(result.getWarnings().get(0).contains("segmentsConfig.replicasPerPartition")); + } + + @Test + public void testMajorMinorParsing() { + assertEquals(DeprecatedTableConfigValidationUtils.majorMinor("1.6.0"), "1.6"); + assertEquals(DeprecatedTableConfigValidationUtils.majorMinor("1.6.0-SNAPSHOT"), "1.6"); + assertEquals(DeprecatedTableConfigValidationUtils.majorMinor("1.6"), "1.6"); + assertEquals(DeprecatedTableConfigValidationUtils.majorMinor("12.345.6"), "12.345"); + assertNull(DeprecatedTableConfigValidationUtils.majorMinor(null)); + assertNull(DeprecatedTableConfigValidationUtils.majorMinor("garbage")); + assertNull(DeprecatedTableConfigValidationUtils.majorMinor("1")); + } + + @Test + public void testSeverityClassification() { + // An unparseable annotation `since` reflects a code-side bug and classifies as ERROR. + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("garbage"), Severity.ERROR); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity(""), Severity.ERROR); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("1"), Severity.ERROR); + } + + @Test + public void testSeverityIsWarningForAllParseableSinceUnderSoftLaunch() { + // Soft-launch policy: every parseable `since` returns WARNING, regardless of the running Pinot version, so + // existing callers that already use deprecated keys keep working. ERROR is reserved for unparseable values + // (which reflect a code-side annotation bug, not user-supplied data). + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("1.6.0", null), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("1.5.0", null), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("0.3.0", null), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("1.6.0", "1.6"), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("1.5.0", "1.6"), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("0.3.0", "1.6"), Severity.WARNING); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("garbage", null), Severity.ERROR); + assertEquals(DeprecatedTableConfigValidationUtils.classifySeverity("garbage", "1.6"), Severity.ERROR); + } + + @Test + public void testRulesDiscoveredFromAnnotations() { + // Sanity check that the annotation walk picks up the expected paths from TableConfig. + boolean foundIndexType = DeprecatedTableConfigValidationUtils.rulesForTesting().stream() + .anyMatch(rule -> rule.pathSegments().equals(List.of("fieldConfigList", "*", "indexType"))); + assertTrue(foundIndexType, "expected fieldConfigList[*].indexType rule"); + + boolean foundNestedMinimize = DeprecatedTableConfigValidationUtils.rulesForTesting().stream() + .anyMatch(rule -> rule.pathSegments().equals(List.of( + "instanceAssignmentConfigMap", "*", "replicaGroupPartitionConfig", "minimizeDataMovement"))); + assertTrue(foundNestedMinimize, "expected nested map-wildcard rule"); + + boolean foundStreamConfigs = DeprecatedTableConfigValidationUtils.rulesForTesting().stream() + .anyMatch(rule -> rule.pathSegments().equals(List.of("tableIndexConfig", "streamConfigs"))); + assertTrue(foundStreamConfigs, "expected tableIndexConfig.streamConfigs rule"); + + boolean foundSegmentPushType = DeprecatedTableConfigValidationUtils.rulesForTesting().stream() + .anyMatch(rule -> rule.pathSegments().equals(List.of("segmentsConfig", "segmentPushType"))); + assertTrue(foundSegmentPushType, "expected segmentsConfig.segmentPushType rule"); + + boolean foundSegmentPushFrequency = DeprecatedTableConfigValidationUtils.rulesForTesting().stream() + .anyMatch(rule -> rule.pathSegments().equals(List.of("segmentsConfig", "segmentPushFrequency"))); + assertTrue(foundSegmentPushFrequency, "expected segmentsConfig.segmentPushFrequency rule"); + } + + /// Provides every rule discovered by the annotation walk so the parameterized test below covers the full set + /// 1:1. If a new {@link org.apache.pinot.spi.config.DeprecatedConfig @DeprecatedConfig} is added on a getter, this + /// test automatically exercises it without needing a new test case. + @DataProvider(name = "allRules") + public Object[][] allRules() { + List rules = DeprecatedTableConfigValidationUtils.rulesForTesting(); + Object[][] data = new Object[rules.size()][]; + for (int i = 0; i < rules.size(); i++) { + data[i] = new Object[] {rules.get(i)}; + } + return data; + } + + @Test(dataProvider = "allRules") + public void testEveryRuleFiresOnSyntheticInputAsArrayWildcard(DeprecatedConfigRule rule) + throws Exception { + runEveryRuleCase(rule, /* arrayWildcard */ true); + } + + @Test(dataProvider = "allRules") + public void testEveryRuleFiresOnSyntheticInputAsMapWildcard(DeprecatedConfigRule rule) + throws Exception { + runEveryRuleCase(rule, /* arrayWildcard */ false); + } + + /// Builds a JSON tree containing the deprecated path and asserts the rule fires. The {@code arrayWildcard} flag + /// controls how `*` segments are realised: as `[ {...} ]` (array branch) or `{"x":...}` (object branch). Running + /// both shapes ensures `collectMatches` is exercised on both wildcard branches for every rule. + private static void runEveryRuleCase(DeprecatedConfigRule rule, boolean arrayWildcard) + throws Exception { + String synthetic = synthesizeJsonForPath(rule.pathSegments(), arrayWildcard); + String expectedPath = expectedPathInMessage(rule.pathSegments(), arrayWildcard); + + Result result = DeprecatedTableConfigValidationUtils.validate(JsonUtils.stringToJsonNode(synthetic), null, null); + if (rule.severity() == Severity.ERROR) { + assertTrue(result.getErrors().stream().anyMatch(m -> m.contains(expectedPath)), + "expected error containing '" + expectedPath + "', got errors=" + result.getErrors() + ", warnings=" + + result.getWarnings()); + } else { + assertTrue(result.getWarnings().stream().anyMatch(m -> m.contains(expectedPath)), + "expected warning containing '" + expectedPath + "', got warnings=" + result.getWarnings() + ", errors=" + + result.getErrors()); + } + } + + private static String synthesizeJsonForPath(List path, boolean arrayWildcard) { + StringBuilder open = new StringBuilder(); + StringBuilder close = new StringBuilder(); + for (String segment : path.subList(0, path.size() - 1)) { + if ("*".equals(segment)) { + if (arrayWildcard) { + open.append("["); + close.insert(0, "]"); + } else { + open.append("{\"x\":"); + close.insert(0, "}"); + } + } else { + open.append("{\"").append(segment).append("\":"); + close.insert(0, "}"); + } + } + String leaf = path.get(path.size() - 1); + if ("*".equals(leaf)) { + open.append(arrayWildcard ? "[\"v\"]" : "{\"x\":\"v\"}"); + } else { + open.append("{\"").append(leaf).append("\":\"v\"}"); + } + return open.append(close).toString(); + } + + private static String expectedPathInMessage(List path, boolean arrayWildcard) { + StringBuilder sb = new StringBuilder(); + for (String segment : path) { + if ("*".equals(segment) && arrayWildcard) { + // The walker emits `[]` (no preceding `.`) for array entries. + sb.append("[0]"); + } else { + String key = "*".equals(segment) ? "x" : segment; + if (sb.length() > 0) { + sb.append('.'); + } + sb.append(key); + } + } + return sb.toString(); + } +} diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java index affeee46be2c..76673e1d92f7 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java @@ -57,6 +57,8 @@ import org.apache.pinot.spi.config.table.assignment.InstancePartitionsType; import org.apache.pinot.spi.config.table.assignment.InstanceReplicaGroupPartitionConfig; import org.apache.pinot.spi.config.table.assignment.InstanceTagPoolConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.config.tenant.Tenant; import org.apache.pinot.spi.config.tenant.TenantRole; import org.apache.pinot.spi.stream.LongMsgOffset; @@ -708,11 +710,7 @@ public void testRebalancePeerDownloadDataLoss() preChecker.init(_helixResourceManager, executorService, 1); TableRebalancer tableRebalancer = new TableRebalancer(_helixManager, null, null, preChecker, _tableSizeReader, null); - TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) - .setNumReplicas(1) - .setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()) - .build(); + TableConfig tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(1).build(); // Create the table addDummySchema(RAW_TABLE_NAME); _helixResourceManager.addTable(tableConfig); @@ -885,7 +883,7 @@ public void testRebalanceWithImplicitRealtimeTablePartitionSelectorAndMinimizeDa new InstanceTagPoolConfig(TagNameUtils.getRealtimeTagForTenant(null), false, 0, null), null, replicaGroupPartitionConfig, InstanceAssignmentConfig.PartitionSelector.IMPLICIT_REALTIME_TABLE_PARTITION_SELECTOR.name(), true); - TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) + TableConfig tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME) .setNumReplicas(numReplicas) .setRoutingConfig( new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE, false)) @@ -988,9 +986,8 @@ public void testRebalanceWithImplicitRealtimeTablePartitionSelectorAndMinimizeDa // "Repartition" and add two new partitions int newNumPartitions = 20; - tableConfig.getIndexingConfig() - .setStreamConfigs( - FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs(newNumPartitions).getStreamConfigsMap()); + setStreamIngestionConfig(tableConfig, + FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs(newNumPartitions).getStreamConfigsMap()); _helixResourceManager.updateTableConfig(tableConfig); // Add segments for the new partitions @@ -1063,6 +1060,24 @@ private int getNumSegmentsMoved(Map> oldSegmentAssig return numSegmentsMoved; } + private static TableConfigBuilder getRealtimeTableConfigBuilder(String tableName) { + return new TableConfigBuilder(TableType.REALTIME).setTableName(tableName) + .setTimeColumnName("timeColumn") + .setTimeType("DAYS") + .setRetentionTimeUnit("DAYS") + .setRetentionTimeValue("5") + .setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()); + } + + private static void setStreamIngestionConfig(TableConfig tableConfig, Map streamConfigs) { + IngestionConfig ingestionConfig = tableConfig.getIngestionConfig(); + if (ingestionConfig == null) { + ingestionConfig = new IngestionConfig(); + tableConfig.setIngestionConfig(ingestionConfig); + } + ingestionConfig.setStreamIngestionConfig(new StreamIngestionConfig(Collections.singletonList(streamConfigs))); + } + @Test public void testRebalanceBatchSizePerServerErrors() throws Exception { @@ -1233,11 +1248,7 @@ public void testRebalancePreCheckerRebalanceConfig() preChecker.init(_helixResourceManager, executorService, 0.5); TableRebalancer tableRebalancer = new TableRebalancer(_helixManager, null, null, preChecker, _tableSizeReader, null); - TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) - .setNumReplicas(2) - .setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()) - .build(); + TableConfig tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(2).build(); // Create the table addDummySchema(RAW_TABLE_NAME); @@ -1279,7 +1290,7 @@ public void testRebalancePreCheckerRebalanceConfig() rebalanceConfig.setUpdateTargetTier(false); rebalanceConfig.setBootstrap(false); rebalanceConfig.setBestEfforts(false); - tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) + tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME) .setTierConfigList(Collections.singletonList( new TierConfig("dummyTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "7d", null, TierFactory.PINOT_SERVER_STORAGE_TYPE, @@ -1293,8 +1304,7 @@ public void testRebalancePreCheckerRebalanceConfig() assertEquals(preCheckerResult.getMessage(), "updateTargetTier should be enabled when tier configs are present"); // trigger downtime warning - TableConfig newTableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(3).build(); + TableConfig newTableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(3).build(); rebalanceConfig.setBootstrap(false); rebalanceConfig.setBestEfforts(false); @@ -1309,7 +1319,7 @@ public void testRebalancePreCheckerRebalanceConfig() "Number of replicas (3) is greater than 1, downtime is not recommended."); // no downtime warning with 1 replica - newTableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(1).build(); + newTableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(1).build(); rebalanceConfig.setDowntime(true); rebalanceResult = tableRebalancer.rebalance(newTableConfig, rebalanceConfig, null); @@ -1330,7 +1340,7 @@ public void testRebalancePreCheckerRebalanceConfig() "Replication of the table is 1, which is not recommended for peer-download enabled tables as it may " + "cause data loss during rebalance"); - newTableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(3).build(); + newTableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(3).build(); newTableConfig.getValidationConfig().setPeerSegmentDownloadScheme("https"); rebalanceResult = tableRebalancer.rebalance(newTableConfig, rebalanceConfig, null); @@ -1379,7 +1389,7 @@ public void testRebalancePreCheckerRebalanceConfig() addFakeServerInstanceToAutoJoinHelixCluster(instanceId, true); // change num replicas from 3 to 4 - newTableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(4).build(); + newTableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(4).build(); // now the new server (the 4th server) should expect to be added all the existing segments (including consuming) rebalanceResult = tableRebalancer.rebalance(newTableConfig, rebalanceConfig, null); @@ -2175,11 +2185,7 @@ public void testRebalanceConsumingSegmentSummary() ConsumingSegmentInfoReader mockConsumingSegmentInfoReader = Mockito.mock(ConsumingSegmentInfoReader.class); TableRebalancer tableRebalancerOriginal = new TableRebalancer(_helixManager, null, null, null, _tableSizeReader, null); - TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) - .setNumReplicas(numReplica) - .setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()) - .build(); + TableConfig tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(numReplica).build(); // Create the table addDummySchema(RAW_TABLE_NAME); @@ -2286,11 +2292,7 @@ public void testRebalanceConsumingSegmentSummaryFailure() TableRebalancer tableRebalancerOriginal = new TableRebalancer(_helixManager, null, null, null, _tableSizeReader, null); - TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME) - .setNumReplicas(numReplica) - .setStreamConfigs(FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap()) - .build(); + TableConfig tableConfig = getRealtimeTableConfigBuilder(RAW_TABLE_NAME).setNumReplicas(numReplica).build(); // Create the table addDummySchema(RAW_TABLE_NAME); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index 367d71112dbe..0e26daac954e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -62,6 +62,7 @@ import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -136,11 +137,11 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert tableConfig.setUpsertConfig(null); } if (maxRows != null) { - tableConfig.getIndexingConfig().getStreamConfigs() + IngestionConfigUtils.getFirstStreamConfigMap(tableConfig) .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, maxRows); } if (maxDuration != null) { - tableConfig.getIndexingConfig().getStreamConfigs() + IngestionConfigUtils.getFirstStreamConfigMap(tableConfig) .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME, maxDuration); } if (tableConfig.getIngestionConfig() == null) { @@ -302,7 +303,7 @@ public void testCommitAfterCatchup() public void testCommitAfterCatchupWithPeriodOffset() throws Exception { TableConfig tableConfig = createTableConfig(); - tableConfig.getIndexingConfig().getStreamConfigs().put( + IngestionConfigUtils.getFirstStreamConfigMap(tableConfig).put( StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), "2d"); FakeRealtimeSegmentDataManager segmentDataManager = @@ -349,7 +350,7 @@ public void testCommitAfterCatchupWithPeriodOffset() public void testCommitAfterCatchupWithTimestampOffset() throws Exception { TableConfig tableConfig = createTableConfig(); - tableConfig.getIndexingConfig().getStreamConfigs().put( + IngestionConfigUtils.getFirstStreamConfigMap(tableConfig).put( StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), Instant.now().toString()); FakeRealtimeSegmentDataManager segmentDataManager = diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index 8d93d5fea1d5..1dab8e8c7181 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -40,6 +40,7 @@ import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; import org.slf4j.Logger; @@ -146,7 +147,7 @@ protected void setupPauselessTable() IngestionConfig ingestionConfig = new IngestionConfig(); ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + new StreamIngestionConfig(List.of(IngestionConfigUtils.getFirstStreamConfigMap(tableConfig)))); ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); tableConfig.getIndexingConfig().setStreamConfigs(null); tableConfig.setIngestionConfig(ingestionConfig); @@ -230,9 +231,7 @@ protected void runValidationAndVerify() _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); } - /** - * Basic test to verify segment assignment and metadata without any failures - */ + /// Basic test to verify segment assignment and metadata without any failures. protected void testBasicSegmentAssignment() { String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/logicaltable/LogicalTableWithTwoRealtimeTableIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/logicaltable/LogicalTableWithTwoRealtimeTableIntegrationTest.java index f09ebdd35e49..88b03ad4b688 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/logicaltable/LogicalTableWithTwoRealtimeTableIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/logicaltable/LogicalTableWithTwoRealtimeTableIntegrationTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.exception.QueryErrorCode; +import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; import org.testng.Assert; @@ -171,7 +172,7 @@ protected TableConfig createRealtimeTableConfig(File sampleAvroFile) { String tableName = REALTIME_TABLE_NAMES.get(tableIndex); Integer partitionId = REALTIME_TABLE_PARTITIONS.get(tableName); tableConfig.setTableName(tableName); - Map streamConfigs = new HashMap<>(tableConfig.getIndexingConfig().getStreamConfigs()); + Map streamConfigs = new HashMap<>(IngestionConfigUtils.getFirstStreamConfigMap(tableConfig)); streamConfigs.put("stream.kafka.partition.ids", String.valueOf(partitionId)); tableConfig.getIndexingConfig().setStreamConfigs(null); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/ingestion/KafkaIncreaseDecreasePartitionsIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/ingestion/KafkaIncreaseDecreasePartitionsIntegrationTest.java index 021eaca9718d..c7082232399d 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/ingestion/KafkaIncreaseDecreasePartitionsIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/ingestion/KafkaIncreaseDecreasePartitionsIntegrationTest.java @@ -18,12 +18,8 @@ */ package org.apache.pinot.integration.tests.realtime.ingestion; -import java.io.IOException; import org.apache.pinot.integration.tests.BaseRealtimeClusterIntegrationTest; -import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.utils.JsonUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.Test; @@ -34,34 +30,26 @@ public class KafkaIncreaseDecreasePartitionsIntegrationTest extends BaseRealtimeClusterIntegrationTest { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaIncreaseDecreasePartitionsIntegrationTest.class); - private static final String KAFKA_TOPIC = "meetup"; private static final int NUM_PARTITIONS = 1; - String createTable() - throws IOException { - Schema schema = createSchema("simpleMeetup_schema.json"); - addSchema(schema); - TableConfig tableConfig = JsonUtils.inputStreamToObject( - getClass().getClassLoader().getResourceAsStream("simpleMeetup_realtime_table_config.json"), TableConfig.class); - addTableConfig(tableConfig); - return tableConfig.getTableName(); + @Override + protected int getNumKafkaPartitions() { + return NUM_PARTITIONS + 2; } @Test public void testDecreasePartitions() throws Exception { LOGGER.info("Starting testDecreasePartitions"); - LOGGER.info("Creating Kafka topic with {} partitions", NUM_PARTITIONS + 2); - createKafkaTopic(KAFKA_TOPIC, NUM_PARTITIONS + 2); - String tableName = createTable(); + String tableName = getTableName(); waitForNumSegmentsInDesiredStateInEV(tableName, CONSUMING, NUM_PARTITIONS + 2, TableType.REALTIME); pauseTable(tableName); LOGGER.info("Deleting Kafka topic"); - deleteKafkaTopic(KAFKA_TOPIC); + deleteKafkaTopic(getKafkaTopic()); LOGGER.info("Creating Kafka topic with {} partitions", NUM_PARTITIONS); - createKafkaTopic(KAFKA_TOPIC, NUM_PARTITIONS); + createKafkaTopic(getKafkaTopic(), NUM_PARTITIONS); resumeTable(tableName); waitForNumSegmentsInDesiredStateInEV(tableName, CONSUMING, NUM_PARTITIONS, TableType.REALTIME); diff --git a/pinot-integration-tests/src/test/resources/chaos-monkey-create-table.json b/pinot-integration-tests/src/test/resources/chaos-monkey-create-table.json index e4bdf46e9611..a40612b38bdd 100644 --- a/pinot-integration-tests/src/test/resources/chaos-monkey-create-table.json +++ b/pinot-integration-tests/src/test/resources/chaos-monkey-create-table.json @@ -2,9 +2,13 @@ "tableName": "myTable", "tableType": "OFFLINE", "segmentsConfig": { - "segmentPushType": "APPEND", "replication": "3" }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tenants": { }, "tableIndexConfig": { @@ -13,5 +17,10 @@ "metadata": { "customConfigs": { } + }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + } } } diff --git a/pinot-integration-tests/src/test/resources/dimDayOfWeek_config.json b/pinot-integration-tests/src/test/resources/dimDayOfWeek_config.json index 20f4f0d303b3..a3083135f5ef 100644 --- a/pinot-integration-tests/src/test/resources/dimDayOfWeek_config.json +++ b/pinot-integration-tests/src/test/resources/dimDayOfWeek_config.json @@ -3,7 +3,6 @@ "tableType": "OFFLINE", "isDimTable": true, "segmentsConfig": { - "segmentPushType": "REFRESH", "replication": "1" }, "tenants": { @@ -14,5 +13,10 @@ "metadata": { "customConfigs": { } + }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "REFRESH" + } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java index 4ddde893b71b..6d78ec2f6887 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java @@ -3593,11 +3593,10 @@ public void testConvertFromLegacyTableConfig() { String expectedPushType = "APPEND"; Map expectedStreamConfigsMap = getTestStreamConfigs(); - TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME) - .setSegmentPushFrequency(expectedPushFrequency) - .setSegmentPushType(expectedPushType) - .setStreamConfigs(expectedStreamConfigsMap) - .build(); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME).build(); + tableConfig.getValidationConfig().setSegmentPushFrequency(expectedPushFrequency); + tableConfig.getValidationConfig().setSegmentPushType(expectedPushType); + tableConfig.getIndexingConfig().setStreamConfigs(expectedStreamConfigsMap); // Before conversion, the ingestion config should be null. assertNull(tableConfig.getIngestionConfig()); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/DeprecatedConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/DeprecatedConfig.java new file mode 100644 index 000000000000..1b79a1ed4365 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/DeprecatedConfig.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.spi.config; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + + +/// Marks a config getter as a deprecated JSON property in `TableConfig` (or any nested config bean). +/// +/// **Metadata-only**: this annotation has no runtime effect on serialization or deserialization. It is consumed by +/// the controller's `DeprecatedTableConfigValidationUtils` to gate creation and update of table configs. Severity +/// is determined by comparing [#since()] against the running Pinot version: properties deprecated in the current +/// major.minor release are reported as **warnings**, while properties deprecated in any earlier release are +/// reported as **errors**. +/// +/// Place this annotation on the Jackson-visible getter (the one that drives JSON property naming) so the discovery +/// walk picks up the correct serialized name. +@Target(ElementType.METHOD) +@Retention(RetentionPolicy.RUNTIME) +public @interface DeprecatedConfig { + + /// Human-readable replacement guidance, surfaced verbatim in the user-facing error/warning. Should describe what + /// the user should set instead, e.g. `"Use 'segmentsConfig.replication' instead."`. + String replacement(); + + /// The Pinot release in which this property was deprecated, e.g. `"1.6.0"`. Used to decide whether a violation is + /// reported as a warning (current major.minor) or an error (older). Format: `MAJOR.MINOR` or `MAJOR.MINOR.PATCH` + /// with an optional `-SNAPSHOT` qualifier; only the leading major.minor pair is compared. + String since(); +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/DedupConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/DedupConfig.java index 3eb03fece31c..aec837037bc3 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/DedupConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/DedupConfig.java @@ -18,11 +18,13 @@ */ package org.apache.pinot.spi.config.table; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.google.common.base.Preconditions; import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; import org.apache.pinot.spi.utils.Enablement; @@ -165,6 +167,8 @@ public void setMetadataManagerConfigs(@Nullable Map metadataMana } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig(replacement = "Use 'dedupConfig.preload' instead.", since = "1.4.0") public boolean isEnablePreload() { return _enablePreload; } @@ -178,6 +182,10 @@ public void setEnablePreload(boolean enablePreload) { } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig( + replacement = "Use 'ingestionConfig.streamIngestionConfig.parallelSegmentConsumptionPolicy' instead.", + since = "1.4.0") public boolean isAllowDedupConsumptionDuringCommit() { return _allowDedupConsumptionDuringCommit; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java index 064526695d45..0c425a547e28 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java @@ -19,6 +19,7 @@ package org.apache.pinot.spi.config.table; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; @@ -28,6 +29,7 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; public class FieldConfig extends BaseJsonConfig { @@ -181,6 +183,8 @@ public EncodingType getEncodingType() { @Nullable @Deprecated + @JsonIgnore + @DeprecatedConfig(replacement = "Use 'fieldConfigList[].indexTypes' instead.", since = "0.9.0") public IndexType getIndexType() { return !_indexTypes.isEmpty() ? _indexTypes.get(0) : null; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index 6a927ded9551..db7ed7cd84b3 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -19,6 +19,7 @@ package org.apache.pinot.spi.config.table; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.JsonNode; import java.util.HashSet; import java.util.List; @@ -26,16 +27,14 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; public class IndexingConfig extends BaseJsonConfig { // Default ratio for overriding dictionary for fixed width columns public static final double DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD = 0.85d; - /** - * This should be equal to the one specified in RangeIndexType. - */ + /// This should be equal to the one specified in `RangeIndexType`. private static final int DEFAULT_RANGE_INDEX_VERSION = 2; private List _invertedIndexColumns; @@ -69,22 +68,16 @@ public class IndexingConfig extends BaseJsonConfig { // we can enable it by default and deprecate this config private boolean _optimizeNoDictStatsCollection = false; - /** - * If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality - * single-valued columns for which rawIndexSize / forwardIndexSize is less than the `noDictionarySizeRatioThreshold`. - * It also disables dictionary for json and text columns. - */ + /// If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality single-valued columns for + /// which rawIndexSize / forwardIndexSize is less than the `noDictionarySizeRatioThreshold`. It also disables + /// dictionary for json and text columns. private boolean _optimizeDictionary; - /** - * Same as `optimizeDictionary` but only for metric columns. - */ + /// Same as `optimizeDictionary` but only for metric columns. private boolean _optimizeDictionaryForMetrics; - /** - * Optimize the dictionary type for var width columns, if values are all the same length then use a fixed-width - * dictionary. Else, use a var-width dictionary. - */ + /// Optimize the dictionary type for var width columns, if values are all the same length then use a fixed-width + /// dictionary. Else, use a var-width dictionary. private boolean _optimizeDictionaryType; private double _noDictionarySizeRatioThreshold = DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD; @@ -101,11 +94,8 @@ public class IndexingConfig extends BaseJsonConfig { @Deprecated private boolean _createInvertedIndexDuringSegmentGeneration; - /** - * The list of columns for which the variable length dictionary needs to be enabled in offline - * segments. This is only valid for string and bytes columns and has no impact for columns of - * other data types. - */ + /// The list of columns for which the variable length dictionary needs to be enabled in offline segments. This is + /// only valid for string and bytes columns and has no impact for columns of other data types. private List _varLengthDictionaryColumns; private JsonNode _tierOverwrites; @@ -137,6 +127,8 @@ public void setRangeIndexVersion(int rangeIndexVersion) { _rangeIndexVersion = rangeIndexVersion; } + @Deprecated + @DeprecatedConfig(replacement = "Use 'tableIndexConfig.jsonIndexConfigs' instead.", since = "0.12.0") public List getJsonIndexColumns() { return _jsonIndexColumns; } @@ -162,6 +154,8 @@ public void setAutoGeneratedInvertedIndex(boolean autoGeneratedInvertedIndex) { } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig(replacement = "Remove this field; it is ignored.", since = "1.6.0") public boolean isCreateInvertedIndexDuringSegmentGeneration() { return _createInvertedIndexDuringSegmentGeneration; } @@ -207,11 +201,12 @@ public void setLoadMode(String loadMode) { _loadMode = loadMode; } - /** - * @deprecated Use List> streamConfigs from - * {@link IngestionConfig#getStreamIngestionConfig()} - */ + /// @deprecated Use `List> streamConfigs` from + /// [org.apache.pinot.spi.config.table.ingestion.IngestionConfig#getStreamIngestionConfig()] @Nullable + @Deprecated + @DeprecatedConfig(replacement = "Use 'ingestionConfig.streamIngestionConfig.streamConfigMaps' instead.", + since = "0.7.1") public Map getStreamConfigs() { return _streamConfigs; } @@ -324,12 +319,10 @@ public void setAggregateMetrics(boolean value) { _aggregateMetrics = value; } - /** - * Returns {@code true} if null handling is enabled at table config level. - * - * Remember that {@link org.apache.pinot.spi.data.Schema} can also have null handling enabled at column level and - * that one has more priority. - */ + /// Returns `true` if null handling is enabled at table config level. + /// + /// Remember that [org.apache.pinot.spi.data.Schema] can also have null handling enabled at column level and that + /// one has more priority. public boolean isNullHandlingEnabled() { return _nullHandlingEnabled; } @@ -420,11 +413,9 @@ public void setMultiColumnTextIndexConfig(MultiColumnTextIndexConfig multiColumn _multiColumnTextIndexConfig = multiColumnTextIndexConfig; } - /** - * Returns all columns referenced in the indexing config. This is useful to construct FieldIndexConfigs in - * IndexLoadingConfig when schema is not provided. Only including the columns referenced by indexes supported in - * FieldIndexConfigs. - */ + /// Returns all columns referenced in the indexing config. This is useful to construct `FieldIndexConfigs` in + /// `IndexLoadingConfig` when schema is not provided. Only including the columns referenced by indexes supported in + /// `FieldIndexConfigs`. @JsonIgnore public Set getAllReferencedColumns() { Set allColumns = new HashSet<>(); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/RoutingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/RoutingConfig.java index b3cde3ea82bb..fade4cab7dc9 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/RoutingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/RoutingConfig.java @@ -23,6 +23,7 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; public class RoutingConfig extends BaseJsonConfig { @@ -55,6 +56,9 @@ public RoutingConfig(@JsonProperty("routingTableBuilderName") @Nullable String r } @Nullable + @Deprecated + @DeprecatedConfig(replacement = "Use 'routing.segmentPrunerTypes' and 'routing.instanceSelectorType' instead.", + since = "0.3.0") public String getRoutingTableBuilderName() { return _routingTableBuilderName; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java index e51466bbf123..a3b3f1c1886d 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java @@ -19,10 +19,10 @@ package org.apache.pinot.spi.config.table; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import java.util.concurrent.TimeUnit; import org.apache.pinot.spi.config.BaseJsonConfig; -import org.apache.pinot.spi.config.table.assignment.InstanceAssignmentConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; import org.apache.pinot.spi.utils.TimeUtils; @@ -99,18 +99,16 @@ public void setDeletedSegmentsRetentionPeriod(String deletedSegmentsRetentionPer _deletedSegmentsRetentionPeriod = deletedSegmentsRetentionPeriod; } - /** - * Returns the retention period for segments replaced by a REFRESH ingestion job. Only applies to tables with - * REFRESH ingestion type; for APPEND tables this setting is ignored and replaced segments are deleted immediately. - * - *

When a lineage entry transitions to COMPLETED state, source segments are preserved for this duration before - * being scheduled for deletion, providing a rollback window. Consumers of this config (e.g. the lineage manager) - * treat a null or unparseable value as a 1 day default. - * - *

Accepts a human-readable period string (e.g. {@code "7d"}, {@code "12h"}) as understood by - * {@code TimeUtils.convertPeriodToMillis}. Setting this value too low (e.g. {@code "0d"}) eliminates the rollback - * window; source segments will be deleted on the next retention pass after the lineage is COMPLETED. - */ + /// Returns the retention period for segments replaced by a REFRESH ingestion job. Only applies to tables with + /// REFRESH ingestion type; for APPEND tables this setting is ignored and replaced segments are deleted immediately. + /// + /// When a lineage entry transitions to COMPLETED state, source segments are preserved for this duration before + /// being scheduled for deletion, providing a rollback window. Consumers of this config (e.g. the lineage manager) + /// treat a null or unparseable value as a 1 day default. + /// + /// Accepts a human-readable period string (e.g. `"7d"`, `"12h"`) as understood by + /// `TimeUtils.convertPeriodToMillis`. Setting this value too low (e.g. `"0d"`) eliminates the rollback window; + /// source segments will be deleted on the next retention pass after the lineage is COMPLETED. public String getReplacedSegmentsRetentionPeriod() { return _replacedSegmentsRetentionPeriod; } @@ -119,14 +117,12 @@ public void setReplacedSegmentsRetentionPeriod(String replacedSegmentsRetentionP _replacedSegmentsRetentionPeriod = replacedSegmentsRetentionPeriod; } - /** - * Returns the retention period before stale IN_PROGRESS or REVERTED lineage entries and their destination segments - * are cleaned up. Consumers of this config (e.g. the lineage manager) treat a null or unparseable value as a - * 1 day default. - * - *

Accepts a human-readable period string (e.g. {@code "7d"}, {@code "12h"}) as understood by - * {@code TimeUtils.convertPeriodToMillis}. - */ + /// Returns the retention period before stale IN_PROGRESS or REVERTED lineage entries and their destination segments + /// are cleaned up. Consumers of this config (e.g. the lineage manager) treat a null or unparseable value as a + /// 1 day default. + /// + /// Accepts a human-readable period string (e.g. `"7d"`, `"12h"`) as understood by + /// `TimeUtils.convertPeriodToMillis`. public String getLineageEntryCleanupRetentionPeriod() { return _lineageEntryCleanupRetentionPeriod; } @@ -135,10 +131,11 @@ public void setLineageEntryCleanupRetentionPeriod(String lineageEntryCleanupRete _lineageEntryCleanupRetentionPeriod = lineageEntryCleanupRetentionPeriod; } - /** - * @deprecated Use {@code segmentIngestionFrequency} from {@link IngestionConfig#getBatchIngestionConfig()} - */ + /// @deprecated Use `segmentIngestionFrequency` from + /// [org.apache.pinot.spi.config.table.ingestion.IngestionConfig#getBatchIngestionConfig()] @Deprecated + @DeprecatedConfig(replacement = "Use 'ingestionConfig.batchIngestionConfig.segmentIngestionFrequency' instead.", + since = "0.8.0") public String getSegmentPushFrequency() { return _segmentPushFrequency; } @@ -148,10 +145,11 @@ public void setSegmentPushFrequency(String segmentPushFrequency) { _segmentPushFrequency = segmentPushFrequency; } - /** - * @deprecated Use {@code segmentIngestionType} from {@link IngestionConfig#getBatchIngestionConfig()} - */ + /// @deprecated Use `segmentIngestionType` from + /// [org.apache.pinot.spi.config.table.ingestion.IngestionConfig#getBatchIngestionConfig()] @Deprecated + @DeprecatedConfig(replacement = "Use 'ingestionConfig.batchIngestionConfig.segmentIngestionType' instead.", + since = "0.8.0") public String getSegmentPushType() { return _segmentPushType; } @@ -161,9 +159,7 @@ public void setSegmentPushType(String segmentPushType) { _segmentPushType = segmentPushType; } - /** - * Try to Use {@link TableConfig#getReplication()} - */ + /// Try to use [TableConfig#getReplication()]. public String getReplication() { return _replication; } @@ -172,31 +168,26 @@ public void setReplication(String replication) { _replication = replication; } - /** - * Try to Use {@link TableConfig#getReplication()} - * @deprecated Use _replication instead - * - * Will be deleted in future version of Pinot - */ + /// Try to use [TableConfig#getReplication()]. + /// + /// @deprecated Use `_replication` instead. Will be deleted in a future version of Pinot. @Deprecated + @DeprecatedConfig(replacement = "Use 'segmentsConfig.replication' instead.", since = "1.1.0") public String getReplicasPerPartition() { return _replicasPerPartition; } - /** - * Try to Use {@link SegmentsValidationAndRetentionConfig#setReplication(String)} - * - * Will be deleted in future version of Pinot - */ + /// Try to use [SegmentsValidationAndRetentionConfig#setReplication(String)]. + /// + /// Will be deleted in a future version of Pinot. @Deprecated public void setReplicasPerPartition(String replicasPerPartition) { _replicasPerPartition = replicasPerPartition; } - /** - * @deprecated Use {@link InstanceAssignmentConfig} instead. - */ + /// @deprecated Use [org.apache.pinot.spi.config.table.assignment.InstanceAssignmentConfig] instead. @Deprecated + @DeprecatedConfig(replacement = "Use 'segmentAssignmentConfigMap' instead.", since = "1.3.0") public ReplicaGroupStrategyConfig getReplicaGroupStrategyConfig() { return _replicaGroupStrategyConfig; } @@ -214,20 +205,16 @@ public void setCompletionConfig(CompletionConfig completionConfig) { _completionConfig = completionConfig; } - /** - * Try to Use {@link TableConfig#getReplication()} - */ + /// Try to use [TableConfig#getReplication()]. @Deprecated @JsonIgnore public int getReplicationNumber() { return Integer.parseInt(_replication); } - /** - * Try to Use {@link TableConfig#getReplication()} - * - * Will be deleted in future version of Pinot - */ + /// Try to use [TableConfig#getReplication()]. + /// + /// Will be deleted in a future version of Pinot. @Deprecated @JsonIgnore public int getReplicasPerPartitionNumber() { @@ -250,10 +237,10 @@ public void setCrypterClassName(String crypterClassName) { _crypterClassName = crypterClassName; } - /** - * @deprecated Use {@link InstanceAssignmentConfig} instead - */ + /// @deprecated Use [org.apache.pinot.spi.config.table.assignment.InstanceAssignmentConfig] instead. @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig(replacement = "Use 'instanceAssignmentConfigMap' instead.", since = "1.3.0") public boolean isMinimizeDataMovement() { return _minimizeDataMovement; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java index bf8fb086d05e..ca95ef808f88 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.spi.config.table; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.google.common.base.Preconditions; import java.util.List; @@ -26,6 +27,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; import org.apache.pinot.spi.config.table.ingestion.TransformConfig; import org.apache.pinot.spi.utils.Enablement; @@ -353,6 +355,8 @@ public void setMetadataManagerConfigs(@Nullable Map metadataMana } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig(replacement = "Use 'upsertConfig.snapshot' instead.", since = "1.4.0") public boolean isEnableSnapshot() { return _enableSnapshot; } @@ -366,6 +370,8 @@ public void setEnableSnapshot(boolean enableSnapshot) { } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig(replacement = "Use 'upsertConfig.preload' instead.", since = "1.4.0") public boolean isEnablePreload() { return _enablePreload; } @@ -379,6 +385,10 @@ public void setEnablePreload(boolean enablePreload) { } @Deprecated + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @DeprecatedConfig( + replacement = "Use 'ingestionConfig.streamIngestionConfig.parallelSegmentConsumptionPolicy' instead.", + since = "1.4.0") public boolean isAllowPartialUpsertConsumptionDuringCommit() { return _allowPartialUpsertConsumptionDuringCommit; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/assignment/InstanceReplicaGroupPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/assignment/InstanceReplicaGroupPartitionConfig.java index 1bc40cba212b..3e1b975f0657 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/assignment/InstanceReplicaGroupPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/assignment/InstanceReplicaGroupPartitionConfig.java @@ -19,10 +19,12 @@ package org.apache.pinot.spi.config.table.assignment; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; +import org.apache.pinot.spi.config.DeprecatedConfig; public class InstanceReplicaGroupPartitionConfig extends BaseJsonConfig { @@ -102,6 +104,9 @@ public int getNumInstancesPerPartition() { return _numInstancesPerPartition; } + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + @Deprecated + @DeprecatedConfig(replacement = "Remove this field; it will be removed in a future release.", since = "1.1.0") public boolean isMinimizeDataMovement() { return _minimizeDataMovement; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index 3b316d337396..0d93adeaa0f1 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -55,7 +55,6 @@ public class TableConfigBuilder { - private static final String DEFAULT_SEGMENT_PUSH_TYPE = "APPEND"; private static final String REFRESH_SEGMENT_PUSH_TYPE = "REFRESH"; private static final String DEFAULT_DELETED_SEGMENTS_RETENTION_PERIOD = "7d"; private static final String DEFAULT_NUM_REPLICAS = "1"; @@ -79,9 +78,8 @@ public class TableConfigBuilder { @Deprecated private String _segmentPushFrequency; - // TODO: Remove 'DEFAULT_SEGMENT_PUSH_TYPE' in the future major release. @Deprecated - private String _segmentPushType = DEFAULT_SEGMENT_PUSH_TYPE; + private String _segmentPushType; private String _peerSegmentDownloadScheme; @Deprecated private ReplicaGroupStrategyConfig _replicaGroupStrategyConfig; @@ -122,7 +120,7 @@ public class TableConfigBuilder { /// @deprecated This flag is ignored. Keep it for backward compatibility during upgrade (especially for JSON ser/de). @Deprecated - private boolean _createInvertedIndexDuringSegmentGeneration; + private Boolean _createInvertedIndexDuringSegmentGeneration; private TableCustomConfig _customConfig; private QuotaConfig _quotaConfig; @@ -217,21 +215,19 @@ public TableConfigBuilder setLineageEntryCleanupRetentionPeriod(String lineageEn return this; } - /** - * @deprecated Use {@code segmentIngestionType} from {@link IngestionConfig#getBatchIngestionConfig()} - */ + /// @deprecated Use `segmentIngestionType` from [IngestionConfig#getBatchIngestionConfig()] + @Deprecated public TableConfigBuilder setSegmentPushType(String segmentPushType) { if (REFRESH_SEGMENT_PUSH_TYPE.equalsIgnoreCase(segmentPushType)) { _segmentPushType = REFRESH_SEGMENT_PUSH_TYPE; } else { - _segmentPushType = DEFAULT_SEGMENT_PUSH_TYPE; + _segmentPushType = "APPEND"; } return this; } - /** - * @deprecated Use {@code segmentIngestionFrequency} from {@link IngestionConfig#getBatchIngestionConfig()} - */ + /// @deprecated Use `segmentIngestionFrequency` from [IngestionConfig#getBatchIngestionConfig()] + @Deprecated public TableConfigBuilder setSegmentPushFrequency(String segmentPushFrequency) { _segmentPushFrequency = segmentPushFrequency; return this; @@ -369,6 +365,8 @@ public TableConfigBuilder setAggregateMetrics(boolean aggregateMetrics) { return this; } + /// @deprecated Use `streamConfigMaps` from [IngestionConfig#getStreamIngestionConfig()] + @Deprecated public TableConfigBuilder setStreamConfigs(Map streamConfigs) { Preconditions.checkState(_tableType == TableType.REALTIME); _streamConfigs = streamConfigs; @@ -536,7 +534,9 @@ public TableConfig build() { indexingConfig.setSortedColumn(Collections.singletonList(_sortedColumn)); } indexingConfig.setInvertedIndexColumns(_invertedIndexColumns); - indexingConfig.setCreateInvertedIndexDuringSegmentGeneration(_createInvertedIndexDuringSegmentGeneration); + if (_createInvertedIndexDuringSegmentGeneration != null) { + indexingConfig.setCreateInvertedIndexDuringSegmentGeneration(_createInvertedIndexDuringSegmentGeneration); + } indexingConfig.setNoDictionaryColumns(_noDictionaryColumns); indexingConfig.setOnHeapDictionaryColumns(_onHeapDictionaryColumns); indexingConfig.setBloomFilterColumns(_bloomFilterColumns); diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/config/TableConfigsSerializationTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/config/TableConfigsSerializationTest.java index 9c546acd6090..b30871342c0b 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/config/TableConfigsSerializationTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/config/TableConfigsSerializationTest.java @@ -289,4 +289,23 @@ public void testRealtimeOnlyTableConfigs() Assert.assertFalse(jsonNode.has("offline"), "offline should not be present when null"); Assert.assertTrue(jsonNode.has("realtime")); } + + /// Locks the invariant relied upon by `TableConfigsRestletResource.subConfigJson` — uppercase `OFFLINE` / + /// `REALTIME` keys in the user-submitted JSON are silently dropped by Jackson at deserialization time because + /// the constructor parameters are annotated `@JsonProperty("offline")` / `@JsonProperty("realtime")`. The + /// deprecation pass therefore needs only to look up the lowercase keys. + @Test + public void testUppercaseOfflineRealtimeKeysAreIgnoredByJackson() + throws Exception { + final String jsonWithUppercaseKeys = "{" + + "\"tableName\":\"" + TEST_TABLE_NAME + "\"," + + "\"schema\":" + createTestSchema().toSingleLineJsonString() + "," + + "\"OFFLINE\":{}," + + "\"REALTIME\":{}" + + "}"; + + final TableConfigs parsed = JsonUtils.stringToObject(jsonWithUppercaseKeys, TableConfigs.class); + Assert.assertNull(parsed.getOffline(), "uppercase OFFLINE key must not populate the offline sub-config"); + Assert.assertNull(parsed.getRealtime(), "uppercase REALTIME key must not populate the realtime sub-config"); + } } diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/builder/TableConfigBuilderTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/builder/TableConfigBuilderTest.java index 57c634ffc75a..69273919291c 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/builder/TableConfigBuilderTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/builder/TableConfigBuilderTest.java @@ -19,8 +19,18 @@ package org.apache.pinot.spi.utils.builder; +import com.fasterxml.jackson.databind.JsonNode; +import java.util.List; +import java.util.Map; +import org.apache.pinot.spi.config.table.DedupConfig; +import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.config.table.assignment.InstanceAssignmentConfig; +import org.apache.pinot.spi.config.table.assignment.InstanceReplicaGroupPartitionConfig; +import org.apache.pinot.spi.config.table.assignment.InstanceTagPoolConfig; +import org.apache.pinot.spi.utils.JsonUtils; import org.testng.Assert; import org.testng.annotations.Test; @@ -42,4 +52,44 @@ public void testValidateSkipSegmentPreprocessFlag() { Assert.assertTrue(tableconfig.getIndexingConfig().isSkipSegmentPreprocess(), "skipSegmentPreprocess will be true"); } + + @Test + public void testBuildOmitsDefaultDeprecatedFieldsFromSerializedJson() + throws Exception { + TableConfig offlineTableConfig = new TableConfigBuilder(TableType.OFFLINE) + .setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setFieldConfigList(List.of(new FieldConfig("c1", FieldConfig.EncodingType.DICTIONARY, + List.of(FieldConfig.IndexType.INVERTED), null, null))) + .build(); + JsonNode offlineJson = JsonUtils.stringToJsonNode(offlineTableConfig.toJsonString()); + Assert.assertFalse(offlineJson.path("segmentsConfig").has("minimizeDataMovement")); + Assert.assertFalse(offlineJson.path("tableIndexConfig").has("createInvertedIndexDuringSegmentGeneration")); + Assert.assertFalse(offlineJson.path("fieldConfigList").get(0).has("indexType")); + + TableConfig upsertTableConfig = new TableConfigBuilder(TableType.REALTIME) + .setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setUpsertConfig(new UpsertConfig()) + .build(); + JsonNode upsertJson = JsonUtils.stringToJsonNode(upsertTableConfig.toJsonString()); + Assert.assertFalse(upsertJson.path("upsertConfig").has("enableSnapshot")); + Assert.assertFalse(upsertJson.path("upsertConfig").has("enablePreload")); + Assert.assertFalse(upsertJson.path("upsertConfig").has("allowPartialUpsertConsumptionDuringCommit")); + + InstanceAssignmentConfig instanceAssignmentConfig = new InstanceAssignmentConfig( + new InstanceTagPoolConfig("testTenant", false, 0, null), null, + new InstanceReplicaGroupPartitionConfig(false, 0, 0, 0, 0, 0, false, null), null, false); + TableConfig dedupTableConfig = new TableConfigBuilder(TableType.REALTIME) + .setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setDedupConfig(new DedupConfig()) + .setInstanceAssignmentConfigMap(Map.of("CONSUMING", instanceAssignmentConfig)) + .build(); + JsonNode dedupJson = JsonUtils.stringToJsonNode(dedupTableConfig.toJsonString()); + Assert.assertFalse(dedupJson.path("dedupConfig").has("enablePreload")); + Assert.assertFalse(dedupJson.path("dedupConfig").has("allowDedupConsumptionDuringCommit")); + Assert.assertFalse(dedupJson.path("instanceAssignmentConfigMap").path("CONSUMING") + .path("replicaGroupPartitionConfig").has("minimizeDataMovement")); + } } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/BootstrapTableTool.java b/pinot-tools/src/main/java/org/apache/pinot/tools/BootstrapTableTool.java index 7bc803b03e93..901d1a06492a 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/BootstrapTableTool.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/BootstrapTableTool.java @@ -274,6 +274,9 @@ private void updatedTableConfig(TableConfig tableConfig, File setupTableTmpDir) throws Exception { final List> batchConfigsMaps = tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps(); + if (batchConfigsMaps == null) { + return; + } for (Map batchConfigsMap : batchConfigsMaps) { String inputDirURI = batchConfigsMap.get(BatchConfigProperties.INPUT_DIR_URI); if (!new File(inputDirURI).exists()) { diff --git a/pinot-tools/src/main/resources/conf/sample_offline_table_config.json b/pinot-tools/src/main/resources/conf/sample_offline_table_config.json index d6c486d0040d..cd132ae8f965 100644 --- a/pinot-tools/src/main/resources/conf/sample_offline_table_config.json +++ b/pinot-tools/src/main/resources/conf/sample_offline_table_config.json @@ -6,9 +6,13 @@ "timeType": "DAYS", "retentionTimeUnit": "DAYS", "retentionTimeValue": "365", - "segmentPushType": "APPEND", "replication": "3" }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tenants": { "broker": "brokerOne", "server": "serverOne" @@ -23,5 +27,10 @@ "metadata": { "customConfigs": { } + }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + } } } diff --git a/pinot-tools/src/main/resources/conf/sample_realtime_table_config.json b/pinot-tools/src/main/resources/conf/sample_realtime_table_config.json index 3487b8a035e6..8501b1860e24 100644 --- a/pinot-tools/src/main/resources/conf/sample_realtime_table_config.json +++ b/pinot-tools/src/main/resources/conf/sample_realtime_table_config.json @@ -6,9 +6,12 @@ "timeType": "DAYS", "retentionTimeUnit": "DAYS", "retentionTimeValue": "5", - "segmentPushType": "APPEND", - "replication": "3", - "replicasPerPartition": "1" + "replication": "3" + }, + "segmentAssignmentConfigMap": { + "COMPLETED": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } }, "tenants": { "broker": "brokerOne", @@ -19,17 +22,23 @@ "invertedIndexColumns": [ "column1", "column2" - ], - "streamConfigs": { - "streamType": "kafka", - "stream.kafka.topic.name": "kafkaTopicName", - "stream.kafka.decoder.class.name": "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", - "stream.kafka.decoder.prop.schema.registry.rest.url": "http://localhost:2222/schemaRegistry", - "stream.kafka.consumer.factory.class.name": "org.apache.pinot.plugin.stream.kafka30.KafkaConsumerFactory", - "stream.kafka.consumer.prop.auto.offset.reset": "largest", - "stream.kafka.broker.list": "localhost:19092", - "realtime.segment.flush.threshold.time": "12h", - "realtime.segment.flush.threshold.size": "100000" + ] + }, + "ingestionConfig": { + "streamIngestionConfig": { + "streamConfigMaps": [ + { + "streamType": "kafka", + "stream.kafka.topic.name": "kafkaTopicName", + "stream.kafka.decoder.class.name": "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", + "stream.kafka.decoder.prop.schema.registry.rest.url": "http://localhost:2222/schemaRegistry", + "stream.kafka.consumer.factory.class.name": "org.apache.pinot.plugin.stream.kafka30.KafkaConsumerFactory", + "stream.kafka.consumer.prop.auto.offset.reset": "largest", + "stream.kafka.broker.list": "localhost:19092", + "realtime.segment.flush.threshold.time": "12h", + "realtime.segment.flush.threshold.size": "100000" + } + ] } }, "metadata": { diff --git a/pinot-tools/src/main/resources/examples/batch/airlineStats/airlineStats_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/airlineStats/airlineStats_offline_table_config.json index b541a059e25f..03d34e82f37d 100644 --- a/pinot-tools/src/main/resources/examples/batch/airlineStats/airlineStats_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/airlineStats/airlineStats_offline_table_config.json @@ -4,9 +4,13 @@ "segmentsConfig": { "timeColumnName": "DaysSinceEpoch", "timeType": "DAYS", - "segmentPushType": "APPEND", "replication": "1" }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tenants": {}, "fieldConfigList": [ { @@ -117,6 +121,9 @@ } ], "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "transformConfigs": [ { "columnName": "ts", diff --git a/pinot-tools/src/main/resources/examples/batch/baseballStats/baseballStats_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/baseballStats/baseballStats_offline_table_config.json index add290e3c7de..3e075a51e728 100644 --- a/pinot-tools/src/main/resources/examples/batch/baseballStats/baseballStats_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/baseballStats/baseballStats_offline_table_config.json @@ -2,9 +2,13 @@ "tableName": "baseballStats", "tableType": "OFFLINE", "segmentsConfig": { - "segmentPushType": "APPEND", "replication": "1" }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tenants": { }, "tableIndexConfig": { @@ -17,5 +21,10 @@ "metadata": { "customConfigs": { } + }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + } } } diff --git a/pinot-tools/src/main/resources/examples/batch/clickstreamFunnel/clickstreamFunnel_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/clickstreamFunnel/clickstreamFunnel_offline_table_config.json index 383d8bb17717..669f6545cb76 100644 --- a/pinot-tools/src/main/resources/examples/batch/clickstreamFunnel/clickstreamFunnel_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/clickstreamFunnel/clickstreamFunnel_offline_table_config.json @@ -2,13 +2,11 @@ "tableName": "clickstreamFunnel", "tableType": "OFFLINE", "segmentsConfig": { - "segmentPushType": "APPEND", "timeColumnName": "ts", "retentionTimeUnit": "DAYS", "retentionTimeValue": "30", "replication": "1", - "deletedSegmentsRetentionPeriod": "7d", - "minimizeDataMovement": false + "deletedSegmentsRetentionPeriod": "7d" }, "tenants": { "broker": "DefaultTenant", @@ -16,7 +14,6 @@ }, "tableIndexConfig": { "loadMode": "MMAP", - "createInvertedIndexDuringSegmentGeneration": true, "rangeIndexVersion": 2, "aggregateMetrics": false, "nullHandlingEnabled": true, @@ -37,10 +34,13 @@ }, "metadata": {}, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "transformConfigs": [], "continueOnError": true, "rowTimeValueCheck": true, "segmentTimeValueCheck": false }, "isDimTable": false -} \ No newline at end of file +} diff --git a/pinot-tools/src/main/resources/examples/batch/dimBaseballTeams/dimBaseballTeams_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/dimBaseballTeams/dimBaseballTeams_offline_table_config.json index 8e6e6a769934..babaca8ac1d8 100644 --- a/pinot-tools/src/main/resources/examples/batch/dimBaseballTeams/dimBaseballTeams_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/dimBaseballTeams/dimBaseballTeams_offline_table_config.json @@ -3,7 +3,6 @@ "tableType": "OFFLINE", "isDimTable": true, "segmentsConfig": { - "segmentPushType": "REFRESH", "replication": "1" }, "tenants": { @@ -14,5 +13,10 @@ "metadata": { "customConfigs": { } + }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "REFRESH" + } } } diff --git a/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json index 2cc5c5a8fd3b..df3a7e0fd92f 100644 --- a/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json @@ -2,11 +2,15 @@ "tableName": "fineFoodReviews", "tableType": "OFFLINE", "segmentsConfig": { - "segmentPushType": "APPEND", "replication": "1" }, "tenants": { }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "MMAP", "noDictionaryColumns": ["Text"], @@ -16,6 +20,11 @@ "columns": ["UserId", "ProductId", "Summary"] } }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + } + }, "metadata": { "customConfigs": { } @@ -23,7 +32,9 @@ "fieldConfigList": [ { "encodingType": "RAW", - "indexType": "VECTOR", + "indexTypes": [ + "VECTOR" + ], "name": "embedding", "properties": { "vectorIndexType": "HNSW", diff --git a/pinot-tools/src/main/resources/examples/batch/githubComplexTypeEvents/githubComplexTypeEvents_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/githubComplexTypeEvents/githubComplexTypeEvents_offline_table_config.json index 031be26219f9..239e46f26050 100644 --- a/pinot-tools/src/main/resources/examples/batch/githubComplexTypeEvents/githubComplexTypeEvents_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/githubComplexTypeEvents/githubComplexTypeEvents_offline_table_config.json @@ -4,7 +4,6 @@ "tenants": { }, "segmentsConfig": { - "segmentPushType": "REFRESH", "replication": "1", "timeColumnName": "created_at_timestamp" }, @@ -12,6 +11,9 @@ "loadMode": "MMAP" }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "REFRESH" + }, "transformConfigs": [ { "columnName": "created_at_timestamp", diff --git a/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_table_config.json index cc640bf1463f..74942125fbf9 100644 --- a/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_table_config.json @@ -4,7 +4,6 @@ "tenants": { }, "segmentsConfig": { - "segmentPushType": "REFRESH", "replication": "1", "timeColumnName": "created_at_timestamp" }, @@ -15,13 +14,16 @@ "repo", "payload" ], - "jsonIndexColumns": [ - "actor", - "repo", - "payload" - ] + "jsonIndexConfigs": { + "actor": {}, + "repo": {}, + "payload": {} + } }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "REFRESH" + }, "transformConfigs": [ { "columnName": "created_at_timestamp", diff --git a/pinot-tools/src/main/resources/examples/batch/starbucksStores/starbucksStores_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/starbucksStores/starbucksStores_offline_table_config.json index 76d937dfcb3c..05e2e695209e 100644 --- a/pinot-tools/src/main/resources/examples/batch/starbucksStores/starbucksStores_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/starbucksStores/starbucksStores_offline_table_config.json @@ -4,16 +4,22 @@ "segmentsConfig": { "retentionTimeUnit": "DAYS", "retentionTimeValue": "1", - "segmentPushType": "APPEND", "replication": "1" }, "tenants": { }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "fieldConfigList": [ { "name": "location_st_point", "encodingType": "RAW", - "indexType": "H3", + "indexTypes": [ + "H3" + ], "properties": { "resolutions": "5" } @@ -25,6 +31,11 @@ "location_st_point" ] }, + "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + } + }, "metadata": { "customConfigs": { } diff --git a/pinot-tools/src/main/resources/examples/batch/testUnnest/testUnnest_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/testUnnest/testUnnest_offline_table_config.json index ef765fbb8c76..8e532b2642ff 100644 --- a/pinot-tools/src/main/resources/examples/batch/testUnnest/testUnnest_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/batch/testUnnest/testUnnest_offline_table_config.json @@ -3,11 +3,9 @@ "tableType": "OFFLINE", "segmentsConfig": { "deletedSegmentsRetentionPeriod": "0d", - "segmentPushType": "APPEND", "timeColumnName": "event_time", "retentionTimeUnit": "DAYS", "retentionTimeValue": "180", - "minimizeDataMovement": false, "replication": "1" }, "tenants": { @@ -24,7 +22,6 @@ "optimizeDictionaryType": false, "enableDynamicStarTreeCreation": false, "columnMajorSegmentBuilderEnabled": true, - "createInvertedIndexDuringSegmentGeneration": true, "optimizeDictionaryForMetrics": false, "noDictionarySizeRatioThreshold": 0, "loadMode": "MMAP", @@ -38,6 +35,9 @@ }, "metadata": {}, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "transformConfigs": [], "enrichmentConfigs": [ { diff --git a/pinot-tools/src/main/resources/examples/minions/batch/baseballStats/baseballStats_offline_table_config.json b/pinot-tools/src/main/resources/examples/minions/batch/baseballStats/baseballStats_offline_table_config.json index 9a0c0848592d..16d654fc904c 100644 --- a/pinot-tools/src/main/resources/examples/minions/batch/baseballStats/baseballStats_offline_table_config.json +++ b/pinot-tools/src/main/resources/examples/minions/batch/baseballStats/baseballStats_offline_table_config.json @@ -2,11 +2,15 @@ "tableName": "baseballStats", "tableType": "OFFLINE", "segmentsConfig": { - "segmentPushType": "APPEND", "replication": "1" }, "tenants": { }, + "segmentAssignmentConfigMap": { + "OFFLINE": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "HEAP", "invertedIndexColumns": [ diff --git a/pinot-tools/src/main/resources/examples/stream/dailySales/dailySales_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/dailySales/dailySales_realtime_table_config.json index def6aefe84fb..f325fc7f30ed 100644 --- a/pinot-tools/src/main/resources/examples/stream/dailySales/dailySales_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/dailySales/dailySales_realtime_table_config.json @@ -2,7 +2,6 @@ "tableName": "dailySales", "tableType": "REALTIME", "segmentsConfig": { - "segmentPushType": "APPEND", "timeColumnName": "daysSinceEpoch", "retentionTimeUnit": "DAYS", "retentionTimeValue": "50000", @@ -10,6 +9,11 @@ }, "tenants": { }, + "segmentAssignmentConfigMap": { + "COMPLETED": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "MMAP", "noDictionaryColumns": [ @@ -18,6 +22,9 @@ ] }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "streamIngestionConfig": { "streamConfigMaps": [ { diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json index 35acf8525c86..3e2f12cce153 100644 --- a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json @@ -2,7 +2,6 @@ "tableName": "fineFoodReviews", "tableType": "REALTIME", "segmentsConfig": { - "segmentPushType": "APPEND", "timeColumnName": "ts", "retentionTimeUnit": "DAYS", "retentionTimeValue": "5", @@ -10,6 +9,11 @@ }, "tenants": { }, + "segmentAssignmentConfigMap": { + "COMPLETED": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "MMAP", "noDictionaryColumns": ["Text"], @@ -23,6 +27,9 @@ ] }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "streamIngestionConfig": { "streamConfigMaps": [ { @@ -52,7 +59,9 @@ "fieldConfigList": [ { "encodingType": "RAW", - "indexType": "VECTOR", + "indexTypes": [ + "VECTOR" + ], "name": "embedding", "properties": { "vectorIndexType": "HNSW", diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_0/fineFoodReviews_part_0_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_0/fineFoodReviews_part_0_realtime_table_config.json index 56472db88a5e..19696d27cf58 100644 --- a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_0/fineFoodReviews_part_0_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_0/fineFoodReviews_part_0_realtime_table_config.json @@ -2,7 +2,6 @@ "tableName": "fineFoodReviews_part_0", "tableType": "REALTIME", "segmentsConfig": { - "segmentPushType": "APPEND", "timeColumnName": "ts", "retentionTimeUnit": "DAYS", "retentionTimeValue": "5", @@ -10,6 +9,11 @@ }, "tenants": { }, + "segmentAssignmentConfigMap": { + "COMPLETED": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "MMAP", "noDictionaryColumns": ["Text"], @@ -23,6 +27,9 @@ ] }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "streamIngestionConfig": { "streamConfigMaps": [ { @@ -53,7 +60,9 @@ "fieldConfigList": [ { "encodingType": "RAW", - "indexType": "VECTOR", + "indexTypes": [ + "VECTOR" + ], "name": "embedding", "properties": { "vectorIndexType": "HNSW", diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_1/fineFoodReviews_part_1_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_1/fineFoodReviews_part_1_realtime_table_config.json index 80464a7caac2..4f47a4a0ebee 100644 --- a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_1/fineFoodReviews_part_1_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews_part_1/fineFoodReviews_part_1_realtime_table_config.json @@ -2,7 +2,6 @@ "tableName": "fineFoodReviews_part_1", "tableType": "REALTIME", "segmentsConfig": { - "segmentPushType": "APPEND", "timeColumnName": "ts", "retentionTimeUnit": "DAYS", "retentionTimeValue": "5", @@ -10,6 +9,11 @@ }, "tenants": { }, + "segmentAssignmentConfigMap": { + "COMPLETED": { + "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy" + } + }, "tableIndexConfig": { "loadMode": "MMAP", "noDictionaryColumns": ["Text"], @@ -23,6 +27,9 @@ ] }, "ingestionConfig": { + "batchIngestionConfig": { + "segmentIngestionType": "APPEND" + }, "streamIngestionConfig": { "streamConfigMaps": [ { @@ -53,7 +60,9 @@ "fieldConfigList": [ { "encodingType": "RAW", - "indexType": "VECTOR", + "indexTypes": [ + "VECTOR" + ], "name": "embedding", "properties": { "vectorIndexType": "HNSW", diff --git a/pinot-tools/src/main/resources/examples/stream/meetupRsvpJson/meetupRsvpJson_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/meetupRsvpJson/meetupRsvpJson_realtime_table_config.json index a21dac74112d..14fc21820180 100644 --- a/pinot-tools/src/main/resources/examples/stream/meetupRsvpJson/meetupRsvpJson_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/meetupRsvpJson/meetupRsvpJson_realtime_table_config.json @@ -15,12 +15,12 @@ "member_json", "venue_json" ], - "jsonIndexColumns": [ - "event_json", - "group_json", - "member_json", - "venue_json" - ] + "jsonIndexConfigs": { + "event_json": {}, + "group_json": {}, + "member_json": {}, + "venue_json": {} + } }, "ingestionConfig": { "streamIngestionConfig": { diff --git a/pinot-tools/src/main/resources/examples/stream/upsertJsonMeetupRsvp/upsertJsonMeetupRsvp_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/upsertJsonMeetupRsvp/upsertJsonMeetupRsvp_realtime_table_config.json index 73c0c5953045..c8aee00d91b5 100644 --- a/pinot-tools/src/main/resources/examples/stream/upsertJsonMeetupRsvp/upsertJsonMeetupRsvp_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/upsertJsonMeetupRsvp/upsertJsonMeetupRsvp_realtime_table_config.json @@ -15,12 +15,12 @@ "member_json", "venue_json" ], - "jsonIndexColumns": [ - "event_json", - "group_json", - "member_json", - "venue_json" - ], + "jsonIndexConfigs": { + "event_json": {}, + "group_json": {}, + "member_json": {}, + "venue_json": {} + }, "columnPartitionMap": { "rsvp_id": { "functionName": "Hashcode", diff --git a/pinot-tools/src/main/resources/examples/stream/upsertMeetupRsvp/upsertMeetupRsvp_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/upsertMeetupRsvp/upsertMeetupRsvp_realtime_table_config.json index 1d984b7129f0..b79038cc707b 100644 --- a/pinot-tools/src/main/resources/examples/stream/upsertMeetupRsvp/upsertMeetupRsvp_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/upsertMeetupRsvp/upsertMeetupRsvp_realtime_table_config.json @@ -55,14 +55,16 @@ }, "upsertConfig": { "mode": "FULL", - "enableSnapshot": true, - "enablePreload": true + "snapshot": "ENABLE", + "preload": "ENABLE" }, "fieldConfigList": [ { "name": "location", "encodingType": "RAW", - "indexType": "H3", + "indexTypes": [ + "H3" + ], "properties": { "resolutions": "5" } diff --git a/pinot-tools/src/main/resources/examples/stream/upsertPartialMeetupRsvp/upsertPartialMeetupRsvp_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/upsertPartialMeetupRsvp/upsertPartialMeetupRsvp_realtime_table_config.json index 27a0cf70282d..9dcc934e924e 100644 --- a/pinot-tools/src/main/resources/examples/stream/upsertPartialMeetupRsvp/upsertPartialMeetupRsvp_realtime_table_config.json +++ b/pinot-tools/src/main/resources/examples/stream/upsertPartialMeetupRsvp/upsertPartialMeetupRsvp_realtime_table_config.json @@ -65,7 +65,9 @@ { "name": "location", "encodingType": "RAW", - "indexType": "H3", + "indexTypes": [ + "H3" + ], "properties": { "resolutions": "5" }