You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/11/12 01:13:24 UTC

[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6247: Add stream and batch to ingestionConfig

Jackie-Jiang commented on a change in pull request #6247:
URL: https://github.com/apache/incubator-pinot/pull/6247#discussion_r521705512



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
##########
@@ -1307,9 +1307,9 @@ public void registerPinotLLCRealtimeSegmentManager(PinotLLCRealtimeSegmentManage
     _pinotLLCRealtimeSegmentManager = pinotLLCRealtimeSegmentManager;
   }
 
-  private void verifyIndexingConfig(String tableNameWithType, IndexingConfig indexingConfig) {
+  private void checkForHLC(String tableNameWithType, TableConfig tableConfig) {

Review comment:
       `verifyStreamConfig()`? We might want to add more checks in the future

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/Batch.java
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.table.ingestion;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.BaseJsonConfig;
+
+
+/**
+ * Contains all configs related to the batch sources for ingestion.
+ */
+public class Batch extends BaseJsonConfig {
+
+  @JsonPropertyDescription("Configs for all the batch sources to ingest from")
+  private final List<Map<String, String>> _batchConfigs;
+
+  @JsonPropertyDescription("Push type APPEND or REFRESH")
+  private final String _segmentPushType;
+
+  @JsonPropertyDescription("Push frequency HOURLY or DAILY")
+  private final String _segmentPushFrequency;
+
+  @JsonCreator
+  public Batch(@JsonProperty("batchConfigs") @Nullable List<Map<String, String>> batchConfigs,

Review comment:
       Don't make the argument nullable. If batch ingestion config is provided, these fields should not be null. This can simplify the null handling

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/Stream.java
##########
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.spi.config.table.ingestion;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.BaseJsonConfig;
+
+
+/**
+ * Contains all the configs related to the streams for ingestion
+ */
+public class Stream extends BaseJsonConfig {
+
+  @JsonPropertyDescription("All configs for the streams from which to ingest")
+  private final List<Map<String, String>> _streamConfigs;
+
+  @JsonCreator
+  public Stream(@JsonProperty("streamConfigs") @Nullable List<Map<String, String>> streamConfigs) {

Review comment:
       `streamConfigs` should not be null if stream ingestion config is provided. Also suggest renaming it to `streamConfigMaps` to differentiate it from `StreamConfig`

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/IngestionUtils.java
##########
@@ -114,4 +122,68 @@ public static Long extractTimeValue(Comparable time) {
     }
     return null;
   }
+
+  /**
+   * Fetches the streamConfig from the given realtime table.
+   * First, the ingestionConfigs->stream->streamConfigs will be checked.
+   * If not found, the indexingConfig->streamConfigs will be checked (which is deprecated).
+   * @param tableConfig realtime table config
+   * @return streamConfigs map
+   */
+  public static Map<String, String> getStreamConfigsMap(TableConfig tableConfig) {

Review comment:
       (nit)
   ```suggestion
     public static Map<String, String> getStreamConfigMap(TableConfig tableConfig) {
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/IngestionUtils.java
##########
@@ -114,4 +122,68 @@ public static Long extractTimeValue(Comparable time) {
     }
     return null;
   }
+
+  /**
+   * Fetches the streamConfig from the given realtime table.
+   * First, the ingestionConfigs->stream->streamConfigs will be checked.
+   * If not found, the indexingConfig->streamConfigs will be checked (which is deprecated).
+   * @param tableConfig realtime table config
+   * @return streamConfigs map
+   */
+  public static Map<String, String> getStreamConfigsMap(TableConfig tableConfig) {

Review comment:
       +1 on moving this to spi. Maybe adding an `IngestionConfigUtils` class for the backward-compatibility handling

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/Batch.java
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.table.ingestion;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.BaseJsonConfig;
+
+
+/**
+ * Contains all configs related to the batch sources for ingestion.
+ */
+public class Batch extends BaseJsonConfig {

Review comment:
       Suggest renaming it to `BatchIngestionConfig`. `Batch` can cause confusion IMO

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/Stream.java
##########
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.spi.config.table.ingestion;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.BaseJsonConfig;
+
+
+/**
+ * Contains all the configs related to the streams for ingestion
+ */
+public class Stream extends BaseJsonConfig {

Review comment:
       Suggest renaming it to `StreamIngestionConfig`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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